From 23fc48e3963a4e26ccaf4692afa834907ece7d18 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Wed, 4 Dec 2024 16:35:17 -0500 Subject: [PATCH 01/13] Implement alternative sidecar broker and pvc decommissioner --- operator/cmd/main.go | 2 + operator/cmd/sidecar/sidecar.go | 150 ++++ operator/config/rbac/bases/operator/role.yaml | 6 + .../config/rbac/decommissioner-role/role.yaml | 91 +++ .../redpanda/redpanda_controller_test.go | 17 +- operator/internal/decommissioning/role.yaml | 91 +++ .../statefulset_decomissioner.go | 643 ++++++++++++++++++ .../statefulset_decommissioner_test.go | 331 +++++++++ .../decommissioning/values_fetcher.go | 116 ++++ operator/internal/testenv/testenv.go | 25 +- operator/pkg/functional/map.go | 66 ++ taskfiles/k8s.yml | 8 + 12 files changed, 1530 insertions(+), 16 deletions(-) create mode 100644 operator/cmd/sidecar/sidecar.go create mode 100644 operator/config/rbac/decommissioner-role/role.yaml create mode 100644 operator/internal/decommissioning/role.yaml create mode 100644 operator/internal/decommissioning/statefulset_decomissioner.go create mode 100644 operator/internal/decommissioning/statefulset_decommissioner_test.go create mode 100644 operator/internal/decommissioning/values_fetcher.go diff --git a/operator/cmd/main.go b/operator/cmd/main.go index afc48a51c..b24e9d297 100644 --- a/operator/cmd/main.go +++ b/operator/cmd/main.go @@ -20,6 +20,7 @@ import ( "github.com/redpanda-data/redpanda-operator/operator/cmd/configurator" "github.com/redpanda-data/redpanda-operator/operator/cmd/envsubst" "github.com/redpanda-data/redpanda-operator/operator/cmd/run" + "github.com/redpanda-data/redpanda-operator/operator/cmd/sidecar" "github.com/redpanda-data/redpanda-operator/operator/cmd/syncclusterconfig" "github.com/redpanda-data/redpanda-operator/operator/cmd/version" ) @@ -45,6 +46,7 @@ func init() { run.Command(), syncclusterconfig.Command(), version.Command(), + sidecar.Command(), ) logOptions.BindFlags(rootCmd.PersistentFlags()) diff --git a/operator/cmd/sidecar/sidecar.go b/operator/cmd/sidecar/sidecar.go new file mode 100644 index 000000000..f768e9295 --- /dev/null +++ b/operator/cmd/sidecar/sidecar.go @@ -0,0 +1,150 @@ +// Copyright 2024 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 sidecar + +import ( + "context" + "errors" + "net/http" + "net/http/pprof" + "time" + + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" + "github.com/spf13/cobra" + "k8s.io/apimachinery/pkg/runtime" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" + clientgoscheme "k8s.io/client-go/kubernetes/scheme" + ctrl "sigs.k8s.io/controller-runtime" + metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" +) + +var ( + schemes = []func(s *runtime.Scheme) error{ + clientgoscheme.AddToScheme, + } +) + +func Command() *cobra.Command { + var ( + metricsAddr string + probeAddr string + pprofAddr string + clusterNamespace string + clusterName string + decommissionTimeout time.Duration + ) + + cmd := &cobra.Command{ + Use: "run", + Short: "Run the redpanda sidecar", + RunE: func(cmd *cobra.Command, args []string) error { + ctx := cmd.Context() + + // Always run a pprof server to facilitate debugging. + go runPProfServer(ctx, pprofAddr) + + return Run( + ctx, + metricsAddr, + probeAddr, + clusterNamespace, + clusterName, + decommissionTimeout, + ) + }, + } + + cmd.Flags().StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.") + cmd.Flags().StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.") + cmd.Flags().StringVar(&pprofAddr, "pprof-bind-address", ":8082", "The address the metric endpoint binds to.") + cmd.Flags().StringVar(&clusterNamespace, "cluster-namespace", "", "The namespace of the cluster that this sidecar manages.") + cmd.Flags().StringVar(&clusterName, "cluster-name", "", "The name of the cluster that this sidecar manages.") + cmd.Flags().DurationVar(&decommissionTimeout, "decommission-timeout", 10*time.Second, "The time period to wait before recheck a broker that is being decommissioned.") + + return cmd +} + +func Run( + ctx context.Context, + metricsAddr string, + probeAddr string, + clusterNamespace string, + clusterName string, + decommissionTimeout time.Duration, +) error { + setupLog := ctrl.LoggerFrom(ctx).WithName("setup") + + if clusterNamespace == "" { + err := errors.New("must specify a cluster-namespace parameter") + setupLog.Error(err, "no cluster namespace provided") + return err + } + + if clusterName == "" { + err := errors.New("must specify a cluster-name parameter") + setupLog.Error(err, "no cluster name provided") + return err + } + + scheme := runtime.NewScheme() + + for _, fn := range schemes { + utilruntime.Must(fn(scheme)) + } + + mgr, err := ctrl.NewManager(ctrl.GetConfigOrDie(), ctrl.Options{ + Metrics: metricsserver.Options{BindAddress: metricsAddr}, + HealthProbeBindAddress: probeAddr, + LeaderElection: true, + LeaderElectionID: clusterName + "." + clusterNamespace + ".redpanda", + Scheme: scheme, + LeaderElectionNamespace: clusterNamespace, + }) + if err != nil { + setupLog.Error(err, "unable to initialize manager") + return err + } + + if err := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), []decommissioning.Option{ + decommissioning.WithFilter(decommissioning.FilterStatefulSetOwner(clusterNamespace, clusterName)), + decommissioning.WithRequeueTimeout(decommissionTimeout), + }...).Setup(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "DecommissionReconciler") + return err + } + + if err := mgr.Start(ctrl.SetupSignalHandler()); err != nil { + setupLog.Error(err, "problem running manager") + return err + } + + return nil +} + +func runPProfServer(ctx context.Context, listenAddr string) { + logger := ctrl.LoggerFrom(ctx) + + pprofMux := http.NewServeMux() + pprofMux.HandleFunc("/debug/pprof/", pprof.Index) + pprofMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) + pprofMux.HandleFunc("/debug/pprof/profile", pprof.Profile) + pprofMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + pprofMux.HandleFunc("/debug/pprof/trace", pprof.Trace) + pprofServer := &http.Server{ + Addr: listenAddr, + Handler: pprofMux, + ReadHeaderTimeout: 3 * time.Second, + } + + logger.Info("starting pprof server...", "addr", listenAddr) + if err := pprofServer.ListenAndServe(); err != nil { + logger.Error(err, "failed to run pprof server") + } +} diff --git a/operator/config/rbac/bases/operator/role.yaml b/operator/config/rbac/bases/operator/role.yaml index 9623a4391..f6b1a738a 100644 --- a/operator/config/rbac/bases/operator/role.yaml +++ b/operator/config/rbac/bases/operator/role.yaml @@ -259,6 +259,12 @@ rules: - patch - update - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - patch - apiGroups: - "" resources: diff --git a/operator/config/rbac/decommissioner-role/role.yaml b/operator/config/rbac/decommissioner-role/role.yaml new file mode 100644 index 000000000..297421727 --- /dev/null +++ b/operator/config/rbac/decommissioner-role/role.yaml @@ -0,0 +1,91 @@ +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + name: decommissioner-role +rules: +- apiGroups: + - "" + resources: + - events + verbs: + - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - patch +- apiGroups: + - "" + resources: + - pods + - secrets + verbs: + - get + - list + - watch +- apiGroups: + - apps + resources: + - statefulsets + verbs: + - get + - list + - watch +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: Role +metadata: + name: decommissioner-role + namespace: default +rules: +- apiGroups: + - "" + resources: + - events + verbs: + - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - patch +- apiGroups: + - "" + resources: + - pods + - secrets + verbs: + - get + - list + - watch +- apiGroups: + - apps + resources: + - statefulsets + verbs: + - get + - list + - watch diff --git a/operator/internal/controller/redpanda/redpanda_controller_test.go b/operator/internal/controller/redpanda/redpanda_controller_test.go index 8c234588f..221368beb 100644 --- a/operator/internal/controller/redpanda/redpanda_controller_test.go +++ b/operator/internal/controller/redpanda/redpanda_controller_test.go @@ -14,7 +14,6 @@ import ( _ "embed" "encoding/json" "fmt" - "math/rand" "slices" "sort" "strings" @@ -798,7 +797,7 @@ func (s *RedpandaControllerSuite) setupRBAC() string { Verbs: []string{"*"}, }) - name := "testenv-" + s.randString(6) + name := "testenv-" + testenv.RandString(6) role.Name = name role.Namespace = s.env.Namespace() @@ -843,22 +842,10 @@ func (s *RedpandaControllerSuite) setupRBAC() string { return name } -func (s *RedpandaControllerSuite) randString(length int) string { - const alphabet = "abcdefghijklmnopqrstuvwxyz0123456789" - - name := "" - for i := 0; i < length; i++ { - //nolint:gosec // not meant to be a secure random string. - name += string(alphabet[rand.Intn(len(alphabet))]) - } - - return name -} - func (s *RedpandaControllerSuite) minimalRP(useFlux bool) *redpandav1alpha2.Redpanda { return &redpandav1alpha2.Redpanda{ ObjectMeta: metav1.ObjectMeta{ - Name: "rp-" + s.randString(6), // GenerateName doesn't play nice with SSA. + Name: "rp-" + testenv.RandString(6), // GenerateName doesn't play nice with SSA. }, Spec: redpandav1alpha2.RedpandaSpec{ ChartRef: redpandav1alpha2.ChartRef{ diff --git a/operator/internal/decommissioning/role.yaml b/operator/internal/decommissioning/role.yaml new file mode 100644 index 000000000..297421727 --- /dev/null +++ b/operator/internal/decommissioning/role.yaml @@ -0,0 +1,91 @@ +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + name: decommissioner-role +rules: +- apiGroups: + - "" + resources: + - events + verbs: + - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - patch +- apiGroups: + - "" + resources: + - pods + - secrets + verbs: + - get + - list + - watch +- apiGroups: + - apps + resources: + - statefulsets + verbs: + - get + - list + - watch +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: Role +metadata: + name: decommissioner-role + namespace: default +rules: +- apiGroups: + - "" + resources: + - events + verbs: + - create + - patch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - delete + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - patch +- apiGroups: + - "" + resources: + - pods + - secrets + verbs: + - get + - list + - watch +- apiGroups: + - apps + resources: + - statefulsets + verbs: + - get + - list + - watch diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go new file mode 100644 index 000000000..7464a9f1a --- /dev/null +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -0,0 +1,643 @@ +// Copyright 2024 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 decommissioning + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "sort" + "strconv" + "strings" + "time" + + "github.com/redpanda-data/common-go/rpadmin" + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" + internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" + "github.com/redpanda-data/redpanda-operator/operator/pkg/client/kubernetes" + "github.com/redpanda-data/redpanda-operator/operator/pkg/collections" + "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" + appsv1 "k8s.io/api/apps/v1" + 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" + corev1ac "k8s.io/client-go/applyconfigurations/core/v1" + "k8s.io/client-go/tools/record" + "k8s.io/utils/ptr" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/builder" + "sigs.k8s.io/controller-runtime/pkg/client" + "sigs.k8s.io/controller-runtime/pkg/handler" + "sigs.k8s.io/controller-runtime/pkg/predicate" +) + +const ( + eventReasonBroker = "DecommissioningBroker" + eventReasonUnboundPersistentVolumeClaims = "DecommissioningUnboundPersistentVolumeClaims" + + k8sManagedByLabelKey = "app.kubernetes.io/managed-by" + k8sInstanceLabelKey = "app.kubernetes.io/instance" + k8sComponentLabelKey = "app.kubernetes.io/component" + k8sNameLabelKey = "app.kubernetes.io/name" + datadirVolume = "datadir" + + traceLevel = 2 + debugLevel = 1 + infoLevel = 0 + + defaultRequeueTimeout = 10 * time.Second +) + +type Option func(*StatefulSetDecomissioner) + +func FilterStatefulSetOwner(ownerNamespace, ownerName string) func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { + filter := filterOwner(ownerNamespace, ownerName) + return func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { + return filter(set), nil + } +} + +func filterOwner(ownerNamespace, ownerName string) func(o client.Object) bool { + return func(o client.Object) bool { + labels := o.GetLabels() + if o.GetNamespace() == ownerNamespace && labels != nil && labels[k8sInstanceLabelKey] == ownerName { + return true + } + return false + } +} + +func WithFilter(filter func(ctx context.Context, set *appsv1.StatefulSet) (bool, error)) Option { + return func(decommissioner *StatefulSetDecomissioner) { + decommissioner.filter = filter + } +} + +func WithFactory(factory internalclient.ClientFactory) Option { + return func(decommissioner *StatefulSetDecomissioner) { + decommissioner.factory = factory + } +} + +func WithRequeueTimeout(timeout time.Duration) Option { + return func(decommissioner *StatefulSetDecomissioner) { + decommissioner.requeueTimeout = timeout + } +} + +type StatefulSetDecomissioner struct { + client client.Client + factory internalclient.ClientFactory + fetcher ValuesFetcher + recorder record.EventRecorder + requeueTimeout time.Duration + filter func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) +} + +func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher ValuesFetcher, options ...Option) *StatefulSetDecomissioner { + k8sClient := mgr.GetClient() + + decommissioner := &StatefulSetDecomissioner{ + recorder: mgr.GetEventRecorderFor("broker-decommissioner"), + client: k8sClient, + fetcher: fetcher, + factory: internalclient.NewFactory(mgr.GetConfig(), k8sClient), + requeueTimeout: defaultRequeueTimeout, + filter: func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { return true, nil }, + } + + for _, opt := range options { + opt(decommissioner) + } + + return decommissioner +} + +// +kubebuilder:rbac:groups=apps,resources=statefulsets,verbs=get;list;watch +// +kubebuilder:rbac:groups=core,resources=pods,verbs=get;list;watch +// +kubebuilder:rbac:groups=core,resources=persistentvolumeclaims,verbs=get;list;watch;delete +// +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=patch +// +kubebuilder:rbac:groups=core,resources=events,verbs=create;patch +// +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch + +// +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets,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;watch;delete +// +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumes,verbs=patch +// +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch +// +kubebuilder:rbac:groups=core,namespace=default,resources=secrets,verbs=get;list;watch + +func (s *StatefulSetDecomissioner) Setup(mgr ctrl.Manager) error { + pvcPredicate, err := predicate.LabelSelectorPredicate( + metav1.LabelSelector{ + MatchExpressions: []metav1.LabelSelectorRequirement{{ + Key: k8sNameLabelKey, // look for only redpanda owned pvcs + Operator: metav1.LabelSelectorOpIn, + Values: []string{"redpanda"}, + }, { + Key: k8sComponentLabelKey, // make sure the PVC is part of the statefulset + Operator: metav1.LabelSelectorOpIn, + Values: []string{"redpanda-statefulset"}, + }, { + Key: k8sInstanceLabelKey, // make sure we have a cluster name + Operator: metav1.LabelSelectorOpExists, + }}, + }, + ) + if err != nil { + return err + } + + return ctrl.NewControllerManagedBy(mgr). + For(&appsv1.StatefulSet{}). + Owns(&corev1.Pod{}). + // PVCs don't have a "true" owner ref, so instead we attempt to map backwards via labels + Watches(&corev1.PersistentVolumeClaim{}, handler.EnqueueRequestsFromMapFunc(func(_ context.Context, o client.Object) []ctrl.Request { + claim := o.(*corev1.PersistentVolumeClaim) + labels := claim.GetLabels() + + // a bit of defensive programming, but we should always have labels due to our use + // of a predicate + if labels == nil { + // we have no labels, so we can't map anything + return nil + } + + release := labels[k8sInstanceLabelKey] + if release == "" { + // we have an invalid release name, so skip + return nil + } + + if !strings.HasPrefix(claim.Name, datadirVolume+"-") { + // we only care about the datadir volume + return nil + } + + // if we are here, it means we can map to a real stateful set + return []ctrl.Request{ + {NamespacedName: types.NamespacedName{ + Name: release, + Namespace: claim.Namespace, + }}, + } + }), builder.WithPredicates(pvcPredicate)). + Complete(s) +} + +func (s *StatefulSetDecomissioner) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { + log := ctrl.LoggerFrom(ctx, "namespace", req.Namespace, "name", req.Name).WithName("StatefulSetDecomissioner.Reconcile") + + set := &appsv1.StatefulSet{} + if err := s.client.Get(ctx, req.NamespacedName, set); err != nil { + if apierrors.IsNotFound(err) { + return ctrl.Result{}, nil + } + log.Error(err, "fetching StatefulSet") + + // avoid the internal controller runtime stacktrace + return ctrl.Result{Requeue: true}, nil + } + + // skip objects already being deleted + if !set.ObjectMeta.DeletionTimestamp.IsZero() { + // TODO: This follows the original implementation, but + // won't this leave orphaned PVCs around? + log.V(traceLevel).Info("StatefulSet is currently deleted, skipping") + + return ctrl.Result{}, nil + } + + requeue, err := s.Decommission(ctx, set) + if err != nil { + // we already logged any error, just requeue directly + return ctrl.Result{Requeue: true}, nil + } + + if requeue { + return ctrl.Result{RequeueAfter: s.requeueTimeout}, nil + } + + return ctrl.Result{}, nil +} + +// Decommission decommissions any stray resources for a StatefulSet. This includes: +// +// 1. Orphaned PVCs created by the StatefulSet +// 2. Any old brokers that exist in Redpanda and need to be explicitly decommissioned via the admin API +// +// It has the following rough flow: +// +// 1. Filter and manage only particular StatefulSets via a specified user filter (for running as a sidecar) +// 2. Find associated unbound PVCs +// 3. If an unbound PVC exists, delete it after attempting to set its backing PV to have a retain policy +// 4. Get the health status of our cluster and compare the existing number of nodes with the desired amount +// 5. If we have more nodes than desired, attempt to decommission the downed nodes via checking: +// a. That the broker's ordinal parsed from its internal advertised address exceeds the max ordinal +// that the stateful set would produce +// b. If it doesn't, check to see if other node ordinals collide with this node ordinal +// c. If they do and one of them is healthy, then any unhealthy nodes with the same ordinal +// can be decommissioned +// 6. If any broker is currently being decommissioned, wait until that process is complete +// 7. Once it is, decommission the brokers starting with the broker with the lowest node id +// +// For PVC deletion and broker decommissioning, each step happens sequentially such that no two brokers should +// attempt to be decommissioned simultaneously. Likewise each PVC is deleted one by one. +func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { + // note that this is best-effort, the decommissioning code needs to be idempotent and deterministic + + log := ctrl.LoggerFrom(ctx, "namespace", set.Namespace, "name", set.Name).WithName("StatefulSetDecommissioner.Decomission") + + // if helm is not managing it, move on. + if managedBy, ok := set.Labels[k8sManagedByLabelKey]; managedBy != "Helm" || !ok { + log.V(traceLevel).Info("not managed by helm") + return false, nil + } + + keep, err := s.filter(ctx, set) + if err != nil { + log.Error(err, "error filtering StatefulSet") + return false, err + } + + if !keep { + log.V(traceLevel).Info("skipping decommission, StatefulSet filtered out") + return false, nil + } + + unboundVolumeClaims, err := s.findUnboundVolumeClaims(ctx, set) + if err != nil { + log.Error(err, "error finding unbound PersistentVolumeClaims") + return false, err + } + + log.V(traceLevel).Info("fetched unbound volume claims", "claims", functional.MapFn(func(claim *corev1.PersistentVolumeClaim) string { + return claim.Name + }, unboundVolumeClaims)) + + // we first clean up any unbound PVCs, ensuring that their PVs have a retain policy + if len(unboundVolumeClaims) > 0 { + claim := unboundVolumeClaims[0] + volume := &corev1.PersistentVolume{ + ObjectMeta: metav1.ObjectMeta{ + Name: claim.Spec.VolumeName, + }, + } + + // ensure that the PV has a retain policy + if err := s.client.Patch(ctx, volume, kubernetes.ApplyPatch(corev1ac.PersistentVolume(volume.Name).WithSpec( + corev1ac.PersistentVolumeSpec().WithPersistentVolumeReclaimPolicy(corev1.PersistentVolumeReclaimRetain), + )), client.ForceOwnership, client.FieldOwner("owner")); err != nil { + log.Error(err, "error patching PersistentVolume spec") + return false, err + } + + // now that we've patched the PV, delete the PVC + if err := s.client.Delete(ctx, claim); err != nil { + log.Error(err, "error deleting PersistentVolumeClaim") + return false, err + } + + message := fmt.Sprintf( + "unbound persistent volume claims: [%s], decommissioning: %s", strings.Join(functional.MapFn(func(claim *corev1.PersistentVolumeClaim) string { + return client.ObjectKeyFromObject(claim).String() + }, unboundVolumeClaims), ", "), client.ObjectKeyFromObject(claim).String(), + ) + + log.V(traceLevel).Info(message) + s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonUnboundPersistentVolumeClaims, message) + + // at this point we should get a requeue anyway due to the ownership watch + // so just delegate to the runtime + return false, nil + } + + // now we check if we can/should decommission any brokers + adminClient, err := s.getAdminClient(ctx, set) + if err != nil { + log.Error(err, "initializing admin client") + return false, err + } + + health, err := adminClient.GetHealthOverview(ctx) + if err != nil { + log.Error(err, "fetching brokers") + return false, err + } + + requestedNodes := int(ptr.Deref(set.Spec.Replicas, 0)) + if len(health.AllNodes) <= requestedNodes { + // we don't need to decommission anything since we're at the proper + // capacity + return false, nil + } + + if len(health.NodesDown) == 0 { + // we don't need to decommission anything since everything is healthy + // and we want to wait until a broker is fully stopped + return false, nil + } + + allNodes := collections.NewSet[int]() + allNodes.Add(health.AllNodes...) + downedNodes := collections.NewSet[int]() + downedNodes.Add(health.NodesDown...) + healthyNodes := allNodes.LeftDisjoint(downedNodes) + + brokerOrdinalMap := map[int]collections.Set[int]{} + brokerMap := map[int]int{} + for _, brokerID := range health.AllNodes { + broker, err := adminClient.Broker(ctx, brokerID) + if err != nil { + log.Error(err, "fetching broker") + return false, err + } + + // NB: We capture the ordinal here because it gives us the + // ability to sort and have an extra check to ensure that we're only decommissioning + // downed brokers who also have ordinals that are higher than what the stateful set + // in its current configuration would actually produce (i.e. we + // don't want to accidentally decommission any random unhealthy brokers). + // + // Additionally, any potential ordinal collisions tell us that maybe the node + // id has changed but the original pod is actually gone (i.e. like a ghost broker) + // when we have an ordinal collision, then we check to make sure we have at least one + // healthy node for the given ordinal before decommissioning. + ordinal, err := ordinalFromFQDN(broker.InternalRPCAddress) + if err != nil { + // continue since we can't tell whether we can decommission this or not + // but make a lot of noise about the fact that we can't map this back to + // an ordinal + log.Error(err, "unexpected error parsing broker pod ordinal", "address", broker.InternalRPCAddress, "broker", broker) + continue + } + + if _, ok := brokerOrdinalMap[ordinal]; !ok { + brokerOrdinalMap[ordinal] = collections.NewSet[int]() + } + + // NB: here we have potentially multiple brokers that align to the same internal RPC address + // if that's the case, then one of them is going to be bad and can be decommissioned + brokerOrdinalMap[ordinal].Add(brokerID) + brokerMap[brokerID] = ordinal + } + + brokersToDecommission := []int{} + brokersToIgnore := []int{} + currentlyDecommissioningBrokers := []int{} + + for _, downedNode := range health.NodesDown { + ordinal, ok := brokerMap[downedNode] + if !ok { + // skip because we can't actually determine whether we should + // decommission it or not without its ordinal + brokersToIgnore = append(brokersToIgnore, downedNode) + } + + status, err := adminClient.DecommissionBrokerStatus(ctx, downedNode) + if err != nil { + if strings.Contains(err.Error(), "is not decommissioning") { + if ordinal >= requestedNodes { + // this broker is old and should be deleted + brokersToDecommission = append(brokersToDecommission, downedNode) + continue + } + + brokers := brokerOrdinalMap[ordinal] + if brokers.Size() == 1 { + // just ignore the node since it may be down, but it probably + // is just having problems + brokersToIgnore = append(brokersToIgnore, downedNode) + continue + } + + // here we have multiple ordinals that align to different nodes + // and we're within our set ordinal range, make sure at least one + // other node in the set is healthy and then we can mark this + // node for decommission, otherwise, we can't distinguish which + // pod is which broker (i.e. they're all down) and whether we + // should actually decommission it or not + hasHealthyBroker := true + for _, broker := range brokers.Values() { + if broker == downedNode { + continue + } + if healthyNodes.HasAny(broker) { + hasHealthyBroker = true + break + } + } + + if hasHealthyBroker { + // we have a healthy broker that isn't us, we can mark this for decommissioning + brokersToDecommission = append(brokersToDecommission, downedNode) + continue + } + + // we can't tell which broker mapped to an ordinal is the current broker that + // may actually correspond to a still-existing pod, so just ignore this broker + brokersToIgnore = append(brokersToIgnore, downedNode) + continue + } + + if strings.Contains(err.Error(), "does not exist") { + // delete the node from our sets + downedNodes.Delete(downedNode) + continue + } + + log.Error(err, "fetching decommission status") + return false, err + } + + if status.Finished { + // skip since we have already decommissioned it, so it should no longer + // show up in the health overview + continue + } + + // add the brokers to the list of what needs to be decommissioned + currentlyDecommissioningBrokers = append(currentlyDecommissioningBrokers, downedNode) + } + + sortBrokers := func(set []int) []int { + // sort by simple node id + sort.SliceStable(set, func(i, j int) bool { + return set[i] < set[j] + }) + return set + } + + healthyBrokers := sortBrokers(healthyNodes.Values()) + brokersToDecommission = sortBrokers(brokersToDecommission) + brokersToIgnore = sortBrokers(brokersToIgnore) + currentlyDecommissioningBrokers = sortBrokers(currentlyDecommissioningBrokers) + + formatBrokerList := func(set []int) string { + return strings.Join(functional.MapFn(strconv.Itoa, set), ", ") + } + + log.V(traceLevel).Info(fmt.Sprintf( + "healthy brokers: [%s], ignored: [%s], to decommission: [%s], decommissioning: [%s]", + formatBrokerList(healthyBrokers), + formatBrokerList(brokersToIgnore), + formatBrokerList(brokersToDecommission), + formatBrokerList(currentlyDecommissioningBrokers), + )) + + if len(currentlyDecommissioningBrokers) != 0 { + // we skip decommissioning our next broker since we already have some node decommissioning in progress + return true, nil + } + + if len(brokersToDecommission) > 0 { + // only record the event here since this is when we trigger a decommission + s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonBroker, "brokers needing decommissioning: [%s], decommissioning: %d", formatBrokerList(brokersToDecommission), brokersToDecommission[0]) + + if err := adminClient.DecommissionBroker(ctx, brokersToDecommission[0]); err != nil { + log.Error(err, "decommissioning broker", "broker", brokersToDecommission[0]) + return false, err + } + } + + // we should have decommissioned something above, so requeue and wait for it to finish + return true, nil +} + +// findUnboundVolumeClaims fetches any PVCs associated with the StatefulSet that aren't actively attached +// to a pod. +// +// Much of this code is copied from the original decommissioner and refactored, but the basic idea +// is: +// +// 1. Pull any pods matching the labels for the stateful set's pod template that are in the same namespace +// 2. Pull any pvcs matching the labels for the stateful set's volume claim template (though the component adds a "NAME-statefulset") +// 3. Find unbound volumes by checking that the pods we pulled reference every volume claim +// +// NB: this follow the original implementation that has a potential race-condition in the cache, where a PVC may come online and be in-cache +// but the corresponding pod has not yet populated into the cache. In this case the PVC could be marked for deletion +// despite the fact that it's still bound to a pod. In such a case the pvc-protection finalizer put in-place by core keeps the +// PVC from being deleted until the pod is deleted. Due to the skip of already-deleted PVCs below, these PVCs should +// just get GC'd when the pod is finally decommissioned. +func (s *StatefulSetDecomissioner) findUnboundVolumeClaims(ctx context.Context, set *appsv1.StatefulSet) ([]*corev1.PersistentVolumeClaim, error) { + pods := &corev1.PodList{} + if err := s.client.List(ctx, pods, client.InNamespace(set.Namespace), client.MatchingLabels(set.Spec.Template.Labels)); err != nil { + return nil, fmt.Errorf("listing pods: %w", err) + } + + dataVolumeLabels := client.MatchingLabels{} + for _, template := range set.Spec.VolumeClaimTemplates { + if template.Name == datadirVolume { + dataVolumeLabels = template.Labels + break + } + } + // the first part of this, "redpanda" is the component name (i.e. redpanda, console, etc.) + dataVolumeLabels[k8sComponentLabelKey] = "redpanda-statefulset" + + // find all pvcs of the data directory for this StatefulSet + pvcs := &corev1.PersistentVolumeClaimList{} + if err := s.client.List(ctx, pvcs, client.InNamespace(set.Namespace), dataVolumeLabels); err != nil { + return nil, fmt.Errorf("listing pvcs: %w", err) + } + + unboundVolumes := map[string]*corev1.PersistentVolumeClaim{} + for _, pvc := range pvcs.Items { + // skip any pvcs that are already deleting + if !pvc.DeletionTimestamp.IsZero() { + continue + } + unboundVolumes[pvc.Name] = pvc.DeepCopy() + } + + for _, pod := range pods.Items { + for _, volume := range pod.Spec.Volumes { + if volume.VolumeSource.PersistentVolumeClaim != nil { + delete(unboundVolumes, volume.VolumeSource.PersistentVolumeClaim.ClaimName) + } + } + } + + unbound := []*corev1.PersistentVolumeClaim{} + for _, claim := range unboundVolumes { + unbound = append(unbound, claim) + } + + sort.SliceStable(unbound, func(i, j int) bool { + first, second := unbound[i], unbound[j] + if first.CreationTimestamp.Before(&second.CreationTimestamp) { + return true + } + return first.Name < second.Name + }) + + return unbound, nil +} + +// getAdminClient initializes an admin API client for a cluster that a statefulset manages. It does this by +// delegating to a "fetcher" which fetches the equivalent values.yaml map from either a Redpanda CR or an +// installed helm release. It then effectively turns this into a Redpanda CR that can be used for initializing +// clients based on existing factory code. +func (s *StatefulSetDecomissioner) getAdminClient(ctx context.Context, set *appsv1.StatefulSet) (*rpadmin.AdminAPI, error) { + release, ok := set.Labels[k8sInstanceLabelKey] + if !ok { + return nil, errors.New("unable to get release name") + } + + values, err := s.fetcher.FetchLatest(ctx, release, set.Namespace) + if err != nil { + return nil, fmt.Errorf("fetching latest values: %w", err) + } + + data, err := json.MarshalIndent(values, "", " ") + if err != nil { + return nil, fmt.Errorf("marshaling values: %w", err) + } + + cluster := &redpandav1alpha2.Redpanda{ + ObjectMeta: metav1.ObjectMeta{ + Name: release, + Namespace: set.Namespace, + }, + Spec: redpandav1alpha2.RedpandaSpec{ClusterSpec: &redpandav1alpha2.RedpandaClusterSpec{}}, + } + + if err := json.Unmarshal(data, cluster.Spec.ClusterSpec); err != nil { + return nil, fmt.Errorf("unmarshaling values: %w", err) + } + + return s.factory.RedpandaAdminClient(ctx, cluster) +} + +// ordinalFromFQDN takes a hostname and attempt to map the +// name back to a stateful set pod ordinal based on the left +// most DNS segment containing the form SETNAME-ORDINAL. +func ordinalFromFQDN(fqdn string) (int, error) { + tokens := strings.Split(fqdn, ".") + if len(tokens) < 2 { + return 0, fmt.Errorf("invalid broker FQDN for ordinal fetching: %s", fqdn) + } + + brokerPod := tokens[0] + brokerTokens := strings.Split(brokerPod, "-") + if len(brokerTokens) < 2 { + return 0, fmt.Errorf("invalid broker FQDN for ordinal fetching: %s", fqdn) + } + + // grab the last item after the "-"" which should be the ordinal and parse it + ordinal, err := strconv.Atoi(brokerTokens[len(brokerTokens)-1]) + if err != nil { + return 0, fmt.Errorf("parsing broker FQDN %q: %w", fqdn, err) + } + + return ordinal, nil +} diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go new file mode 100644 index 000000000..72b508603 --- /dev/null +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -0,0 +1,331 @@ +// Copyright 2024 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 decommissioning_test + +import ( + "context" + _ "embed" + "encoding/json" + "testing" + "time" + + "github.com/go-logr/logr/testr" + "github.com/redpanda-data/common-go/rpadmin" + "github.com/redpanda-data/helm-charts/pkg/helm" + "github.com/redpanda-data/helm-charts/pkg/kube" + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" + "github.com/redpanda-data/redpanda-operator/operator/internal/testenv" + internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" + "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" + "github.com/stretchr/testify/suite" + corev1 "k8s.io/api/core/v1" + rbacv1 "k8s.io/api/rbac/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" + utilruntime "k8s.io/apimachinery/pkg/util/runtime" + "k8s.io/apimachinery/pkg/util/wait" + clientgoscheme "k8s.io/client-go/kubernetes/scheme" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" +) + +//go:embed role.yaml +var decommissionerRBAC []byte + +func TestStatefulSetDecommissioner(t *testing.T) { + if testing.Short() { + t.Skip("skipping long running test as -short was specified") + } + suite.Run(t, new(StatefulSetDecommissionerSuite)) +} + +type StatefulSetDecommissionerSuite struct { + suite.Suite + + ctx context.Context + env *testenv.Env + client client.Client + helm *helm.Client + clientFactory internalclient.ClientFactory +} + +var _ suite.SetupAllSuite = (*StatefulSetDecommissionerSuite)(nil) + +func (s *StatefulSetDecommissionerSuite) TestBasicDecommission() { + chart := s.installChart("basic", "", map[string]any{ + "statefulset": map[string]any{ + "replicas": 5, + }, + }) + + s.upgradeChart(chart, map[string]any{ + "statefulset": map[string]any{ + "replicas": 4, + }, + }) + + s.waitFor(func(ctx context.Context) (bool, error) { + var pvcs corev1.PersistentVolumeClaimList + if err := s.client.List(ctx, &pvcs, client.InNamespace(s.env.Namespace())); err != nil { + return false, err + } + // make sure we've deleted the PVC + return len(pvcs.Items) == 4, nil + }) + + adminClient := s.adminClientFor(chart) + + s.waitFor(func(ctx context.Context) (bool, error) { + health, err := adminClient.GetHealthOverview(ctx) + if err != nil { + return false, err + } + // make sure that we've removed all stale nodes + return len(health.NodesDown) == 0, nil + }) + + s.cleanupChart(chart) +} + +func (s *StatefulSetDecommissionerSuite) SetupSuite() { + t := s.T() + + scheme := runtime.NewScheme() + utilruntime.Must(clientgoscheme.AddToScheme(scheme)) + + log := testr.New(t).V(10) + if testing.Verbose() { + log = log.V(10) + } + + s.ctx = context.Background() + s.env = testenv.New(t, testenv.Options{ + // We need our own cluster for these tests since we need additional + // agents. Otherwise we can just turn up the default... but we'll + // need a different cluster to manipulate for node cleanup anyway. + Name: "decommissioning", + Agents: 5, + Scheme: scheme, + Logger: log, + }) + + s.client = s.env.Client() + + s.env.SetupManager(s.setupRBAC(), func(mgr ctrl.Manager) error { + helmClient, err := helm.New(helm.Options{ + KubeConfig: mgr.GetConfig(), + }) + if err != nil { + return err + } + if err := helmClient.RepoAdd(s.ctx, "redpandadata", "https://charts.redpanda.com"); err != nil { + return err + } + + s.helm = helmClient + dialer := kube.NewPodDialer(mgr.GetConfig()) + s.clientFactory = internalclient.NewFactory(mgr.GetConfig(), mgr.GetClient()).WithDialer(dialer.DialContext) + + decommissioner := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), decommissioning.WithFactory(s.clientFactory)) + if err := decommissioner.Setup(mgr); err != nil { + return err + } + + return nil + }) +} + +type chart struct { + name string + version string + release helm.Release + values map[string]any +} + +func (s *StatefulSetDecommissionerSuite) installChart(name, version string, overrides map[string]any) *chart { + values := map[string]any{ + "statefulset": map[string]any{ + "replicas": 1, + }, + "console": map[string]any{ + "enabled": false, + }, + "external": map[string]any{ + "enabled": false, + }, + "image": map[string]any{ + "repository": "redpandadata/redpanda-unstable", + "tag": "v24.3.1-rc8", + }, + } + + if overrides != nil { + values = functional.MergeMaps(values, overrides) + } + + release, err := s.helm.Install(s.ctx, "redpandadata/redpanda", helm.InstallOptions{ + Version: version, + CreateNamespace: true, + Name: name, + Namespace: s.env.Namespace(), + Values: values, + }) + s.Require().NoError(err) + + return &chart{ + name: name, + version: version, + values: values, + release: release, + } +} + +func (s *StatefulSetDecommissionerSuite) adminClientFor(chart *chart) *rpadmin.AdminAPI { + data, err := json.Marshal(chart.values) + s.Require().NoError(err) + + cluster := &redpandav1alpha2.Redpanda{ + ObjectMeta: metav1.ObjectMeta{ + Name: chart.name, + Namespace: s.env.Namespace(), + }, + Spec: redpandav1alpha2.RedpandaSpec{ClusterSpec: &redpandav1alpha2.RedpandaClusterSpec{}}, + } + + err = json.Unmarshal(data, cluster) + s.Require().NoError(err) + + adminClient, err := s.clientFactory.RedpandaAdminClient(s.ctx, cluster) + s.Require().NoError(err) + + return adminClient +} + +func (s *StatefulSetDecommissionerSuite) upgradeChart(chart *chart, overrides map[string]any) { + values := functional.MergeMaps(chart.values, overrides) + release, err := s.helm.Upgrade(s.ctx, chart.release.Name, "redpandadata/redpanda", helm.UpgradeOptions{ + Version: chart.version, + Namespace: s.env.Namespace(), + Values: values, + }) + s.Require().NoError(err) + + chart.release = release + chart.values = values +} + +func (s *StatefulSetDecommissionerSuite) cleanupChart(chart *chart) { + s.Require().NoError(s.helm.Uninstall(s.ctx, chart.release)) +} + +func (s *StatefulSetDecommissionerSuite) setupRBAC() string { + roles, err := kube.DecodeYAML(decommissionerRBAC, s.client.Scheme()) + s.Require().NoError(err) + + role := roles[1].(*rbacv1.Role) + clusterRole := roles[0].(*rbacv1.ClusterRole) + + // Inject additional permissions required for running in testenv. + role.Rules = append(role.Rules, rbacv1.PolicyRule{ + APIGroups: []string{""}, + Resources: []string{"pods/portforward"}, + Verbs: []string{"*"}, + }) + + name := "testenv-" + testenv.RandString(6) + + role.Name = name + role.Namespace = s.env.Namespace() + clusterRole.Name = name + clusterRole.Namespace = s.env.Namespace() + + s.applyAndWait(roles...) + s.applyAndWait( + &corev1.ServiceAccount{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + }, + &rbacv1.RoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + Subjects: []rbacv1.Subject{ + {Kind: "ServiceAccount", Namespace: s.env.Namespace(), Name: name}, + }, + RoleRef: rbacv1.RoleRef{ + APIGroup: "rbac.authorization.k8s.io", + Kind: "Role", + Name: role.Name, + }, + }, + &rbacv1.ClusterRoleBinding{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + }, + Subjects: []rbacv1.Subject{ + {Kind: "ServiceAccount", Namespace: s.env.Namespace(), Name: name}, + }, + RoleRef: rbacv1.RoleRef{ + APIGroup: "rbac.authorization.k8s.io", + Kind: "ClusterRole", + Name: clusterRole.Name, + }, + }, + ) + + return name +} + +func (s *StatefulSetDecommissionerSuite) applyAndWait(objs ...client.Object) { + s.applyAndWaitFor(func(obj client.Object) bool { + switch obj := obj.(type) { + case *corev1.Secret, *corev1.ConfigMap, *corev1.ServiceAccount, + *rbacv1.ClusterRole, *rbacv1.Role, *rbacv1.RoleBinding, *rbacv1.ClusterRoleBinding: + return true + + default: + s.T().Fatalf("unhandled object %T in applyAndWait", obj) + panic("unreachable") + } + }, objs...) +} + +func (s *StatefulSetDecommissionerSuite) applyAndWaitFor(cond func(client.Object) bool, objs ...client.Object) { + for _, obj := range objs { + gvk, err := s.client.GroupVersionKindFor(obj) + s.NoError(err) + + obj.SetManagedFields(nil) + obj.GetObjectKind().SetGroupVersionKind(gvk) + + s.Require().NoError(s.client.Patch(s.ctx, obj, client.Apply, client.ForceOwnership, client.FieldOwner("tests"))) + } + + for _, obj := range objs { + s.NoError(wait.PollUntilContextTimeout(s.ctx, 5*time.Second, 5*time.Minute, false, func(ctx context.Context) (done bool, err error) { + if err := s.client.Get(ctx, client.ObjectKeyFromObject(obj), obj); err != nil { + return false, err + } + + if cond(obj) { + return true, nil + } + + s.T().Logf("waiting for %T %q to be ready", obj, obj.GetName()) + return false, nil + })) + } +} + +func (s *StatefulSetDecommissionerSuite) waitFor(cond func(ctx context.Context) (bool, error)) { + s.NoError(wait.PollUntilContextTimeout(s.ctx, 5*time.Second, 5*time.Minute, false, cond)) +} diff --git a/operator/internal/decommissioning/values_fetcher.go b/operator/internal/decommissioning/values_fetcher.go new file mode 100644 index 000000000..f92da2d08 --- /dev/null +++ b/operator/internal/decommissioning/values_fetcher.go @@ -0,0 +1,116 @@ +// Copyright 2024 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 decommissioning + +import ( + "bytes" + "compress/gzip" + "context" + "encoding/base64" + "encoding/json" + "errors" + "io" + + corev1 "k8s.io/api/core/v1" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" +) + +// the format logic for helm releases can be found: +// https://github.com/helm/helm/blob/2cea1466d3c27491364eb44bafc7be1ca5461b2d/pkg/storage/driver/util.go#L58 + +var ( + gzipHeader = []byte{0x1f, 0x8b, 0x08} +) + +type ValuesFetcher interface { + FetchLatest(ctx context.Context, name, namespace string) (map[string]any, error) +} + +type HelmFetcher struct { + client client.Client +} + +func NewHelmFetcher(mgr ctrl.Manager) *HelmFetcher { + return &HelmFetcher{client: mgr.GetClient()} +} + +func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) (map[string]any, error) { + log := ctrl.LoggerFrom(ctx, "namespace", namespace, "name", name).WithName("HelmFetcher.FetchLatest") + + var secrets corev1.SecretList + + if err := f.client.List(ctx, &secrets, client.MatchingLabels{ + "name": name, + "owner": "helm", + }, client.InNamespace(namespace)); err != nil { + log.Error(err, "fetching secrets list") + return nil, err + } + + latestVersion := 0 + var latestValues map[string]any + for _, item := range secrets.Items { + values, version, err := f.decode(item.Data["release"]) + if err != nil { + log.Error(err, "decoding secret", "secret", item.Name) + continue + } + if version > latestVersion { + latestVersion = version + latestValues = values + } + } + + if latestValues != nil { + return latestValues, nil + } + + err := errors.New("unable to find latest value") + log.Error(err, "no secrets were decodable") + return nil, err +} + +type partialChart struct { + Config map[string]any `json:"config"` + Version int `json:"version"` +} + +func (f *HelmFetcher) decode(data []byte) (map[string]any, int, error) { + decoded := make([]byte, base64.StdEncoding.DecodedLen(len(data))) + n, err := base64.StdEncoding.Decode(decoded, data) + if err != nil { + return nil, 0, err + } + decoded = decoded[:n] + + if len(decoded) > 3 && bytes.Equal(decoded[0:3], gzipHeader) { + reader, err := gzip.NewReader(bytes.NewReader(decoded)) + if err != nil { + return nil, 0, err + } + defer reader.Close() + unzipped, err := io.ReadAll(reader) + if err != nil { + return nil, 0, err + } + decoded = unzipped + } + + var chart partialChart + if err := json.Unmarshal(decoded, &chart); err != nil { + return nil, 0, err + } + + // We only care about the chart.Config here and not the + // merged values with the chart values because our + // client initialization code already does the merging. + return chart.Config, chart.Version, nil +} diff --git a/operator/internal/testenv/testenv.go b/operator/internal/testenv/testenv.go index c2c3a977e..8874a3320 100644 --- a/operator/internal/testenv/testenv.go +++ b/operator/internal/testenv/testenv.go @@ -18,6 +18,7 @@ import ( "github.com/go-logr/logr" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "golang.org/x/exp/rand" "golang.org/x/sync/errgroup" corev1 "k8s.io/api/core/v1" apiextensionsv1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1" @@ -56,6 +57,8 @@ type Env struct { } type Options struct { + Name string + Agents int Scheme *runtime.Scheme CRDs []*apiextensionsv1.CustomResourceDefinition Logger logr.Logger @@ -71,6 +74,14 @@ type Options struct { // Due to the shared nature, the k3d cluster will NOT be shutdown at the end of // tests. func New(t *testing.T, options Options) *Env { + if options.Agents == 0 { + options.Agents = 3 + } + + if options.Name == "" { + options.Name = k3dClusterName + } + if options.Scheme == nil { options.Scheme = goclientscheme.Scheme } @@ -80,7 +91,7 @@ func New(t *testing.T, options Options) *Env { } // TODO maybe allow customizing name? - cluster, err := k3d.GetOrCreate(k3dClusterName) + cluster, err := k3d.GetOrCreate(options.Name, k3d.WithAgents(options.Agents)) require.NoError(t, err) if len(options.CRDs) > 0 { @@ -247,3 +258,15 @@ func (e *Env) shutdown() { }), "stalled waiting for Namespace %q to finish deleting", e.namespace.Name) } } + +func RandString(length int) string { + const alphabet = "abcdefghijklmnopqrstuvwxyz0123456789" + + name := "" + for i := 0; i < length; i++ { + //nolint:gosec // not meant to be a secure random string. + name += string(alphabet[rand.Intn(len(alphabet))]) + } + + return name +} diff --git a/operator/pkg/functional/map.go b/operator/pkg/functional/map.go index fc6c07204..febb67c3e 100644 --- a/operator/pkg/functional/map.go +++ b/operator/pkg/functional/map.go @@ -16,3 +16,69 @@ func MapFn[T any, U any](fn func(T) U, a []T) []U { } return s } + +func deepCopyMap(m map[string]any) map[string]any { + copied := make(map[string]any) + for k, v := range m { + switch cast := v.(type) { + case map[string]any: + copied[k] = deepCopyMap(cast) + case []any: + copied[k] = deepCopyElements(cast) + default: + copied[k] = v + } + } + return copied +} + +func deepCopy(v any) any { + switch cast := v.(type) { + case map[string]any: + return deepCopyMap(cast) + case []any: + return deepCopyElements(cast) + default: + return v + } +} + +func deepCopyElements(v []any) []any { + copied := make([]any, len(v)) + for _, value := range v { + switch cast := value.(type) { + case map[string]any: + copied = append(copied, deepCopyMap(cast)) + case []any: + copied = append(copied, deepCopyElements(cast)) + default: + return v + } + } + + return copied +} + +func MergeMaps(first, second map[string]any) map[string]any { + merged := deepCopyMap(first) + for k, v := range second { + if original, ok := merged[k]; ok { + switch cast := original.(type) { + case map[string]any: + // the types must match, otherwise we can't merge them + if vmap, ok := v.(map[string]any); ok { + merged[k] = MergeMaps(cast, deepCopyMap(vmap)) + } + case []any: + // the types must match, otherwise we can't merge them + if varray, ok := v.([]any); ok { + merged[k] = append(cast, deepCopyElements(varray)) + } + } + } + + merged[k] = deepCopy(v) + } + + return merged +} diff --git a/taskfiles/k8s.yml b/taskfiles/k8s.yml index a704047fb..1cc66e7ab 100644 --- a/taskfiles/k8s.yml +++ b/taskfiles/k8s.yml @@ -34,6 +34,14 @@ tasks: # controller-gen won't output to multiple directories and we need a copy # of resources for tests, so cp it is. - cp ./config/rbac/v2-manager-role/*.yaml ./internal/controller/redpanda/ + - | + controller-gen \ + paths='./internal/decommissioning/...' \ + rbac:roleName=decommissioner-role \ + output:rbac:artifacts:config=config/rbac/decommissioner-role + # controller-gen won't output to multiple directories and we need a copy + # of resources for tests, so cp it is. + - cp ./config/rbac/decommissioner-role/*.yaml ./internal/decommissioning/ - | controller-gen \ paths='./internal/controller/vectorized/...' \ From ca31f20c562bb353b227ca0f0ab69687c2d0a5dc Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Wed, 4 Dec 2024 16:48:49 -0500 Subject: [PATCH 02/13] run linter fix --- operator/cmd/sidecar/sidecar.go | 11 +++++------ .../decommissioning/statefulset_decomissioner.go | 11 ++++++----- .../statefulset_decommissioner_test.go | 15 ++++++++------- .../internal/decommissioning/values_fetcher.go | 4 +--- 4 files changed, 20 insertions(+), 21 deletions(-) diff --git a/operator/cmd/sidecar/sidecar.go b/operator/cmd/sidecar/sidecar.go index f768e9295..acc440e70 100644 --- a/operator/cmd/sidecar/sidecar.go +++ b/operator/cmd/sidecar/sidecar.go @@ -16,21 +16,20 @@ import ( "net/http/pprof" "time" - "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" "github.com/spf13/cobra" "k8s.io/apimachinery/pkg/runtime" utilruntime "k8s.io/apimachinery/pkg/util/runtime" clientgoscheme "k8s.io/client-go/kubernetes/scheme" ctrl "sigs.k8s.io/controller-runtime" metricsserver "sigs.k8s.io/controller-runtime/pkg/metrics/server" -) -var ( - schemes = []func(s *runtime.Scheme) error{ - clientgoscheme.AddToScheme, - } + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" ) +var schemes = []func(s *runtime.Scheme) error{ + clientgoscheme.AddToScheme, +} + func Command() *cobra.Command { var ( metricsAddr string diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 7464a9f1a..29a909a24 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -20,11 +20,6 @@ import ( "time" "github.com/redpanda-data/common-go/rpadmin" - redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" - internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" - "github.com/redpanda-data/redpanda-operator/operator/pkg/client/kubernetes" - "github.com/redpanda-data/redpanda-operator/operator/pkg/collections" - "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" appsv1 "k8s.io/api/apps/v1" corev1 "k8s.io/api/core/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" @@ -38,6 +33,12 @@ import ( "sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/handler" "sigs.k8s.io/controller-runtime/pkg/predicate" + + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" + internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" + "github.com/redpanda-data/redpanda-operator/operator/pkg/client/kubernetes" + "github.com/redpanda-data/redpanda-operator/operator/pkg/collections" + "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" ) const ( diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go index 72b508603..f95fc268b 100644 --- a/operator/internal/decommissioning/statefulset_decommissioner_test.go +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -18,13 +18,6 @@ import ( "github.com/go-logr/logr/testr" "github.com/redpanda-data/common-go/rpadmin" - "github.com/redpanda-data/helm-charts/pkg/helm" - "github.com/redpanda-data/helm-charts/pkg/kube" - redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" - "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" - "github.com/redpanda-data/redpanda-operator/operator/internal/testenv" - internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" - "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" "github.com/stretchr/testify/suite" corev1 "k8s.io/api/core/v1" rbacv1 "k8s.io/api/rbac/v1" @@ -35,6 +28,14 @@ import ( clientgoscheme "k8s.io/client-go/kubernetes/scheme" ctrl "sigs.k8s.io/controller-runtime" "sigs.k8s.io/controller-runtime/pkg/client" + + "github.com/redpanda-data/helm-charts/pkg/helm" + "github.com/redpanda-data/helm-charts/pkg/kube" + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" + "github.com/redpanda-data/redpanda-operator/operator/internal/testenv" + internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" + "github.com/redpanda-data/redpanda-operator/operator/pkg/functional" ) //go:embed role.yaml diff --git a/operator/internal/decommissioning/values_fetcher.go b/operator/internal/decommissioning/values_fetcher.go index f92da2d08..4da139ca3 100644 --- a/operator/internal/decommissioning/values_fetcher.go +++ b/operator/internal/decommissioning/values_fetcher.go @@ -26,9 +26,7 @@ import ( // the format logic for helm releases can be found: // https://github.com/helm/helm/blob/2cea1466d3c27491364eb44bafc7be1ca5461b2d/pkg/storage/driver/util.go#L58 -var ( - gzipHeader = []byte{0x1f, 0x8b, 0x08} -) +var gzipHeader = []byte{0x1f, 0x8b, 0x08} type ValuesFetcher interface { FetchLatest(ctx context.Context, name, namespace string) (map[string]any, error) From 5ab6828a571c0214497a49ec76d6f947189cc219 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Thu, 5 Dec 2024 15:49:06 -0500 Subject: [PATCH 03/13] Add ghost broker portion of test --- .../statefulset_decomissioner.go | 4 +- .../statefulset_decommissioner_test.go | 59 ++++++++++++++++++- operator/pkg/functional/filter.go | 20 +++++++ 3 files changed, 81 insertions(+), 2 deletions(-) create mode 100644 operator/pkg/functional/filter.go diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 29a909a24..14797274a 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -427,7 +427,7 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 // node for decommission, otherwise, we can't distinguish which // pod is which broker (i.e. they're all down) and whether we // should actually decommission it or not - hasHealthyBroker := true + hasHealthyBroker := false for _, broker := range brokers.Values() { if broker == downedNode { continue @@ -529,6 +529,8 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 // despite the fact that it's still bound to a pod. In such a case the pvc-protection finalizer put in-place by core keeps the // PVC from being deleted until the pod is deleted. Due to the skip of already-deleted PVCs below, these PVCs should // just get GC'd when the pod is finally decommissioned. +// +// TODO: will this cause issues if a PVC is deleted and before it's GC'd a Pod comes up? func (s *StatefulSetDecomissioner) findUnboundVolumeClaims(ctx context.Context, set *appsv1.StatefulSet) ([]*corev1.PersistentVolumeClaim, error) { pods := &corev1.PodList{} if err := s.client.List(ctx, pods, client.InNamespace(set.Namespace), client.MatchingLabels(set.Spec.Template.Labels)); err != nil { diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go index f95fc268b..66842b78a 100644 --- a/operator/internal/decommissioning/statefulset_decommissioner_test.go +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -20,9 +20,11 @@ import ( "github.com/redpanda-data/common-go/rpadmin" "github.com/stretchr/testify/suite" corev1 "k8s.io/api/core/v1" + policyv1 "k8s.io/api/policy/v1" rbacv1 "k8s.io/api/rbac/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/types" utilruntime "k8s.io/apimachinery/pkg/util/runtime" "k8s.io/apimachinery/pkg/util/wait" clientgoscheme "k8s.io/client-go/kubernetes/scheme" @@ -60,7 +62,7 @@ type StatefulSetDecommissionerSuite struct { var _ suite.SetupAllSuite = (*StatefulSetDecommissionerSuite)(nil) -func (s *StatefulSetDecommissionerSuite) TestBasicDecommission() { +func (s *StatefulSetDecommissionerSuite) TestDecommission() { chart := s.installChart("basic", "", map[string]any{ "statefulset": map[string]any{ "replicas": 5, @@ -93,9 +95,64 @@ func (s *StatefulSetDecommissionerSuite) TestBasicDecommission() { return len(health.NodesDown) == 0, nil }) + var firstBroker corev1.Pod + s.Require().NoError(s.client.Get(s.ctx, types.NamespacedName{Namespace: s.env.Namespace(), Name: chart.name + "-0"}, &firstBroker)) + var firstPVC corev1.PersistentVolumeClaim + s.Require().NoError(s.client.Get(s.ctx, types.NamespacedName{Namespace: s.env.Namespace(), Name: "datadir-" + chart.name + "-0"}, &firstPVC)) + + // now we simulate node failure by tainting a node with NoSchedule and evicting the pod + firstBrokerNode := firstBroker.Spec.NodeName + s.taintNode(firstBrokerNode) + s.T().Cleanup(func() { + s.untaintNode(firstBrokerNode) + }) + s.client.SubResource("eviction").Create(s.ctx, &firstBroker, &policyv1.Eviction{}) + + s.waitFor(func(ctx context.Context) (bool, error) { + health, err := adminClient.GetHealthOverview(ctx) + if err != nil { + return false, err + } + // make sure that the pod has been taken offline + return len(health.NodesDown) == 1, nil + }) + + // we have to manually delete both the broker and its PVC, which would normally + // be done by the PVC unbinder + s.Require().NoError(s.client.Delete(s.ctx, &firstPVC)) + s.Require().NoError(s.client.Delete(s.ctx, &firstBroker)) + + s.waitFor(func(ctx context.Context) (bool, error) { + health, err := adminClient.GetHealthOverview(ctx) + if err != nil { + return false, err + } + // now make sure it comes back online and the broker is decommissioned + return len(health.NodesDown) == 0, nil + }) + s.cleanupChart(chart) } +func (s *StatefulSetDecommissionerSuite) taintNode(name string) { + var node corev1.Node + s.Require().NoError(s.client.Get(s.ctx, types.NamespacedName{Name: name}, &node)) + node.Spec.Taints = append(node.Spec.Taints, corev1.Taint{ + Key: "decommission-test", + Effect: corev1.TaintEffectNoSchedule, + }) + s.Require().NoError(s.client.Update(s.ctx, &node)) +} + +func (s *StatefulSetDecommissionerSuite) untaintNode(name string) { + var node corev1.Node + s.Require().NoError(s.client.Get(s.ctx, types.NamespacedName{Name: name}, &node)) + node.Spec.Taints = functional.Filter(node.Spec.Taints, func(taint corev1.Taint) bool { + return taint.Key != "decommission-test" + }) + s.Require().NoError(s.client.Update(s.ctx, &node)) +} + func (s *StatefulSetDecommissionerSuite) SetupSuite() { t := s.T() diff --git a/operator/pkg/functional/filter.go b/operator/pkg/functional/filter.go new file mode 100644 index 000000000..0ec94ce7d --- /dev/null +++ b/operator/pkg/functional/filter.go @@ -0,0 +1,20 @@ +// Copyright 2024 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 functional + +func Filter[T any](array []T, filter func(T) bool) []T { + var filtered []T + for _, value := range array { + if filter(value) { + filtered = append(filtered, value) + } + } + return filtered +} From 835735eb977a4243bee87737d8739023ce61217d Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Thu, 5 Dec 2024 16:20:35 -0500 Subject: [PATCH 04/13] check error in tests --- .../internal/decommissioning/statefulset_decommissioner_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go index 66842b78a..71dec6ee8 100644 --- a/operator/internal/decommissioning/statefulset_decommissioner_test.go +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -106,7 +106,7 @@ func (s *StatefulSetDecommissionerSuite) TestDecommission() { s.T().Cleanup(func() { s.untaintNode(firstBrokerNode) }) - s.client.SubResource("eviction").Create(s.ctx, &firstBroker, &policyv1.Eviction{}) + s.Require().NoError(s.client.SubResource("eviction").Create(s.ctx, &firstBroker, &policyv1.Eviction{})) s.waitFor(func(ctx context.Context) (bool, error) { health, err := adminClient.GetHealthOverview(ctx) From 1f6658d291720f94f5218989ae6764a052cf8a98 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Thu, 5 Dec 2024 18:04:58 -0500 Subject: [PATCH 05/13] Add delayed cache --- operator/cmd/sidecar/sidecar.go | 63 ++--- .../internal/decommissioning/delayed_cache.go | 151 +++++++++++ .../decommissioning/delayed_cache_test.go | 75 ++++++ .../statefulset_decomissioner.go | 252 +++++++++++++----- .../statefulset_decommissioner_test.go | 17 +- operator/internal/testenv/testenv.go | 1 - 6 files changed, 450 insertions(+), 109 deletions(-) create mode 100644 operator/internal/decommissioning/delayed_cache.go create mode 100644 operator/internal/decommissioning/delayed_cache_test.go diff --git a/operator/cmd/sidecar/sidecar.go b/operator/cmd/sidecar/sidecar.go index acc440e70..0927e81c9 100644 --- a/operator/cmd/sidecar/sidecar.go +++ b/operator/cmd/sidecar/sidecar.go @@ -12,8 +12,6 @@ package sidecar import ( "context" "errors" - "net/http" - "net/http/pprof" "time" "github.com/spf13/cobra" @@ -32,12 +30,14 @@ var schemes = []func(s *runtime.Scheme) error{ func Command() *cobra.Command { var ( - metricsAddr string - probeAddr string - pprofAddr string - clusterNamespace string - clusterName string - decommissionTimeout time.Duration + metricsAddr string + probeAddr string + pprofAddr string + clusterNamespace string + clusterName string + decommissionRequeueTimeout time.Duration + decommissionVoteInterval time.Duration + decommissionMaxVoteCount int ) cmd := &cobra.Command{ @@ -46,16 +46,16 @@ func Command() *cobra.Command { RunE: func(cmd *cobra.Command, args []string) error { ctx := cmd.Context() - // Always run a pprof server to facilitate debugging. - go runPProfServer(ctx, pprofAddr) - return Run( ctx, metricsAddr, probeAddr, + pprofAddr, clusterNamespace, clusterName, - decommissionTimeout, + decommissionRequeueTimeout, + decommissionVoteInterval, + decommissionMaxVoteCount, ) }, } @@ -65,7 +65,9 @@ func Command() *cobra.Command { cmd.Flags().StringVar(&pprofAddr, "pprof-bind-address", ":8082", "The address the metric endpoint binds to.") cmd.Flags().StringVar(&clusterNamespace, "cluster-namespace", "", "The namespace of the cluster that this sidecar manages.") cmd.Flags().StringVar(&clusterName, "cluster-name", "", "The name of the cluster that this sidecar manages.") - cmd.Flags().DurationVar(&decommissionTimeout, "decommission-timeout", 10*time.Second, "The time period to wait before recheck a broker that is being decommissioned.") + cmd.Flags().DurationVar(&decommissionRequeueTimeout, "decommission-requeue-timeout", 10*time.Second, "The time period to wait before rechecking a broker that is being decommissioned.") + cmd.Flags().DurationVar(&decommissionVoteInterval, "decommission-vote-interval", 30*time.Second, "The time period between incrementing decommission vote counts since the last decommission conditions were met.") + cmd.Flags().IntVar(&decommissionMaxVoteCount, "decommission-vote-count", 2, "The number of times that a vote must be tallied when a resource meets decommission conditions for it to actually be decommissioned.") return cmd } @@ -74,9 +76,12 @@ func Run( ctx context.Context, metricsAddr string, probeAddr string, + pprofAddr string, clusterNamespace string, clusterName string, - decommissionTimeout time.Duration, + decommissionRequeueTimeout time.Duration, + decommissionVoteInterval time.Duration, + decommissionMaxVoteCount int, ) error { setupLog := ctrl.LoggerFrom(ctx).WithName("setup") @@ -101,6 +106,7 @@ func Run( mgr, err := ctrl.NewManager(ctrl.GetConfigOrDie(), ctrl.Options{ Metrics: metricsserver.Options{BindAddress: metricsAddr}, HealthProbeBindAddress: probeAddr, + PprofBindAddress: pprofAddr, LeaderElection: true, LeaderElectionID: clusterName + "." + clusterNamespace + ".redpanda", Scheme: scheme, @@ -113,9 +119,11 @@ func Run( if err := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), []decommissioning.Option{ decommissioning.WithFilter(decommissioning.FilterStatefulSetOwner(clusterNamespace, clusterName)), - decommissioning.WithRequeueTimeout(decommissionTimeout), - }...).Setup(mgr); err != nil { - setupLog.Error(err, "unable to create controller", "controller", "DecommissionReconciler") + decommissioning.WithRequeueTimeout(decommissionRequeueTimeout), + decommissioning.WithDelayedCacheInterval(decommissionVoteInterval), + decommissioning.WithDelayedCacheMaxCount(decommissionMaxVoteCount), + }...).SetupWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create controller", "controller", "StatefulSetDecommissioner") return err } @@ -126,24 +134,3 @@ func Run( return nil } - -func runPProfServer(ctx context.Context, listenAddr string) { - logger := ctrl.LoggerFrom(ctx) - - pprofMux := http.NewServeMux() - pprofMux.HandleFunc("/debug/pprof/", pprof.Index) - pprofMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - pprofMux.HandleFunc("/debug/pprof/profile", pprof.Profile) - pprofMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - pprofMux.HandleFunc("/debug/pprof/trace", pprof.Trace) - pprofServer := &http.Server{ - Addr: listenAddr, - Handler: pprofMux, - ReadHeaderTimeout: 3 * time.Second, - } - - logger.Info("starting pprof server...", "addr", listenAddr) - if err := pprofServer.ListenAndServe(); err != nil { - logger.Error(err, "failed to run pprof server") - } -} diff --git a/operator/internal/decommissioning/delayed_cache.go b/operator/internal/decommissioning/delayed_cache.go new file mode 100644 index 000000000..4653bd367 --- /dev/null +++ b/operator/internal/decommissioning/delayed_cache.go @@ -0,0 +1,151 @@ +// Copyright 2024 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 decommissioning + +import ( + "iter" + "slices" + "sync" + "time" +) + +type cacheInfo struct { + lastMarked time.Time + count int +} + +// CategorizedDelayedCache keeps track of any item that meets some threshold +// cound over a period of time and then processes an item when it meets that +// threshold. +type CategorizedDelayedCache[Category, Entry comparable] struct { + data map[Category]map[Entry]*cacheInfo + mutex sync.RWMutex + maxCount int + countInterval time.Duration +} + +// NewCategorizedDelayedCache creates a delayed cache with an additional level of organization by category. +func NewCategorizedDelayedCache[Category, Entry comparable](maxCount int, interval time.Duration) *CategorizedDelayedCache[Category, Entry] { + return &CategorizedDelayedCache[Category, Entry]{ + data: make(map[Category]map[Entry]*cacheInfo), + maxCount: maxCount, + countInterval: interval, + } +} + +// Clean removes all of the entries for the given category. +func (d *CategorizedDelayedCache[Category, Entry]) Clean(category Category) { + d.mutex.Lock() + defer d.mutex.Unlock() + + delete(d.data, category) +} + +// Size returns the number of entries in the cache for the given category. +func (d *CategorizedDelayedCache[Category, Entry]) Size(category Category) int { + d.mutex.RLock() + defer d.mutex.RUnlock() + + if entries, ok := d.data[category]; ok { + return len(entries) + } + + return 0 +} + +// Entries returns an iterator over the entries of a category. +func (d *CategorizedDelayedCache[Category, Entry]) Entries(category Category) iter.Seq[Entry] { + return func(yield func(Entry) bool) { + d.mutex.RLock() + defer d.mutex.RUnlock() + + if entries, ok := d.data[category]; ok { + for entry := range entries { + if !yield(entry) { + return + } + } + } + } +} + +// Filter keeps only the entries in the cache which are equal to the given entries. +func (d *CategorizedDelayedCache[Category, Entry]) Filter(category Category, entries ...Entry) { + d.mutex.Lock() + defer d.mutex.Unlock() + + if stored, ok := d.data[category]; ok { + for entry := range stored { + if !slices.Contains(entries, entry) { + delete(stored, entry) + } + } + } +} + +// Prune deletes any entries in the cache matching the given entries, it is the opposite of Filter. +func (d *CategorizedDelayedCache[Category, Entry]) Prune(category Category, entries ...Entry) { + d.mutex.Lock() + defer d.mutex.Unlock() + + if stored, ok := d.data[category]; ok { + for _, entry := range entries { + delete(stored, entry) + } + } +} + +// Mark marks the entry in the cache, incrementing its count if it hasn't been marked in +// over the configurable interval. +func (d *CategorizedDelayedCache[Category, Entry]) Mark(category Category, entry Entry) { + d.mutex.Lock() + defer d.mutex.Unlock() + + if _, ok := d.data[category]; !ok { + d.data[category] = make(map[Entry]*cacheInfo) + } + entries := d.data[category] + + if info, ok := entries[entry]; ok { + if time.Since(info.lastMarked) > d.countInterval { + info.count++ + info.lastMarked = time.Now() + } + return + } + + entries[entry] = &cacheInfo{ + lastMarked: time.Now(), + count: 1, + } + + return +} + +// Process processes and removes a cache item, calling the callback if the minimum threshold count is met. +func (d *CategorizedDelayedCache[Category, Entry]) Process(category Category, entry Entry, onThresholdMet func() error) (bool, error) { + d.mutex.Lock() + defer d.mutex.Unlock() + + processed := false + if entries, ok := d.data[category]; ok { + if info, ok := entries[entry]; ok { + if d.maxCount <= info.count { + processed = true + if err := onThresholdMet(); err != nil { + return processed, err + } + delete(entries, entry) + } + } + } + + return processed, nil +} diff --git a/operator/internal/decommissioning/delayed_cache_test.go b/operator/internal/decommissioning/delayed_cache_test.go new file mode 100644 index 000000000..19716eb61 --- /dev/null +++ b/operator/internal/decommissioning/delayed_cache_test.go @@ -0,0 +1,75 @@ +// Copyright 2024 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 decommissioning_test + +import ( + "errors" + "slices" + "testing" + "time" + + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" + "github.com/stretchr/testify/require" +) + +func TestCategorizedDelayedCache(t *testing.T) { + // we need to be seen 10 times, with at least 1 millisecond between each viewing + cache := decommissioning.NewCategorizedDelayedCache[int, int](10, time.Millisecond) + + category := 0 + for second := 0; second < 9; second++ { + for entry := 0; entry < 10; entry++ { + cache.Mark(category, entry) + // call Process to make sure nothing is actually getting processed + processed, err := cache.Process(category, entry, func() error { + return errors.New("this shouldn't be called") + }) + require.NoError(t, err) + require.False(t, processed) + } + require.Equal(t, cache.Size(category), 10) + // sleep 2 milliseconds so that we trip the cache interval + time.Sleep(2 * time.Millisecond) + } + + time.Sleep(2 * time.Millisecond) + for entry := 0; entry < 10; entry++ { + even := entry%2 == 0 + if even { + cache.Mark(category, entry) + } + + processed, err := cache.Process(category, entry, func() error { + return errors.New("this should error") + }) + if even { + require.True(t, processed) + require.Error(t, err) + } else { + require.False(t, processed) + require.NoError(t, err) + } + + // doing this should eject the entry from the cache + processed, err = cache.Process(category, entry, func() error { + return nil + }) + require.NoError(t, err) + require.Equal(t, even, processed) + } + + require.ElementsMatch(t, []int{1, 3, 5, 7, 9}, slices.Collect(cache.Entries(category))) + cache.Filter(category, 1, 3, 5) + require.ElementsMatch(t, []int{1, 3, 5}, slices.Collect(cache.Entries(category))) + cache.Prune(category, 3, 5) + require.ElementsMatch(t, []int{1}, slices.Collect(cache.Entries(category))) + cache.Clean(category) + require.Equal(t, 0, cache.Size(category)) +} diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 14797274a..eca5cecf7 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -25,6 +25,7 @@ import ( apierrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/types" + "k8s.io/apimachinery/pkg/util/wait" corev1ac "k8s.io/client-go/applyconfigurations/core/v1" "k8s.io/client-go/tools/record" "k8s.io/utils/ptr" @@ -56,6 +57,11 @@ const ( infoLevel = 0 defaultRequeueTimeout = 10 * time.Second + // these defaults give us roughly 1 minute before + // we decommission a failing broker that appears to + // need decommissioning + defaultDelayedCacheInterval = 30 * time.Second + defaultDelayedMaxCacheCount = 2 ) type Option func(*StatefulSetDecomissioner) @@ -95,31 +101,52 @@ func WithRequeueTimeout(timeout time.Duration) Option { } } +func WithDelayedCacheInterval(interval time.Duration) Option { + return func(decommissioner *StatefulSetDecomissioner) { + decommissioner.delayedCacheInterval = interval + } +} + +func WithDelayedCacheMaxCount(count int) Option { + return func(decommissioner *StatefulSetDecomissioner) { + decommissioner.delayedCacheMaxCount = count + } +} + type StatefulSetDecomissioner struct { - client client.Client - factory internalclient.ClientFactory - fetcher ValuesFetcher - recorder record.EventRecorder - requeueTimeout time.Duration - filter func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) + client client.Client + factory internalclient.ClientFactory + fetcher ValuesFetcher + recorder record.EventRecorder + requeueTimeout time.Duration + delayedCacheInterval time.Duration + delayedCacheMaxCount int + delayedBrokerIDCache *CategorizedDelayedCache[types.NamespacedName, int] + delayedVolumeCache *CategorizedDelayedCache[types.NamespacedName, types.NamespacedName] + filter func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) } func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher ValuesFetcher, options ...Option) *StatefulSetDecomissioner { k8sClient := mgr.GetClient() decommissioner := &StatefulSetDecomissioner{ - recorder: mgr.GetEventRecorderFor("broker-decommissioner"), - client: k8sClient, - fetcher: fetcher, - factory: internalclient.NewFactory(mgr.GetConfig(), k8sClient), - requeueTimeout: defaultRequeueTimeout, - filter: func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { return true, nil }, + recorder: mgr.GetEventRecorderFor("broker-decommissioner"), + client: k8sClient, + fetcher: fetcher, + factory: internalclient.NewFactory(mgr.GetConfig(), k8sClient), + requeueTimeout: defaultRequeueTimeout, + delayedCacheInterval: defaultDelayedCacheInterval, + delayedCacheMaxCount: defaultDelayedMaxCacheCount, + filter: func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { return true, nil }, } for _, opt := range options { opt(decommissioner) } + decommissioner.delayedBrokerIDCache = NewCategorizedDelayedCache[types.NamespacedName, int](decommissioner.delayedCacheMaxCount, decommissioner.delayedCacheInterval) + decommissioner.delayedVolumeCache = NewCategorizedDelayedCache[types.NamespacedName, types.NamespacedName](decommissioner.delayedCacheMaxCount, decommissioner.delayedCacheInterval) + return decommissioner } @@ -137,7 +164,7 @@ func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher ValuesFetcher, optio // +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch // +kubebuilder:rbac:groups=core,namespace=default,resources=secrets,verbs=get;list;watch -func (s *StatefulSetDecomissioner) Setup(mgr ctrl.Manager) error { +func (s *StatefulSetDecomissioner) SetupWithManager(mgr ctrl.Manager) error { pvcPredicate, err := predicate.LabelSelectorPredicate( metav1.LabelSelector{ MatchExpressions: []metav1.LabelSelectorRequirement{{ @@ -201,6 +228,10 @@ func (s *StatefulSetDecomissioner) Reconcile(ctx context.Context, req ctrl.Reque set := &appsv1.StatefulSet{} if err := s.client.Get(ctx, req.NamespacedName, set); err != nil { if apierrors.IsNotFound(err) { + // clean up the caches if this is finally deleted + s.delayedBrokerIDCache.Clean(req.NamespacedName) + s.delayedVolumeCache.Clean(req.NamespacedName) + return ctrl.Result{}, nil } log.Error(err, "fetching StatefulSet") @@ -220,12 +251,15 @@ func (s *StatefulSetDecomissioner) Reconcile(ctx context.Context, req ctrl.Reque requeue, err := s.Decommission(ctx, set) if err != nil { - // we already logged any error, just requeue directly + // we already logged any error, just requeue directly, delegating to the + // exponential backoff behavior return ctrl.Result{Requeue: true}, nil } if requeue { - return ctrl.Result{RequeueAfter: s.requeueTimeout}, nil + // wait up to a 10% additional jitter factor, but requeue again + timeout := wait.Jitter(s.requeueTimeout, 0.10) + return ctrl.Result{RequeueAfter: timeout}, nil } return ctrl.Result{}, nil @@ -253,6 +287,24 @@ func (s *StatefulSetDecomissioner) Reconcile(ctx context.Context, req ctrl.Reque // // For PVC deletion and broker decommissioning, each step happens sequentially such that no two brokers should // attempt to be decommissioned simultaneously. Likewise each PVC is deleted one by one. +// +// NB: most of these operations are highly sensitive to the behavior of the client-side Kubernetes cache, the +// population of which can introduce race conditions for prematurely considering either a PVC unbound or a broker +// ready to decommission due to a stale read on either the StatefulSet or a partial list of Pods. Consider: +// +// - A StatefulSet scales up and the pod comes up and registers with the cluster but has not yet become fully healthy +// - Reconciliation is triggered due to the pod coming online but the StatefulSet has been slow to propagate through +// cache +// - The requested replicas for the StatefulSet are out-of-date and it seems like we have one extra broker than we need +// and that broker is currently unhealthy in our cluster health check +// - We mark it as needing to be decommissioned even though it's new +// +// These types of stale cache scenarios are effectively eliminated by introducing a "delayed" cache that essentially +// counts the number of times that we've marked a broker as ready to decommission across a certain window. If it's been +// marked as needing to be decommissioned some m times with delays of n in between each mark, then it can be decommissioned +// because we've given the client-side cache enough time to fill. We also add this guard for PVCs. In every loop through +// reconciliation we expunge the broker and/or PVC entries that no longer meet the decommission criteria, so if a broker +// or PVC should no longer be decommissioned, we reset our count. func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1.StatefulSet) (bool, error) { // note that this is best-effort, the decommissioning code needs to be idempotent and deterministic @@ -285,41 +337,64 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 return claim.Name }, unboundVolumeClaims)) - // we first clean up any unbound PVCs, ensuring that their PVs have a retain policy - if len(unboundVolumeClaims) > 0 { - claim := unboundVolumeClaims[0] - volume := &corev1.PersistentVolume{ - ObjectMeta: metav1.ObjectMeta{ - Name: claim.Spec.VolumeName, - }, - } + setCacheKey := client.ObjectKeyFromObject(set) - // ensure that the PV has a retain policy - if err := s.client.Patch(ctx, volume, kubernetes.ApplyPatch(corev1ac.PersistentVolume(volume.Name).WithSpec( - corev1ac.PersistentVolumeSpec().WithPersistentVolumeReclaimPolicy(corev1.PersistentVolumeReclaimRetain), - )), client.ForceOwnership, client.FieldOwner("owner")); err != nil { - log.Error(err, "error patching PersistentVolume spec") - return false, err - } + // remove any volumes from the cache that are no longer considered unbound + s.delayedVolumeCache.Filter(setCacheKey, functional.MapFn(func(claim *corev1.PersistentVolumeClaim) types.NamespacedName { + return client.ObjectKeyFromObject(claim) + }, unboundVolumeClaims)...) - // now that we've patched the PV, delete the PVC - if err := s.client.Delete(ctx, claim); err != nil { - log.Error(err, "error deleting PersistentVolumeClaim") - return false, err - } + // first mark all of the claims as needing potential expiration + for _, claim := range unboundVolumeClaims { + s.delayedVolumeCache.Mark(setCacheKey, client.ObjectKeyFromObject(claim)) + } - message := fmt.Sprintf( - "unbound persistent volume claims: [%s], decommissioning: %s", strings.Join(functional.MapFn(func(claim *corev1.PersistentVolumeClaim) string { - return client.ObjectKeyFromObject(claim).String() - }, unboundVolumeClaims), ", "), client.ObjectKeyFromObject(claim).String(), - ) + // now we attempt to clean up the first of the PVCs that meets the treshold of the cache, + // ensuring that their PVs have a retain policy, and short-circuiting the rest of reconciliation + // if we actually delete a claim + for _, claim := range unboundVolumeClaims { + deleted, err := s.delayedVolumeCache.Process(setCacheKey, client.ObjectKeyFromObject(claim), func() error { + // first mark all of the claims as needing potential expiration + volume := &corev1.PersistentVolume{ + ObjectMeta: metav1.ObjectMeta{ + Name: claim.Spec.VolumeName, + }, + } - log.V(traceLevel).Info(message) - s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonUnboundPersistentVolumeClaims, message) + // ensure that the PV has a retain policy + if err := s.client.Patch(ctx, volume, kubernetes.ApplyPatch(corev1ac.PersistentVolume(volume.Name).WithSpec( + corev1ac.PersistentVolumeSpec().WithPersistentVolumeReclaimPolicy(corev1.PersistentVolumeReclaimRetain), + )), client.ForceOwnership, client.FieldOwner("owner")); err != nil { + log.Error(err, "error patching PersistentVolume spec") + return err + } - // at this point we should get a requeue anyway due to the ownership watch - // so just delegate to the runtime - return false, nil + // now that we've patched the PV, delete the PVC + if err := s.client.Delete(ctx, claim); err != nil { + log.Error(err, "error deleting PersistentVolumeClaim") + return err + } + + message := fmt.Sprintf( + "unbound persistent volume claims: [%s], decommissioning: %s", strings.Join(functional.MapFn(func(claim *corev1.PersistentVolumeClaim) string { + return client.ObjectKeyFromObject(claim).String() + }, unboundVolumeClaims), ", "), client.ObjectKeyFromObject(claim).String(), + ) + + log.V(traceLevel).Info(message) + s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonUnboundPersistentVolumeClaims, message) + + return nil + }) + if err != nil { + return false, err + } + + // if anything was actually deleted, just return immediately and we'll pick things up on + // the next pass + if deleted { + return true, nil + } } // now we check if we can/should decommission any brokers @@ -335,16 +410,29 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 return false, err } + // NB: we don't have to take into account cache counts yet at this point + // since we're not actually deleting anything if these conditions hold + // the counting happens below as a guard for *when we actually do clean + // something up* + requestedNodes := int(ptr.Deref(set.Spec.Replicas, 0)) if len(health.AllNodes) <= requestedNodes { // we don't need to decommission anything since we're at the proper - // capacity + // capacity, we also clear the cache here because nothing should + // be considered for decommissioning until we actually have more nodes + // than are desired + + s.delayedVolumeCache.Clean(setCacheKey) return false, nil } if len(health.NodesDown) == 0 { // we don't need to decommission anything since everything is healthy - // and we want to wait until a broker is fully stopped + // and we want to wait until a broker is fully stopped, we also clear the cache + // here because everything is healthy and we don't want to accidentally pick up something + // later and have old cache entries count against it + + s.delayedVolumeCache.Clean(setCacheKey) return false, nil } @@ -478,6 +566,16 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 return set } + // NB: here is where we need to start to check the cache for any + // nodes that meet the threshold, otherwise they should just be considered + // ignored as well + + // mark any brokers that currently meet our criteria for decommissioning + s.delayedBrokerIDCache.Filter(setCacheKey, brokersToDecommission...) + for _, broker := range brokersToDecommission { + s.delayedBrokerIDCache.Mark(setCacheKey, broker) + } + healthyBrokers := sortBrokers(healthyNodes.Values()) brokersToDecommission = sortBrokers(brokersToDecommission) brokersToIgnore = sortBrokers(brokersToIgnore) @@ -495,23 +593,41 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 formatBrokerList(currentlyDecommissioningBrokers), )) + // we marked the brokers to decommission above, but don't attempt a decommission yet + // because something is already in progress + if len(currentlyDecommissioningBrokers) != 0 { // we skip decommissioning our next broker since we already have some node decommissioning in progress return true, nil } - if len(brokersToDecommission) > 0 { - // only record the event here since this is when we trigger a decommission - s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonBroker, "brokers needing decommissioning: [%s], decommissioning: %d", formatBrokerList(brokersToDecommission), brokersToDecommission[0]) + // now attempt to decommission something, if we have actually done something, then requeue + for _, broker := range brokersToDecommission { + decommissioned, err := s.delayedBrokerIDCache.Process(setCacheKey, broker, func() error { + // only record the event here since this is when we trigger a decommission + s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonBroker, "brokers needing decommissioning: [%s], decommissioning: %d", formatBrokerList(brokersToDecommission), brokersToDecommission[0]) + + if err := adminClient.DecommissionBroker(ctx, brokersToDecommission[0]); err != nil { + log.Error(err, "decommissioning broker", "broker", brokersToDecommission[0]) + return err + } - if err := adminClient.DecommissionBroker(ctx, brokersToDecommission[0]); err != nil { - log.Error(err, "decommissioning broker", "broker", brokersToDecommission[0]) + return nil + }) + if err != nil { return false, err } + + if decommissioned { + // we decommissioned something, return immediately and wait until the process is fully complete + // before attempting our next decommission + return true, nil + } } - // we should have decommissioned something above, so requeue and wait for it to finish - return true, nil + // we may not have decommissioned anything above, but go ahead and requeue if we have anything + // in either of our caches since we have something that might need decommissioning soon + return s.delayedBrokerIDCache.Size(setCacheKey) > 0 || s.delayedVolumeCache.Size(setCacheKey) > 0, nil } // findUnboundVolumeClaims fetches any PVCs associated with the StatefulSet that aren't actively attached @@ -524,13 +640,14 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 // 2. Pull any pvcs matching the labels for the stateful set's volume claim template (though the component adds a "NAME-statefulset") // 3. Find unbound volumes by checking that the pods we pulled reference every volume claim // -// NB: this follow the original implementation that has a potential race-condition in the cache, where a PVC may come online and be in-cache -// but the corresponding pod has not yet populated into the cache. In this case the PVC could be marked for deletion +// NB: this bit follows the original implementation that has a potential race-condition in the cache, where a PVC may come online +// and be in-cache but the corresponding pod has not yet populated into the cache. In this case the PVC could be marked for deletion // despite the fact that it's still bound to a pod. In such a case the pvc-protection finalizer put in-place by core keeps the // PVC from being deleted until the pod is deleted. Due to the skip of already-deleted PVCs below, these PVCs should -// just get GC'd when the pod is finally decommissioned. -// -// TODO: will this cause issues if a PVC is deleted and before it's GC'd a Pod comes up? +// just get GC'd when the pod is finally decommissioned. HOWEVER, this is addressed by wrapping the volume claim deletion +// logic in an extra caching layer in the main reconciliation loop that only deletes a volume after it has been seen +// as unbound n times with m amount of time between checks. This gives the pod and PVC time to both enter cache +// so that the PVC will not be decommissioned while still being legitimately bound to a pod. func (s *StatefulSetDecomissioner) findUnboundVolumeClaims(ctx context.Context, set *appsv1.StatefulSet) ([]*corev1.PersistentVolumeClaim, error) { pods := &corev1.PodList{} if err := s.client.List(ctx, pods, client.InNamespace(set.Namespace), client.MatchingLabels(set.Spec.Template.Labels)); err != nil { @@ -630,16 +747,21 @@ func ordinalFromFQDN(fqdn string) (int, error) { return 0, fmt.Errorf("invalid broker FQDN for ordinal fetching: %s", fqdn) } - brokerPod := tokens[0] - brokerTokens := strings.Split(brokerPod, "-") - if len(brokerTokens) < 2 { - return 0, fmt.Errorf("invalid broker FQDN for ordinal fetching: %s", fqdn) + return ordinalFromResourceName(tokens[0]) +} + +// ordinalFromResourceName takes a ordinal suffixed resource and returns +// the ordinal at the end. +func ordinalFromResourceName(name string) (int, error) { + resourceTokens := strings.Split(name, "-") + if len(resourceTokens) < 2 { + return 0, fmt.Errorf("invalid resource name for ordinal fetching: %s", name) } // grab the last item after the "-"" which should be the ordinal and parse it - ordinal, err := strconv.Atoi(brokerTokens[len(brokerTokens)-1]) + ordinal, err := strconv.Atoi(resourceTokens[len(resourceTokens)-1]) if err != nil { - return 0, fmt.Errorf("parsing broker FQDN %q: %w", fqdn, err) + return 0, fmt.Errorf("parsing resource name %q: %w", name, err) } return ordinal, nil diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go index 71dec6ee8..e21429392 100644 --- a/operator/internal/decommissioning/statefulset_decommissioner_test.go +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -160,9 +160,6 @@ func (s *StatefulSetDecommissionerSuite) SetupSuite() { utilruntime.Must(clientgoscheme.AddToScheme(scheme)) log := testr.New(t).V(10) - if testing.Verbose() { - log = log.V(10) - } s.ctx = context.Background() s.env = testenv.New(t, testenv.Options{ @@ -192,8 +189,18 @@ func (s *StatefulSetDecommissionerSuite) SetupSuite() { dialer := kube.NewPodDialer(mgr.GetConfig()) s.clientFactory = internalclient.NewFactory(mgr.GetConfig(), mgr.GetClient()).WithDialer(dialer.DialContext) - decommissioner := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), decommissioning.WithFactory(s.clientFactory)) - if err := decommissioner.Setup(mgr); err != nil { + options := []decommissioning.Option{ + // override this so we can dial directly to our Redpanda pods + decommissioning.WithFactory(s.clientFactory), + // set these low so that we don't have to wait forever in the test + // these settings should give about a 5-10 second window before + // actually running a decommission + decommissioning.WithDelayedCacheInterval(5 * time.Second), + decommissioning.WithDelayedCacheMaxCount(2), + decommissioning.WithRequeueTimeout(2 * time.Second), + } + decommissioner := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), options...) + if err := decommissioner.SetupWithManager(mgr); err != nil { return err } diff --git a/operator/internal/testenv/testenv.go b/operator/internal/testenv/testenv.go index 8874a3320..056f1a707 100644 --- a/operator/internal/testenv/testenv.go +++ b/operator/internal/testenv/testenv.go @@ -90,7 +90,6 @@ func New(t *testing.T, options Options) *Env { options.Logger = logr.Discard() } - // TODO maybe allow customizing name? cluster, err := k3d.GetOrCreate(options.Name, k3d.WithAgents(options.Agents)) require.NoError(t, err) From 0574e70674a23145359d8d1631b6cc0acf2907c0 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Thu, 5 Dec 2024 23:37:34 -0500 Subject: [PATCH 06/13] fix linter --- operator/internal/decommissioning/delayed_cache.go | 2 -- operator/internal/decommissioning/delayed_cache_test.go | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/operator/internal/decommissioning/delayed_cache.go b/operator/internal/decommissioning/delayed_cache.go index 4653bd367..83195ce91 100644 --- a/operator/internal/decommissioning/delayed_cache.go +++ b/operator/internal/decommissioning/delayed_cache.go @@ -125,8 +125,6 @@ func (d *CategorizedDelayedCache[Category, Entry]) Mark(category Category, entry lastMarked: time.Now(), count: 1, } - - return } // Process processes and removes a cache item, calling the callback if the minimum threshold count is met. diff --git a/operator/internal/decommissioning/delayed_cache_test.go b/operator/internal/decommissioning/delayed_cache_test.go index 19716eb61..e517a0412 100644 --- a/operator/internal/decommissioning/delayed_cache_test.go +++ b/operator/internal/decommissioning/delayed_cache_test.go @@ -15,8 +15,9 @@ import ( "testing" "time" - "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" "github.com/stretchr/testify/require" + + "github.com/redpanda-data/redpanda-operator/operator/internal/decommissioning" ) func TestCategorizedDelayedCache(t *testing.T) { From 7d073a8655d191c94dece3b527c435fab5da3dc0 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Thu, 5 Dec 2024 23:44:28 -0500 Subject: [PATCH 07/13] Fix early return conditions --- .../decommissioning/statefulset_decomissioner.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index eca5cecf7..129f8f19a 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -422,8 +422,10 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 // be considered for decommissioning until we actually have more nodes // than are desired - s.delayedVolumeCache.Clean(setCacheKey) - return false, nil + s.delayedBrokerIDCache.Clean(setCacheKey) + // reqeueue if we have any pending volumes for decommission, and if not + // settle until the next thing trigger reconciliation + return s.delayedVolumeCache.Size(setCacheKey) > 0, nil } if len(health.NodesDown) == 0 { @@ -432,8 +434,10 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 // here because everything is healthy and we don't want to accidentally pick up something // later and have old cache entries count against it - s.delayedVolumeCache.Clean(setCacheKey) - return false, nil + s.delayedBrokerIDCache.Clean(setCacheKey) + // reqeueue if we have any pending volumes for decommission, and if not + // settle until the next thing trigger reconciliation + return s.delayedVolumeCache.Size(setCacheKey) > 0, nil } allNodes := collections.NewSet[int]() From def31f5774d9e831ff111bdb5d3fef0ac391a37a Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Fri, 6 Dec 2024 12:12:42 -0500 Subject: [PATCH 08/13] Add alternative fetcher implementations --- operator/cmd/sidecar/sidecar.go | 12 +- operator/go.mod | 4 +- .../decommissioning/chained_fetcher.go | 50 ++++ operator/internal/decommissioning/fetcher.go | 20 ++ .../{values_fetcher.go => helm_fetcher.go} | 41 +++- .../decommissioning/redpanda_fetcher.go | 41 ++++ .../decommissioning/rpk_profile_fetcher.go | 42 ++++ .../statefulset_decomissioner.go | 27 +-- operator/pkg/client/cluster.go | 35 ++- operator/pkg/client/factory.go | 129 ++++++++--- operator/pkg/client/rpk.go | 214 ++++++++++++++++++ operator/pkg/client/spec.go | 25 +- 12 files changed, 537 insertions(+), 103 deletions(-) create mode 100644 operator/internal/decommissioning/chained_fetcher.go create mode 100644 operator/internal/decommissioning/fetcher.go rename operator/internal/decommissioning/{values_fetcher.go => helm_fetcher.go} (69%) create mode 100644 operator/internal/decommissioning/redpanda_fetcher.go create mode 100644 operator/internal/decommissioning/rpk_profile_fetcher.go create mode 100644 operator/pkg/client/rpk.go diff --git a/operator/cmd/sidecar/sidecar.go b/operator/cmd/sidecar/sidecar.go index 0927e81c9..2b4b02886 100644 --- a/operator/cmd/sidecar/sidecar.go +++ b/operator/cmd/sidecar/sidecar.go @@ -38,6 +38,7 @@ func Command() *cobra.Command { decommissionRequeueTimeout time.Duration decommissionVoteInterval time.Duration decommissionMaxVoteCount int + redpandaYAMLPath string ) cmd := &cobra.Command{ @@ -56,6 +57,7 @@ func Command() *cobra.Command { decommissionRequeueTimeout, decommissionVoteInterval, decommissionMaxVoteCount, + redpandaYAMLPath, ) }, } @@ -68,6 +70,7 @@ func Command() *cobra.Command { cmd.Flags().DurationVar(&decommissionRequeueTimeout, "decommission-requeue-timeout", 10*time.Second, "The time period to wait before rechecking a broker that is being decommissioned.") cmd.Flags().DurationVar(&decommissionVoteInterval, "decommission-vote-interval", 30*time.Second, "The time period between incrementing decommission vote counts since the last decommission conditions were met.") cmd.Flags().IntVar(&decommissionMaxVoteCount, "decommission-vote-count", 2, "The number of times that a vote must be tallied when a resource meets decommission conditions for it to actually be decommissioned.") + cmd.Flags().StringVar(&redpandaYAMLPath, "redpanda-yaml", "/etc/redpanda/redpanda.yaml", "Path to redpanda.yaml") return cmd } @@ -82,6 +85,7 @@ func Run( decommissionRequeueTimeout time.Duration, decommissionVoteInterval time.Duration, decommissionMaxVoteCount int, + redpandaYAMLPath string, ) error { setupLog := ctrl.LoggerFrom(ctx).WithName("setup") @@ -117,7 +121,13 @@ func Run( return err } - if err := decommissioning.NewStatefulSetDecommissioner(mgr, decommissioning.NewHelmFetcher(mgr), []decommissioning.Option{ + fetcher := decommissioning.NewChainedFetcher( + // prefer RPK profile first and then move on to fetch from helm values + decommissioning.NewRPKProfileFetcher(redpandaYAMLPath), + decommissioning.NewHelmFetcher(mgr), + ) + + if err := decommissioning.NewStatefulSetDecommissioner(mgr, fetcher, []decommissioning.Option{ decommissioning.WithFilter(decommissioning.FilterStatefulSetOwner(clusterNamespace, clusterName)), decommissioning.WithRequeueTimeout(decommissionRequeueTimeout), decommissioning.WithDelayedCacheInterval(decommissionVoteInterval), diff --git a/operator/go.mod b/operator/go.mod index 3f060e908..dfa230d65 100644 --- a/operator/go.mod +++ b/operator/go.mod @@ -26,6 +26,7 @@ require ( github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring v0.74.0 github.com/prometheus/client_golang v1.20.0 github.com/prometheus/common v0.55.0 + github.com/redpanda-data/common-go/net v0.1.0 github.com/redpanda-data/common-go/rpadmin v0.1.9 github.com/redpanda-data/console/backend v0.0.0-20240303221210-05d5d9e85f20 github.com/redpanda-data/helm-charts v0.0.0-20241203151858-926cfe070c6e @@ -42,6 +43,7 @@ require ( github.com/twmb/franz-go/pkg/sasl/kerberos v1.1.0 github.com/twmb/franz-go/pkg/sr v1.2.0 go.uber.org/zap v1.27.0 + golang.org/x/exp v0.0.0-20240904232852-e7e105dedf7e golang.org/x/sync v0.8.0 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 @@ -338,7 +340,6 @@ require ( github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/procfs v0.15.1 // indirect - github.com/redpanda-data/common-go/net v0.1.0 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/rs/xid v1.5.0 // indirect @@ -408,7 +409,6 @@ require ( go.step.sm/crypto v0.40.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/crypto v0.26.0 // indirect - golang.org/x/exp v0.0.0-20240904232852-e7e105dedf7e // indirect golang.org/x/mod v0.20.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/oauth2 v0.22.0 // indirect diff --git a/operator/internal/decommissioning/chained_fetcher.go b/operator/internal/decommissioning/chained_fetcher.go new file mode 100644 index 000000000..f74db26ac --- /dev/null +++ b/operator/internal/decommissioning/chained_fetcher.go @@ -0,0 +1,50 @@ +// Copyright 2024 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 decommissioning + +import ( + "context" + "errors" + "fmt" +) + +// ChainedFetcher delegates fetching behavior to a list of sub fetchers +// moving down the list if an error occurs in the previous fetcher. +type ChainedFetcher struct { + fetchers []Fetcher +} + +var _ Fetcher = (*ChainedFetcher)(nil) + +func NewChainedFetcher(fetchers ...Fetcher) *ChainedFetcher { + return &ChainedFetcher{fetchers: fetchers} +} + +func (c *ChainedFetcher) FetchLatest(ctx context.Context, name, namespace string) (any, error) { + if len(c.fetchers) == 0 { + return nil, errors.New("chained fetcher does not have any supplied sub-fetchers") + } + + errs := []error{} + + for _, fetcher := range c.fetchers { + object, err := fetcher.FetchLatest(ctx, name, namespace) + if err != nil { + errs = append(errs, err) + continue + } + + if object != nil { + return object, nil + } + } + + return nil, fmt.Errorf("all sub-fetchers failed: %w", errors.Join(errs...)) +} diff --git a/operator/internal/decommissioning/fetcher.go b/operator/internal/decommissioning/fetcher.go new file mode 100644 index 000000000..b84346d3d --- /dev/null +++ b/operator/internal/decommissioning/fetcher.go @@ -0,0 +1,20 @@ +// Copyright 2024 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 decommissioning + +import ( + "context" +) + +// Fetcher acts as a mechanism for fetching an object that can be used in initializing +// a connection to Redpanda. This can come in the form of a Redpanda CR or an RPK profile. +type Fetcher interface { + FetchLatest(ctx context.Context, name, namespace string) (any, error) +} diff --git a/operator/internal/decommissioning/values_fetcher.go b/operator/internal/decommissioning/helm_fetcher.go similarity index 69% rename from operator/internal/decommissioning/values_fetcher.go rename to operator/internal/decommissioning/helm_fetcher.go index 4da139ca3..5ef3c67dd 100644 --- a/operator/internal/decommissioning/values_fetcher.go +++ b/operator/internal/decommissioning/helm_fetcher.go @@ -16,11 +16,15 @@ import ( "encoding/base64" "encoding/json" "errors" + "fmt" "io" corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ctrl "sigs.k8s.io/controller-runtime" "sigs.k8s.io/controller-runtime/pkg/client" + + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" ) // the format logic for helm releases can be found: @@ -28,19 +32,22 @@ import ( var gzipHeader = []byte{0x1f, 0x8b, 0x08} -type ValuesFetcher interface { - FetchLatest(ctx context.Context, name, namespace string) (map[string]any, error) -} - +// HelmFetcher fetches a Redpanda CR via initializing it virtually with a +// Helm values file stored in a secret. This is to maintain backwards +// compatibility with our current mechanism for decommissioning, but +// it should likely be dropped in the future with preference to using +// an RPK profile. type HelmFetcher struct { client client.Client } +var _ Fetcher = (*HelmFetcher)(nil) + func NewHelmFetcher(mgr ctrl.Manager) *HelmFetcher { return &HelmFetcher{client: mgr.GetClient()} } -func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) (map[string]any, error) { +func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) (any, error) { log := ctrl.LoggerFrom(ctx, "namespace", namespace, "name", name).WithName("HelmFetcher.FetchLatest") var secrets corev1.SecretList @@ -49,8 +56,7 @@ func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) ( "name": name, "owner": "helm", }, client.InNamespace(namespace)); err != nil { - log.Error(err, "fetching secrets list") - return nil, err + return nil, fmt.Errorf("fetching secrets list: %w", err) } latestVersion := 0 @@ -58,6 +64,8 @@ func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) ( for _, item := range secrets.Items { values, version, err := f.decode(item.Data["release"]) if err != nil { + // just log the error and move on rather than making it terminal + // in case there's some secret that's just badly formatted log.Error(err, "decoding secret", "secret", item.Name) continue } @@ -68,7 +76,24 @@ func (f *HelmFetcher) FetchLatest(ctx context.Context, name, namespace string) ( } if latestValues != nil { - return latestValues, nil + data, err := json.Marshal(latestValues) + if err != nil { + return nil, fmt.Errorf("marshaling values: %w", err) + } + + cluster := &redpandav1alpha2.Redpanda{ + ObjectMeta: metav1.ObjectMeta{ + Name: name, + Namespace: namespace, + }, + Spec: redpandav1alpha2.RedpandaSpec{ClusterSpec: &redpandav1alpha2.RedpandaClusterSpec{}}, + } + + if err := json.Unmarshal(data, cluster.Spec.ClusterSpec); err != nil { + return nil, fmt.Errorf("unmarshaling values: %w", err) + } + + return cluster, nil } err := errors.New("unable to find latest value") diff --git a/operator/internal/decommissioning/redpanda_fetcher.go b/operator/internal/decommissioning/redpanda_fetcher.go new file mode 100644 index 000000000..49ea991f2 --- /dev/null +++ b/operator/internal/decommissioning/redpanda_fetcher.go @@ -0,0 +1,41 @@ +// Copyright 2024 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 decommissioning + +import ( + "context" + "fmt" + + "k8s.io/apimachinery/pkg/types" + ctrl "sigs.k8s.io/controller-runtime" + "sigs.k8s.io/controller-runtime/pkg/client" + + redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" +) + +// RedpandaFetcher fetches a Redpanda cluster via CR. +type RedpandaFetcher struct { + client client.Client +} + +var _ Fetcher = (*RedpandaFetcher)(nil) + +func NewRedpandaFetcher(mgr ctrl.Manager) *RedpandaFetcher { + return &RedpandaFetcher{client: mgr.GetClient()} +} + +func (f *RedpandaFetcher) FetchLatest(ctx context.Context, name, namespace string) (any, error) { + var cluster redpandav1alpha2.Redpanda + if err := f.client.Get(ctx, types.NamespacedName{Namespace: namespace, Name: name}, &cluster); err != nil { + return nil, fmt.Errorf("fetching cluster: %w", err) + } + + return cluster, nil +} diff --git a/operator/internal/decommissioning/rpk_profile_fetcher.go b/operator/internal/decommissioning/rpk_profile_fetcher.go new file mode 100644 index 000000000..fb355ab6e --- /dev/null +++ b/operator/internal/decommissioning/rpk_profile_fetcher.go @@ -0,0 +1,42 @@ +// Copyright 2024 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 decommissioning + +import ( + "context" + + rpkconfig "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" + "github.com/spf13/afero" +) + +// RPKProfileFetcher loads up an RPK profile used in initializing a cluster +// connection. It should only ever be used when we only care about managing +// a single cluster, as it ignores the name and the namespace handed it and +// solely uses a profile found on disk. +type RPKProfileFetcher struct { + configPath string +} + +var _ Fetcher = (*RPKProfileFetcher)(nil) + +func NewRPKProfileFetcher(configPath string) *RPKProfileFetcher { + return &RPKProfileFetcher{configPath: configPath} +} + +func (f *RPKProfileFetcher) FetchLatest(_ context.Context, _, _ string) (any, error) { + params := rpkconfig.Params{ConfigFlag: f.configPath} + + config, err := params.Load(afero.NewOsFs()) + if err != nil { + return nil, err + } + + return config.VirtualProfile(), nil +} diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 129f8f19a..48ba8b0af 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -11,7 +11,6 @@ package decommissioning import ( "context" - "encoding/json" "errors" "fmt" "sort" @@ -35,7 +34,6 @@ import ( "sigs.k8s.io/controller-runtime/pkg/handler" "sigs.k8s.io/controller-runtime/pkg/predicate" - redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" internalclient "github.com/redpanda-data/redpanda-operator/operator/pkg/client" "github.com/redpanda-data/redpanda-operator/operator/pkg/client/kubernetes" "github.com/redpanda-data/redpanda-operator/operator/pkg/collections" @@ -116,7 +114,7 @@ func WithDelayedCacheMaxCount(count int) Option { type StatefulSetDecomissioner struct { client client.Client factory internalclient.ClientFactory - fetcher ValuesFetcher + fetcher Fetcher recorder record.EventRecorder requeueTimeout time.Duration delayedCacheInterval time.Duration @@ -126,7 +124,7 @@ type StatefulSetDecomissioner struct { filter func(ctx context.Context, set *appsv1.StatefulSet) (bool, error) } -func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher ValuesFetcher, options ...Option) *StatefulSetDecomissioner { +func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher Fetcher, options ...Option) *StatefulSetDecomissioner { k8sClient := mgr.GetClient() decommissioner := &StatefulSetDecomissioner{ @@ -717,29 +715,12 @@ func (s *StatefulSetDecomissioner) getAdminClient(ctx context.Context, set *apps return nil, errors.New("unable to get release name") } - values, err := s.fetcher.FetchLatest(ctx, release, set.Namespace) + fetched, err := s.fetcher.FetchLatest(ctx, release, set.Namespace) if err != nil { return nil, fmt.Errorf("fetching latest values: %w", err) } - data, err := json.MarshalIndent(values, "", " ") - if err != nil { - return nil, fmt.Errorf("marshaling values: %w", err) - } - - cluster := &redpandav1alpha2.Redpanda{ - ObjectMeta: metav1.ObjectMeta{ - Name: release, - Namespace: set.Namespace, - }, - Spec: redpandav1alpha2.RedpandaSpec{ClusterSpec: &redpandav1alpha2.RedpandaClusterSpec{}}, - } - - if err := json.Unmarshal(data, cluster.Spec.ClusterSpec); err != nil { - return nil, fmt.Errorf("unmarshaling values: %w", err) - } - - return s.factory.RedpandaAdminClient(ctx, cluster) + return s.factory.RedpandaAdminClient(ctx, fetched) } // ordinalFromFQDN takes a hostname and attempt to map the diff --git a/operator/pkg/client/cluster.go b/operator/pkg/client/cluster.go index 57989df19..4791c156a 100644 --- a/operator/pkg/client/cluster.go +++ b/operator/pkg/client/cluster.go @@ -11,17 +11,14 @@ package client import ( "github.com/redpanda-data/common-go/rpadmin" - "github.com/redpanda-data/console/backend/pkg/config" "github.com/twmb/franz-go/pkg/kgo" - "github.com/twmb/franz-go/pkg/sasl" - "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/pkg/sr" "github.com/redpanda-data/helm-charts/pkg/redpanda" redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" ) -// RedpandaAdminForCluster returns a simple kgo.Client able to communicate with the given cluster specified via a Redpanda cluster. +// redpandaAdminForCluster returns a simple rpadmin.AdminAPI able to communicate with the given cluster specified via a Redpanda cluster. func (c *Factory) redpandaAdminForCluster(cluster *redpandav1alpha2.Redpanda) (*rpadmin.AdminAPI, error) { dot, err := cluster.GetDot(c.config) if err != nil { @@ -43,7 +40,7 @@ func (c *Factory) redpandaAdminForCluster(cluster *redpandav1alpha2.Redpanda) (* return client, nil } -// schemaRegistryForCluster returns a simple kgo.Client able to communicate with the given cluster specified via a Redpanda cluster. +// schemaRegistryForCluster returns a simple sr.Client able to communicate with the given cluster specified via a Redpanda cluster. func (c *Factory) schemaRegistryForCluster(cluster *redpandav1alpha2.Redpanda) (*sr.Client, error) { dot, err := cluster.GetDot(c.config) if err != nil { @@ -65,7 +62,7 @@ func (c *Factory) schemaRegistryForCluster(cluster *redpandav1alpha2.Redpanda) ( return client, nil } -// KafkaForCluster returns a simple kgo.Client able to communicate with the given cluster specified via a Redpanda cluster. +// kafkaForCluster returns a simple kgo.Client able to communicate with the given cluster specified via a Redpanda cluster. func (c *Factory) kafkaForCluster(cluster *redpandav1alpha2.Redpanda, opts ...kgo.Opt) (*kgo.Client, error) { dot, err := cluster.GetDot(c.config) if err != nil { @@ -77,23 +74,19 @@ func (c *Factory) kafkaForCluster(cluster *redpandav1alpha2.Redpanda, opts ...kg return nil, err } - if c.userAuth != nil { - auth := scram.Auth{ - User: c.userAuth.Username, - Pass: c.userAuth.Password, - } + authOpt, err := c.kafkaUserAuth() + if err != nil { + // close the client since it's no longer usable + client.Close() - var mechanism sasl.Mechanism - switch c.userAuth.Mechanism { - case config.SASLMechanismScramSHA256: - mechanism = auth.AsSha256Mechanism() - case config.SASLMechanismScramSHA512: - mechanism = auth.AsSha512Mechanism() - default: - return nil, ErrUnsupportedSASLMechanism - } + return nil, err + } + + if authOpt != nil { + // close this client since we're not going to use it anymore + client.Close() - return kgo.NewClient(append(client.Opts(), kgo.SASL(mechanism))...) + return kgo.NewClient(append(client.Opts(), authOpt)...) } return client, nil diff --git a/operator/pkg/client/factory.go b/operator/pkg/client/factory.go index cc4beaa72..4acc38584 100644 --- a/operator/pkg/client/factory.go +++ b/operator/pkg/client/factory.go @@ -14,14 +14,20 @@ import ( "errors" "github.com/redpanda-data/common-go/rpadmin" + "github.com/redpanda-data/console/backend/pkg/config" + "github.com/spf13/afero" "github.com/twmb/franz-go/pkg/kadm" "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/sasl" + "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/pkg/sr" apierrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/types" "k8s.io/client-go/rest" "sigs.k8s.io/controller-runtime/pkg/client" + rpkconfig "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" + "github.com/redpanda-data/helm-charts/pkg/redpanda" redpandav1alpha2 "github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2" "github.com/redpanda-data/redpanda-operator/operator/pkg/client/acls" @@ -30,12 +36,13 @@ import ( ) var ( - ErrInvalidClusterRef = errors.New("clusterRef refers to a cluster that does not exist") - ErrEmptyBrokerList = errors.New("empty broker list") - ErrEmptyURLList = errors.New("empty url list") - ErrInvalidKafkaClientObject = errors.New("cannot initialize Kafka API client from given object") - ErrInvalidRedpandaClientObject = errors.New("cannot initialize Redpanda Admin API client from given object") - ErrUnsupportedSASLMechanism = errors.New("unsupported SASL mechanism") + ErrInvalidClusterRef = errors.New("clusterRef refers to a cluster that does not exist") + ErrEmptyBrokerList = errors.New("empty broker list") + ErrEmptyURLList = errors.New("empty url list") + ErrInvalidKafkaClientObject = errors.New("cannot initialize Kafka API client from given object") + ErrInvalidRedpandaClientObject = errors.New("cannot initialize Redpanda Admin API client from given object") + ErrInvalidSchemaRegistryClientObject = errors.New("cannot initialize Schema Registry API client from given object") + ErrUnsupportedSASLMechanism = errors.New("unsupported SASL mechanism") ) // UserAuth allows you to override the auth credentials used in establishing a client connection @@ -57,20 +64,20 @@ type UserAuth struct { // at method invocation. type ClientFactory interface { // KafkaClient initializes a kgo.Client based on the spec of the passed in struct. - // The struct *must* implement either the v1alpha2.KafkaConnectedObject interface or the v1alpha2.ClusterReferencingObject - // interface to properly initialize. Callers should always call Close on the returned *kgo.Client, or it will leak - // goroutines. - KafkaClient(ctx context.Context, object client.Object, opts ...kgo.Opt) (*kgo.Client, error) + // The struct *must* either be an RPK profile, Redpanda CR, or implement either the v1alpha2.KafkaConnectedObject interface + // or the v1alpha2.ClusterReferencingObject interface to properly initialize. Callers should always call Close on the returned *kgo.Client, + // or it will leak goroutines. + KafkaClient(ctx context.Context, object any, opts ...kgo.Opt) (*kgo.Client, error) // RedpandaAdminClient initializes a rpadmin.AdminAPI client based on the spec of the passed in struct. - // The struct *must* implement either the v1alpha2.AdminConnectedObject interface or the v1alpha2.ClusterReferencingObject - // interface to properly initialize. - RedpandaAdminClient(ctx context.Context, object client.Object) (*rpadmin.AdminAPI, error) + // The struct *must* either be an RPK profile, Redpanda CR, or implement either the v1alpha2.AdminConnectedObject interface + // or the v1alpha2.ClusterReferencingObject interface to properly initialize. + RedpandaAdminClient(ctx context.Context, object any) (*rpadmin.AdminAPI, error) // SchemaRegistryClient initializes an sr.Client based on the spec of the passed in struct. - // The struct *must* implement either the v1alpha2.SchemaRegistryConnectedObject interface or the v1alpha2.ClusterReferencingObject - // interface to properly initialize. - SchemaRegistryClient(ctx context.Context, object client.Object) (*sr.Client, error) + // The struct *must* either be an RPK profile, Redpanda CR, or implement either the v1alpha2.SchemaRegistryConnectedObject interface + // or the v1alpha2.ClusterReferencingObject interface to properly initialize. + SchemaRegistryClient(ctx context.Context, object any) (*sr.Client, error) // ACLs returns a high-level client for synchronizing ACLs. Callers should always call Close on the returned *acls.Syncer, or it will leak // goroutines. @@ -87,6 +94,7 @@ type ClientFactory interface { type Factory struct { client.Client config *rest.Config + fs afero.Fs dialer redpanda.DialContextFunc userAuth *UserAuth @@ -97,6 +105,7 @@ var _ ClientFactory = (*Factory)(nil) func NewFactory(config *rest.Config, kubeclient client.Client) *Factory { return &Factory{ config: rest.CopyConfig(config), + fs: afero.NewOsFs(), Client: kubeclient, } } @@ -106,26 +115,47 @@ func (c *Factory) WithDialer(dialer redpanda.DialContextFunc) *Factory { Client: c.Client, config: c.config, userAuth: c.userAuth, + fs: c.fs, dialer: dialer, } } +func (c *Factory) WithFS(fs afero.Fs) *Factory { + return &Factory{ + Client: c.Client, + config: c.config, + userAuth: c.userAuth, + dialer: c.dialer, + fs: fs, + } +} + func (c *Factory) WithUserAuth(userAuth *UserAuth) *Factory { return &Factory{ Client: c.Client, config: c.config, dialer: c.dialer, + fs: c.fs, userAuth: userAuth, } } -func (c *Factory) KafkaClient(ctx context.Context, obj client.Object, opts ...kgo.Opt) (*kgo.Client, error) { +func (c *Factory) KafkaClient(ctx context.Context, obj any, opts ...kgo.Opt) (*kgo.Client, error) { // if we pass in a Redpanda cluster, just use it if cluster, ok := obj.(*redpandav1alpha2.Redpanda); ok { return c.kafkaForCluster(cluster, opts...) } - cluster, err := c.getCluster(ctx, obj) + if profile, ok := obj.(*rpkconfig.RpkProfile); ok { + return c.kafkaForRPKProfile(profile, opts...) + } + + o, ok := obj.(client.Object) + if !ok { + return nil, ErrInvalidKafkaClientObject + } + + cluster, err := c.getCluster(ctx, o) if err != nil { return nil, err } @@ -134,20 +164,29 @@ func (c *Factory) KafkaClient(ctx context.Context, obj client.Object, opts ...kg return c.kafkaForCluster(cluster, opts...) } - if spec := c.getKafkaSpec(obj); spec != nil { - return c.kafkaForSpec(ctx, obj.GetNamespace(), c.getKafkaMetricNamespace(obj), spec, opts...) + if spec := c.getKafkaSpec(o); spec != nil { + return c.kafkaForSpec(ctx, o.GetNamespace(), c.getKafkaMetricNamespace(o), spec, opts...) } return nil, ErrInvalidKafkaClientObject } -func (c *Factory) RedpandaAdminClient(ctx context.Context, obj client.Object) (*rpadmin.AdminAPI, error) { +func (c *Factory) RedpandaAdminClient(ctx context.Context, obj any) (*rpadmin.AdminAPI, error) { // if we pass in a Redpanda cluster, just use it if cluster, ok := obj.(*redpandav1alpha2.Redpanda); ok { return c.redpandaAdminForCluster(cluster) } - cluster, err := c.getCluster(ctx, obj) + if profile, ok := obj.(*rpkconfig.RpkProfile); ok { + return c.redpandaAdminForRPKProfile(profile) + } + + o, ok := obj.(client.Object) + if !ok { + return nil, ErrInvalidRedpandaClientObject + } + + cluster, err := c.getCluster(ctx, o) if err != nil { return nil, err } @@ -156,20 +195,29 @@ func (c *Factory) RedpandaAdminClient(ctx context.Context, obj client.Object) (* return c.redpandaAdminForCluster(cluster) } - if spec := c.getAdminSpec(obj); spec != nil { - return c.redpandaAdminForSpec(ctx, obj.GetNamespace(), spec) + if spec := c.getAdminSpec(o); spec != nil { + return c.redpandaAdminForSpec(ctx, o.GetNamespace(), spec) } return nil, ErrInvalidRedpandaClientObject } -func (c *Factory) SchemaRegistryClient(ctx context.Context, obj client.Object) (*sr.Client, error) { +func (c *Factory) SchemaRegistryClient(ctx context.Context, obj any) (*sr.Client, error) { // if we pass in a Redpanda cluster, just use it if cluster, ok := obj.(*redpandav1alpha2.Redpanda); ok { return c.schemaRegistryForCluster(cluster) } - cluster, err := c.getCluster(ctx, obj) + if profile, ok := obj.(*rpkconfig.RpkProfile); ok { + return c.schemaRegistryForRPKProfile(profile) + } + + o, ok := obj.(client.Object) + if !ok { + return nil, ErrInvalidSchemaRegistryClientObject + } + + cluster, err := c.getCluster(ctx, o) if err != nil { return nil, err } @@ -178,11 +226,11 @@ func (c *Factory) SchemaRegistryClient(ctx context.Context, obj client.Object) ( return c.schemaRegistryForCluster(cluster) } - if spec := c.getSchemaRegistrySpec(obj); spec != nil { - return c.schemaRegistryForSpec(ctx, obj.GetNamespace(), spec) + if spec := c.getSchemaRegistrySpec(o); spec != nil { + return c.schemaRegistryForSpec(ctx, o.GetNamespace(), spec) } - return nil, ErrInvalidRedpandaClientObject + return nil, ErrInvalidSchemaRegistryClientObject } func (c *Factory) Schemas(ctx context.Context, obj redpandav1alpha2.ClusterReferencingObject) (*schemas.Syncer, error) { @@ -282,3 +330,26 @@ func (c *Factory) getSchemaRegistrySpec(obj client.Object) *redpandav1alpha2.Sch return nil } + +func (c *Factory) kafkaUserAuth() (kgo.Opt, error) { + if c.userAuth != nil { + auth := scram.Auth{ + User: c.userAuth.Username, + Pass: c.userAuth.Password, + } + + var mechanism sasl.Mechanism + switch c.userAuth.Mechanism { + case config.SASLMechanismScramSHA256: + mechanism = auth.AsSha256Mechanism() + case config.SASLMechanismScramSHA512: + mechanism = auth.AsSha512Mechanism() + default: + return nil, ErrUnsupportedSASLMechanism + } + + return kgo.SASL(mechanism), nil + } + + return nil, nil +} diff --git a/operator/pkg/client/rpk.go b/operator/pkg/client/rpk.go new file mode 100644 index 000000000..fba91dc60 --- /dev/null +++ b/operator/pkg/client/rpk.go @@ -0,0 +1,214 @@ +// Copyright 2024 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 client + +import ( + "fmt" + "net" + "net/http" + "strings" + "time" + + commonnet "github.com/redpanda-data/common-go/net" + "github.com/redpanda-data/common-go/rpadmin" + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/sasl/plain" + "github.com/twmb/franz-go/pkg/sasl/scram" + "github.com/twmb/franz-go/pkg/sr" + + rpkconfig "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" +) + +// this roughly implements https://github.com/redpanda-data/redpanda/blob/f5a7a13f7fca3f69a4380f0bbfa8fbc3e7f899d6/src/go/rpk/pkg/adminapi/admin.go#L46 +// but without the OIDC support, that said, it seems real odd that we delegate to the KafkaAPI stanza for an admin API connection. +func getAdminAuth(p *rpkconfig.RpkProfile) rpadmin.Auth { + if p.HasSASLCredentials() { + return &rpadmin.BasicAuth{Username: p.KafkaAPI.SASL.User, Password: p.KafkaAPI.SASL.Password} + } + + // we explicitly don't support anything else at the moment (i.e. OIDC) + return &rpadmin.NopAuth{} +} + +// redpandaAdminForRPKProfile returns a simple rpadmin.AdminAPI able to communicate with a cluster based on the given RPK profile. +func (c *Factory) redpandaAdminForRPKProfile(profile *rpkconfig.RpkProfile) (*rpadmin.AdminAPI, error) { + tls, err := profile.AdminAPI.TLS.Config(c.fs) + if err != nil { + return nil, fmt.Errorf("unable to create admin api tls config: %v", err) + } + + client, err := rpadmin.NewAdminAPIWithDialer(profile.AdminAPI.Addresses, getAdminAuth(profile), tls, c.dialer) + if err != nil { + return nil, fmt.Errorf("initializing admin client: %w", err) + } + + if c.userAuth != nil { + client.SetAuth(&rpadmin.BasicAuth{ + Username: c.userAuth.Username, + Password: c.userAuth.Password, + }) + } + + return client, nil +} + +// this implements the basic logic found here https://github.com/redpanda-data/redpanda/blob/8e1ccaab1975150ed4b2aec630b4fed6f06a16bf/src/go/rpk/pkg/schemaregistry/client.go#L14 +func normalizeSchemaRegistryURLs(profile *rpkconfig.RpkProfile) ([]string, error) { + urls := profile.SR.Addresses + for i, url := range urls { + scheme, _, err := commonnet.ParseHostMaybeScheme(url) + if err != nil { + return nil, fmt.Errorf("unable to parse schema registry address %q: %v", url, err) + } + switch scheme { + case "http", "https": + continue + case "": + if profile.SR.TLS != nil { + urls[i] = "https://" + url + continue + } + + urls[i] = "http://" + url + default: + return nil, fmt.Errorf("unsupported scheme %q in the schema registry address %q", scheme, url) + } + } + + return urls, nil +} + +// schemaRegistryForRPKProfile returns a simple sr.Client able to communicate with a cluster based on the given RPK profile. +func (c *Factory) schemaRegistryForRPKProfile(profile *rpkconfig.RpkProfile) (*sr.Client, error) { + urls, err := normalizeSchemaRegistryURLs(profile) + if err != nil { + return nil, err + } + + tls, err := profile.SR.TLS.Config(c.fs) + if err != nil { + return nil, err + } + + // These transport values come from the TLS client options found here: + // https://github.com/twmb/franz-go/blob/cea7aa5d803781e5f0162187795482ba1990c729/pkg/sr/clientopt.go#L48-L68 + transport := &http.Transport{ + Proxy: http.ProxyFromEnvironment, + ForceAttemptHTTP2: true, + MaxIdleConns: 100, + MaxIdleConnsPerHost: 100, + DialContext: c.dialer, + IdleConnTimeout: 90 * time.Second, + TLSHandshakeTimeout: 10 * time.Second, + ExpectContinueTimeout: 1 * time.Second, + } + + if c.dialer == nil { + transport.DialContext = (&net.Dialer{ + Timeout: 30 * time.Second, + KeepAlive: 30 * time.Second, + }).DialContext + } + + if tls != nil { + transport.TLSClientConfig = tls + } + + opts := []sr.ClientOpt{sr.HTTPClient(&http.Client{ + Timeout: 5 * time.Second, + Transport: transport, + })} + + if profile.HasSASLCredentials() { + opts = append(opts, sr.BasicAuth(profile.KafkaAPI.SASL.User, profile.KafkaAPI.SASL.Password)) + } + + opts = append(opts, sr.URLs(urls...)) + + if c.userAuth != nil { + // override explicit user auth + opts = append(opts, sr.BasicAuth(c.userAuth.Username, c.userAuth.Password)) + } + + return sr.NewClient(opts...) +} + +// this implements roughly https://github.com/redpanda-data/redpanda/blob/dev/src/go/rpk/pkg/kafka/client_franz.go#L100 +func getKafkaAuth(profile *rpkconfig.RpkProfile) (kgo.Opt, error) { + if profile.HasSASLCredentials() { + auth := scram.Auth{ + User: profile.KafkaAPI.SASL.User, + Pass: profile.KafkaAPI.SASL.Password, + } + + switch name := strings.ToUpper(profile.KafkaAPI.SASL.Mechanism); name { + case "SCRAM-SHA-256", "": // we default to SCRAM-SHA-256 -- people commonly specify user & pass without --sasl-mechanism + return kgo.SASL(auth.AsSha256Mechanism()), nil + case "SCRAM-SHA-512": + return kgo.SASL(auth.AsSha512Mechanism()), nil + case "PLAIN": + return kgo.SASL((&plain.Auth{ + User: profile.KafkaAPI.SASL.User, + Pass: profile.KafkaAPI.SASL.Password, + }).AsMechanism()), nil + default: + return nil, fmt.Errorf("unknown SASL mechanism %q, supported: [SCRAM-SHA-256, SCRAM-SHA-512, PLAIN]", name) + } + } + + return nil, nil +} + +// kafkaForRPKProfile returns a simple kgo.Client able to communicate with a cluster based on the given RPK profile. +func (c *Factory) kafkaForRPKProfile(profile *rpkconfig.RpkProfile, opts ...kgo.Opt) (*kgo.Client, error) { + kopts := []kgo.Opt{ + kgo.SeedBrokers(profile.KafkaAPI.Brokers...), + } + + tls, err := profile.KafkaAPI.TLS.Config(c.fs) + if err != nil { + return nil, err + } + + authOpt, err := getKafkaAuth(profile) + if err != nil { + return nil, err + } + + if authOpt != nil { + kopts = append(kopts, authOpt) + } + + if tls != nil { + // we can only specify one of DialTLSConfig or Dialer + if c.dialer == nil { + kopts = append(kopts, kgo.DialTLSConfig(tls)) + } else { + kopts = append(kopts, kgo.Dialer(wrapTLSDialer(c.dialer, tls))) + } + } else if c.dialer != nil { + kopts = append(kopts, kgo.Dialer(c.dialer)) + } + + // append all user specified opts after + kopts = append(kopts, opts...) + + // and finally handle factory-level auth override + authOpt, err = c.kafkaUserAuth() + if err != nil { + return nil, err + } + + if authOpt != nil { + kopts = append(kopts, authOpt) + } + + return kgo.NewClient(kopts...) +} diff --git a/operator/pkg/client/spec.go b/operator/pkg/client/spec.go index 8a890f931..2dd99d6bc 100644 --- a/operator/pkg/client/spec.go +++ b/operator/pkg/client/spec.go @@ -17,10 +17,7 @@ import ( "time" "github.com/redpanda-data/common-go/rpadmin" - "github.com/redpanda-data/console/backend/pkg/config" "github.com/twmb/franz-go/pkg/kgo" - "github.com/twmb/franz-go/pkg/sasl" - "github.com/twmb/franz-go/pkg/sasl/scram" "github.com/twmb/franz-go/pkg/sr" "sigs.k8s.io/controller-runtime/pkg/log" @@ -57,23 +54,13 @@ func (c *Factory) kafkaForSpec(ctx context.Context, namespace string, metricName kopts = append(kopts, saslOpt) } - if c.userAuth != nil { - auth := scram.Auth{ - User: c.userAuth.Username, - Pass: c.userAuth.Password, - } - - var mechanism sasl.Mechanism - switch c.userAuth.Mechanism { - case config.SASLMechanismScramSHA256: - mechanism = auth.AsSha256Mechanism() - case config.SASLMechanismScramSHA512: - mechanism = auth.AsSha512Mechanism() - default: - return nil, ErrUnsupportedSASLMechanism - } + authOpt, err := c.kafkaUserAuth() + if err != nil { + return nil, err + } - kopts = append(kopts, kgo.SASL(mechanism)) + if authOpt != nil { + kopts = append(kopts, authOpt) } if spec.TLS != nil { From 774ea6dd4d2b05a3161db3232929eed214c2aa27 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Fri, 6 Dec 2024 12:20:14 -0500 Subject: [PATCH 09/13] Don't initialize new FS every fetch call --- operator/internal/decommissioning/rpk_profile_fetcher.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/operator/internal/decommissioning/rpk_profile_fetcher.go b/operator/internal/decommissioning/rpk_profile_fetcher.go index fb355ab6e..e3d1a703b 100644 --- a/operator/internal/decommissioning/rpk_profile_fetcher.go +++ b/operator/internal/decommissioning/rpk_profile_fetcher.go @@ -22,18 +22,19 @@ import ( // solely uses a profile found on disk. type RPKProfileFetcher struct { configPath string + fs afero.Fs } var _ Fetcher = (*RPKProfileFetcher)(nil) func NewRPKProfileFetcher(configPath string) *RPKProfileFetcher { - return &RPKProfileFetcher{configPath: configPath} + return &RPKProfileFetcher{configPath: configPath, fs: afero.NewOsFs()} } func (f *RPKProfileFetcher) FetchLatest(_ context.Context, _, _ string) (any, error) { params := rpkconfig.Params{ConfigFlag: f.configPath} - config, err := params.Load(afero.NewOsFs()) + config, err := params.Load(f.fs) if err != nil { return nil, err } From f785107c6260dbad740c4edd3e3c62f2d34adf1c Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Fri, 6 Dec 2024 12:25:27 -0500 Subject: [PATCH 10/13] Add memoized profile so we don't read it every time --- .../decommissioning/rpk_profile_fetcher.go | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/operator/internal/decommissioning/rpk_profile_fetcher.go b/operator/internal/decommissioning/rpk_profile_fetcher.go index e3d1a703b..0e521feee 100644 --- a/operator/internal/decommissioning/rpk_profile_fetcher.go +++ b/operator/internal/decommissioning/rpk_profile_fetcher.go @@ -11,6 +11,7 @@ package decommissioning import ( "context" + "sync" rpkconfig "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" "github.com/spf13/afero" @@ -23,6 +24,8 @@ import ( type RPKProfileFetcher struct { configPath string fs afero.Fs + profile *rpkconfig.RpkProfile + mutex sync.Mutex } var _ Fetcher = (*RPKProfileFetcher)(nil) @@ -32,6 +35,16 @@ func NewRPKProfileFetcher(configPath string) *RPKProfileFetcher { } func (f *RPKProfileFetcher) FetchLatest(_ context.Context, _, _ string) (any, error) { + f.mutex.Lock() + defer f.mutex.Unlock() + + if f.profile != nil { + // returned the memoized profile so we don't have to keep reading it, if we need + // to handle the profile on disk changing, then we should implement something like + // an fsnotify watcher that clears the memoized profile when the file changes on disk + return f.profile, nil + } + params := rpkconfig.Params{ConfigFlag: f.configPath} config, err := params.Load(f.fs) @@ -39,5 +52,7 @@ func (f *RPKProfileFetcher) FetchLatest(_ context.Context, _, _ string) (any, er return nil, err } - return config.VirtualProfile(), nil + f.profile = config.VirtualProfile() + + return f.profile, nil } From 7544a80bf54afc978bdd9abb9eb85155819def26 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Tue, 10 Dec 2024 12:20:43 -0500 Subject: [PATCH 11/13] Address review comments --- .../statefulset_decomissioner.go | 36 ++++++++++++------- .../statefulset_decommissioner_test.go | 8 +++-- operator/pkg/functional/map.go | 4 +++ 3 files changed, 32 insertions(+), 16 deletions(-) diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 48ba8b0af..6d41991dd 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -154,25 +154,24 @@ func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher Fetcher, options ... // +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=patch // +kubebuilder:rbac:groups=core,resources=events,verbs=create;patch // +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch +// +kubebuilder:rbac:groups=coordination,resources=leases,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets,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;watch;delete -// +kubebuilder:rbac:groups=core,namespace=default,resources=persistentvolumes,verbs=patch // +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch // +kubebuilder:rbac:groups=core,namespace=default,resources=secrets,verbs=get;list;watch +// +kubebuilder:rbac:groups=coordination,namespace=default,resources=leases,verbs=get;list;watch;create;update;patch;delete func (s *StatefulSetDecomissioner) SetupWithManager(mgr ctrl.Manager) error { pvcPredicate, err := predicate.LabelSelectorPredicate( metav1.LabelSelector{ MatchExpressions: []metav1.LabelSelectorRequirement{{ - Key: k8sNameLabelKey, // look for only redpanda owned pvcs - Operator: metav1.LabelSelectorOpIn, - Values: []string{"redpanda"}, + Key: k8sNameLabelKey, // make sure we have a name + Operator: metav1.LabelSelectorOpExists, }, { - Key: k8sComponentLabelKey, // make sure the PVC is part of the statefulset - Operator: metav1.LabelSelectorOpIn, - Values: []string{"redpanda-statefulset"}, + Key: k8sComponentLabelKey, // make sure we have a component label + Operator: metav1.LabelSelectorOpExists, }, { Key: k8sInstanceLabelKey, // make sure we have a cluster name Operator: metav1.LabelSelectorOpExists, @@ -607,10 +606,10 @@ func (s *StatefulSetDecomissioner) Decommission(ctx context.Context, set *appsv1 for _, broker := range brokersToDecommission { decommissioned, err := s.delayedBrokerIDCache.Process(setCacheKey, broker, func() error { // only record the event here since this is when we trigger a decommission - s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonBroker, "brokers needing decommissioning: [%s], decommissioning: %d", formatBrokerList(brokersToDecommission), brokersToDecommission[0]) + s.recorder.Eventf(set, corev1.EventTypeNormal, eventReasonBroker, "brokers needing decommissioning: [%s], decommissioning: %d", formatBrokerList(brokersToDecommission), broker) - if err := adminClient.DecommissionBroker(ctx, brokersToDecommission[0]); err != nil { - log.Error(err, "decommissioning broker", "broker", brokersToDecommission[0]) + if err := adminClient.DecommissionBroker(ctx, broker); err != nil { + log.Error(err, "decommissioning broker", "broker", broker) return err } @@ -656,15 +655,26 @@ func (s *StatefulSetDecomissioner) findUnboundVolumeClaims(ctx context.Context, return nil, fmt.Errorf("listing pods: %w", err) } + // This code for volume labels differs from what the original decommission controller did + // because it attempts to follow the actual label provisioning convention of the statefulset + // controller code here: + // https://github.com/kubernetes/kubernetes/blob/a499facee693a1a83daadb82d88f7b51d324ffc5/pkg/controller/statefulset/stateful_set_utils.go#L391 + // + // The original does some oddities around overwriting the component label with a `-statefulset` suffix. This comes from the fact + // that the original code failed to merge in the MatchLabels, which in the helm chart we explicitly set to have a `-statefulset` + // suffix. Here we just add the MatchLabels merging code found in the statefulset controller, so we should be good. dataVolumeLabels := client.MatchingLabels{} for _, template := range set.Spec.VolumeClaimTemplates { if template.Name == datadirVolume { - dataVolumeLabels = template.Labels + for key, value := range template.Labels { + dataVolumeLabels[key] = value + } + for key, value := range set.Spec.Selector.MatchLabels { + dataVolumeLabels[key] = value + } break } } - // the first part of this, "redpanda" is the component name (i.e. redpanda, console, etc.) - dataVolumeLabels[k8sComponentLabelKey] = "redpanda-statefulset" // find all pvcs of the data directory for this StatefulSet pvcs := &corev1.PersistentVolumeClaimList{} diff --git a/operator/internal/decommissioning/statefulset_decommissioner_test.go b/operator/internal/decommissioning/statefulset_decommissioner_test.go index e21429392..af9d512f8 100644 --- a/operator/internal/decommissioning/statefulset_decommissioner_test.go +++ b/operator/internal/decommissioning/statefulset_decommissioner_test.go @@ -159,7 +159,9 @@ func (s *StatefulSetDecommissionerSuite) SetupSuite() { scheme := runtime.NewScheme() utilruntime.Must(clientgoscheme.AddToScheme(scheme)) - log := testr.New(t).V(10) + log := testr.NewWithOptions(t, testr.Options{ + Verbosity: 10, + }) s.ctx = context.Background() s.env = testenv.New(t, testenv.Options{ @@ -227,8 +229,8 @@ func (s *StatefulSetDecommissionerSuite) installChart(name, version string, over "enabled": false, }, "image": map[string]any{ - "repository": "redpandadata/redpanda-unstable", - "tag": "v24.3.1-rc8", + "repository": "redpandadata/redpanda", + "tag": "v24.3.1", }, } diff --git a/operator/pkg/functional/map.go b/operator/pkg/functional/map.go index febb67c3e..cbe14021d 100644 --- a/operator/pkg/functional/map.go +++ b/operator/pkg/functional/map.go @@ -59,6 +59,10 @@ func deepCopyElements(v []any) []any { return copied } +// MergeMaps attempts to merge all elements in a generic map +// including merging arrays of matching keyed arrays. Note that +// the array merging behavior is *not* the same as that of Helm +// so this should not be used as a replacement for that. func MergeMaps(first, second map[string]any) map[string]any { merged := deepCopyMap(first) for k, v := range second { From 71d7f1586f3e8d5570cd1848aa733b8ae6908859 Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Tue, 10 Dec 2024 12:28:51 -0500 Subject: [PATCH 12/13] re-run task generate and update flags --- operator/cmd/sidecar/sidecar.go | 6 ++-- operator/config/rbac/bases/operator/role.yaml | 19 ++++++++---- .../config/rbac/decommissioner-role/role.yaml | 30 +++++++++++++++---- operator/internal/decommissioning/role.yaml | 30 +++++++++++++++---- 4 files changed, 64 insertions(+), 21 deletions(-) diff --git a/operator/cmd/sidecar/sidecar.go b/operator/cmd/sidecar/sidecar.go index 2b4b02886..c9bf81a46 100644 --- a/operator/cmd/sidecar/sidecar.go +++ b/operator/cmd/sidecar/sidecar.go @@ -65,12 +65,12 @@ func Command() *cobra.Command { cmd.Flags().StringVar(&metricsAddr, "metrics-bind-address", ":8080", "The address the metric endpoint binds to.") cmd.Flags().StringVar(&probeAddr, "health-probe-bind-address", ":8081", "The address the probe endpoint binds to.") cmd.Flags().StringVar(&pprofAddr, "pprof-bind-address", ":8082", "The address the metric endpoint binds to.") - cmd.Flags().StringVar(&clusterNamespace, "cluster-namespace", "", "The namespace of the cluster that this sidecar manages.") - cmd.Flags().StringVar(&clusterName, "cluster-name", "", "The name of the cluster that this sidecar manages.") + cmd.Flags().StringVar(&clusterNamespace, "redpanda-cluster-namespace", "", "The namespace of the cluster that this sidecar manages.") + cmd.Flags().StringVar(&clusterName, "redpanda-cluster-name", "", "The name of the cluster that this sidecar manages.") cmd.Flags().DurationVar(&decommissionRequeueTimeout, "decommission-requeue-timeout", 10*time.Second, "The time period to wait before rechecking a broker that is being decommissioned.") cmd.Flags().DurationVar(&decommissionVoteInterval, "decommission-vote-interval", 30*time.Second, "The time period between incrementing decommission vote counts since the last decommission conditions were met.") cmd.Flags().IntVar(&decommissionMaxVoteCount, "decommission-vote-count", 2, "The number of times that a vote must be tallied when a resource meets decommission conditions for it to actually be decommissioned.") - cmd.Flags().StringVar(&redpandaYAMLPath, "redpanda-yaml", "/etc/redpanda/redpanda.yaml", "Path to redpanda.yaml") + cmd.Flags().StringVar(&redpandaYAMLPath, "redpanda-yaml", "/etc/redpanda/redpanda.yaml", "Path to redpanda.yaml whose rpk stanza will be used for connecting to a Redpanda cluster.") return cmd } diff --git a/operator/config/rbac/bases/operator/role.yaml b/operator/config/rbac/bases/operator/role.yaml index f6b1a738a..77a197ce1 100644 --- a/operator/config/rbac/bases/operator/role.yaml +++ b/operator/config/rbac/bases/operator/role.yaml @@ -138,6 +138,18 @@ rules: - patch - update - watch +- apiGroups: + - coordination + resources: + - leases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch - apiGroups: - networking.k8s.io resources: @@ -259,12 +271,6 @@ rules: - patch - update - watch -- apiGroups: - - "" - resources: - - persistentvolumes - verbs: - - patch - apiGroups: - "" resources: @@ -331,6 +337,7 @@ rules: - update - watch - apiGroups: + - coordination - coordination.k8s.io resources: - leases diff --git a/operator/config/rbac/decommissioner-role/role.yaml b/operator/config/rbac/decommissioner-role/role.yaml index 297421727..f7a6256b8 100644 --- a/operator/config/rbac/decommissioner-role/role.yaml +++ b/operator/config/rbac/decommissioner-role/role.yaml @@ -43,6 +43,18 @@ rules: - get - list - watch +- apiGroups: + - coordination + resources: + - leases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch --- apiVersion: rbac.authorization.k8s.io/v1 kind: Role @@ -66,12 +78,6 @@ rules: - get - list - watch -- apiGroups: - - "" - resources: - - persistentvolumes - verbs: - - patch - apiGroups: - "" resources: @@ -89,3 +95,15 @@ rules: - get - list - watch +- apiGroups: + - coordination + resources: + - leases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch diff --git a/operator/internal/decommissioning/role.yaml b/operator/internal/decommissioning/role.yaml index 297421727..f7a6256b8 100644 --- a/operator/internal/decommissioning/role.yaml +++ b/operator/internal/decommissioning/role.yaml @@ -43,6 +43,18 @@ rules: - get - list - watch +- apiGroups: + - coordination + resources: + - leases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch --- apiVersion: rbac.authorization.k8s.io/v1 kind: Role @@ -66,12 +78,6 @@ rules: - get - list - watch -- apiGroups: - - "" - resources: - - persistentvolumes - verbs: - - patch - apiGroups: - "" resources: @@ -89,3 +95,15 @@ rules: - get - list - watch +- apiGroups: + - coordination + resources: + - leases + verbs: + - create + - delete + - get + - list + - patch + - update + - watch From 90ffa2bdb42a9ab02726b2dcc9d68688c355279a Mon Sep 17 00:00:00 2001 From: Andrew Stucki Date: Tue, 10 Dec 2024 12:32:37 -0500 Subject: [PATCH 13/13] fix group for leader election --- operator/config/rbac/bases/operator/role.yaml | 3 +-- operator/config/rbac/decommissioner-role/role.yaml | 4 ++-- operator/internal/decommissioning/role.yaml | 4 ++-- .../internal/decommissioning/statefulset_decomissioner.go | 4 ++-- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/operator/config/rbac/bases/operator/role.yaml b/operator/config/rbac/bases/operator/role.yaml index 77a197ce1..e6de8aeaa 100644 --- a/operator/config/rbac/bases/operator/role.yaml +++ b/operator/config/rbac/bases/operator/role.yaml @@ -139,7 +139,7 @@ rules: - update - watch - apiGroups: - - coordination + - coordination.k8s.io resources: - leases verbs: @@ -337,7 +337,6 @@ rules: - update - watch - apiGroups: - - coordination - coordination.k8s.io resources: - leases diff --git a/operator/config/rbac/decommissioner-role/role.yaml b/operator/config/rbac/decommissioner-role/role.yaml index f7a6256b8..648fd34de 100644 --- a/operator/config/rbac/decommissioner-role/role.yaml +++ b/operator/config/rbac/decommissioner-role/role.yaml @@ -44,7 +44,7 @@ rules: - list - watch - apiGroups: - - coordination + - coordination.k8s.io resources: - leases verbs: @@ -96,7 +96,7 @@ rules: - list - watch - apiGroups: - - coordination + - coordination.k8s.io resources: - leases verbs: diff --git a/operator/internal/decommissioning/role.yaml b/operator/internal/decommissioning/role.yaml index f7a6256b8..648fd34de 100644 --- a/operator/internal/decommissioning/role.yaml +++ b/operator/internal/decommissioning/role.yaml @@ -44,7 +44,7 @@ rules: - list - watch - apiGroups: - - coordination + - coordination.k8s.io resources: - leases verbs: @@ -96,7 +96,7 @@ rules: - list - watch - apiGroups: - - coordination + - coordination.k8s.io resources: - leases verbs: diff --git a/operator/internal/decommissioning/statefulset_decomissioner.go b/operator/internal/decommissioning/statefulset_decomissioner.go index 6d41991dd..e69186348 100644 --- a/operator/internal/decommissioning/statefulset_decomissioner.go +++ b/operator/internal/decommissioning/statefulset_decomissioner.go @@ -154,14 +154,14 @@ func NewStatefulSetDecommissioner(mgr ctrl.Manager, fetcher Fetcher, options ... // +kubebuilder:rbac:groups=core,resources=persistentvolumes,verbs=patch // +kubebuilder:rbac:groups=core,resources=events,verbs=create;patch // +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch -// +kubebuilder:rbac:groups=coordination,resources=leases,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=apps,namespace=default,resources=statefulsets,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;watch;delete // +kubebuilder:rbac:groups=core,namespace=default,resources=events,verbs=create;patch // +kubebuilder:rbac:groups=core,namespace=default,resources=secrets,verbs=get;list;watch -// +kubebuilder:rbac:groups=coordination,namespace=default,resources=leases,verbs=get;list;watch;create;update;patch;delete +// +kubebuilder:rbac:groups=coordination.k8s.io,namespace=default,resources=leases,verbs=get;list;watch;create;update;patch;delete func (s *StatefulSetDecomissioner) SetupWithManager(mgr ctrl.Manager) error { pvcPredicate, err := predicate.LabelSelectorPredicate(