Skip to content

Commit

Permalink
Add Redpanda CRD flag that switch of Flux reconciliation
Browse files Browse the repository at this point in the history
New `Usefulx` boolean flag can turn off Flux reconciliation of HelmRelease and
HelmRepository via `suspend` flag in both resources. Regardless of the Flux
controllers execution (inside or outside Redpanda operator) Redpanda deployment
can be unchanged which gives users not only ability to interveen if something
break, but it can act as switch to new execution model without flux controllers.

Before setting UseFlux flag to `false` users need to alight ChartVersion to at
least `5.9.3` version of the Redpanda chart.

The Redpanda reconciller can leave behind Flux custom resources (HelmRelease,
HelmChart and HelmRepository) and take ownership of all resources generated by
the helm chart. When suspend flag is enabled on mentioned resources user could
potenetially delete HelmRelease (using kubectl), but all resources would be
unchanged. Current Redpanda operator implementation would quickly re-create
HelmRelease CR, but that would still not affect any K8S primitive resource. The
analogy could be `kubectl delete --cascade=orphan ...` where main resource is
deleted, but the child resources are left unchanged.

Reference:
https://fluxcd.io/flux/components/helm/helmreleases/#suspend
https://fluxcd.io/flux/components/source/helmrepositories/#suspend
  • Loading branch information
RafalKorepta committed Sep 24, 2024
1 parent ec8c1e6 commit 5536cca
Show file tree
Hide file tree
Showing 5 changed files with 163 additions and 19 deletions.
16 changes: 16 additions & 0 deletions src/go/k8s/api/redpanda/v1alpha2/redpanda_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,22 @@ type ChartRef struct {
Timeout *metav1.Duration `json:"timeout,omitempty"`
// Defines how to handle upgrades, including failures.
Upgrade *HelmUpgrade `json:"upgrade,omitempty"`
// NOTE! Alpha feature
// UseFlux flag set to `false` will prevent helm controller from reconciling helm chart. The operator would be
// tight with `go` based Redpanda helm chart version. The rest of the ChartRef fields would be ignored.
//
// Before setting UseFlux flag to `false` please alight your ChartVersion to at least `5.9.3`
// version of the Redpanda chart.
//
// RedpandaStatus might not be accurate if flag is set to `false` and HelmRelease is manually deleted.
//
// To achieve dynamic switch for Flux controllers (HelmRelease and HelmRepository) the resources
// would not be removed, but they will be put in suspended mode (if flag is provided and set to `false`).
//
// https://fluxcd.io/flux/components/helm/helmreleases/#suspend
// https://fluxcd.io/flux/components/source/helmrepositories/#suspend
// +optional
UseFlux *bool `json:"useFlux,omitempty"`
}

// RedpandaSpec defines the desired state of the Redpanda cluster.
Expand Down
5 changes: 5 additions & 0 deletions src/go/k8s/api/redpanda/v1alpha2/zz_generated.deepcopy.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

34 changes: 34 additions & 0 deletions src/go/k8s/config/crd/bases/cluster.redpanda.com_redpandas.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,23 @@ spec:
type: string
type: object
type: object
useFlux:
description: |-
NOTE! Alpha feature
UseFlux flag set to `false` will prevent helm controller from reconciling helm chart. The operator would be
tight with `go` based Redpanda helm chart version. The rest of the ChartRef fields would be ignored.

Before setting UseFlux flag to `false` please alight your ChartVersion to at least `5.9.3`
version of the Redpanda chart.

RedpandaStatus might not be accurate if flag is set to `false` and HelmRelease is manually deleted.

To achieve dynamic switch for Flux controllers (HelmRelease and HelmRepository) the resources
would not be removed, but they will be put in suspended mode (if flag is provided and set to `false`).

https://fluxcd.io/flux/components/helm/helmreleases/#suspend
https://fluxcd.io/flux/components/source/helmrepositories/#suspend
type: boolean
type: object
clusterSpec:
description: Defines the Helm values to use to deploy the cluster.
Expand Down Expand Up @@ -9873,6 +9890,23 @@ spec:
type: string
type: object
type: object
useFlux:
description: |-
NOTE! Alpha feature
UseFlux flag set to `false` will prevent helm controller from reconciling helm chart. The operator would be
tight with `go` based Redpanda helm chart version. The rest of the ChartRef fields would be ignored.

Before setting UseFlux flag to `false` please alight your ChartVersion to at least `5.9.3`
version of the Redpanda chart.

RedpandaStatus might not be accurate if flag is set to `false` and HelmRelease is manually deleted.

To achieve dynamic switch for Flux controllers (HelmRelease and HelmRepository) the resources
would not be removed, but they will be put in suspended mode (if flag is provided and set to `false`).

https://fluxcd.io/flux/components/helm/helmreleases/#suspend
https://fluxcd.io/flux/components/source/helmrepositories/#suspend
type: boolean
type: object
clusterSpec:
description: Defines the Helm values to use to deploy the cluster.
Expand Down
37 changes: 37 additions & 0 deletions src/go/k8s/config/crd/bases/redpanda_controller_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// 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 redpanda

