Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
38 changes: 27 additions & 11 deletions operator/pkg/resources/statefulset_update.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,15 +94,23 @@ func (r *StatefulSetResource) runUpdate(
// Check if we should run update on this specific STS.

log.V(logger.DebugLevel).Info("Checking that we should update")
update, err := r.shouldUpdate(current, modified)
stsChanged, nodePoolRestarting, err := r.shouldUpdate(current, modified)
if err != nil {
return fmt.Errorf("unable to determine the update procedure: %w", err)
}

if !update {
if !stsChanged && !nodePoolRestarting {
return nil
}

// When a StatefulSet change is detected, mark all Node Pool Pods as requiring a Pod rolling restart
if stsChanged {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe add a quick comment here indicating why the logic is this way so future editors don't have to reverse engineer the reasoning?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, will add comment.

log.V(logger.DebugLevel).Info("Setting ClusterUpdate condition on pods")
if err = r.MarkPodsForUpdate(ctx, ClusterUpdateReasonVersion); err != nil {
return fmt.Errorf("unable to mark pods for update: %w", err)
}
}

// At this point, we have seen a diff and want to update the StatefulSet.
npStatus := r.getNodePoolStatus()

Expand Down Expand Up @@ -612,17 +620,19 @@ func (r *StatefulSetResource) updateStatefulSet(
return nil
}

// shouldUpdate returns true if changes on the CR require update
// shouldUpdate compares the generated appsv1.StatefulSet from the StatefulSetResource.obj
// function (referred to as `modified`) with appsv1.StatefulSet retrieved from
// the Kubernetes API (referred to as `current`). If a difference is detected, the function returns
// `true` as the first value. Certain fields are ignored because they are not relevant to a Pod
// rolling upgrade (e.g. `Status`, `VolumeClaimTemplate`, or `Annotations`).
//
// If no differences are found, the function returns `false` as first value. The second value
// depends on the Node Pool Status, indicating whether rolling restart is still in progress.
func (r *StatefulSetResource) shouldUpdate(
current, modified *appsv1.StatefulSet,
) (bool, error) {
) (bool, bool, error) {
log := r.logger.WithName("shouldUpdate")

npStatus := r.getNodePoolStatus()

if npStatus.Restarting || r.pandaCluster.Status.Restarting {
return true, nil
}
prepareResourceForPatch(current, modified)
opts := []patch.CalculateOption{
patch.IgnoreStatusFields(),
Expand All @@ -632,10 +642,16 @@ func (r *StatefulSetResource) shouldUpdate(
}
patchResult, err := patch.NewPatchMaker(patch.NewAnnotator(redpandaAnnotatorKey), &patch.K8sStrategicMergePatcher{}, &patch.BaseJSONMergePatcher{}).Calculate(current, modified, opts...)
if err != nil || patchResult.IsEmpty() {
return false, err
npStatus := r.getNodePoolStatus()

if npStatus.Restarting || r.pandaCluster.Status.Restarting {
return false, true, nil
}

return false, false, err
}
log.Info("Detected diff", "patchResult", string(patchResult.Patch))
return true, nil
return true, false, nil
}

func (r *StatefulSetResource) getRestartingStatus() bool {
Expand Down
30 changes: 26 additions & 4 deletions operator/pkg/resources/statefulset_update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,14 +163,36 @@ func TestShouldUpdate_AnnotationChange(t *testing.T) {
},
},
}
update, err := ssres.shouldUpdate(sts, stsWithAnnotation)
stsChange, _, err := ssres.shouldUpdate(sts, stsWithAnnotation)
require.NoError(t, err)
require.True(t, update)
require.True(t, stsChange)

// same statefulset with same annotation
update, err = ssres.shouldUpdate(stsWithAnnotation, stsWithAnnotation)
stsChange, _, err = ssres.shouldUpdate(stsWithAnnotation, stsWithAnnotation)
require.NoError(t, err)
require.False(t, update)
require.False(t, stsChange)

ssres = StatefulSetResource{
nodePool: vectorizedv1alpha1.NodePoolSpecWithDeleted{
NodePoolSpec: vectorizedv1alpha1.NodePoolSpec{
Name: "default",
},
},
pandaCluster: &vectorizedv1alpha1.Cluster{
Status: vectorizedv1alpha1.ClusterStatus{
Restarting: false,
NodePools: map[string]vectorizedv1alpha1.NodePoolStatus{
"default": {
Restarting: true,
},
},
},
},
}
// same statefulset with same annotation, but with node pool status restarting
_, nodePoolRestarting, err := ssres.shouldUpdate(stsWithAnnotation, stsWithAnnotation)
require.NoError(t, err)
require.True(t, nodePoolRestarting)
}

func TestPutInMaintenanceMode(t *testing.T) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ metadata:
name: additional-configuration
spec:
image: "redpandadata/redpanda"
version: "v24.2.5"
version: "v25.1.11"
replicas: 1
resources:
requests:
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
apiVersion: kuttl.dev/v1beta1
kind: TestStep
commands:
- command: ./verify-config.sh -r 10 -b v24.2
- command: ./verify-config.sh -r 10 -b v25.1
namespaced: true
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
apiVersion: kuttl.dev/v1beta1
kind: TestStep
commands:
- command: ./verify-config.sh -r 11 -b v24.2
- command: ./verify-config.sh -r 11 -b v25.1
namespaced: true
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,8 @@ cloud_storage_segment_max_upload_interval_sec: 1800
default_topic_partitions: 3
enable_idempotence: true
enable_rack_awareness: true
internal_topic_replication_factor: 3
kafka_nodelete_topics: [_internal_connectors_configs _internal_connectors_offsets _internal_connectors_status _audit __consumer_offsets _redpanda_e2e_probe _schemas]
log_segment_size: 536870912
EOF
)
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
#!/usr/bin/env bash

