Skip to content

Commit

Permalink
operator v1: store NodePoolSpec in STS annotations
Browse files Browse the repository at this point in the history
we encountered corner cases, where it becomes extremely difficult to
synthesize a NodePoolSpec just by looking at the StatefulSet - which is
our fallback, if a nodePool was removed from the spec.
AdditionalCommandlineArguments is hard to reconstruct, because we'd need
to pull out of of the args field in the pod spec of the STS, removing
all "other default" args - very error prone.

Instead, we now store the NodePoolSpec used to create the STS in the STS
as an annotation. This way we can always find the NodePoolSpec to create
the (deleted) STS.

In addition, we take this chance to remove small special cases for
handling delete nodepools:
- Do not set replicas=currentReplicas anymore. It was more of a trick.
  Instead, we now set for a deleted nodePool replicas=0, which exactly
  represents what should happen with it (scale down to zero).
- Add check for Deleted bool in scale-down handler. It prevented
  replicas=currentReplicas being accepted as "do notthing"
  if it's a deleted nodepool. Then, the control flow would proceed and
  downscaling happens. This was not very explicit and very hard to find
  out, why downscale even works in deleted NodePools. With the refactor,
  replicas is 0, and no special case is needed for deleting anymore.
  • Loading branch information
birdayz committed Nov 21, 2024
1 parent 2880652 commit fc0b719
Show file tree
Hide file tree
Showing 4 changed files with 25 additions and 64 deletions.
5 changes: 5 additions & 0 deletions operator/pkg/labels/labels.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,11 @@ const (
// PodNodeIDKey is used to store the Redpanda NodeID of this pod.
PodNodeIDKey = "operator.redpanda.com/node-id"

// NodePoolSpecKey is used to store the NodePoolSpec in a StatefulSet's annotations.
// This allows the operator to correctly reconstruct a NodePoolSpec even
// after it was removed from Spec already.
NodePoolSpecKey = "cluster.redpanda.com/node-pool-spec"

nameKeyRedpandaVal = "redpanda"
nameKeyConsoleVal = "redpanda-console"
managedByOperatorVal = "redpanda-operator"
Expand Down
73 changes: 11 additions & 62 deletions operator/pkg/nodepools/pools.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,14 @@ package nodepools

import (
"context"
"encoding/json"
"fmt"
"slices"
"strings"

vectorizedv1alpha1 "github.com/redpanda-data/redpanda-operator/operator/api/vectorized/v1alpha1"
"github.com/redpanda-data/redpanda-operator/operator/pkg/labels"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/api/resource"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/utils/ptr"
"sigs.k8s.io/controller-runtime/pkg/client"
Expand Down Expand Up @@ -89,70 +88,20 @@ outer:
continue
}

replicas := sts.Spec.Replicas
if st, ok := cluster.Status.NodePools[npName]; ok {
replicas = &st.CurrentReplicas
}

var redpandaContainer *corev1.Container
for i := range sts.Spec.Template.Spec.Containers {
container := sts.Spec.Template.Spec.Containers[i]
if container.Name == "redpanda" {
redpandaContainer = &container
break
var np vectorizedv1alpha1.NodePoolSpec
if nodePoolSpecJSON, ok := sts.Annotations[labels.NodePoolSpecKey]; ok {
if err := json.Unmarshal([]byte(nodePoolSpecJSON), &np); err != nil {
return nil, fmt.Errorf("failed to synthesize deleted nodePool %s from its annotation %s", npName, labels.NodePoolSpecKey)
}
}
if redpandaContainer == nil {
return nil, fmt.Errorf("redpanda container not defined in STS %s template", sts.Name)
}

var datadirVcCapacity resource.Quantity
var datadirVcStorageClassName string

var cacheVcExists bool
var cacheVcCapacity resource.Quantity
var cacheVcStorageClassName string

for i := range sts.Spec.VolumeClaimTemplates {
vct := sts.Spec.VolumeClaimTemplates[i]
if vct.Name == "datadir" {
datadirVcCapacity = vct.Spec.Resources.Requests[corev1.ResourceStorage]
if vct.Spec.StorageClassName != nil {
datadirVcStorageClassName = ptr.Deref(vct.Spec.StorageClassName, "")
}
}
if vct.Name == "shadow-index-cache" {
cacheVcExists = true
cacheVcCapacity = vct.Spec.Resources.Requests[corev1.ResourceStorage]
if vct.Spec.StorageClassName != nil {
cacheVcStorageClassName = ptr.Deref(vct.Spec.StorageClassName, "")
}
}
}
// Desired replicas for deleted NodePools is always zero.
np.Replicas = ptr.To(int32(0))

np := vectorizedv1alpha1.NodePoolSpecWithDeleted{
NodePoolSpec: vectorizedv1alpha1.NodePoolSpec{
Name: npName,
Replicas: replicas,
Resources: vectorizedv1alpha1.RedpandaResourceRequirements{
ResourceRequirements: redpandaContainer.Resources,
},
Tolerations: sts.Spec.Template.Spec.Tolerations,
NodeSelector: sts.Spec.Template.Spec.NodeSelector,
Storage: vectorizedv1alpha1.StorageSpec{
Capacity: datadirVcCapacity,
StorageClassName: datadirVcStorageClassName,
},
},
Deleted: true,
}
if cacheVcExists {
np.CloudCacheStorage = vectorizedv1alpha1.StorageSpec{
Capacity: cacheVcCapacity,
StorageClassName: cacheVcStorageClassName,
}
}
nodePoolsWithDeleted = append(nodePoolsWithDeleted, &np)
nodePoolsWithDeleted = append(nodePoolsWithDeleted, &vectorizedv1alpha1.NodePoolSpecWithDeleted{
NodePoolSpec: np,
Deleted: true,
})
}
return nodePoolsWithDeleted, nil
}
8 changes: 8 additions & 0 deletions operator/pkg/resources/statefulset.go
Original file line number Diff line number Diff line change
Expand Up @@ -428,11 +428,19 @@ func (r *StatefulSetResource) obj(
nodePoolSelector = clusterLabels.AsAPISelector()
}

nodePoolSpecJSON, err := json.Marshal(r.nodePool.NodePoolSpec)
if err != nil {
return nil, fmt.Errorf("failed to marshal NodePoolSpec as JSON: %w", err)
}

ss := &appsv1.StatefulSet{
ObjectMeta: metav1.ObjectMeta{
Namespace: r.Key().Namespace,
Name: r.Key().Name,
Labels: nodePoolLabels,
Annotations: map[string]string{
labels.NodePoolSpecKey: string(nodePoolSpecJSON),
},
},
TypeMeta: metav1.TypeMeta{
Kind: "StatefulSet",
Expand Down
3 changes: 1 addition & 2 deletions operator/pkg/resources/statefulset_scale.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,9 +102,8 @@ func (r *StatefulSetResource) handleScaling(ctx context.Context) error {
return r.setCurrentReplicas(ctx, *r.nodePool.Replicas, r.nodePool.Name, r.logger)
}

if ptr.Deref(r.nodePool.Replicas, 0) == npCurrentReplicas && !r.nodePool.Deleted {
if ptr.Deref(r.nodePool.Replicas, 0) == npCurrentReplicas {
log.V(logger.DebugLevel).Info("No scaling changes required for this nodepool", "replicas", *r.nodePool.Replicas, "spec replicas", *r.LastObservedState.Spec.Replicas) // No changes to replicas, we do nothing here

return nil
}

Expand Down

0 comments on commit fc0b719

Please sign in to comment.