Skip to content

Commit

Permalink
operator v1: add separate test suite with feature flags
Browse files Browse the repository at this point in the history
it's not possible to toggle feature flags on a per test case basis.
therefore, as per advice from Rafal, we're adding a new test suite.
it also uses k8s 1.28, as this is what cloud uses, and is required by
the new code under test.
  • Loading branch information
birdayz committed Sep 2, 2024
1 parent d533730 commit 824cea8
Show file tree
Hide file tree
Showing 28 changed files with 474 additions and 46 deletions.
64 changes: 64 additions & 0 deletions .buildkite/pipeline.yml
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,70 @@ steps:
failed: true
branches:
- main
- key: k8s-operator-with-flags
label: K8s Operator tests with flags
timeout_in_minutes: 180
notify:
- github_commit_status:
context: k8s-operator
commands:
- |
TAG_NAME=$(ci/scripts/tag-check.sh) ./ci/scripts/run-in-nix-docker.sh ./task ci:run-k8s-tests-with-flags
agents:
queue: amd64-builders
artifact_paths:
- src/go/k8s/*.tar.gz
- src/go/k8s/tests/_e2e_artifacts/kuttl-report.xml
plugins:
- seek-oss/aws-sm#v2.3.2: &aws-sm-plugin
json-to-env:
- json-key: .
secret-id: sdlc/prod/buildkite/active_directory
- json-key: .
secret-id: sdlc/prod/buildkite/buildkite_analytics_token
- json-key: .
secret-id: sdlc/prod/buildkite/buildkite_api_token
- json-key: .
secret-id: sdlc/prod/buildkite/cdt_gcp
- json-key: .
secret-id: sdlc/prod/buildkite/cdt_runner_aws
- json-key: .
secret-id: sdlc/prod/buildkite/ci_db
- json-key: .
secret-id: sdlc/prod/buildkite/cloudsmith
- json-key: .
secret-id: sdlc/prod/buildkite/dockerhub
- json-key: .
secret-id: sdlc/prod/buildkite/gh_token
- json-key: .
secret-id: sdlc/prod/buildkite/github_api_token
- json-key: .
secret-id: sdlc/prod/buildkite/goreleaser_key
- json-key: .
secret-id: sdlc/prod/buildkite/grafana_token
- json-key: .
secret-id: sdlc/prod/buildkite/redpanda_sample_license
- json-key: .
secret-id: sdlc/prod/buildkite/redpanda_second_sample_license
- json-key: .
secret-id: sdlc/prod/buildkite/rpk_test_client
- json-key: .
secret-id: sdlc/prod/buildkite/seceng_audit_aws
- json-key: .
secret-id: sdlc/prod/buildkite/slack_vbot_token
- json-key: .
secret-id: sdlc/prod/buildkite/teleport_bot_token
- json-key: .
secret-id: sdlc/prod/buildkite/test_result_dsn
- https://[email protected]/redpanda-data/step-slack-notify-buildkite-plugin.git#main:
message: ":cloud: K8s Operator v1 Jobs failed"
channel_name: "kubernetes-tests"
slack_token_env_var_name: "SLACK_VBOT_TOKEN"
conditions:
failed: true
branches:
- main


- group: K8s Operator v2 Jobs
if: |
Expand Down
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
/dbuild
bin
_e2e_artifacts/
_e2e_with_flags_artifacts/
_e2e_unstable_artifacts/
_helm_e2e_artifacts/
src/go/k8s/testbin/*
Expand Down
15 changes: 15 additions & 0 deletions src/go/k8s/config/e2e-tests-with-flags/kustomization.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
---
apiVersion: kustomize.config.k8s.io/v1beta1
kind: Kustomization
resources:
- ../e2e-tests
patches:
- patch: |-
- op: add
path: /spec/template/spec/containers/0/args/-
value: --auto-delete-pvcs
target:
group: apps
version: v1
kind: Deployment
name: controller-manager
2 changes: 1 addition & 1 deletion src/go/k8s/go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ require (
k8s.io/apimachinery v0.29.5
k8s.io/client-go v0.29.5
k8s.io/component-helpers v0.29.0
k8s.io/kubectl v0.29.0
k8s.io/utils v0.0.0-20240310230437-4693a0247e57
pgregory.net/rapid v1.1.0
sigs.k8s.io/controller-runtime v0.17.2
Expand Down Expand Up @@ -407,7 +408,6 @@ require (
k8s.io/component-base v0.29.5 // indirect
k8s.io/klog/v2 v2.120.1 // indirect
k8s.io/kube-openapi v0.0.0-20240103051144-eec4567ac022 // indirect
k8s.io/kubectl v0.29.0 // indirect
oras.land/oras-go v1.2.5 // indirect
sigs.k8s.io/gateway-api v1.0.0 // indirect
sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect
Expand Down
22 changes: 6 additions & 16 deletions src/go/k8s/internal/controller/redpanda/cluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
adminutils "github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/admin"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/labels"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/networking"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/patch"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/resources"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/resources/featuregates"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/utils"
Expand Down Expand Up @@ -141,7 +142,7 @@ func (r *ClusterReconciler) Reconcile(
// - Set OperatorQuiescent condition, based on our best knowledge if there is
// any outstanding work to do for the controller.
defer func() {
_, patchErr := patchStatus(ctx, r.Client, &vectorizedCluster, func(cluster *vectorizedv1alpha1.Cluster) {
_, patchErr := patch.PatchStatus(ctx, r.Client, &vectorizedCluster, func(cluster *vectorizedv1alpha1.Cluster) {
// Set quiescent
cond := getQuiescentCondition(cluster)

Expand Down Expand Up @@ -361,9 +362,9 @@ func (r *ClusterReconciler) removePodFinalizer(
log := l.WithName("removePodFinalizer")
if controllerutil.ContainsFinalizer(pod, FinalizerKey) {
log.V(logger.DebugLevel).WithValues("namespace", pod.Namespace, "name", pod.Name).Info("removing finalizer")
patch := client.MergeFrom(pod.DeepCopy())
p := client.MergeFrom(pod.DeepCopy())
controllerutil.RemoveFinalizer(pod, FinalizerKey)
if err := r.Patch(ctx, pod, patch); err != nil {
if err := r.Patch(ctx, pod, p); err != nil {
return fmt.Errorf("unable to remove pod (%s/%s) finalizer: %w", pod.Namespace, pod.Name, err)
}
}
Expand Down Expand Up @@ -765,9 +766,9 @@ func (r *ClusterReconciler) removeFinalizers(

if controllerutil.ContainsFinalizer(redpandaCluster, FinalizerKey) {
log.V(logger.DebugLevel).Info("removing finalizers from cluster custom resource")
patch := client.MergeFrom(redpandaCluster.DeepCopy())
p := client.MergeFrom(redpandaCluster.DeepCopy())
controllerutil.RemoveFinalizer(redpandaCluster, FinalizerKey)
if err := r.Patch(ctx, redpandaCluster, patch); err != nil {
if err := r.Patch(ctx, redpandaCluster, p); err != nil {
return fmt.Errorf("unable to remove Cluster finalizer: %w", err)
}
}
Expand Down Expand Up @@ -1131,17 +1132,6 @@ func isRedpandaClusterVersionManaged(
return true
}

func patchStatus(ctx context.Context, c client.Client, observedCluster *vectorizedv1alpha1.Cluster, mutator func(cluster *vectorizedv1alpha1.Cluster)) (vectorizedv1alpha1.ClusterStatus, error) {
clusterPatch := client.MergeFrom(observedCluster.DeepCopy())
mutator(observedCluster)

if err := c.Status().Patch(ctx, observedCluster, clusterPatch); err != nil {
return vectorizedv1alpha1.ClusterStatus{}, fmt.Errorf("failed to update cluster status: %w", err)
}

return observedCluster.Status, nil
}

func getQuiescentCondition(redpandaCluster *vectorizedv1alpha1.Cluster) vectorizedv1alpha1.ClusterCondition {
condition := vectorizedv1alpha1.ClusterCondition{
Type: vectorizedv1alpha1.OperatorQuiescentConditionType,
Expand Down
12 changes: 12 additions & 0 deletions src/go/k8s/kind-for-cloud.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
kind: Cluster
apiVersion: kind.x-k8s.io/v1alpha4
nodes:
- role: control-plane
# Need to run KIND 0.19 based image; >0.19 does not support cgroupsv1/missing cgroupns - and this is required for CI at the moment.
image: kindest/node:v1.28.0@sha256:dad5a6238c5e41d7cac405fae3b5eda2ad1de6f1190fa8bfc64ff5bb86173213
- role: worker
image: kindest/node:v1.28.0@sha256:dad5a6238c5e41d7cac405fae3b5eda2ad1de6f1190fa8bfc64ff5bb86173213
- role: worker
image: kindest/node:v1.28.0@sha256:dad5a6238c5e41d7cac405fae3b5eda2ad1de6f1190fa8bfc64ff5bb86173213
- role: worker
image: kindest/node:v1.28.0@sha256:dad5a6238c5e41d7cac405fae3b5eda2ad1de6f1190fa8bfc64ff5bb86173213
30 changes: 30 additions & 0 deletions src/go/k8s/kuttl-test-with-flags.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
apiVersion: kuttl.dev/v1beta1
kind: TestSuite
startKIND: true
kindContainers:
- localhost/redpanda-operator:dev
- localhost/configurator:dev
- localhost/redpanda:dev
testDirs:
- ./tests/e2e-with-flags
kindConfig: ./kind-for-cloud.yaml
kindNodeCache: false
commands:
- command: kubectl taint node kind-control-plane node-role.kubernetes.io/control-plane-
- command: "mkdir -p tests/_e2e_with_flags_artifacts"
- command: "./hack/install-cert-manager.sh tests/_e2e_with_flags_artifacts"
background: true
ignoreFailure: true
- command: "kubectl create -f https://raw.githubusercontent.com/prometheus-operator/prometheus-operator/e23ff77fceba6a5d9f190f5d1a123c87701dc964/bundle.yaml"
background: true
ignoreFailure: true
- command: "sh -c 'until kustomize build config/e2e-tests-with-flags 2>> tests/_e2e_with_flags_artifacts/kustomize-output.txt | kubectl apply --server-side -f - 1>> tests/_e2e_with_flags_artifacts/kubectl-output.txt 2>> tests/_e2e_with_flags_artifacts/kubectl-error-output.txt; do sleep 0.5; done'"
background: true
- command: "./hack/wait-for-webhook-ready.sh"
artifactsDir: tests/_e2e_with_flags_artifacts
timeout: 330
reportFormat: xml
parallel: 2
namespace: redpanda-system
suppress:
- events
25 changes: 25 additions & 0 deletions src/go/k8s/pkg/patch/patch.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
// Package patch is a utility package that provides utils around patching a resource.
// It has its own package, because of a dependency conflict; pkg/utils may not
// import types/v1alpha1, types/v1alpha1 imports pkg/utils (cycle).
package patch

import (
"context"
"fmt"

vectorizedv1alpha1 "github.com/redpanda-data/redpanda-operator/src/go/k8s/api/vectorized/v1alpha1"
"sigs.k8s.io/controller-runtime/pkg/client"
)

// PatchStatus persforms a mutation as done by mutator, calls k8s-api with PATCH, and then returns the
// new status.
func PatchStatus(ctx context.Context, c client.Client, observedCluster *vectorizedv1alpha1.Cluster, mutator func(cluster *vectorizedv1alpha1.Cluster)) (vectorizedv1alpha1.ClusterStatus, error) {
clusterPatch := client.MergeFrom(observedCluster.DeepCopy())
mutator(observedCluster)

if err := c.Status().Patch(ctx, observedCluster, clusterPatch); err != nil {
return vectorizedv1alpha1.ClusterStatus{}, fmt.Errorf("failed to update cluster status: %w", err)
}

return observedCluster.Status, nil
}
20 changes: 3 additions & 17 deletions src/go/k8s/pkg/resources/statefulset_scale.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
vectorizedv1alpha1 "github.com/redpanda-data/redpanda-operator/src/go/k8s/api/vectorized/v1alpha1"
adminutils "github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/admin"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/labels"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/patch"
"github.com/redpanda-data/redpanda-operator/src/go/k8s/pkg/resources/featuregates"
)

Expand Down Expand Up @@ -448,28 +449,13 @@ func setCurrentReplicas(

log.Info("Scaling StatefulSet", "replicas", replicas)

err := retry.RetryOnConflict(retry.DefaultRetry, func() error {
cluster := &vectorizedv1alpha1.Cluster{}
err := c.Get(ctx, types.NamespacedName{
Name: pandaCluster.Name,
Namespace: pandaCluster.Namespace,
}, cluster)
if err != nil {
return err
}

result, err := patch.PatchStatus(ctx, c, pandaCluster, func(cluster *vectorizedv1alpha1.Cluster) {
cluster.Status.CurrentReplicas = replicas

err = c.Status().Update(ctx, cluster)
if err == nil {
// sync original cluster variable to avoid conflicts on subsequent operations
pandaCluster.Status = cluster.Status
}
return err
})
if err != nil {
return fmt.Errorf("could not scale cluster %s to %d replicas: %w", pandaCluster.Name, pandaCluster.GetCurrentReplicas(), err)
}
pandaCluster.Status = result

log.Info("StatefulSet scaled", "replicas", replicas)
return nil
Expand Down
36 changes: 36 additions & 0 deletions src/go/k8s/tests/e2e-with-flags/decommission/00-assert.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
apiVersion: kuttl.dev/v1beta1
kind: TestAssert
commands:
- script: |
kubectl wait --for=condition=ClusterConfigured=True cluster/up-img-admin-tls --timeout 300s --namespace $NAMESPACE
kubectl wait --for=condition=OperatorQuiescent=True cluster/up-img-admin-tls --timeout 300s --namespace $NAMESPACE
---
apiVersion: v1
kind: Pod
metadata:
name: decommission-0
status:
phase: Running
---
apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: decommission
status:
replicas: 3
currentReplicas: 3
readyReplicas: 3
---
apiVersion: apps/v1
kind: StatefulSet
metadata:
name: decommission
spec:
persistentVolumeClaimRetentionPolicy:
whenDeleted: Delete
whenScaled: Delete
---
apiVersion: kuttl.dev/v1beta1
kind: TestAssert
collectors:
- command: ../../../hack/get-redpanda-info.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: decommission
spec:
image: "localhost/redpanda"
version: "dev"
replicas: 3
resources:
requests:
cpu: 1
memory: 1Gi
limits:
cpu: 1
memory: 1Gi
configuration:
rpcServer:
port: 33145
kafkaApi:
- port: 9092
adminApi:
- port: 9644
pandaproxyApi:
- port: 8082
developerMode: true
additionalCommandlineArguments:
dump-memory-diagnostics-on-alloc-failure-kind: all
abort-on-seastar-bad-alloc: ''
20 changes: 20 additions & 0 deletions src/go/k8s/tests/e2e-with-flags/decommission/01-assert.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
apiVersion: kuttl.dev/v1beta1
kind: TestAssert
commands:
- script: |
kubectl wait --for=condition=ClusterConfigured=True cluster/up-img-admin-tls --timeout 300s --namespace $NAMESPACE
kubectl wait --for=condition=OperatorQuiescent=True cluster/up-img-admin-tls --timeout 300s --namespace $NAMESPACE
---
apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: decommission
status:
replicas: 2
currentReplicas: 2
readyReplicas: 2
---
apiVersion: kuttl.dev/v1beta1
kind: TestAssert
collectors:
- command: ../../../hack/get-redpanda-info.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
apiVersion: redpanda.vectorized.io/v1alpha1
kind: Cluster
metadata:
name: decommission
spec:
replicas: 2
18 changes: 18 additions & 0 deletions src/go/k8s/tests/e2e-with-flags/decommission/02-assert.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
apiVersion: v1
kind: Pod
metadata:
labels:
job-name: get-broker-count
status:
containerStatuses:
- name: curl
state:
terminated:
exitCode: 0
reason: Completed
phase: Running
---
apiVersion: kuttl.dev/v1beta1
kind: TestAssert
collectors:
- command: ../../../hack/get-redpanda-info.sh
Loading

0 comments on commit 824cea8

Please sign in to comment.