expected=$(
cat <<EOF
redpanda:
data_directory: /var/lib/redpanda/data
empty_seed_starts_cluster: false
seed_servers:
- host:
address: additional-configuration-0.additional-configuration.${NAMESPACE}.svc.cluster.local.
port: 33145
rpc_server:
address: 0.0.0.0
port: 33145
kafka_api:
- address: 0.0.0.0
port: 9092
name: kafka
authentication_method: none
admin:
- address: 0.0.0.0
port: 9644
name: admin
advertised_rpc_api:
address: additional-configuration-0.additional-configuration.${NAMESPACE}.svc.cluster.local.
port: 33145
advertised_kafka_api:
- address: additional-configuration-0.additional-configuration.${NAMESPACE}.svc.cluster.local.
port: 9092
name: kafka
developer_mode: true
auto_create_topics_enabled: true
fetch_reads_debounce_timeout: 10
group_initial_rebalance_delay: 0
group_topic_partitions: 3
log_segment_size_min: 1
storage_min_free_bytes: 10485760
topic_partitions_per_shard: 1000
write_caching_default: "true"
rpk:
overprovisioned: true
tune_network: true
tune_disk_scheduler: true
tune_disk_nomerges: true
tune_disk_write_cache: true
tune_disk_irq: true
tune_cpu: true
tune_aio_events: true
tune_clocksource: true
tune_swappiness: true
coredump_dir: /var/lib/redpanda/coredump
tune_ballast_file: true
pandaproxy:
pandaproxy_api:
- address: 0.0.0.0
port: 8082
name: proxy
advertised_pandaproxy_api:
- address: additional-configuration-0.additional-configuration.${NAMESPACE}.svc.cluster.local.
port: 8082
name: proxy
pandaproxy_client:
brokers:
- address: additional-configuration.${NAMESPACE}.svc.cluster.local.
port: 9092
retries: ${PANDAPROXY_RETRIES}
schema_registry:
schema_registry_api:
- address: 0.0.0.0
port: 8081
name: external
EOF
)

expected_bootstrap=$(
cat <<EOF
auto_create_topics_enabled: false
cloud_storage_segment_max_upload_interval_sec: 1800
default_topic_partitions: 3
enable_idempotence: true
enable_rack_awareness: true
log_segment_size: 536870912
EOF
)
Loading