Skip to content

Commit

Permalink
Add license status to cluster output (#312)
Browse files Browse the repository at this point in the history
* Add license status to cluster output

* Add new condition to e2e tests

* Attempt to fix failing tests

* Reorganize statuses since kuttl assertions are order-dependent

* change assertion that fails non-deterministically when the previous assertions run too slowly

* rev failure attempts due to extra time

* remove condition assertions since they are reflected in other status fields

* Convert upgrade rollback test to go code
  • Loading branch information
andrewstucki authored Nov 19, 2024
1 parent 585a918 commit 6b168a6
Show file tree
Hide file tree
Showing 25 changed files with 285 additions and 228 deletions.
4 changes: 4 additions & 0 deletions operator/api/redpanda/v1alpha2/redpanda_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,9 @@ const (
// ClusterConfigSynced is a condition indicating whether or not the
// redpanda cluster's configuration is up to date with the desired config.
ClusterConfigSynced = "ClusterConfigSynced"
// ClusterLicenseValid is a condition indicating whether or not the
// redpanda cluster has a valid license.
ClusterLicenseValid = "ClusterLicenseValid"
)

type ChartRef struct {
Expand Down Expand Up @@ -160,6 +163,7 @@ type HelmUpgrade struct {
// +kubebuilder:subresource:status
// +kubebuilder:resource:path=redpandas
// +kubebuilder:resource:shortName=rp
// +kubebuilder:printcolumn:name="License",type="string",JSONPath=`.status.conditions[?(@.type=="ClusterLicenseValid")].message`,description=""
// +kubebuilder:printcolumn:name="Ready",type="string",JSONPath=".status.conditions[?(@.type==\"Ready\")].status",description=""
// +kubebuilder:printcolumn:name="Status",type="string",JSONPath=".status.conditions[?(@.type==\"Ready\")].message",description=""
// +kubebuilder:storageversion
Expand Down
3 changes: 3 additions & 0 deletions operator/config/crd/bases/cluster.redpanda.com_redpandas.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -9804,6 +9804,9 @@ spec:
subresources:
status: {}
- additionalPrinterColumns:
- jsonPath: .status.conditions[?(@.type=="ClusterLicenseValid")].message
name: License
type: string
- jsonPath: .status.conditions[?(@.type=="Ready")].status
name: Ready
type: string
Expand Down
117 changes: 93 additions & 24 deletions operator/internal/controller/redpanda/redpanda_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import (
"sigs.k8s.io/controller-runtime/pkg/predicate"
"sigs.k8s.io/yaml"

"github.com/redpanda-data/common-go/rpadmin"
"github.com/redpanda-data/helm-charts/charts/redpanda"
"github.com/redpanda-data/helm-charts/pkg/gotohelm/helmette"
"github.com/redpanda-data/helm-charts/pkg/kube"
Expand Down Expand Up @@ -174,10 +175,10 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr

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

log.Info("Starting reconcile loop")
log.V(logger.TraceLevel).Info("Starting reconcile loop")

rp := &v1alpha2.Redpanda{}
if err := r.Client.Get(ctx, req.NamespacedName, rp); err != nil {
Expand All @@ -203,7 +204,7 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr

_, ok := rp.GetAnnotations()[resources.ManagedDecommissionAnnotation]
if ok {
log.Info("Managed decommission")
log.V(logger.TraceLevel).Info("Managed decommission")
return ctrl.Result{}, nil
}

Expand All @@ -230,6 +231,10 @@ func (r *RedpandaReconciler) Reconcile(c context.Context, req ctrl.Request) (ctr
return ctrl.Result{}, err
}

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

if err := r.reconcileClusterConfig(ctx, rp); err != nil {
return ctrl.Result{}, err
}
Expand Down Expand Up @@ -310,7 +315,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2
log := ctrl.LoggerFrom(ctx)

if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) {
log.Info("useFlux is true; skipping non-flux reconciliation...")
log.V(logger.TraceLevel).Info("useFlux is true; skipping non-flux reconciliation...")
return nil
}

Expand Down Expand Up @@ -373,7 +378,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2
obj.SetAnnotations(annos)

if _, ok := annos["helm.sh/hook"]; ok {
log.Info(fmt.Sprintf("skipping helm hook %T: %q", obj, obj.GetName()))
log.V(logger.TraceLevel).Info(fmt.Sprintf("skipping helm hook %T: %q", obj, obj.GetName()))
continue
}

Expand All @@ -396,7 +401,7 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2
// to be a no-op.
// This check could likely be hoisted above the deployment loop as well.
if rp.Generation == rp.Status.ObservedGeneration && rp.Generation != 0 {
log.Info("observed generation is up to date. skipping garbage collection", "generation", rp.Generation, "observedGeneration", rp.Status.ObservedGeneration)
log.V(logger.TraceLevel).Info("observed generation is up to date. skipping garbage collection", "generation", rp.Generation, "observedGeneration", rp.Status.ObservedGeneration)
return nil
}

Expand All @@ -408,24 +413,13 @@ func (r *RedpandaReconciler) reconcileDefluxed(ctx context.Context, rp *v1alpha2
return nil
}

func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1alpha2.Redpanda) error {
func (r *RedpandaReconciler) ratelimitCondition(ctx context.Context, rp *v1alpha2.Redpanda, conditionType string) bool {
log := ctrl.LoggerFrom(ctx)

if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) {
apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{
Type: v1alpha2.ClusterConfigSynced,
Status: metav1.ConditionUnknown,
ObservedGeneration: rp.Generation,
Reason: "HandledByFlux",
Message: "cluster configuration is not managed by the operator when Flux is enabled",
})
return nil
}

cond := apimeta.FindStatusCondition(rp.Status.Conditions, v1alpha2.ClusterConfigSynced)
cond := apimeta.FindStatusCondition(rp.Status.Conditions, conditionType)
if cond == nil {
cond = &metav1.Condition{
Type: v1alpha2.ClusterConfigSynced,
Type: conditionType,
Status: metav1.ConditionUnknown,
}
}
Expand All @@ -437,15 +431,15 @@ func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1a
// NB: This controller re-queues fairly frequently as is (Watching STS
// which watches Pods), so we're largely relying on that to ensure we eventually run our rechecks.
if previouslySynced && !(generationChanged || recheck) {
return nil
return true
}

redpandaReady := !apimeta.IsStatusConditionTrue(rp.Status.Conditions, meta.ReadyCondition)
redpandaReady := apimeta.IsStatusConditionTrue(rp.Status.Conditions, meta.ReadyCondition)

if !(rp.GenerationObserved() || redpandaReady) {
log.Info("redpanda not yet ready. skipping cluster config reconciliation.")
log.V(logger.TraceLevel).Info(fmt.Sprintf("redpanda not yet ready. skipping %s reconciliation.", conditionType))
apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{
Type: v1alpha2.ClusterConfigSynced,
Type: conditionType,
Status: metav1.ConditionUnknown,
ObservedGeneration: rp.Generation,
Reason: "RedpandaNotReady",
Expand All @@ -454,6 +448,81 @@ func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1a

// NB: Redpanda becoming ready and/or observing it's generation will
// trigger a re-queue for us.
return true
}

return false
}

func (r *RedpandaReconciler) reconcileLicense(ctx context.Context, rp *v1alpha2.Redpanda) error {
if r.ratelimitCondition(ctx, rp, v1alpha2.ClusterLicenseValid) {
return nil
}

client, err := r.ClientFactory.RedpandaAdminClient(ctx, rp)
if err != nil {
return err
}

features, err := client.GetEnterpriseFeatures(ctx)
if err != nil {
if internalclient.IsTerminalClientError(err) {
apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{
Type: v1alpha2.ClusterLicenseValid,
Status: metav1.ConditionUnknown,
ObservedGeneration: rp.Generation,
Reason: "TerminalError",
Message: err.Error(),
})

return nil
}
return err
}

var message string
var reason string
status := metav1.ConditionUnknown

switch features.LicenseStatus {
case rpadmin.LicenseStatusExpired:
status = metav1.ConditionFalse
reason = "LicenseExpired"
message = "Expired"
case rpadmin.LicenseStatusNotPresent:
status = metav1.ConditionFalse
reason = "LicenseNotPresent"
message = "Not Present"
case rpadmin.LicenseStatusValid:
status = metav1.ConditionTrue
reason = "LicenseValid"
message = "Valid"
}

apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{
Type: v1alpha2.ClusterLicenseValid,
Status: status,
ObservedGeneration: rp.Generation,
Reason: reason,
Message: message,
})

return nil
}

func (r *RedpandaReconciler) reconcileClusterConfig(ctx context.Context, rp *v1alpha2.Redpanda) error {
if ptr.Deref(rp.Spec.ChartRef.UseFlux, true) {
apimeta.SetStatusCondition(rp.GetConditions(), metav1.Condition{
Type: v1alpha2.ClusterConfigSynced,
Status: metav1.ConditionUnknown,
ObservedGeneration: rp.Generation,
Reason: "HandledByFlux",
Message: "cluster configuration is not managed by the operator when Flux is enabled",
})
return nil
}

if r.ratelimitCondition(ctx, rp, v1alpha2.ClusterConfigSynced) {
return nil
}

Expand Down
Loading

0 comments on commit 6b168a6

Please sign in to comment.