Skip to content

Commit

Permalink
replace HelmRelease logic with SSA
Browse files Browse the repository at this point in the history
Prior to this commit the HelmRelease object was very carefully managed across
its lifecycle in a way that could be argued to be antithetical to the
Kubernetes mantra.

This logic, seemingly, was avoiding updates to the HelmRelease. I suspect this
may be due to an incorrect belief that the underlying flux controller would
aggressively roll out updates even if no such updates needed to be made.

This commit replaces all of this logic with a single Patch call using server
side apply, significantly improving the readability. Aside from the lack of
certain logs, there does not appear to be any observable difference in
behavior.
  • Loading branch information
chrisseto authored and RafalKorepta committed Oct 21, 2024
1 parent bc225a5 commit e1cebd3
Showing 1 changed file with 14 additions and 156 deletions.
170 changes: 14 additions & 156 deletions operator/internal/controller/redpanda/redpanda_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"encoding/base64"
"errors"
"fmt"
"reflect"
"sort"
"strings"
"time"
Expand All @@ -27,7 +26,6 @@ import (
"github.com/fluxcd/pkg/apis/meta"
"github.com/fluxcd/pkg/runtime/logger"
sourcev1 "github.com/fluxcd/source-controller/api/v1beta2"
"github.com/go-logr/logr"
appsv1 "k8s.io/api/apps/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
apimeta "k8s.io/apimachinery/pkg/api/meta"
Expand All @@ -41,7 +39,6 @@ import (
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
"sigs.k8s.io/controller-runtime/pkg/predicate"
v2 "sigs.k8s.io/controller-runtime/pkg/webhook/conversion/testdata/api/v2"

"github.com/redpanda-data/redpanda-operator/operator/api/redpanda/v1alpha2"
"github.com/redpanda-data/redpanda-operator/operator/pkg/resources"
Expand Down Expand Up @@ -299,25 +296,13 @@ func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpand
}

// Check if HelmRelease exists or create it also
rp, hr, err := r.reconcileHelmRelease(ctx, rp)
if err != nil {
return rp, err
}
if hr.Name == "" {
log.Info(fmt.Sprintf("Created HelmRelease for '%s/%s', will requeue", rp.Namespace, rp.Name))
if err := r.reconcileHelmRelease(ctx, rp); err != nil {
return rp, err
}

isGenerationCurrent := hr.Generation != hr.Status.ObservedGeneration
isStatusConditionReady := apimeta.IsStatusConditionTrue(hr.Status.Conditions, meta.ReadyCondition) || apimeta.IsStatusConditionTrue(hr.Status.Conditions, helmv2beta2.RemediatedCondition)
msgNotReady := fmt.Sprintf(resourceNotReadyStrFmt, resourceTypeHelmRelease, hr.GetNamespace(), hr.GetName())
msgReady := fmt.Sprintf(resourceReadyStrFmt, resourceTypeHelmRelease, hr.GetNamespace(), hr.GetName())
isStatusReadyNILorTRUE := ptr.Equal(rp.Status.HelmReleaseReady, ptr.To(true))
isStatusReadyNILorFALSE := ptr.Equal(rp.Status.HelmReleaseReady, ptr.To(false))

isResourceReady := r.checkIfResourceIsReady(log, msgNotReady, msgReady, resourceTypeHelmRelease, isGenerationCurrent, isStatusConditionReady, isStatusReadyNILorTRUE, isStatusReadyNILorFALSE, rp)
if !isResourceReady {
if !ptr.Deref(rp.Status.HelmReleaseReady, false) {
// strip out all of the requeues since this will get requeued based on the Owns in the setup of the reconciler
msgNotReady := fmt.Sprintf(resourceNotReadyStrFmt, resourceTypeHelmRelease, rp.GetNamespace(), rp.GetHelmReleaseName())
return v1alpha2.RedpandaNotReady(rp, "ArtifactFailed", msgNotReady), nil
}

Expand All @@ -344,78 +329,23 @@ func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpand
return v1alpha2.RedpandaReady(rp), nil
}

func (r *RedpandaReconciler) checkIfResourceIsReady(log logr.Logger, msgNotReady, msgReady, kind string, isGenerationCurrent, isStatusConditionReady, isStatusReadyNILorTRUE, isStatusReadyNILorFALSE bool, rp *v1alpha2.Redpanda) bool {
if isGenerationCurrent || !isStatusConditionReady {
// capture event only
if isStatusReadyNILorTRUE {
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityInfo, msgNotReady)
}

switch kind {
case resourceTypeHelmRelease:
rp.Status.HelmReleaseReady = ptr.To(false)
}

log.Info(msgNotReady)
return false
} else if isStatusConditionReady && isStatusReadyNILorFALSE {
// here since the condition should be true, we update the value to
// be true, and send an event
switch kind {
case resourceTypeHelmRelease:
rp.Status.HelmReleaseReady = ptr.To(true)
}

r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityInfo, msgReady)
}

return true
}