import (
_ "embed"
"encoding/json"
"testing"

"github.com/redpanda-data/redpanda-operator/src/go/k8s/internal/controller/redpanda"
"github.com/stretchr/testify/require"
v1 "k8s.io/apiextensions-apiserver/pkg/apis/apiextensions/v1"
"k8s.io/apimachinery/pkg/util/yaml"
)

//go:embed cluster.redpanda.com_redpandas.yaml
var redpandaCRD []byte

func TestDefluxedMinimumVersion(t *testing.T) {
crd := v1.CustomResourceDefinition{}

red, err := yaml.ToJSON(redpandaCRD)
require.NoError(t, err)

require.NoError(t, json.Unmarshal(red, &crd))
// 0 is v1alpha1 and 1 is v1alpha2
recursiveProperties := crd.Spec.Versions[1].Schema.OpenAPIV3Schema.Properties

require.Contains(t, recursiveProperties["spec"].Properties["chartRef"].Properties["useFlux"].Description, redpanda.HelmChartConstraint)
}
90 changes: 71 additions & 19 deletions src/go/k8s/internal/controller/redpanda/redpanda_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"strings"
"time"

"github.com/Masterminds/semver/v3"
helmv2beta1 "github.com/fluxcd/helm-controller/api/v2beta1"
helmv2beta2 "github.com/fluxcd/helm-controller/api/v2beta2"
"github.com/fluxcd/pkg/apis/meta"
Expand Down Expand Up @@ -58,6 +59,8 @@ const (

revisionPath = "/revision"
componentLabelValue = "redpanda-statefulset"

HelmChartConstraint = "5.9.3"
)

var errWaitForReleaseDeletion = errors.New("wait for helm release deletion")
Expand Down Expand Up @@ -168,6 +171,11 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr
start := time.Now()
log := ctrl.LoggerFrom(ctx).WithName("RedpandaReconciler.Reconcile")

defer func() {
durationMsg := fmt.Sprintf("reconciliation finished in %s", time.Since(start).String())
log.Info(durationMsg)
}()

log.Info("Starting reconcile loop")

rp := &v1alpha2.Redpanda{}
Expand Down Expand Up @@ -208,22 +216,64 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr
}
}

rp, result, err := r.reconcile(ctx, rp)
rp, err := r.reconcile(ctx, rp)
if err != nil {
return ctrl.Result{}, err
}

err = r.reconcileDefluxed(ctx, rp)
if err != nil {
return ctrl.Result{}, err
}

// Update status after reconciliation.
if updateStatusErr := r.patchRedpandaStatus(ctx, rp); updateStatusErr != nil {
log.Error(updateStatusErr, "unable to update status after reconciliation")
return ctrl.Result{}, updateStatusErr
}

// Log reconciliation duration
durationMsg := fmt.Sprintf("reconciliation finished in %s", time.Since(start).String())
log.Info(durationMsg)
return ctrl.Result{}, nil
}

func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2.Redpanda) error {
log := ctrl.LoggerFrom(ctx)
log.WithName("RedpandaReconciler.reconcileDefluxed")

if !ptr.Deref(rp.Spec.ChartRef.UseFlux, true) {
// TODO (Rafal) Implement Redpanda helm chart templating with Redpanda Status Report
// In the Redpanda.Status there will be only Conditions and Failures that would be used.

if !atLeast(rp.Spec.ChartRef.ChartVersion) {
log.Error(fmt.Errorf("chart version needs to be at least %s", HelmChartConstraint), "", "chart version", rp.Spec.ChartRef.ChartVersion)
v1alpha2.RedpandaNotReady(rp, "ChartRefUnsupported", fmt.Sprintf("chart version needs to be at least %s. Currently it is %s", HelmChartConstraint, rp.Spec.ChartRef.ChartVersion))
r.EventRecorder.Eventf(rp, "Warning", v1alpha2.EventSeverityError, fmt.Sprintf("chart version needs to be at least %s. Currently it is %s", HelmChartConstraint, rp.Spec.ChartRef.ChartVersion))
// Do not error out to not requeue. User needs to first migrate helm release to at least 5.9.3 version
return nil
}
}
return nil
}

func atLeast(version string) bool {
if version == "" {
return true
}

c, err := semver.NewConstraint(fmt.Sprintf(">= %s", HelmChartConstraint))
if err != nil {
// Handle constraint not being parsable.
return false
}

v, err := semver.NewVersion(version)
if err != nil {
return false
}

return result, err
return c.Check(v)
}

func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpanda) (*v1alpha2.Redpanda, ctrl.Result, error) {
func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpanda) (*v1alpha2.Redpanda, error) {
log := ctrl.LoggerFrom(ctx)
log.WithName("RedpandaReconciler.reconcile")

Expand All @@ -233,24 +283,24 @@ func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpand
rp = v1alpha2.RedpandaProgressing(rp)
if updateStatusErr := r.patchRedpandaStatus(ctx, rp); updateStatusErr != nil {
log.Error(updateStatusErr, "unable to update status after generation update")
return rp, ctrl.Result{}, updateStatusErr
return rp, updateStatusErr
}
}

