diff --git a/src/go/k8s/Dockerfile b/src/go/k8s/Dockerfile index 70f46a419153..aa13e52862f8 100644 --- a/src/go/k8s/Dockerfile +++ b/src/go/k8s/Dockerfile @@ -3,7 +3,7 @@ FROM --platform=$BUILDPLATFORM public.ecr.aws/docker/library/golang:1.21.0 as bu ARG TARGETARCH ARG TARGETOS -# Copy the rpk as a close depedency +# Copy the rpk as a close dependency WORKDIR /workspace COPY rpk/ rpk/ diff --git a/src/go/k8s/Makefile b/src/go/k8s/Makefile index 94ae699e9654..dcefe7723c50 100644 --- a/src/go/k8s/Makefile +++ b/src/go/k8s/Makefile @@ -12,6 +12,8 @@ NAMESPACE ?= "redpanda" ifeq (aarch64,$(shell uname -m)) TARGETARCH = arm64 +else ifeq (x86_64,$(shell uname -m)) + TARGETARCH = amd64 else TARGETARCH ?= $(shell uname -m) endif @@ -117,7 +119,7 @@ e2e-tests: kuttl test docker-build docker-build-configurator # Execute end to end tests v2 e2e-tests-v2: kuttl - echo "~~~ Running kuttl tests :k8s:" + echo "~~~ Running kuttl tests :k8s:redpanda" $(KUTTL) test $(TEST_ONLY_FLAG) --config kuttl-v2-test.yaml $(KUTTL_TEST_FLAGS) # Execute end to end unstable tests @@ -201,11 +203,10 @@ gofumpt-lint: gofumpt-install goimports: goimports-install goimports -w . - .PHONY: golangci-lint-install golangci-lint-install: mkdir -p $(GOLANGCI_LINT) - curl -sSfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh | sh -s -- -b $(GOLANGCI_LINT) v1.52.2 + curl -sSfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh | sh -s -- -b $(GOLANGCI_LINT) v1.54.2 .PHONY: golangci-lint golangci-lint: diff --git a/src/go/k8s/config/rbac/bases/operator/kustomization.yaml b/src/go/k8s/config/rbac/bases/operator/kustomization.yaml new file mode 100644 index 000000000000..c449da9e23ea --- /dev/null +++ b/src/go/k8s/config/rbac/bases/operator/kustomization.yaml @@ -0,0 +1,5 @@ +apiVersion: kustomize.config.k8s.io/v1beta1 +kind: Kustomization +resources: +- role.yaml +- role_binding.yaml diff --git a/src/go/k8s/config/rbac/bases/operator/role.yaml b/src/go/k8s/config/rbac/bases/operator/role.yaml index e53554a0cb74..3d29cd0fdd85 100644 --- a/src/go/k8s/config/rbac/bases/operator/role.yaml +++ b/src/go/k8s/config/rbac/bases/operator/role.yaml @@ -6,18 +6,6 @@ metadata: creationTimestamp: null name: manager-role rules: -- apiGroups: - - apps - resources: - - configmaps - verbs: - - create - - delete - - get - - list - - patch - - update - - watch - apiGroups: - apps resources: @@ -131,6 +119,17 @@ rules: - get - list - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - delete + - get + - list + - patch + - update + - watch - apiGroups: - "" resources: @@ -328,6 +327,13 @@ rules: - get - patch - update +- apiGroups: + - apps + resources: + - statefulsets/status + verbs: + - patch + - update - apiGroups: - batch resources: @@ -441,6 +447,25 @@ rules: verbs: - create - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - "" + resources: + - pods + verbs: + - get + - list + - watch - apiGroups: - "" resources: diff --git a/src/go/k8s/controllers/redpanda/console_controller.go b/src/go/k8s/controllers/redpanda/console_controller.go index 1dc6ca545b01..4ec5711ebe91 100644 --- a/src/go/k8s/controllers/redpanda/console_controller.go +++ b/src/go/k8s/controllers/redpanda/console_controller.go @@ -66,7 +66,7 @@ const ( ) //+kubebuilder:rbac:groups=apps,resources=deployments,verbs=get;list;watch;create;update;patch;delete -//+kubebuilder:rbac:groups=apps,resources=configmaps,verbs=get;list;watch;create;update;patch;delete +//+kubebuilder:rbac:groups=core,resources=configmaps,verbs=get;list;watch;create;update;patch;delete //+kubebuilder:rbac:groups=core,resources=events,verbs=get;list;watch;create;update;patch //+kubebuilder:rbac:groups=redpanda.vectorized.io,resources=consoles,verbs=get;list;watch;create;update;patch;delete diff --git a/src/go/k8s/controllers/redpanda/redpanda_controller_utils.go b/src/go/k8s/controllers/redpanda/redpanda_controller_utils.go index bdeee30b9ec1..0c75e8305b0b 100644 --- a/src/go/k8s/controllers/redpanda/redpanda_controller_utils.go +++ b/src/go/k8s/controllers/redpanda/redpanda_controller_utils.go @@ -1,6 +1,8 @@ package redpanda import ( + "context" + "fmt" "io" "net" "net/http" @@ -8,14 +10,85 @@ import ( "path/filepath" "time" - "k8s.io/utils/pointer" - + "github.com/fluxcd/pkg/runtime/logger" "github.com/fluxcd/source-controller/controllers" "github.com/go-logr/logr" + "helm.sh/helm/v3/pkg/action" + "helm.sh/helm/v3/pkg/cli" "helm.sh/helm/v3/pkg/registry" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/errors" + "k8s.io/utils/pointer" + "sigs.k8s.io/controller-runtime/pkg/client" + "sigs.k8s.io/controller-runtime/pkg/event" + "sigs.k8s.io/controller-runtime/pkg/predicate" +) + +const ( + K8sInstanceLabelKey = "app.kubernetes.io/instance" + K8sNameLabelKey = "app.kubernetes.io/name" + K8sComponentLabelKey = "app.kubernetes.io/component" + K8sManagedByLabelKey = "app.kubernetes.io/managed-by" + + EnvHelmReleaseNameKey = "REDPANDA_HELM_RELEASE_NAME" ) +var UpdateEventFilter = predicate.Funcs{ + CreateFunc: func(e event.CreateEvent) bool { return false }, + UpdateFunc: func(e event.UpdateEvent) bool { return true }, + DeleteFunc: func(e event.DeleteEvent) bool { return false }, + GenericFunc: func(e event.GenericEvent) bool { return false }, +} + +var DeleteEventFilter = predicate.Funcs{ + CreateFunc: func(e event.CreateEvent) bool { return false }, + UpdateFunc: func(e event.UpdateEvent) bool { return false }, + DeleteFunc: func(e event.DeleteEvent) bool { return true }, + GenericFunc: func(e event.GenericEvent) bool { return false }, +} + +// Check to see if the release name of a helm chart matches the name of a redpanda object +// this is by design for the operator +func isValidReleaseName(releaseName string, redpandaNameList []string) bool { + for i := range redpandaNameList { + if releaseName == redpandaNameList[i] { + return true + } + } + return false +} + +func getHelmValues(log logr.Logger, releaseName, namespace string) (map[string]interface{}, error) { + settings := cli.New() + actionConfig := new(action.Configuration) + if err := actionConfig.Init(settings.RESTClientGetter(), namespace, os.Getenv("HELM_DRIVER"), func(format string, v ...interface{}) { Debugf(log, format, v) }); err != nil { + return nil, fmt.Errorf("could not create action-config for helm driver: %w", err) + } + + gv := action.NewGetValues(actionConfig) + gv.AllValues = true + + return gv.Run(releaseName) +} + +func bestTrySetRetainPV(c client.Client, log logr.Logger, ctx context.Context, name, namespace string) { + log.WithName("RedpandaNodePVCReconciler.bestTrySetRetainPV") + pv := &corev1.PersistentVolume{} + if getErr := c.Get(ctx, types.NamespacedName{Name: name, Namespace: namespace}, pv); getErr != nil { + Infof(log, "could not change retain policy of pv %s", pv.Name) + return + } + // try to set reclaim policy, fail if we cannot set this to avoid data loss + if pv.Spec.PersistentVolumeReclaimPolicy != corev1.PersistentVolumeReclaimRetain { + pv.Spec.PersistentVolumeReclaimPolicy = corev1.PersistentVolumeReclaimRetain + if updateErr := c.Update(ctx, pv); updateErr != nil { + // no need to place error here. we simply move on and not attempt to remove the pv + Infof(log, "could not set reclaim policy for %s; continuing: %s", pv.Name, updateErr.Error()) + } + } +} + func ClientGenerator(isLogin bool) (*registry.Client, string, error) { if isLogin { // create a temporary file to store the credentials @@ -107,3 +180,15 @@ func StartFileServer(path, address string, l logr.Logger) { func IsBoolPointerNILorEqual(a *bool, b bool) bool { return a == nil || pointer.BoolEqual(a, pointer.Bool(b)) } + +func Infof(log logr.Logger, format string, a ...interface{}) { + log.Info(fmt.Sprintf(format, a...)) +} + +func Debugf(log logr.Logger, format string, a ...interface{}) { + log.V(logger.DebugLevel).Info(fmt.Sprintf(format, a...)) +} + +func Tracef(log logr.Logger, format string, a ...interface{}) { + log.V(logger.TraceLevel).Info(fmt.Sprintf(format, a...)) +} diff --git a/src/go/k8s/controllers/redpanda/redpanda_decommission_controller.go b/src/go/k8s/controllers/redpanda/redpanda_decommission_controller.go new file mode 100644 index 000000000000..b219613fce20 --- /dev/null +++ b/src/go/k8s/controllers/redpanda/redpanda_decommission_controller.go @@ -0,0 +1,688 @@ +// Copyright 2023 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package redpanda + +import ( + "context" + "crypto/tls" + "errors" + "fmt" + "os" + "sort" + "strings" + "time" + + "github.com/go-logr/logr" + "github.com/redpanda-data/redpanda/src/go/rpk/pkg/api/admin" + appsv1 "k8s.io/api/apps/v1" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/utils/pointer" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/redpanda-data/redpanda/src/go/k8s/apis/redpanda/v1alpha1" +) + +// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch; +// +kubebuilder:rbac:groups=core,namespace=default,resources=pods,verbs=get;list;watch; +// +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumeclaims,verbs=get;list;update;patch;delete;watch +// +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=get;list;update;patch;watch +// +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets/status,verbs=update;patch + +const ( + DecommissionCondition = "DecommissionPhase" + + DecomConditionFalseReasonMsg = "Decommission process is in waiting phase." + DecomConditionTrueReasonMsg = "Decommission process is actively running." + DecomConditionUnknownReasonMsg = "Decommission process has completed or in an unknown state." +) + +var ConditionUnknown = appsv1.StatefulSetCondition{ + Type: DecommissionCondition, + Status: corev1.ConditionUnknown, + Message: DecomConditionUnknownReasonMsg, +} + +type DecommissionReconciler struct { + client.Client + OperatorMode bool +} + +// SetupWithManager sets up the controller with the Manager. +func (r *DecommissionReconciler) SetupWithManager(mgr ctrl.Manager) error { + return ctrl.NewControllerManagedBy(mgr). + For(&appsv1.StatefulSet{}).WithEventFilter(UpdateEventFilter).Complete(r) +} + +func (r *DecommissionReconciler) Reconcile(c context.Context, req ctrl.Request) (ctrl.Result, error) { + ctx, done := context.WithCancel(c) + defer done() + + start := time.Now() + log := ctrl.LoggerFrom(ctx).WithName("DecommissionReconciler.Reconcile") + + sts := &appsv1.StatefulSet{} + if err := r.Client.Get(ctx, req.NamespacedName, sts); err != nil { + return ctrl.Result{}, fmt.Errorf("could not retrieve the statefulset: %w", err) + } + + // Examine if the object is under deletion + if !sts.ObjectMeta.DeletionTimestamp.IsZero() { + log.Info(fmt.Sprintf("the statefulset %q is being deleted", req.NamespacedName)) + return ctrl.Result{}, nil + } + + decomCondition, _ := getConditionOfTypeAndListWithout(DecommissionCondition, sts.Status.Conditions) + if decomCondition == nil { + decomCondition = &ConditionUnknown + } + + var err error + var result ctrl.Result + + switch decomCondition.Status { + case corev1.ConditionUnknown: + // we have been notified, check to see if we need to decommission + result, err = r.verifyIfNeedDecommission(ctx, sts) + case corev1.ConditionFalse: + // we have verified we need to decommission, so we need to start, update the condition to do so + patch := client.MergeFrom(sts.DeepCopy()) + + // create condition here + newCondition := appsv1.StatefulSetCondition{ + Type: DecommissionCondition, + Status: corev1.ConditionTrue, + Message: DecomConditionTrueReasonMsg, + Reason: "Waiting period over, ready to start Decommission process.", + LastTransitionTime: metav1.Now(), + } + + if updatedConditions, ok := updateStatefulSetDecommissionConditions(&newCondition, sts.Status.Conditions); ok { + sts.Status.Conditions = updatedConditions + if errPatch := r.Client.Status().Patch(ctx, sts, patch); errPatch != nil { + return ctrl.Result{}, fmt.Errorf("unable to update sts status %q with condition: %w", sts.Name, errPatch) + } + log.Info("Updating true condition successfully") + } + result = ctrl.Result{Requeue: true, RequeueAfter: 10 * time.Second} + case corev1.ConditionTrue: + // condition updated to true, so we proceed to decommission + log.Info("decommission started") + result, err = r.reconcileDecommission(ctx, sts) + } + + // Log reconciliation duration + durationMsg := fmt.Sprintf("succesfull reconciliation finished in %s", time.Since(start).String()) + if result.RequeueAfter > 0 { + durationMsg = fmt.Sprintf("%s, next run in %s", durationMsg, result.RequeueAfter.String()) + } + if err != nil { + durationMsg = fmt.Sprintf("found error, we will requeue, reconciliation attempt finished in %s", time.Since(start).String()) + } + + log.Info(durationMsg) + + return result, err +} + +// verifyIfNeedDecommission checks if decommission is necessary or not. +// The checks happen in a few steps: +// 1. We first check whether statefulset is a redpanda statefulset or not +// 2. If we are, we check if another processes is already in chart of decommission +// 3. We then check to see if we have contact with the adminAPI and verify health +// 4. We ensure we are in a good state by seeing if requestedReplicas and health.AllNodes are neither one empty +// 5. Check that the AdminAPI is reporting that we have more brokers than the number of requestedReplicas of the sts resource +// 6. If we have this situation, we are most likely decommission since we have a signal of scaling, set condition and requeue +// The requeue process at the end of the above process allows time the node to get enter maintenance mode. +// nolint:funlen // the length is ok +func (r *DecommissionReconciler) verifyIfNeedDecommission(ctx context.Context, sts *appsv1.StatefulSet) (ctrl.Result, error) { + log := ctrl.LoggerFrom(ctx).WithName("DecommissionReconciler.verifyIfNeedDecommission") + Infof(log, "verify if we need to decommission: %s/%s", sts.Namespace, sts.Name) + + namespace := sts.Namespace + + if len(sts.Labels) == 0 { + return ctrl.Result{}, nil + } + + // if helm is not managing it, move on. + if managedBy, ok := sts.Labels[K8sManagedByLabelKey]; managedBy != "Helm" || !ok { + Infof(log, "not managed by helm, moving on: managed-by: %s, ok: %t", managedBy, ok) + return ctrl.Result{}, nil + } + + redpandaNameList := make([]string, 0) + if r.OperatorMode { + opts := &client.ListOptions{Namespace: namespace} + redpandaList := &v1alpha1.RedpandaList{} + if errGET := r.Client.List(ctx, redpandaList, opts); errGET != nil { + return ctrl.Result{}, fmt.Errorf("could not GET list of Redpandas in namespace: %w", errGET) + } + + for i := range redpandaList.Items { + item := redpandaList.Items[i] + redpandaNameList = append(redpandaNameList, item.Name) + } + } else { + releaseName, ok := os.LookupEnv(EnvHelmReleaseNameKey) + if !ok { + log.Info(fmt.Sprintf("Skipping reconciliation, expected to find release-name env var: %q", EnvHelmReleaseNameKey)) + return ctrl.Result{}, nil + } + + redpandaNameList = append(redpandaNameList, releaseName) + } + + var releaseName string + if val, ok := sts.Labels[K8sInstanceLabelKey]; !ok || !isValidReleaseName(val, redpandaNameList) { + Infof(log, "could not find instance label or unable retrieve valid releaseName: %s", val) + return ctrl.Result{}, nil + } else { + releaseName = val + } + + requestedReplicas := pointer.Int32Deref(sts.Spec.Replicas, 0) + + valuesMap, err := getHelmValues(log, releaseName, namespace) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not retrieve values, probably not a valid managed helm release: %w", err) + } + + // we are in operator mode, check if another controller has ownership here + if r.OperatorMode { + enabledControllerSideCar, ok, errGetBool := unstructured.NestedBool(valuesMap, "statefulset", "sideCars", "controllers", "enabled") + if errGetBool != nil { + return ctrl.Result{}, fmt.Errorf("could not retrieve sideCar state: %w", errGetBool) + } + if ok && enabledControllerSideCar { + log.Info("another controller has ownership, moving on ") + return ctrl.Result{}, nil + } + } + + adminAPI, err := buildAdminAPI(releaseName, namespace, requestedReplicas, valuesMap) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not reconcile, error creating adminapi: %w", err) + } + + health, err := watchClusterHealth(ctx, adminAPI) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not make request to admin-api: %w", err) + } + + // strange error case here + if requestedReplicas == 0 || len(health.AllNodes) == 0 { + Infof(log, "requested replicas %q, or number of nodes registered %q are invalid, stopping reconciliation", requestedReplicas, health.AllNodes) + return ctrl.Result{}, nil + } + + Debugf(log, "health is found to be %+v", health) + + Infof(log, "all-nodes/requestedReps: %d/%d", len(health.AllNodes), int(requestedReplicas)) + if len(health.AllNodes) > int(requestedReplicas) { + log.Info("we are downscaling, attempt to add condition with status false") + // we are in decommission mode, we should probably wait here some time to verify + // that we are committed and after x amount of time perform the decommission task after. + + patch := client.MergeFrom(sts.DeepCopy()) + // create condition here + newCondition := appsv1.StatefulSetCondition{ + Type: DecommissionCondition, + Status: corev1.ConditionFalse, + Message: DecomConditionFalseReasonMsg, + Reason: "Need for Decommission detected", + LastTransitionTime: metav1.Now(), + } + + updatedConditions, ok := updateStatefulSetDecommissionConditions(&newCondition, sts.Status.Conditions) + if ok { + sts.Status.Conditions = updatedConditions + if errPatch := r.Client.Status().Patch(ctx, sts, patch); errPatch != nil { + return ctrl.Result{}, fmt.Errorf("unable to update sts status %q with condition: %w", sts.Name, errPatch) + } + log.Info("Updating false condition successfully") + } + + log.Info("we are entering decommission and updated conditions, waiting to begin") + // we exit but requeue to allow actual decommission later + return ctrl.Result{Requeue: true, RequeueAfter: 10 * time.Second}, nil + } + + return ctrl.Result{}, nil +} + +// reconcileDecommission performs decommission task after verifying that we should decommission the sts given +// 1. After requeue from decommission due to condition we have set, now we verify and perform tasks. +// 2. Retrieve sts information again, this time focusing on replicas and state +// 3. If we observe that we have not completed deletion, we requeue +// 4. We wait until the cluster is not ready, else requeue, this means that we have reached a steady state that we can proceed from +// 5. As in previous function, we build adminAPI client and get values files +// 6. Check if we have more nodes registered than requested, proceed since this is the first clue we need to decommission +// 7. We are in steady state, proceed if we have more or the same number of downed nodes then are in allNodes registered minus requested +// 8. For all the downed nodes, we get decommission-status, since we have waited for steady state we should be OK to do so +// 9. Any failures captured will force us to requeue and try again. +// 10. Attempt to delete the pvc and retain volumes if possible. +// 11. Finally, reset condition state to unknown if we have been successful so far. +// +//nolint:funlen // length looks good +func (r *DecommissionReconciler) reconcileDecommission(ctx context.Context, sts *appsv1.StatefulSet) (ctrl.Result, error) { + log := ctrl.LoggerFrom(ctx).WithName("DecommissionReconciler.reconcileDecommission") + Infof(log, "reconciling: %s/%s", sts.Namespace, sts.Name) + + namespace := sts.Namespace + + releaseName, ok := sts.Labels[K8sInstanceLabelKey] + if !ok { + log.Info("could not find instance label to retrieve releaseName") + return ctrl.Result{}, nil + } + + requestedReplicas := pointer.Int32Deref(sts.Spec.Replicas, 0) + statusReplicas := sts.Status.Replicas + availableReplicas := sts.Status.AvailableReplicas + + // we have started decommission, but we want to requeue if we have not transitioned here. This should + // avoid decommissioning the wrong node (broker) id + if statusReplicas != requestedReplicas && sts.Status.UpdatedReplicas == 0 { + log.Info("have not finished terminating and restarted largest ordinal, requeue here") + return ctrl.Result{Requeue: true, RequeueAfter: 10 * time.Second}, nil + } + + // This helps avoid decommissioning nodes that are starting up where, say, a node has been removed + // and you need to move it and start a new one + if availableReplicas != 0 { + log.Info("have not reached steady state yet, requeue here") + return ctrl.Result{Requeue: true, RequeueAfter: 10 * time.Second}, nil + } + + valuesMap, err := getHelmValues(log, releaseName, namespace) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not retrieve values, probably not a valid managed helm release: %w", err) + } + + adminAPI, err := buildAdminAPI(releaseName, namespace, requestedReplicas, valuesMap) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not reconcile, error creating adminAPI: %w", err) + } + + health, err := watchClusterHealth(ctx, adminAPI) + if err != nil { + return ctrl.Result{}, fmt.Errorf("could not make request to admin-api: %w", err) + } + + // decommission looks like this: + // 1) replicas = 2, and health: AllNodes:[0 1 2] NodesDown:[2] + // will not heal on its own, we need to remove these downed nodes + // 2) Downed node was replaced due to node being removed, + + if requestedReplicas == 0 || len(health.AllNodes) == 0 { + return ctrl.Result{}, nil + } + + var errList error + // nolint:nestif // this is ok + if len(health.AllNodes) > int(requestedReplicas) { + // we are in decommission mode + + // first check if we have a controllerID before we perform the decommission, else requeue immediately + // this happens when the controllerID node is being terminated, may show more than one node down at this point + if health.ControllerID < 0 { + log.Info("controllerID is not defined yet, we will requeue") + return ctrl.Result{Requeue: true, RequeueAfter: 10 * time.Second}, nil + } + + // perform decommission on down down-nodes but only if down nodes match count of all-nodes-replicas + // the greater case takes care of the situation where we may also have additional ids here. + if len(health.NodesDown) >= (len(health.AllNodes) - int(requestedReplicas)) { + // TODO guard against intermittent situations where a node is coming up after it being brought down + // how do we get a signal of this, it would be easy if we can compare previous situation + for i := range health.NodesDown { + item := health.NodesDown[i] + + // Now we check the decommission status before continuing + doDecommission := false + status, decommStatusError := adminAPI.DecommissionBrokerStatus(ctx, item) + if decommStatusError != nil { + Infof(log, "error found for decommission status: %s", decommStatusError.Error()) + // nolint:gocritic // no need for a switch, this is ok + if strings.Contains(decommStatusError.Error(), "is not decommissioning") { + doDecommission = true + } else if strings.Contains(decommStatusError.Error(), "does not exists") { + Infof(log, "nodeID %d does not exist, skipping: %s", item, decommStatusError.Error()) + } else { + errList = errors.Join(errList, fmt.Errorf("could get decommission status of broker: %w", decommStatusError)) + } + } + Debugf(log, "decommission status: %v", status) + + if doDecommission { + Infof(log, "all checks pass, attempting to decommission: %d", item) + // we want a clear signal to avoid 400s here, the suspicion here is an invalid transitional state + decomErr := adminAPI.DecommissionBroker(ctx, item) + if decomErr != nil && !strings.Contains(decomErr.Error(), "failed: Not Found") && !strings.Contains(decomErr.Error(), "failed: Bad Request") { + errList = errors.Join(errList, fmt.Errorf("could not decommission broker: %w", decomErr)) + } + } + } + } + } + + // now we check pvcs + if err = r.reconcilePVCs(log.WithName("DecommissionReconciler.reconcilePVCs"), ctx, sts, valuesMap); err != nil { + errList = errors.Join(errList, fmt.Errorf("could not reconcile pvcs: %w", err)) + } + + if errList != nil { + return ctrl.Result{RequeueAfter: 30 * time.Second}, fmt.Errorf("found errors %w", errList) + } + + // now we need to + patch := client.MergeFrom(sts.DeepCopy()) + // create condition here + newCondition := appsv1.StatefulSetCondition{ + Type: DecommissionCondition, + Status: corev1.ConditionUnknown, + Message: DecomConditionUnknownReasonMsg, + Reason: "Decommission completed", + LastTransitionTime: metav1.Now(), + } + + if updatedConditions, isUpdated := updateStatefulSetDecommissionConditions(&newCondition, sts.Status.Conditions); isUpdated { + sts.Status.Conditions = updatedConditions + if errPatch := r.Client.Status().Patch(ctx, sts, patch); errPatch != nil { + return ctrl.Result{}, fmt.Errorf("unable to update sts status %q condition: %w", sts.Name, errPatch) + } + log.Info("Updating unknown condition successfully, was able to decommission") + } + + return ctrl.Result{}, nil +} + +func (r *DecommissionReconciler) reconcilePVCs(log logr.Logger, ctx context.Context, sts *appsv1.StatefulSet, valuesMap map[string]interface{}) error { + Infof(log, "reconciling: %s/%s", sts.Namespace, sts.Name) + + Infof(log, "checking storage") + persistentStorageEnabled, ok, err := unstructured.NestedBool(valuesMap, "storage", "persistentVolume", "enabled") + if !ok || err != nil { + return fmt.Errorf("could not retrieve persistent storage settings: %w", err) + } + + if !persistentStorageEnabled { + return nil + } + + log.Info("persistent storage enabled, checking if we need to remove something") + podLabels := client.MatchingLabels{} + + for k, v := range sts.Spec.Template.Labels { + podLabels[k] = v + } + + podOpts := []client.ListOption{ + client.InNamespace(sts.Namespace), + podLabels, + } + + podList := &corev1.PodList{} + if listPodErr := r.Client.List(ctx, podList, podOpts...); listPodErr != nil { + return fmt.Errorf("could not list pods: %w", listPodErr) + } + + templates := sts.Spec.VolumeClaimTemplates + var vctLabels map[string]string + for i := range templates { + template := templates[i] + if template.Name == "datadir" { + vctLabels = template.Labels + break + } + } + + vctMatchingLabels := client.MatchingLabels{} + + for k, v := range vctLabels { + // TODO is this expected + vctMatchingLabels[k] = v + if k == K8sComponentLabelKey { + vctMatchingLabels[k] = fmt.Sprintf("%s-statefulset", v) + } + } + + vctOpts := []client.ListOption{ + client.InNamespace(sts.Namespace), + vctMatchingLabels, + } + + // find the dataDir template + // now cycle through pvcs, retain volumes for future but delete claims + pvcList := &corev1.PersistentVolumeClaimList{} + if listErr := r.Client.List(ctx, pvcList, vctOpts...); listErr != nil { + return fmt.Errorf("could not get pvc list: %w", listErr) + } + + pvcsBound := make(map[string]bool, 0) + for i := range pvcList.Items { + item := pvcList.Items[i] + pvcsBound[item.Name] = false + } + + for j := range podList.Items { + pod := podList.Items[j] + // skip pods that are being terminated + if pod.GetDeletionTimestamp() != nil { + continue + } + volumes := pod.Spec.Volumes + for i := range volumes { + volume := volumes[i] + if volume.VolumeSource.PersistentVolumeClaim != nil { + pvcsBound[volume.VolumeSource.PersistentVolumeClaim.ClaimName] = true + } + } + } + + Infof(log, "pvc name list, binding processed: %+v", pvcsBound) + + if pvcErrorList := r.tryToDeletePVC(log, ctx, pointer.Int32Deref(sts.Spec.Replicas, 0), pvcsBound, pvcList); pvcErrorList != nil { + return fmt.Errorf("errors found: %w", pvcErrorList) + } + + return nil +} + +func (r *DecommissionReconciler) tryToDeletePVC(log logr.Logger, ctx context.Context, replicas int32, isBoundList map[string]bool, pvcList *corev1.PersistentVolumeClaimList) error { + var pvcErrorList error + + // here we sort the list of items, should be ordered by ordinal, and we remove the last first so we sort first then remove + // only the first n matching the number of replicas + keys := make([]string, 0) + for k := range isBoundList { + keys = append(keys, k) + } + + // sort the pvc strings + sort.Strings(keys) + + // remove first + keys = keys[int(replicas):] + + Debugf(log, "pvcs to delete: %+v", keys) + + // TODO: may want to not processes cases where we have more than 1 pvcs, so we force the 1 node at a time policy + // this will avoid dead locking the cluster since you cannot add new nodes, or decomm + + for i := range pvcList.Items { + item := pvcList.Items[i] + + if isBoundList[item.Name] || !isNameInList(item.Name, keys) { + continue + } + + // we are being deleted, before moving forward, try to update PV to avoid data loss + bestTrySetRetainPV(r.Client, log, ctx, item.Spec.VolumeName, item.Namespace) + + Debugf(log, "getting ready to remove %s", item.Name) + + // now we are ready to delete PVC + if errDelete := r.Client.Delete(ctx, &item); errDelete != nil { + pvcErrorList = errors.Join(pvcErrorList, fmt.Errorf("could not delete PVC %q: %w", item.Name, errDelete)) + } + } + + return pvcErrorList +} + +func isNameInList(name string, keys []string) bool { + for i := range keys { + if name == keys[i] { + return true + } + } + return false +} + +func buildAdminAPI(releaseName, namespace string, replicas int32, values map[string]interface{}) (*admin.AdminAPI, error) { + tlsEnabled, ok, err := unstructured.NestedBool(values, "tls", "enabled") + if !ok || err != nil { + // probably not a correct helm release, bail + return nil, fmt.Errorf("tlsEnabled found not to be ok %t, err: %w", tlsEnabled, err) + } + + // need some additional checks to see if this is a redpanda + + // Now try to either use the URL if it is not empty or build the service name to get the sts information + // proper command, note that we are using curl and ignoring tls, we will continue + // curl -k https://redpanda-2.redpanda.redpanda.svc.cluster.local.:9644/v1/cluster/health_overview + // sample response: + + // this can be ://..svc.cluster.local.: + // http scheme will be determined by tls option below: + + var tlsConfig *tls.Config = nil + if tlsEnabled { + //nolint:gosec // will not pull secrets unless working in chart + tlsConfig = &tls.Config{InsecureSkipVerify: true} + } + + urls, err := createBrokerURLs(releaseName, namespace, replicas, values) + if err != nil { + return nil, fmt.Errorf("could not create broker url: %w", err) + } + + // TODO we do not tls, but we may need sasl items here. + return admin.NewAdminAPI(urls, admin.BasicCredentials{}, tlsConfig) +} + +func createBrokerURLs(release, namespace string, replicas int32, values map[string]interface{}) ([]string, error) { + brokerList := make([]string, 0) + + fullnameOverride, ok, err := unstructured.NestedString(values, "fullnameOverride") + if !ok || err != nil { + return brokerList, fmt.Errorf("could not retrieve fullnameOverride: %s; error %w", fullnameOverride, err) + } + + serviceName := release + if fullnameOverride != "" { + serviceName = fullnameOverride + } + + // unstructured things that ports are flot64 types this is by design for json conversion + portFloat, ok, err := unstructured.NestedFloat64(values, "listeners", "admin", "port") + if !ok || err != nil { + return brokerList, fmt.Errorf("could not retrieve admin port %f, error: %w", portFloat, err) + } + + port := int(portFloat) + + domain, ok, err := unstructured.NestedString(values, "clusterDomain") + if !ok || err != nil { + return brokerList, fmt.Errorf("could not retrieve clusterDomain: %s; error: %w", domain, err) + } + + for i := 0; i < int(replicas); i++ { + brokerList = append(brokerList, fmt.Sprintf("%s-%d.%s.%s.svc.%s:%d", release, i, serviceName, namespace, domain, port)) + } + + return brokerList, nil +} + +func watchClusterHealth(ctx context.Context, adminAPI *admin.AdminAPI) (*admin.ClusterHealthOverview, error) { + start := time.Now() + stop := start.Add(60 * time.Second) + + var health admin.ClusterHealthOverview + var err error + for time.Now().Before(stop) { + health, err = adminAPI.GetHealthOverview(ctx) + + if err == nil && len(health.NodesDown) > 0 { + return &health, err + } + time.Sleep(2) // nolint:staticcheck // this is ok, we do not want hit the api too hard + } + + return &health, err +} + +func updateStatefulSetDecommissionConditions(toAdd *appsv1.StatefulSetCondition, conditions []appsv1.StatefulSetCondition) ([]appsv1.StatefulSetCondition, bool) { + if len(conditions) == 0 { + conditions = make([]appsv1.StatefulSetCondition, 0) + conditions = append(conditions, *toAdd) + return conditions, true + } + + oldCondition, newConditions := getConditionOfTypeAndListWithout(toAdd.Type, conditions) + if oldCondition == nil { + newConditions = append(newConditions, *toAdd) + return newConditions, true + } + + switch oldCondition.Status { + case corev1.ConditionUnknown: + // we can only transition to a "false" state, leave old condition alone + if toAdd.Status != corev1.ConditionFalse { + return conditions, false + } + case corev1.ConditionFalse: + if toAdd.Status != corev1.ConditionTrue { + // we can only transition to a "true" state, leave old alone otherwise + return conditions, false + } + case corev1.ConditionTrue: + if toAdd.Status != corev1.ConditionUnknown { + // we can only transition to a "unknown" state, leave old alone otherwise + return conditions, false + } + } + + newConditions = append(newConditions, *toAdd) + return newConditions, true +} + +func getConditionOfTypeAndListWithout(conditionType appsv1.StatefulSetConditionType, conditions []appsv1.StatefulSetCondition) (oldCondition *appsv1.StatefulSetCondition, newConditions []appsv1.StatefulSetCondition) { + if len(conditions) == 0 { + return nil, conditions + } + + newConditions = make([]appsv1.StatefulSetCondition, 0) + for i := range conditions { + c := conditions[i] + // we expect only to have one decommission condition + if c.Type == conditionType { + oldCondition = &c + continue + } + newConditions = append(newConditions, c) + } + + return oldCondition, newConditions +} diff --git a/src/go/k8s/controllers/redpanda/redpanda_node_pvc_controller.go b/src/go/k8s/controllers/redpanda/redpanda_node_pvc_controller.go new file mode 100644 index 000000000000..9c974bdf0236 --- /dev/null +++ b/src/go/k8s/controllers/redpanda/redpanda_node_pvc_controller.go @@ -0,0 +1,182 @@ +// Copyright 2023 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package redpanda + +import ( + "context" + "errors" + "fmt" + "os" + "time" + + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + "k8s.io/component-helpers/storage/volume" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/redpanda-data/redpanda/src/go/k8s/apis/redpanda/v1alpha1" +) + +// +kubebuilder:rbac:groups=cluster.redpanda.com,namespace=default,resources=redpandas,verbs=get;list;watch; +// +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumeclaims,verbs=get;list;update;patch;delete +// +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=get;list;update;patch;delete +// +kubebuilder:rbac:groups=core,resources=nodes,verbs=get;list;watch + +// RedpandaNodePVCReconciler watches node objects, and sets annotation to PVC to mark them for deletion +type RedpandaNodePVCReconciler struct { + client.Client + OperatorMode bool +} + +// SetupWithManager sets up the controller with the Manager. +func (r *RedpandaNodePVCReconciler) SetupWithManager(mgr ctrl.Manager) error { + return ctrl.NewControllerManagedBy(mgr). + For(&corev1.Node{}).WithEventFilter(DeleteEventFilter).Complete(r) +} + +func (r *RedpandaNodePVCReconciler) Reconcile(c context.Context, req ctrl.Request) (ctrl.Result, error) { + ctx, done := context.WithCancel(c) + defer done() + + start := time.Now() + log := ctrl.LoggerFrom(ctx).WithName("RedpandaNodePVCReconciler.Reconcile") + + Infof(log, "Node %q was found to be deleted, checking for existing PVCs", req.Name) + + result, err := r.reconcile(ctx, req) + + // Log reconciliation duration + durationMsg := fmt.Sprintf("reconciliation finished in %s", time.Since(start).String()) + if result.RequeueAfter > 0 { + durationMsg = fmt.Sprintf("%s, next run in %s", durationMsg, result.RequeueAfter.String()) + } + log.Info(durationMsg) + + return result, err +} + +// nolint:funlen,unparam // the length is ok +func (r *RedpandaNodePVCReconciler) reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { + log := ctrl.LoggerFrom(ctx).WithName("RedpandaNodePVCReconciler.reconcile") + Infof(log, "detected node %s deleted; checking if any PVC should be removed", req.Name) + + redpandaNameList := make([]string, 0) + if r.OperatorMode { + opts := &client.ListOptions{Namespace: req.Namespace} + redpandaList := &v1alpha1.RedpandaList{} + if errList := r.Client.List(ctx, redpandaList, opts); errList != nil { + return ctrl.Result{}, fmt.Errorf("could not GET list of Redpandas in namespace: %w", errList) + } + + for i := range redpandaList.Items { + item := redpandaList.Items[i] + redpandaNameList = append(redpandaNameList, item.Name) + } + } else { + releaseName, ok := os.LookupEnv(EnvHelmReleaseNameKey) + if !ok { + Infof(log, "Skipping reconciliation, expected to find release-name env var: %q", EnvHelmReleaseNameKey) + return ctrl.Result{}, nil + } + + redpandaNameList = append(redpandaNameList, releaseName) + } + + opts := &client.ListOptions{Namespace: req.Namespace} + pvcList := &corev1.PersistentVolumeClaimList{} + if errGET := r.Client.List(ctx, pvcList, opts); errGET != nil { + return ctrl.Result{}, fmt.Errorf("could not GET list of PVCs: %w", errGET) + } + + var errs error + // creating map of release name to sideCarEnabled + doTaskMap := make(map[string]bool, 0) + // this could be a lot of redpandas, usually 1 but may be a few + for i := range pvcList.Items { + pvc := pvcList.Items[i] + + // first check if the application label is redpanda, if not continue + if len(pvc.Labels) == 0 { + continue + } + + if _, ok := pvc.Labels[K8sNameLabelKey]; !ok { + continue + } + + if len(pvc.Annotations) == 0 { + continue + } + + // Now we check if the node where the PVC was originally located was deleted + // if it is, then we change the PV to change the reclaim policy and then mark the pvc for deletion + workerNode, ok := pvc.Annotations[volume.AnnSelectedNode] + + if !ok || workerNode == "" { + errs = errors.Join(errs, fmt.Errorf("worker node annotation not found or node name is empty: %q", workerNode)) //nolint:goerr113 // joining since we do not error here + } + + // we are not being deleted, move on + if workerNode != req.Name { + continue + } + + // Now check if we are allowed to operate on this pvc as there may be another controller working + var releaseName string + if val, ok := pvc.Labels[K8sInstanceLabelKey]; !ok || !isValidReleaseName(val, redpandaNameList) { + Infof(log, "could not find instance label or unable retrieve valid releaseName: %s", val) + continue + } else { + releaseName = val + } + + // we are in operator mode, check if another controller has ownership here + // we will silently fail where we cannot find get values, parse the values file etc.. + if r.OperatorMode { // nolint:nestif // this is ok + doTask, foundKey := doTaskMap[releaseName] + if foundKey && !doTask { + continue + } + + if !foundKey { + valuesMap, err := getHelmValues(log, releaseName, req.Namespace) + if err != nil { + Infof(log, "could not retrieve values for release %q, probably not a valid managed helm release: %s", releaseName, err) + continue + } + + enabledControllerSideCar, okSideCar, errGetBool := unstructured.NestedBool(valuesMap, "statefulset", "sideCars", "controllers", "enabled") + if errGetBool != nil { + Infof(log, "could not retrieve sideCar state for release %q: %s", releaseName, errGetBool) + continue + } + + doTaskMap[releaseName] = !enabledControllerSideCar + if okSideCar && enabledControllerSideCar { + log.Info("another controller has ownership, moving on") + continue + } + } + } + + // we are being deleted, before moving forward, try to update PV to avoid data loss + // this is by best effort, if we cannot, then we move on, + pvName := pvc.Spec.VolumeName + bestTrySetRetainPV(r.Client, log, ctx, pvName, req.Namespace) + + // now we are ready to delete PVC + if deleteErr := r.Client.Delete(ctx, &pvc); deleteErr != nil { + errs = errors.Join(errs, fmt.Errorf("could not delete PVC %q: %w", pvc.Name, deleteErr)) //nolint:goerr113 // joining since we do not error here + } + } + + return ctrl.Result{}, errs +} diff --git a/src/go/k8s/controllers/redpanda/state.go b/src/go/k8s/controllers/redpanda/state.go index 31c02ed4b16f..072581e4f6ee 100644 --- a/src/go/k8s/controllers/redpanda/state.go +++ b/src/go/k8s/controllers/redpanda/state.go @@ -1,3 +1,12 @@ +// Copyright 2023 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + package redpanda import ( diff --git a/src/go/k8s/controllers/redpanda/cluster_controller_common_test.go b/src/go/k8s/controllers/test/cluster_controller_common_test.go similarity index 98% rename from src/go/k8s/controllers/redpanda/cluster_controller_common_test.go rename to src/go/k8s/controllers/test/cluster_controller_common_test.go index 4224c2869bff..d83f3423b29a 100644 --- a/src/go/k8s/controllers/redpanda/cluster_controller_common_test.go +++ b/src/go/k8s/controllers/test/cluster_controller_common_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" diff --git a/src/go/k8s/controllers/redpanda/cluster_controller_configuration_test.go b/src/go/k8s/controllers/test/cluster_controller_configuration_test.go similarity index 99% rename from src/go/k8s/controllers/redpanda/cluster_controller_configuration_test.go rename to src/go/k8s/controllers/test/cluster_controller_configuration_test.go index b8bdea9af935..b47d2637d21b 100644 --- a/src/go/k8s/controllers/redpanda/cluster_controller_configuration_test.go +++ b/src/go/k8s/controllers/test/cluster_controller_configuration_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" @@ -40,7 +41,7 @@ const ( versionWithoutCentralizedConfiguration = "v21.11.1-dev" // no centralized config, shadow index enabled ) -var _ = Describe("RedPandaCluster configuration controller", func() { +var _ = Describe("RedpandaCluster configuration controller", func() { const ( timeout = time.Second * 30 interval = time.Millisecond * 100 diff --git a/src/go/k8s/controllers/redpanda/cluster_controller_scale_test.go b/src/go/k8s/controllers/test/cluster_controller_scale_test.go similarity index 99% rename from src/go/k8s/controllers/redpanda/cluster_controller_scale_test.go rename to src/go/k8s/controllers/test/cluster_controller_scale_test.go index 78d024d20cba..a94d596e3425 100644 --- a/src/go/k8s/controllers/redpanda/cluster_controller_scale_test.go +++ b/src/go/k8s/controllers/test/cluster_controller_scale_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" diff --git a/src/go/k8s/controllers/redpanda/cluster_controller_test.go b/src/go/k8s/controllers/test/cluster_controller_test.go similarity index 99% rename from src/go/k8s/controllers/redpanda/cluster_controller_test.go rename to src/go/k8s/controllers/test/cluster_controller_test.go index eee4a4fca41f..15f6f6d63d54 100644 --- a/src/go/k8s/controllers/redpanda/cluster_controller_test.go +++ b/src/go/k8s/controllers/test/cluster_controller_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" @@ -17,6 +18,7 @@ import ( . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" types2 "github.com/onsi/gomega/types" + "github.com/redpanda-data/redpanda/src/go/k8s/controllers/redpanda" appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" v1 "k8s.io/api/rbac/v1" @@ -30,7 +32,6 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client/fake" "github.com/redpanda-data/redpanda/src/go/k8s/apis/vectorized/v1alpha1" - "github.com/redpanda-data/redpanda/src/go/k8s/controllers/redpanda" res "github.com/redpanda-data/redpanda/src/go/k8s/pkg/resources" ) diff --git a/src/go/k8s/controllers/redpanda/console_controller_test.go b/src/go/k8s/controllers/test/console_controller_test.go similarity index 99% rename from src/go/k8s/controllers/redpanda/console_controller_test.go rename to src/go/k8s/controllers/test/console_controller_test.go index 35b7e9ec4e30..01373a5beab0 100644 --- a/src/go/k8s/controllers/redpanda/console_controller_test.go +++ b/src/go/k8s/controllers/test/console_controller_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" diff --git a/src/go/k8s/controllers/test/redpanda_controller_common_test.go b/src/go/k8s/controllers/test/redpanda_controller_common_test.go new file mode 100644 index 000000000000..3ae71e3fc957 --- /dev/null +++ b/src/go/k8s/controllers/test/redpanda_controller_common_test.go @@ -0,0 +1,34 @@ +// Copyright 2022 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +// nolint:testpackage // this name is ok +package test + +import ( + "strings" + + "github.com/moby/moby/pkg/namesgenerator" + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" +) + +func getRandomizedNamespacedName(name string) (types.NamespacedName, *corev1.Namespace) { + ns := strings.Replace(namesgenerator.GetRandomName(0), "_", "-", 1) + key := types.NamespacedName{ + Name: name, + Namespace: ns, + } + namespace := &corev1.Namespace{ + ObjectMeta: metav1.ObjectMeta{ + Name: ns, + }, + } + return key, namespace +} diff --git a/src/go/k8s/controllers/redpanda/redpanda_controller_test.go b/src/go/k8s/controllers/test/redpanda_controller_test.go similarity index 86% rename from src/go/k8s/controllers/redpanda/redpanda_controller_test.go rename to src/go/k8s/controllers/test/redpanda_controller_test.go index e8d006e416b8..d2804d656836 100644 --- a/src/go/k8s/controllers/redpanda/redpanda_controller_test.go +++ b/src/go/k8s/controllers/test/redpanda_controller_test.go @@ -7,23 +7,20 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" "fmt" - "strings" "time" "github.com/fluxcd/helm-controller/api/v2beta1" "github.com/fluxcd/source-controller/api/v1beta2" - "github.com/moby/moby/pkg/namesgenerator" . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" - corev1 "k8s.io/api/core/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - "k8s.io/apimachinery/pkg/types" "sigs.k8s.io/controller-runtime/pkg/client" "github.com/redpanda-data/redpanda/src/go/k8s/apis/redpanda/v1alpha1" @@ -95,17 +92,3 @@ var _ = Describe("Redpanda Controller", func() { }) }) }) - -func getRandomizedNamespacedName(name string) (types.NamespacedName, *corev1.Namespace) { - ns := strings.Replace(namesgenerator.GetRandomName(0), "_", "-", 1) - key := types.NamespacedName{ - Name: name, - Namespace: ns, - } - namespace := &corev1.Namespace{ - ObjectMeta: metav1.ObjectMeta{ - Name: ns, - }, - } - return key, namespace -} diff --git a/src/go/k8s/controllers/redpanda/suite_test.go b/src/go/k8s/controllers/test/suite_test.go similarity index 90% rename from src/go/k8s/controllers/redpanda/suite_test.go rename to src/go/k8s/controllers/test/suite_test.go index 7d4f824583a2..880b5978dcd9 100644 --- a/src/go/k8s/controllers/redpanda/suite_test.go +++ b/src/go/k8s/controllers/test/suite_test.go @@ -7,7 +7,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -package redpanda_test +// nolint:testpackage // this name is ok +package test import ( "context" @@ -28,6 +29,7 @@ import ( . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" "github.com/onsi/gomega/gexec" + "github.com/redpanda-data/redpanda/src/go/rpk/pkg/api/admin" "helm.sh/helm/v3/pkg/getter" "k8s.io/client-go/kubernetes/scheme" "k8s.io/client-go/rest" @@ -38,11 +40,9 @@ import ( logf "sigs.k8s.io/controller-runtime/pkg/log" "sigs.k8s.io/controller-runtime/pkg/log/zap" - "github.com/redpanda-data/redpanda/src/go/rpk/pkg/api/admin" - redpandav1alpha1 "github.com/redpanda-data/redpanda/src/go/k8s/apis/redpanda/v1alpha1" vectorizedv1alpha1 "github.com/redpanda-data/redpanda/src/go/k8s/apis/vectorized/v1alpha1" - redpandacontrollers "github.com/redpanda-data/redpanda/src/go/k8s/controllers/redpanda" + "github.com/redpanda-data/redpanda/src/go/k8s/controllers/redpanda" adminutils "github.com/redpanda-data/redpanda/src/go/k8s/pkg/admin" consolepkg "github.com/redpanda-data/redpanda/src/go/k8s/pkg/console" "github.com/redpanda-data/redpanda/src/go/k8s/pkg/resources" @@ -155,7 +155,7 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { return testKafkaAdmin, nil } - err = (&redpandacontrollers.ClusterReconciler{ + err = (&redpanda.ClusterReconciler{ Client: k8sManager.GetClient(), Log: logf.Log.WithName("controllers").WithName("core").WithName("RedpandaCluster"), Scheme: k8sManager.GetScheme(), @@ -169,7 +169,7 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { Expect(err).ToNot(HaveOccurred()) driftCheckPeriod := 500 * time.Millisecond - err = (&redpandacontrollers.ClusterConfigurationDriftReconciler{ + err = (&redpanda.ClusterConfigurationDriftReconciler{ Client: k8sManager.GetClient(), Log: logf.Log.WithName("controllers").WithName("core").WithName("RedpandaCluster"), Scheme: k8sManager.GetScheme(), @@ -178,7 +178,7 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { }).WithClusterDomain("cluster.local").SetupWithManager(k8sManager) Expect(err).ToNot(HaveOccurred()) - err = (&redpandacontrollers.ConsoleReconciler{ + err = (&redpanda.ConsoleReconciler{ Client: k8sManager.GetClient(), Scheme: k8sManager.GetScheme(), Log: logf.Log.WithName("controllers").WithName("redpanda").WithName("Console"), @@ -190,8 +190,8 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { Expect(err).ToNot(HaveOccurred()) storageAddr := ":9090" - storageAdvAddr := redpandacontrollers.DetermineAdvStorageAddr(storageAddr, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) - storage := redpandacontrollers.MustInitStorage("/tmp", storageAdvAddr, 60*time.Second, 2, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) + storageAdvAddr := redpanda.DetermineAdvStorageAddr(storageAddr, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) + storage := redpanda.MustInitStorage("/tmp", storageAdvAddr, 60*time.Second, 2, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) metricsH := helper.MustMakeMetrics(k8sManager) // TODO fill this in with options @@ -215,7 +215,7 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { // Helm Chart Controller helmChart := helmSourceController.HelmChartReconciler{ Client: k8sManager.GetClient(), - RegistryClientGenerator: redpandacontrollers.ClientGenerator, + RegistryClientGenerator: redpanda.ClientGenerator, Getters: getters, Metrics: metricsH, Storage: storage, @@ -243,10 +243,11 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { // to handle that. <-k8sManager.Elected() - redpandacontrollers.StartFileServer(storage.BasePath, storageAddr, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) + redpanda.StartFileServer(storage.BasePath, storageAddr, logf.Log.WithName("controllers").WithName("core").WithName("Redpanda")) }() - err = (&redpandacontrollers.RedpandaReconciler{ + // Redpanda Reconciler + err = (&redpanda.RedpandaReconciler{ Client: k8sManager.GetClient(), Scheme: k8sManager.GetScheme(), EventRecorder: k8sManager.GetEventRecorderFor("RedpandaReconciler"), @@ -254,6 +255,18 @@ var _ = BeforeSuite(func(suiteCtx SpecContext) { }).SetupWithManager(k8sManager) Expect(err).ToNot(HaveOccurred()) + err = (&redpanda.DecommissionReconciler{ + Client: k8sManager.GetClient(), + OperatorMode: false, + }).SetupWithManager(k8sManager) + Expect(err).ToNot(HaveOccurred()) + + err = (&redpanda.RedpandaNodePVCReconciler{ + Client: k8sManager.GetClient(), + OperatorMode: false, + }).SetupWithManager(k8sManager) + Expect(err).ToNot(HaveOccurred()) + go func() { err = k8sManager.Start(ctx) Expect(err).ToNot(HaveOccurred()) diff --git a/src/go/k8s/controllers/redpanda/testdata/metrics.golden.txt b/src/go/k8s/controllers/test/testdata/metrics.golden.txt similarity index 100% rename from src/go/k8s/controllers/redpanda/testdata/metrics.golden.txt rename to src/go/k8s/controllers/test/testdata/metrics.golden.txt diff --git a/src/go/k8s/go.mod b/src/go/k8s/go.mod index c00db2986337..e436f5f09d74 100644 --- a/src/go/k8s/go.mod +++ b/src/go/k8s/go.mod @@ -41,6 +41,7 @@ require ( k8s.io/apiextensions-apiserver v0.26.1 k8s.io/apimachinery v0.26.2 k8s.io/client-go v0.26.2 + k8s.io/component-helpers v0.26.0 k8s.io/utils v0.0.0-20230220204549-a5ecb0141aa5 sigs.k8s.io/controller-runtime v0.14.6 ) diff --git a/src/go/k8s/go.sum b/src/go/k8s/go.sum index 55e163c420b4..cebfba020c32 100644 --- a/src/go/k8s/go.sum +++ b/src/go/k8s/go.sum @@ -1945,6 +1945,8 @@ k8s.io/client-go v0.26.2 h1:s1WkVujHX3kTp4Zn4yGNFK+dlDXy1bAAkIl+cFAiuYI= k8s.io/client-go v0.26.2/go.mod h1:u5EjOuSyBa09yqqyY7m3abZeovO/7D/WehVVlZ2qcqU= k8s.io/component-base v0.26.2 h1:IfWgCGUDzrD6wLLgXEstJKYZKAFS2kO+rBRi0p3LqcI= k8s.io/component-base v0.26.2/go.mod h1:DxbuIe9M3IZPRxPIzhch2m1eT7uFrSBJUBuVCQEBivs= +k8s.io/component-helpers v0.26.0 h1:KNgwqs3EUdK0HLfW4GhnbD+q/Zl9U021VfIU7qoVYFk= +k8s.io/component-helpers v0.26.0/go.mod h1:jHN01qS/Jdj95WCbTe9S2VZ9yxpxXNY488WjF+yW4fo= k8s.io/klog/v2 v2.100.1 h1:7WCHKK6K8fNhTqfBhISHQ97KrnJNFZMcQvKp7gP/tmg= k8s.io/klog/v2 v2.100.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= k8s.io/kube-openapi v0.0.0-20230217203603-ff9a8e8fa21d h1:oFDpQ7FfzinCtrFOl4izwOWsdTprlS2A9IXBENMW0UA= diff --git a/src/go/k8s/kind.yaml b/src/go/k8s/kind.yaml index 4d8f1a66d07f..d25d03cedb07 100644 --- a/src/go/k8s/kind.yaml +++ b/src/go/k8s/kind.yaml @@ -11,3 +11,9 @@ nodes: image: kindest/node:v1.24.7@sha256:577c630ce8e509131eab1aea12c022190978dd2f745aac5eb1fe65c0807eb315 - role: worker image: kindest/node:v1.24.7@sha256:577c630ce8e509131eab1aea12c022190978dd2f745aac5eb1fe65c0807eb315 +- role: worker + image: kindest/node:v1.24.7@sha256:577c630ce8e509131eab1aea12c022190978dd2f745aac5eb1fe65c0807eb315 +- role: worker + image: kindest/node:v1.24.7@sha256:577c630ce8e509131eab1aea12c022190978dd2f745aac5eb1fe65c0807eb315 +- role: worker + image: kindest/node:v1.24.7@sha256:577c630ce8e509131eab1aea12c022190978dd2f745aac5eb1fe65c0807eb315 diff --git a/src/go/k8s/kuttl-v2-test.yaml b/src/go/k8s/kuttl-v2-test.yaml index 08173e7881c1..1ef341cd6f3a 100644 --- a/src/go/k8s/kuttl-v2-test.yaml +++ b/src/go/k8s/kuttl-v2-test.yaml @@ -3,6 +3,8 @@ kind: TestSuite startKIND: true kindContainers: - localhost/redpanda-operator:dev +# - docker.redpanda.com/redpandadata/redpanda:v23.2.8 +# - docker.redpanda.com/redpandadata/console:v2.3.2 testDirs: - ./tests/e2e-v2 kindConfig: ./kind.yaml @@ -12,12 +14,12 @@ commands: - command: kubectl taint nodes -l node-role.kubernetes.io/master= node-role.kubernetes.io/master:NoSchedule- ignoreFailure: true - command: ./hack/install-cert-manager.sh tests/_e2e_artifacts_v2 - - command: bash -c "kubectl kustomize ./config/crd | kubectl apply -f -" + - script: kubectl kustomize ./config/crd | kubectl apply -f - - command: helm repo add redpanda https://charts.redpanda.com - command: helm repo update - command: helm install --set logLevel=trace --set image.tag=dev - --set image.repository=localhost/redpanda-operator --namespace redpanda --create-namespace - redpanda-operator redpanda/operator + --set image.repository=localhost/redpanda-operator --namespace redpanda --create-namespace redpanda-operator + redpanda/operator --set rbac.createAdditionalControllerCRs=true --set additionalCmdFlags="{--additional-controllers=all}" artifactsDir: tests/_e2e_artifacts_v2 timeout: 720 reportFormat: xml diff --git a/src/go/k8s/main.go b/src/go/k8s/main.go index bcae441bfd65..d3811fda8b8c 100644 --- a/src/go/k8s/main.go +++ b/src/go/k8s/main.go @@ -9,10 +9,12 @@ package main import ( + "fmt" "log" "net/http" "net/http/pprof" "os" + "strings" "time" cmapiv1 "github.com/cert-manager/cert-manager/pkg/apis/certmanager/v1" @@ -47,8 +49,25 @@ import ( redpandawebhooks "github.com/redpanda-data/redpanda/src/go/k8s/webhooks/redpanda" ) +type RedpandaController string + +type OperatorState string + +func (r RedpandaController) toString() string { + return string(r) +} + const ( defaultConfiguratorContainerImage = "vectorized/configurator" + + AllControllers = RedpandaController("all") + NodeController = RedpandaController("nodeWatcher") + DecommissionController = RedpandaController("decommission") + + OperatorV1Mode = OperatorState("Clustered-v1") + OperatorV2Mode = OperatorState("Namespaced-v2") + ClusterControllerMode = OperatorState("Clustered-Controllers") + NamespaceControllerMode = OperatorState("Namespaced-Controllers") ) var ( @@ -70,6 +89,11 @@ var ( logOptions logger.Options storageAdvAddr string + + availableControllers = []string{ + NodeController.toString(), + DecommissionController.toString(), + } ) //nolint:wsl // the init was generated by kubebuilder @@ -88,7 +112,7 @@ func init() { // +kubebuilder:rbac:groups=core,namespace=default,resources=configmaps,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch -//nolint:funlen // length looks good +//nolint:funlen,gocyclo // length looks good func main() { var ( clusterDomain string @@ -105,6 +129,8 @@ func main() { restrictToRedpandaVersion string namespace string eventsAddr string + additionalControllers []string + operatorMode bool // allowPVCDeletion controls the PVC deletion feature in the Cluster custom resource. // PVCs will be deleted when its Pod has been deleted and the Node that Pod is assigned to @@ -138,6 +164,8 @@ func main() { flag.StringVar(&namespace, "namespace", "", "If namespace is set to not empty value, it changes scope of Redpanda operator to work in single namespace") flag.BoolVar(&ghostbuster, "unsafe-decommission-failed-brokers", false, "Set to enable decommissioning a failed broker that is configured but does not exist in the StatefulSet (ghost broker). This may result in invalidating valid data") _ = flag.CommandLine.MarkHidden("unsafe-decommission-failed-brokers") + flag.StringSliceVar(&additionalControllers, "additional-controllers", []string{""}, fmt.Sprintf("which controllers to run, available: all, %s", strings.Join(availableControllers, ", "))) + flag.BoolVar(&operatorMode, "operator-mode", true, "enables to run as an operator, setting this to false will disable cluster (deprecated), redpanda resources reconciliation.") logOptions.BindFlags(flag.CommandLine) clientOptions.BindFlags(flag.CommandLine) @@ -185,9 +213,82 @@ func main() { ImagePullPolicy: corev1.PullPolicy(configuratorImagePullPolicy), } - // we are in v2 mode if we are namespace Scoped, otherwise we are in clusterScope or v1 mode - if namespace != "" { // nolint:nestif // this is ok - ctrl.Log.Info("running in v2", "mode", "namespaced", "namespace", namespace) + // init running state values if we are not in operator mode + operatorRunningState := ClusterControllerMode + if namespace != "" { + operatorRunningState = NamespaceControllerMode + } + + // but if we are in operator mode, then the run state is different + if operatorMode { + operatorRunningState = OperatorV1Mode + if namespace != "" { + operatorRunningState = OperatorV2Mode + } + } + + // Now we start different processes depending on state + switch operatorRunningState { + case OperatorV1Mode: + ctrl.Log.Info("running in v1", "mode", OperatorV1Mode) + + if err = (&redpandacontrollers.ClusterReconciler{ + Client: mgr.GetClient(), + Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("Cluster"), + Scheme: mgr.GetScheme(), + AdminAPIClientFactory: adminutils.NewInternalAdminAPI, + DecommissionWaitInterval: decommissionWaitInterval, + MetricsTimeout: metricsTimeout, + RestrictToRedpandaVersion: restrictToRedpandaVersion, + GhostDecommissioning: ghostbuster, + }).WithClusterDomain(clusterDomain).WithConfiguratorSettings(configurator).WithAllowPVCDeletion(allowPVCDeletion).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "Unable to create controller", "controller", "Cluster") + os.Exit(1) + } + + if err = (&redpandacontrollers.ClusterConfigurationDriftReconciler{ + Client: mgr.GetClient(), + Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("ClusterConfigurationDrift"), + Scheme: mgr.GetScheme(), + AdminAPIClientFactory: adminutils.NewInternalAdminAPI, + RestrictToRedpandaVersion: restrictToRedpandaVersion, + }).WithClusterDomain(clusterDomain).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "Unable to create controller", "controller", "ClusterConfigurationDrift") + os.Exit(1) + } + + if err = redpandacontrollers.NewClusterMetricsController(mgr.GetClient()). + SetupWithManager(mgr); err != nil { + setupLog.Error(err, "Unable to create controller", "controller", "ClustersMetrics") + os.Exit(1) + } + + if err = (&redpandacontrollers.ConsoleReconciler{ + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("Console"), + AdminAPIClientFactory: adminutils.NewInternalAdminAPI, + Store: consolepkg.NewStore(mgr.GetClient(), mgr.GetScheme()), + EventRecorder: mgr.GetEventRecorderFor("Console"), + KafkaAdminClientFactory: consolepkg.NewKafkaAdmin, + }).WithClusterDomain(clusterDomain).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "Console") + os.Exit(1) + } + + // Setup webhooks + if webhookEnabled { + setupLog.Info("Setup webhook") + if err = (&vectorizedv1alpha1.Cluster{}).SetupWebhookWithManager(mgr); err != nil { + setupLog.Error(err, "Unable to create webhook", "webhook", "RedpandaCluster") + os.Exit(1) + } + hookServer := mgr.GetWebhookServer() + hookServer.Register("/mutate-redpanda-vectorized-io-v1alpha1-console", &webhook.Admission{Handler: &redpandawebhooks.ConsoleDefaulter{Client: mgr.GetClient()}}) + hookServer.Register("/validate-redpanda-vectorized-io-v1alpha1-console", &webhook.Admission{Handler: &redpandawebhooks.ConsoleValidator{Client: mgr.GetClient()}}) + } + case OperatorV2Mode: + ctrl.Log.Info("running in v2", "mode", OperatorV2Mode, "namespace", namespace) storageAddr := ":9090" storageAdvAddr = redpandacontrollers.DetermineAdvStorageAddr(storageAddr, setupLog) storage := redpandacontrollers.MustInitStorage("/tmp", storageAdvAddr, 60*time.Second, 2, setupLog) @@ -268,8 +369,8 @@ func main() { redpandacontrollers.StartFileServer(storage.BasePath, storageAddr, setupLog) }() - // Redpanda Reconciler + // Redpanda Reconciler var redpandaEventRecorder *events.Recorder if redpandaEventRecorder, err = events.NewRecorder(mgr, ctrl.Log, eventsAddr, "RedpandaReconciler"); err != nil { setupLog.Error(err, "unable to create event recorder for: RedpandaReconciler") @@ -285,80 +386,72 @@ func main() { setupLog.Error(err, "unable to create controller", "controller", "Redpanda") os.Exit(1) } - } else { - ctrl.Log.Info("running in v1", "mode", "clustered") - if err = (&redpandacontrollers.ClusterReconciler{ - Client: mgr.GetClient(), - Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("Cluster"), - Scheme: mgr.GetScheme(), - AdminAPIClientFactory: adminutils.NewInternalAdminAPI, - DecommissionWaitInterval: decommissionWaitInterval, - MetricsTimeout: metricsTimeout, - RestrictToRedpandaVersion: restrictToRedpandaVersion, - GhostDecommissioning: ghostbuster, - }).WithClusterDomain(clusterDomain).WithConfiguratorSettings(configurator).WithAllowPVCDeletion(allowPVCDeletion).SetupWithManager(mgr); err != nil { - setupLog.Error(err, "Unable to create controller", "controller", "Cluster") + var topicEventRecorder *events.Recorder + if topicEventRecorder, err = events.NewRecorder(mgr, ctrl.Log, eventsAddr, "TopicReconciler"); err != nil { + setupLog.Error(err, "unable to create event recorder for: TopicReconciler") os.Exit(1) } - if err = (&redpandacontrollers.ClusterConfigurationDriftReconciler{ - Client: mgr.GetClient(), - Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("ClusterConfigurationDrift"), - Scheme: mgr.GetScheme(), - AdminAPIClientFactory: adminutils.NewInternalAdminAPI, - RestrictToRedpandaVersion: restrictToRedpandaVersion, - }).WithClusterDomain(clusterDomain).SetupWithManager(mgr); err != nil { - setupLog.Error(err, "Unable to create controller", "controller", "ClusterConfigurationDrift") + if err = (&clusterredpandacomcontrollers.TopicReconciler{ + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + EventRecorder: topicEventRecorder, + }).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "Topic") os.Exit(1) } - if err = redpandacontrollers.NewClusterMetricsController(mgr.GetClient()). - SetupWithManager(mgr); err != nil { - setupLog.Error(err, "Unable to create controller", "controller", "ClustersMetrics") - os.Exit(1) + if runThisController(NodeController, additionalControllers) { + if err = (&redpandacontrollers.RedpandaNodePVCReconciler{ + Client: mgr.GetClient(), + OperatorMode: operatorMode, + }).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "RedpandaNodePVCReconciler") + os.Exit(1) + } } - if err = (&redpandacontrollers.ConsoleReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - Log: ctrl.Log.WithName("controllers").WithName("redpanda").WithName("Console"), - AdminAPIClientFactory: adminutils.NewInternalAdminAPI, - Store: consolepkg.NewStore(mgr.GetClient(), mgr.GetScheme()), - EventRecorder: mgr.GetEventRecorderFor("Console"), - KafkaAdminClientFactory: consolepkg.NewKafkaAdmin, - }).WithClusterDomain(clusterDomain).SetupWithManager(mgr); err != nil { - setupLog.Error(err, "unable to create controller", "controller", "Console") - os.Exit(1) + if runThisController(DecommissionController, additionalControllers) { + if err = (&redpandacontrollers.DecommissionReconciler{ + Client: mgr.GetClient(), + OperatorMode: operatorMode, + }).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "DecommissionReconciler") + os.Exit(1) + } } - // Setup webhooks - if webhookEnabled { - setupLog.Info("Setup webhook") - if err = (&vectorizedv1alpha1.Cluster{}).SetupWebhookWithManager(mgr); err != nil { - setupLog.Error(err, "Unable to create webhook", "webhook", "RedpandaCluster") + case ClusterControllerMode: + ctrl.Log.Info("running as a cluster controller", "mode", ClusterControllerMode) + setupLog.Error(err, "unable to create cluster controllers, not supported") + os.Exit(1) + case NamespaceControllerMode: + ctrl.Log.Info("running as a namespace controller", "mode", NamespaceControllerMode, "namespace", namespace) + if runThisController(NodeController, additionalControllers) { + if err = (&redpandacontrollers.RedpandaNodePVCReconciler{ + Client: mgr.GetClient(), + OperatorMode: operatorMode, + }).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "RedpandaNodePVCReconciler") os.Exit(1) } - hookServer := mgr.GetWebhookServer() - hookServer.Register("/mutate-redpanda-vectorized-io-v1alpha1-console", &webhook.Admission{Handler: &redpandawebhooks.ConsoleDefaulter{Client: mgr.GetClient()}}) - hookServer.Register("/validate-redpanda-vectorized-io-v1alpha1-console", &webhook.Admission{Handler: &redpandawebhooks.ConsoleValidator{Client: mgr.GetClient()}}) } - } - var topicEventRecorder *events.Recorder - if topicEventRecorder, err = events.NewRecorder(mgr, ctrl.Log, eventsAddr, "TopicReconciler"); err != nil { - setupLog.Error(err, "unable to create event recorder for: TopicReconciler") + if runThisController(DecommissionController, additionalControllers) { + if err = (&redpandacontrollers.DecommissionReconciler{ + Client: mgr.GetClient(), + OperatorMode: operatorMode, + }).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "DecommissionReconciler") + os.Exit(1) + } + } + default: + setupLog.Error(err, "unable unknown state, not supported") os.Exit(1) } - if err = (&clusterredpandacomcontrollers.TopicReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - EventRecorder: topicEventRecorder, - }).SetupWithManager(mgr); err != nil { - setupLog.Error(err, "unable to create controller", "controller", "Topic") - os.Exit(1) - } //+kubebuilder:scaffold:builder if err := mgr.AddHealthzCheck("health", healthz.Ping); err != nil { @@ -390,3 +483,16 @@ func main() { os.Exit(1) } } + +func runThisController(rc RedpandaController, controllers []string) bool { + if len(controllers) == 0 { + return false + } + + for _, c := range controllers { + if RedpandaController(c) == AllControllers || RedpandaController(c) == rc { + return true + } + } + return false +} diff --git a/src/go/k8s/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml new file mode 100644 index 000000000000..3b1b6c30feb8 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/00-assert-create-redpanda.yaml @@ -0,0 +1,54 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + finalizers: + - operator.redpanda.com/finalizer + name: redpanda +status: + conditions: + - message: Redpanda reconciliation succeeded + reason: RedpandaClusterDeployed + status: "True" + type: Ready + helmRelease: redpanda + helmReleaseReady: true + helmRepository: redpanda-repository + helmRepositoryReady: true + observedGeneration: 1 +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 5 + collisionCount: 0 + currentReplicas: 5 + observedGeneration: 1 + readyReplicas: 5 + replicas: 5 + updatedReplicas: 5 +--- +apiVersion: helm.toolkit.fluxcd.io/v2beta1 +kind: HelmRelease +metadata: + name: redpanda +status: + conditions: + - message: Release reconciliation succeeded + reason: ReconciliationSucceeded + status: "True" + type: Ready + - message: Helm install succeeded + reason: InstallSucceeded + status: "True" + type: Released + helmChart: redpanda/redpanda-redpanda + 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/src/go/k8s/tests/e2e-v2/decommission/00-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/00-create-redpanda.yaml new file mode 100644 index 000000000000..2324ee29a2b9 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/00-create-redpanda.yaml @@ -0,0 +1,11 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + name: redpanda +spec: + chartRef: + chartVersion: "5.3.2" + clusterSpec: + statefulset: + replicas: 5 diff --git a/src/go/k8s/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml new file mode 100644 index 000000000000..b7d5f1982bd0 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/01-assert-scale-down-redpanda.yaml @@ -0,0 +1,54 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + finalizers: + - operator.redpanda.com/finalizer + name: redpanda +status: + conditions: + - message: Redpanda reconciliation succeeded + reason: RedpandaClusterDeployed + status: "True" + type: Ready + helmRelease: redpanda + helmReleaseReady: true + helmRepository: redpanda-repository + helmRepositoryReady: true + observedGeneration: 2 +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 4 + collisionCount: 0 + currentReplicas: 4 + observedGeneration: 2 + readyReplicas: 4 + replicas: 4 + updatedReplicas: 4 +--- +apiVersion: helm.toolkit.fluxcd.io/v2beta1 +kind: HelmRelease +metadata: + name: redpanda +status: + conditions: + - message: Release reconciliation succeeded + reason: ReconciliationSucceeded + status: "True" + type: Ready + - message: Helm upgrade succeeded + reason: UpgradeSucceeded + status: "True" + type: Released + helmChart: redpanda/redpanda-redpanda + 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/src/go/k8s/tests/e2e-v2/decommission/01-scale-down-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/01-scale-down-redpanda.yaml new file mode 100644 index 000000000000..9ea24a22a5e6 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/01-scale-down-redpanda.yaml @@ -0,0 +1,11 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + name: redpanda +spec: + chartRef: + chartVersion: "5.3.2" + clusterSpec: + statefulset: + replicas: 4 diff --git a/src/go/k8s/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml new file mode 100644 index 000000000000..0de91e471c42 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/02-assert-scale-down-redpanda.yaml @@ -0,0 +1,54 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + finalizers: + - operator.redpanda.com/finalizer + name: redpanda +status: + conditions: + - message: Redpanda reconciliation succeeded + reason: RedpandaClusterDeployed + status: "True" + type: Ready + helmRelease: redpanda + helmReleaseReady: true + helmRepository: redpanda-repository + helmRepositoryReady: true + observedGeneration: 3 +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 3 + collisionCount: 0 + currentReplicas: 3 + observedGeneration: 3 + readyReplicas: 3 + replicas: 3 + updatedReplicas: 3 +--- +apiVersion: helm.toolkit.fluxcd.io/v2beta1 +kind: HelmRelease +metadata: + name: redpanda +status: + conditions: + - message: Release reconciliation succeeded + reason: ReconciliationSucceeded + status: "True" + type: Ready + - message: Helm upgrade succeeded + reason: UpgradeSucceeded + status: "True" + type: Released + helmChart: redpanda/redpanda-redpanda + 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/src/go/k8s/tests/e2e-v2/decommission/02-scale-down-redpanda.yaml b/src/go/k8s/tests/e2e-v2/decommission/02-scale-down-redpanda.yaml new file mode 100644 index 000000000000..7ae496488fc1 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/02-scale-down-redpanda.yaml @@ -0,0 +1,11 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + name: redpanda +spec: + chartRef: + chartVersion: "5.3.2" + clusterSpec: + statefulset: + replicas: 3 diff --git a/src/go/k8s/tests/e2e-v2/decommission/03-assert.yaml b/src/go/k8s/tests/e2e-v2/decommission/03-assert.yaml new file mode 100644 index 000000000000..9cd7bb2844f4 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/03-assert.yaml @@ -0,0 +1,4 @@ +apiVersion: kuttl.dev/v1beta1 +kind: TestAssert +collectors: +- command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/src/go/k8s/tests/e2e-v2/decommission/03-delete-redpandas.yaml b/src/go/k8s/tests/e2e-v2/decommission/03-delete-redpandas.yaml new file mode 100644 index 000000000000..1c8658708176 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/decommission/03-delete-redpandas.yaml @@ -0,0 +1,28 @@ +--- +apiVersion: kuttl.dev/v1beta1 +kind: TestStep +delete: + - apiVersion: cluster.redpanda.com/v1alpha1 + kind: Redpanda + name: redpanda + - apiVersion: batch/v1 + kind: Job + name: redpanda-configuration + - apiVersion: batch/v1 + kind: Job + name: redpanda-post-upgrade + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-0 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-1 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-2 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-3 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-4 diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml new file mode 100644 index 000000000000..3b1b6c30feb8 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/00-assert-create-redpanda.yaml @@ -0,0 +1,54 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + finalizers: + - operator.redpanda.com/finalizer + name: redpanda +status: + conditions: + - message: Redpanda reconciliation succeeded + reason: RedpandaClusterDeployed + status: "True" + type: Ready + helmRelease: redpanda + helmReleaseReady: true + helmRepository: redpanda-repository + helmRepositoryReady: true + observedGeneration: 1 +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 5 + collisionCount: 0 + currentReplicas: 5 + observedGeneration: 1 + readyReplicas: 5 + replicas: 5 + updatedReplicas: 5 +--- +apiVersion: helm.toolkit.fluxcd.io/v2beta1 +kind: HelmRelease +metadata: + name: redpanda +status: + conditions: + - message: Release reconciliation succeeded + reason: ReconciliationSucceeded + status: "True" + type: Ready + - message: Helm install succeeded + reason: InstallSucceeded + status: "True" + type: Released + helmChart: redpanda/redpanda-redpanda + 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/src/go/k8s/tests/e2e-v2/node-deleted/00-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/00-create-redpanda.yaml new file mode 100644 index 000000000000..2324ee29a2b9 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/00-create-redpanda.yaml @@ -0,0 +1,11 @@ +--- +apiVersion: cluster.redpanda.com/v1alpha1 +kind: Redpanda +metadata: + name: redpanda +spec: + chartRef: + chartVersion: "5.3.2" + clusterSpec: + statefulset: + replicas: 5 diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/01-assert-delete-node.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/01-assert-delete-node.yaml new file mode 100644 index 000000000000..a8491ec16dca --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/01-assert-delete-node.yaml @@ -0,0 +1,14 @@ +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 5 + collisionCount: 0 + currentReplicas: 5 + observedGeneration: 1 + readyReplicas: 5 + replicas: 5 + updatedReplicas: 5 +# TestAssert cannot be used, since TestStep is used and this assertion is used there diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/01-delete-node.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/01-delete-node.yaml new file mode 100644 index 000000000000..a6ec7253f889 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/01-delete-node.yaml @@ -0,0 +1,6 @@ +apiVersion: kuttl.dev/v1beta1 +kind: TestStep +commands: + - script: kubectl -n redpanda get pod redpanda-1 -o jsonpath='{.spec.nodeName}' | xargs kubectl delete node +assert: + - 01-assert-delete-node.yaml diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/02-assert-delete-node-again.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/02-assert-delete-node-again.yaml new file mode 100644 index 000000000000..a8491ec16dca --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/02-assert-delete-node-again.yaml @@ -0,0 +1,14 @@ +--- +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: redpanda +status: + availableReplicas: 5 + collisionCount: 0 + currentReplicas: 5 + observedGeneration: 1 + readyReplicas: 5 + replicas: 5 + updatedReplicas: 5 +# TestAssert cannot be used, since TestStep is used and this assertion is used there diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/02-delete-node-again.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/02-delete-node-again.yaml new file mode 100644 index 000000000000..ab5092824929 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/02-delete-node-again.yaml @@ -0,0 +1,6 @@ +apiVersion: kuttl.dev/v1beta1 +kind: TestStep +commands: + - script: kubectl -n redpanda get pod redpanda-2 -o jsonpath='{.spec.nodeName}' | xargs kubectl delete node +assert: + - 02-assert-delete-node-again.yaml diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/03-assert.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/03-assert.yaml new file mode 100644 index 000000000000..9cd7bb2844f4 --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/03-assert.yaml @@ -0,0 +1,4 @@ +apiVersion: kuttl.dev/v1beta1 +kind: TestAssert +collectors: +- command: ../../../hack/get-redpanda-info.sh redpanda ../../_e2e_artifacts_v2 diff --git a/src/go/k8s/tests/e2e-v2/node-deleted/03-delete-redpandas.yaml b/src/go/k8s/tests/e2e-v2/node-deleted/03-delete-redpandas.yaml new file mode 100644 index 000000000000..09e1398c559e --- /dev/null +++ b/src/go/k8s/tests/e2e-v2/node-deleted/03-delete-redpandas.yaml @@ -0,0 +1,28 @@ +--- +apiVersion: kuttl.dev/v1beta1 +kind: TestStep +delete: + - apiVersion: cluster.redpanda.com/v1alpha1 + kind: Redpanda + name: redpanda + - apiVersion: batch/v1 + kind: Job + name: redpanda-configuration + - apiVersion: batch/v1 + kind: Job + name: redpanda-post-upgrade + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-0 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-1 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-2 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-3 + - apiVersion: v1 + kind: PersistentVolumeClaim + name: datadir-redpanda-4 \ No newline at end of file diff --git a/src/go/k8s/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml index a5402e5e5cee..39bba406cacf 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-rollback/00-create-redpanda.yaml @@ -5,7 +5,7 @@ metadata: name: redpanda spec: chartRef: - chartVersion: "5.0.7" + chartVersion: "5.3.2" clusterSpec: statefulset: replicas: 1 diff --git a/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml index 69c49c5771cc..1512ae7c7a27 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-assert-upgrade-bad-redpanda.yaml @@ -34,8 +34,8 @@ metadata: name: redpanda status: helmChart: redpanda/redpanda-redpanda - lastAppliedRevision: 5.0.7 - lastAttemptedRevision: 5.0.7 + lastAppliedRevision: 5.3.2 + lastAttemptedRevision: 5.3.2 --- apiVersion: kuttl.dev/v1beta1 kind: TestAssert diff --git a/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml index b73e02165120..3a0ee44e28b3 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-rollback/01-upgrade-bad-redpanda.yaml @@ -6,7 +6,7 @@ metadata: spec: chartRef: timeout: 1m - chartVersion: "5.0.7" + chartVersion: "5.3.2" clusterSpec: image: tag: v23.99.99 diff --git a/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml index 0af44d8c8709..c279cd0890d1 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-assert-upgrade-good-redpanda.yaml @@ -33,8 +33,8 @@ metadata: name: redpanda status: helmChart: redpanda/redpanda-redpanda - lastAppliedRevision: 5.0.7 - lastAttemptedRevision: 5.0.7 + lastAppliedRevision: 5.3.2 + lastAttemptedRevision: 5.3.2 --- apiVersion: kuttl.dev/v1beta1 kind: TestAssert diff --git a/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml index fc085d534caa..1379171245da 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-rollback/02-upgrade-good-redpanda.yaml @@ -6,7 +6,7 @@ metadata: spec: chartRef: timeout: 2m - chartVersion: "5.0.7" + chartVersion: "5.3.2" clusterSpec: image: tag: v23.2.3 diff --git a/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml index f639d61293f6..92ca3f8d7b1b 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-assert-create-redpanda.yaml @@ -45,8 +45,8 @@ status: status: "True" type: Released helmChart: redpanda/redpanda-redpanda - lastAppliedRevision: 5.0.9 - lastAttemptedRevision: 5.0.9 + lastAppliedRevision: 5.3.2 + lastAttemptedRevision: 5.3.2 lastAttemptedValuesChecksum: 9a164b2eba37330d0bb90cd1dff1e40779cab130 lastReleaseRevision: 1 observedGeneration: 1 diff --git a/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-create-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-create-redpanda.yaml index f490498dd0ae..51c1701b136e 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-create-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-values-check/00-create-redpanda.yaml @@ -5,7 +5,7 @@ metadata: name: redpanda spec: chartRef: - chartVersion: "5.0.9" + chartVersion: "5.3.2" clusterSpec: statefulset: replicas: 1 diff --git a/src/go/k8s/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml b/src/go/k8s/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml index 5d8c6f4dba74..d00b2e029b5a 100644 --- a/src/go/k8s/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml +++ b/src/go/k8s/tests/e2e-v2/upgrade-values-check/01-assert-upgrade-redpanda.yaml @@ -45,8 +45,8 @@ status: status: "True" type: Released helmChart: redpanda/redpanda-redpanda - lastAppliedRevision: 5.0.9 - lastAttemptedRevision: 5.0.9 + lastAppliedRevision: 5.3.2 + lastAttemptedRevision: 5.3.2 lastAttemptedValuesChecksum: 1c9b2eec287c945a5a66d5562e998e108de18393 lastReleaseRevision: 2 observedGeneration: 2