func (r *RedpandaReconciler) reconcileHelmRelease(ctx context.Context, rp *v1alpha2.Redpanda) (*v1alpha2.Redpanda, *helmv2beta2.HelmRelease, error) {
var err error

// Check if HelmRelease exists or create it
hr := &helmv2beta2.HelmRelease{}

// have we recorded a helmRelease, if not assume we have not created it
if rp.Status.HelmRelease == "" {
// did not find helmRelease, then create it
hr, err = r.createHelmRelease(ctx, rp)
return rp, hr, err
}

// if we are not empty, then we assume at some point this existed, let's check
key := types.NamespacedName{Namespace: rp.Namespace, Name: rp.Status.GetHelmRelease()}
err = r.Client.Get(ctx, key, hr)
func (r *RedpandaReconciler) reconcileHelmRelease(ctx context.Context, rp *v1alpha2.Redpanda) error {
hr, err := r.createHelmReleaseFromTemplate(ctx, rp)
if err != nil {
if apierrors.IsNotFound(err) {
rp.Status.HelmRelease = ""
hr, err = r.createHelmRelease(ctx, rp)
return rp, hr, err
}
// if this is a not found error
return rp, hr, fmt.Errorf("failed to get HelmRelease '%s/%s': %w", rp.Namespace, rp.Status.HelmRelease, err)
return err
}

// Check if we need to update here
hrTemplate, errTemplated := r.createHelmReleaseFromTemplate(ctx, rp)
if errTemplated != nil {
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityError, errTemplated.Error())
return rp, hr, errTemplated
if err := r.apply(ctx, hr); err != nil {
return err
}

if r.helmReleaseRequiresUpdate(ctx, hr, hrTemplate) {
hr.Spec = hrTemplate.Spec
if err = r.Client.Update(ctx, hr); err != nil {
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityError, err.Error())
return rp, hr, err
}
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityInfo, fmt.Sprintf("HelmRelease '%s/%s' updated", rp.Namespace, rp.GetHelmReleaseName()))
rp.Status.HelmRelease = rp.GetHelmReleaseName()
}
isGenerationCurrent := hr.Generation == hr.Status.ObservedGeneration
isStatusConditionReady := apimeta.IsStatusConditionTrue(hr.Status.Conditions, meta.ReadyCondition) || apimeta.IsStatusConditionTrue(hr.Status.Conditions, helmv2beta2.RemediatedCondition)

return rp, hr, nil
rp.Status.HelmRelease = hr.Name
rp.Status.HelmReleaseReady = ptr.To(isGenerationCurrent && isStatusConditionReady)

return nil
}