// pull our deployments and stateful sets
redpandaStatefulSets, err := redpandaStatefulSetsForCluster(ctx, r.Client, rp)
if err != nil {
return rp, ctrl.Result{}, err
return rp, err
}
consoleDeployments, err := consoleDeploymentsForCluster(ctx, r.Client, rp)
if err != nil {
return rp, ctrl.Result{}, err
return rp, err
}

// Check if HelmRepository exists or create it
rp, repo, err := r.reconcileHelmRepository(ctx, rp)
if err != nil {
return rp, ctrl.Result{}, err
return rp, err
}

isGenerationCurrent := repo.Generation != repo.Status.ObservedGeneration
Expand All @@ -263,17 +313,17 @@ func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpand
isResourceReady := r.checkIfResourceIsReady(log, msgNotReady, msgReady, resourceTypeHelmRepository, isGenerationCurrent, isStatusConditionReady, isStatusReadyNILorTRUE, isStatusReadyNILorFALSE, rp)
if !isResourceReady {
// strip out all of the requeues since this will get requeued based on the Owns in the setup of the reconciler
return v1alpha2.RedpandaNotReady(rp, "ArtifactFailed", msgNotReady), ctrl.Result{}, nil
return v1alpha2.RedpandaNotReady(rp, "ArtifactFailed", msgNotReady), nil
}

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

isGenerationCurrent = hr.Generation != hr.Status.ObservedGeneration
Expand All @@ -286,30 +336,30 @@ func (r *RedpandaReconciler) reconcile(ctx context.Context, rp *v1alpha2.Redpand
isResourceReady = r.checkIfResourceIsReady(log, msgNotReady, msgReady, resourceTypeHelmRelease, isGenerationCurrent, isStatusConditionReady, isStatusReadyNILorTRUE, isStatusReadyNILorFALSE, rp)
if !isResourceReady {
// strip out all of the requeues since this will get requeued based on the Owns in the setup of the reconciler
return v1alpha2.RedpandaNotReady(rp, "ArtifactFailed", msgNotReady), ctrl.Result{}, nil
return v1alpha2.RedpandaNotReady(rp, "ArtifactFailed", msgNotReady), nil
}

for _, sts := range redpandaStatefulSets {
decommission, err := needsDecommission(ctx, sts, log)
if err != nil {
return rp, ctrl.Result{}, err
return rp, err
}
if decommission {
return v1alpha2.RedpandaNotReady(rp, "RedpandaPodsNotReady", "Cluster currently decommissioning dead nodes"), ctrl.Result{}, nil
return v1alpha2.RedpandaNotReady(rp, "RedpandaPodsNotReady", "Cluster currently decommissioning dead nodes"), nil
}
}

// check to make sure that our stateful set pods are all current
if message, ready := checkStatefulSetStatus(redpandaStatefulSets); !ready {
return v1alpha2.RedpandaNotReady(rp, "RedpandaPodsNotReady", message), ctrl.Result{}, nil
return v1alpha2.RedpandaNotReady(rp, "RedpandaPodsNotReady", message), nil
}

// check to make sure that our deployment pods are all current
if message, ready := checkDeploymentsStatus(consoleDeployments); !ready {
return v1alpha2.RedpandaNotReady(rp, "ConsolePodsNotReady", message), ctrl.Result{}, nil
return v1alpha2.RedpandaNotReady(rp, "ConsolePodsNotReady", message), nil
}

return v1alpha2.RedpandaReady(rp), ctrl.Result{}, nil
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 {
Expand Down Expand Up @@ -529,6 +579,7 @@ func (r *RedpandaReconciler) createHelmReleaseFromTemplate(ctx context.Context,
OwnerReferences: []metav1.OwnerReference{rp.OwnerShipRefObj()},
},
Spec: helmv2beta2.HelmReleaseSpec{
Suspend: !ptr.Deref(rp.Spec.ChartRef.UseFlux, true),
Chart: helmv2beta2.HelmChartTemplate{
Spec: helmv2beta2.HelmChartTemplateSpec{
Chart: "redpanda",
Expand Down Expand Up @@ -557,6 +608,7 @@ func (r *RedpandaReconciler) createHelmRepositoryFromTemplate(rp *v1alpha2.Redpa
OwnerReferences: []metav1.OwnerReference{rp.OwnerShipRefObj()},
},
Spec: sourcev1.HelmRepositorySpec{
Suspend: !ptr.Deref(rp.Spec.ChartRef.UseFlux, true),
Interval: metav1.Duration{Duration: 30 * time.Second},
URL: v1alpha2.RedpandaChartRepository,
},
Expand Down

0 comments on commit 5536cca

Please sign in to comment.