func (r *RedpandaReconciler) reconcileHelmRepository(ctx context.Context, rp *v1alpha2.Redpanda) error {
Expand Down Expand Up @@ -447,31 +377,6 @@ func (r *RedpandaReconciler) reconcileDelete(ctx context.Context, rp *v1alpha2.R
return ctrl.Result{}, nil
}

func (r *RedpandaReconciler) createHelmRelease(ctx context.Context, rp *v1alpha2.Redpanda) (*helmv2beta2.HelmRelease, error) {
// create helmRelease resource from template
hRelease, err := r.createHelmReleaseFromTemplate(ctx, rp)
if err != nil {
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityError, fmt.Sprintf("could not create helm release template: %s", err))
return hRelease, fmt.Errorf("could not create HelmRelease template: %w", err)
}

// create helmRelease object here
if err := r.Client.Create(ctx, hRelease); err != nil {
if !apierrors.IsAlreadyExists(err) {
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityError, err.Error())
return hRelease, fmt.Errorf("failed to create HelmRelease '%s/%s': %w", rp.Namespace, rp.Status.HelmRelease, err)
}
// we already exist, then update the status to rp
rp.Status.HelmRelease = rp.GetHelmReleaseName()
}

// we have created the resource, so we are ok to update events, and update the helmRelease name on the status object
r.event(rp, rp.Status.LastAttemptedRevision, v1alpha2.EventSeverityInfo, fmt.Sprintf("HelmRelease '%s/%s' created ", rp.Namespace, rp.GetHelmReleaseName()))
rp.Status.HelmRelease = rp.GetHelmReleaseName()

return hRelease, nil
}

func (r *RedpandaReconciler) deleteHelmRelease(ctx context.Context, rp *v1alpha2.Redpanda) error {
if rp.Status.HelmRelease == "" {
return nil
Expand Down Expand Up @@ -597,37 +502,6 @@ func (r *RedpandaReconciler) patchRedpandaStatus(ctx context.Context, rp *v1alph
return r.Client.Status().Patch(ctx, rp, client.MergeFrom(latest))
}

// event emits a Kubernetes event and forwards the event to notification controller if configured.
func (r *RedpandaReconciler) event(rp *v1alpha2.Redpanda, revision, severity, msg string) {
var metaData map[string]string
if revision != "" {
metaData = map[string]string{v2.GroupVersion.Group + revisionPath: revision}
}
eventType := "Normal"
if severity == v1alpha2.EventSeverityError {
eventType = "Warning"
}
r.EventRecorder.AnnotatedEventf(rp, metaData, eventType, severity, msg)
}

func (r *RedpandaReconciler) helmReleaseRequiresUpdate(ctx context.Context, hr, hrTemplate *helmv2beta2.HelmRelease) bool {
log := ctrl.LoggerFrom(ctx).WithName("RedpandaReconciler.helmReleaseRequiresUpdate")

switch {
case !reflect.DeepEqual(hr.GetValues(), hrTemplate.GetValues()):
log.Info("values found different")
return true
case helmChartRequiresUpdate(log, &hr.Spec.Chart, &hrTemplate.Spec.Chart):
log.Info("chartTemplate found different")
return true
case hr.Spec.Interval != hrTemplate.Spec.Interval:
log.Info("interval found different")
return true
default:
return false
}
}

func (r *RedpandaReconciler) apply(ctx context.Context, obj client.Object) error {
gvk, err := r.Client.GroupVersionKindFor(obj)
if err != nil {
Expand All @@ -640,22 +514,6 @@ func (r *RedpandaReconciler) apply(ctx context.Context, obj client.Object) error
return r.Client.Patch(ctx, obj, client.Apply, client.ForceOwnership, client.FieldOwner("redpanda-operator"))
}

// helmChartRequiresUpdate compares the v2beta1.HelmChartTemplate of the
// v2beta1.HelmRelease to the given v1beta2.HelmChart to determine if an
// update is required.
func helmChartRequiresUpdate(log logr.Logger, template, chart *helmv2beta2.HelmChartTemplate) bool {
switch {
case template.Spec.Chart != chart.Spec.Chart:
log.Info("chart is different")
return true
case template.Spec.Version != "" && template.Spec.Version != chart.Spec.Version:
log.Info("spec version is different")
return true
default:
return false
}
}

func isRedpandaManaged(ctx context.Context, redpandaCluster *v1alpha2.Redpanda) bool {
log := ctrl.LoggerFrom(ctx).WithName("RedpandaReconciler.isRedpandaManaged")

Expand Down

0 comments on commit e1cebd3

Please sign in to comment.