diff --git a/internal/controller/etcd/reconcile_complete.go b/internal/controller/etcd/reconcile_complete.go new file mode 100644 index 000000000..7c567df4b --- /dev/null +++ b/internal/controller/etcd/reconcile_complete.go @@ -0,0 +1,62 @@ +package etcd + +import ( + druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" + "github.com/gardener/etcd-druid/internal/component" + ctrlutils "github.com/gardener/etcd-druid/internal/controller/utils" + + v1beta1constants "github.com/gardener/gardener/pkg/apis/core/v1beta1/constants" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" +) + +func (r *Reconciler) completeReconcile(ctx component.OperatorContext, etcdObjectKey client.ObjectKey) ctrlutils.ReconcileStepResult { + rLog := r.logger.WithValues("etcd", etcdObjectKey, "operation", "completeReconcile").WithValues("runID", ctx.RunID) + ctx.SetLogger(rLog) + + reconcileCompletionStepFns := []reconcileFn{ + r.updateObservedGeneration, + r.removeOperationAnnotation, + } + + for _, fn := range reconcileCompletionStepFns { + if stepResult := fn(ctx, etcdObjectKey); ctrlutils.ShortCircuitReconcileFlow(stepResult) { + return r.recordIncompleteReconcileOperation(ctx, etcdObjectKey, stepResult) + } + } + ctx.Logger.Info("Finished reconciliation completion flow") + return ctrlutils.ContinueReconcile() +} + +func (r *Reconciler) updateObservedGeneration(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { + etcd := &druidv1alpha1.Etcd{} + if result := ctrlutils.GetLatestEtcd(ctx, r.client, etcdObjKey, etcd); ctrlutils.ShortCircuitReconcileFlow(result) { + return result + } + originalEtcd := etcd.DeepCopy() + etcd.Status.ObservedGeneration = &etcd.Generation + if err := r.client.Status().Patch(ctx, etcd, client.MergeFrom(originalEtcd)); err != nil { + ctx.Logger.Error(err, "failed to patch status.ObservedGeneration") + return ctrlutils.ReconcileWithError(err) + } + ctx.Logger.Info("patched status.ObservedGeneration", "ObservedGeneration", etcd.Generation) + return ctrlutils.ContinueReconcile() +} + +func (r *Reconciler) removeOperationAnnotation(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { + etcdPartialObjMeta := ctrlutils.EmptyEtcdPartialObjectMetadata() + if result := ctrlutils.GetLatestEtcdPartialObjectMeta(ctx, r.client, etcdObjKey, etcdPartialObjMeta); ctrlutils.ShortCircuitReconcileFlow(result) { + return result + } + + if metav1.HasAnnotation(etcdPartialObjMeta.ObjectMeta, v1beta1constants.GardenerOperation) { + ctx.Logger.Info("Removing operation annotation") + withOpAnnotation := etcdPartialObjMeta.DeepCopy() + delete(etcdPartialObjMeta.Annotations, v1beta1constants.GardenerOperation) + if err := r.client.Patch(ctx, etcdPartialObjMeta, client.MergeFrom(withOpAnnotation)); err != nil { + ctx.Logger.Error(err, "failed to remove operation annotation") + return ctrlutils.ReconcileWithError(err) + } + } + return ctrlutils.ContinueReconcile() +} diff --git a/internal/controller/etcd/reconcile_spec.go b/internal/controller/etcd/reconcile_spec.go index ac309468e..07baca77b 100644 --- a/internal/controller/etcd/reconcile_spec.go +++ b/internal/controller/etcd/reconcile_spec.go @@ -17,7 +17,6 @@ import ( v1beta1constants "github.com/gardener/gardener/pkg/apis/core/v1beta1/constants" "github.com/gardener/gardener/pkg/controllerutils" corev1 "k8s.io/api/core/v1" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil" ) @@ -25,21 +24,16 @@ import ( // syncRetryInterval will be used by both sync and preSync stages for a component and should be used when there is a need to requeue for retrying after a specific interval. const syncRetryInterval = 10 * time.Second -func (r *Reconciler) triggerReconcileSpecFlow(ctx component.OperatorContext, etcdObjectKey client.ObjectKey) ctrlutils.ReconcileStepResult { +func (r *Reconciler) reconcileSpec(ctx component.OperatorContext, etcdObjectKey client.ObjectKey) ctrlutils.ReconcileStepResult { + rLog := r.logger.WithValues("etcd", etcdObjectKey, "operation", "reconcileSpec").WithValues("runID", ctx.RunID) + ctx.SetLogger(rLog) + reconcileStepFns := []reconcileFn{ r.recordReconcileStartOperation, r.ensureFinalizer, r.preSyncEtcdResources, r.syncEtcdResources, - r.updateObservedGeneration, r.recordReconcileSuccessOperation, - // Removing the operation annotation after last operation recording seems counter-intuitive. - // If we reverse the order where we first remove the operation annotation and then record the last operation then - // in case the operation annotation removal succeeds but the last operation recording fails, then the control - // will never enter this flow again and the last operation will never be recorded. - // Reason: there is a predicate check done in `reconciler.canReconcile` prior to entering this flow. - // That check will no longer succeed once the reconcile operation annotation has been removed. - r.removeOperationAnnotation, } for _, fn := range reconcileStepFns { @@ -51,23 +45,6 @@ func (r *Reconciler) triggerReconcileSpecFlow(ctx component.OperatorContext, etc return ctrlutils.ContinueReconcile() } -func (r *Reconciler) removeOperationAnnotation(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { - etcdPartialObjMeta := ctrlutils.EmptyEtcdPartialObjectMetadata() - if result := ctrlutils.GetLatestEtcdPartialObjectMeta(ctx, r.client, etcdObjKey, etcdPartialObjMeta); ctrlutils.ShortCircuitReconcileFlow(result) { - return result - } - if metav1.HasAnnotation(etcdPartialObjMeta.ObjectMeta, v1beta1constants.GardenerOperation) { - ctx.Logger.Info("Removing operation annotation") - withOpAnnotation := etcdPartialObjMeta.DeepCopy() - delete(etcdPartialObjMeta.Annotations, v1beta1constants.GardenerOperation) - if err := r.client.Patch(ctx, etcdPartialObjMeta, client.MergeFrom(withOpAnnotation)); err != nil { - ctx.Logger.Error(err, "failed to remove operation annotation") - return ctrlutils.ReconcileWithError(err) - } - } - return ctrlutils.ContinueReconcile() -} - func (r *Reconciler) ensureFinalizer(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { etcdPartialObjMeta := ctrlutils.EmptyEtcdPartialObjectMetadata() if result := ctrlutils.GetLatestEtcdPartialObjectMeta(ctx, r.client, etcdObjKey, etcdPartialObjMeta); ctrlutils.ShortCircuitReconcileFlow(result) { @@ -123,21 +100,6 @@ func (r *Reconciler) syncEtcdResources(ctx component.OperatorContext, etcdObjKey return ctrlutils.ContinueReconcile() } -func (r *Reconciler) updateObservedGeneration(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { - etcd := &druidv1alpha1.Etcd{} - if result := ctrlutils.GetLatestEtcd(ctx, r.client, etcdObjKey, etcd); ctrlutils.ShortCircuitReconcileFlow(result) { - return result - } - originalEtcd := etcd.DeepCopy() - etcd.Status.ObservedGeneration = &etcd.Generation - if err := r.client.Status().Patch(ctx, etcd, client.MergeFrom(originalEtcd)); err != nil { - ctx.Logger.Error(err, "failed to patch status.ObservedGeneration") - return ctrlutils.ReconcileWithError(err) - } - ctx.Logger.Info("patched status.ObservedGeneration", "ObservedGeneration", etcd.Generation) - return ctrlutils.ContinueReconcile() -} - func (r *Reconciler) recordReconcileStartOperation(ctx component.OperatorContext, etcdObjKey client.ObjectKey) ctrlutils.ReconcileStepResult { if err := r.lastOpErrRecorder.RecordStart(ctx, etcdObjKey, druidv1alpha1.LastOperationTypeReconcile); err != nil { ctx.Logger.Error(err, "failed to record etcd reconcile start operation") @@ -162,14 +124,14 @@ func (r *Reconciler) recordIncompleteReconcileOperation(ctx component.OperatorCo return exitReconcileStepResult } -// canReconcileSpec assesses whether the Etcd spec should undergo reconciliation. +// shouldReconcileSpec assesses whether the Etcd spec should undergo reconciliation. // // Reconciliation decision follows these rules: // - Skipped if 'druid.gardener.cloud/suspend-etcd-spec-reconcile' annotation is present, signaling a pause in reconciliation. // - Automatic reconciliation occurs if EnableEtcdSpecAutoReconcile is true. // - If 'gardener.cloud/operation: reconcile' annotation exists and 'druid.gardener.cloud/suspend-etcd-spec-reconcile' annotation is not set, reconciliation proceeds upon Etcd spec changes. // - Reconciliation is not initiated if EnableEtcdSpecAutoReconcile is false and none of the relevant annotations are present. -func (r *Reconciler) canReconcileSpec(etcd *druidv1alpha1.Etcd) bool { +func (r *Reconciler) shouldReconcileSpec(etcd *druidv1alpha1.Etcd) bool { // Check if spec reconciliation has been suspended, if yes, then record the event and return false. if suspendReconcileAnnotKey := druidv1alpha1.GetSuspendEtcdSpecReconcileAnnotationKey(etcd.ObjectMeta); suspendReconcileAnnotKey != nil { r.recordEtcdSpecReconcileSuspension(etcd, *suspendReconcileAnnotKey) diff --git a/internal/controller/etcd/reconciler.go b/internal/controller/etcd/reconciler.go index 1901e2c71..68ffaeae7 100644 --- a/internal/controller/etcd/reconciler.go +++ b/internal/controller/etcd/reconciler.go @@ -85,28 +85,57 @@ type reconcileFn func(ctx component.OperatorContext, objectKey client.ObjectKey) // Reconcile manages the reconciliation of the Etcd component to align it with its desired specifications. // // The reconciliation process involves the following steps: -// 1. Deletion Handling: If the Etcd component has a deletionTimestamp, initiate the deletion workflow. On error, requeue the request. -// 2. Spec Reconciliation : Determine whether the Etcd spec should be reconciled based on annotations and flags and if there is a need then reconcile spec. -// 3. Status Reconciliation: Always update the status of the Etcd component to reflect its current state. -// 4. Scheduled Requeue: Requeue the reconciliation request after a defined period (EtcdStatusSyncPeriod) to maintain sync. +// 1. Deletion Handling: If the Etcd component has a deletionTimestamp, initiate the deletion workflow. +// On error, requeue the request. +// 2. Spec Reconciliation : Determine whether the Etcd spec should be reconciled based on annotations and flags, +// and if there is a need then reconcile spec. +// 3. Status Reconciliation: Always update the status of the Etcd component to reflect its current state, +// as well as status fields derived from spec reconciliation. +// 4. Remove operation-reconcile annotation if it was set and if spec reconciliation had succeeded. +// 5. Scheduled Requeue: Requeue the reconciliation request after a defined period (EtcdStatusSyncPeriod) to maintain sync. func (r *Reconciler) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { runID := string(controller.ReconcileIDFromContext(ctx)) operatorCtx := component.NewOperatorContext(ctx, r.logger, runID) if result := r.reconcileEtcdDeletion(operatorCtx, req.NamespacedName); ctrlutils.ShortCircuitReconcileFlow(result) { return result.ReconcileResult() } + + etcd := &druidv1alpha1.Etcd{} + if result := ctrlutils.GetLatestEtcd(ctx, r.client, req.NamespacedName, etcd); ctrlutils.ShortCircuitReconcileFlow(result) { + return result.ReconcileResult() + } + shouldReconcileSpec := r.shouldReconcileSpec(etcd) + var reconcileSpecResult ctrlutils.ReconcileStepResult - if result := r.reconcileSpec(operatorCtx, req.NamespacedName); ctrlutils.ShortCircuitReconcileFlow(result) { - reconcileSpecResult = result + if shouldReconcileSpec { + reconcileSpecResult = r.reconcileSpec(operatorCtx, req.NamespacedName) } if result := r.reconcileStatus(operatorCtx, req.NamespacedName); ctrlutils.ShortCircuitReconcileFlow(result) { r.logger.Error(result.GetCombinedError(), "Failed to reconcile status") return result.ReconcileResult() } + if reconcileSpecResult.NeedsRequeue() { return reconcileSpecResult.ReconcileResult() } + + // Spec reconciliation involves some steps that must be executed after status reconciliation, + // to ensure consistency of the status and to ensure that any intermittent failures result in a + // requeue to re-attempt the spec reconciliation. + // Specifically, status.observedGeneration must be updated after the rest of the status fields are updated, + // because consumers of the Etcd status must check the observed generation to confirm that reconciliation is + // in fact complete, and the status fields reflect the latest possible state of the etcd cluster after the + // spec was reconciled. + // Additionally, the operation annotation needs to be removed only at the end of reconciliation, to ensure that + // if any failure is encountered during reconciliation, then reconciliation is re-attempted upon the next requeue. + // r.completeReconcile() is executed only if the spec was reconciled, as denoted by the `shouldReconcileSpec` flag. + if shouldReconcileSpec { + if result := r.completeReconcile(operatorCtx, req.NamespacedName); ctrlutils.ShortCircuitReconcileFlow(result) { + return result.ReconcileResult() + } + } + return ctrlutils.ReconcileAfter(r.config.EtcdStatusSyncPeriod, "Periodic Requeue").ReconcileResult() } @@ -142,16 +171,3 @@ func (r *Reconciler) reconcileEtcdDeletion(ctx component.OperatorContext, etcdOb } return ctrlutils.ContinueReconcile() } - -func (r *Reconciler) reconcileSpec(ctx component.OperatorContext, etcdObjectKey client.ObjectKey) ctrlutils.ReconcileStepResult { - etcd := &druidv1alpha1.Etcd{} - if result := ctrlutils.GetLatestEtcd(ctx, r.client, etcdObjectKey, etcd); ctrlutils.ShortCircuitReconcileFlow(result) { - return result - } - if r.canReconcileSpec(etcd) { - rLog := r.logger.WithValues("etcd", etcdObjectKey, "operation", "reconcileSpec").WithValues("runID", ctx.RunID) - ctx.SetLogger(rLog) - return r.triggerReconcileSpecFlow(ctx, etcdObjectKey) - } - return ctrlutils.ContinueReconcile() -} diff --git a/internal/health/condition/check_all_members_updated_test.go b/internal/health/condition/check_all_members_updated_test.go index a6852427a..f9b418f6c 100644 --- a/internal/health/condition/check_all_members_updated_test.go +++ b/internal/health/condition/check_all_members_updated_test.go @@ -6,7 +6,6 @@ package condition_test import ( "context" - "k8s.io/utils/pointer" druidv1alpha1 "github.com/gardener/etcd-druid/api/v1alpha1" "github.com/gardener/etcd-druid/internal/health/condition" @@ -15,6 +14,7 @@ import ( appsv1 "k8s.io/api/apps/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/utils/pointer" "sigs.k8s.io/controller-runtime/pkg/client" . "github.com/onsi/ginkgo/v2" diff --git a/internal/utils/miscellaneous_test.go b/internal/utils/miscellaneous_test.go new file mode 100644 index 000000000..52d2e4da7 --- /dev/null +++ b/internal/utils/miscellaneous_test.go @@ -0,0 +1,206 @@ +package utils + +import ( + "testing" + "time" + + "sigs.k8s.io/controller-runtime/pkg/client" + + . "github.com/onsi/gomega" +) + +func TestMergeMaps(t *testing.T) { + testCases := []struct { + name string + input []map[string]int + expected map[string]int + }{ + { + name: "Nil input", + input: nil, + expected: nil, + }, + { + name: "Empty maps", + input: []map[string]int{}, + expected: map[string]int{}, + }, + { + name: "Non-overlapping keys", + input: []map[string]int{{"a": 1}, {"b": 2}}, + expected: map[string]int{"a": 1, "b": 2}, + }, + { + name: "Overlapping keys", + input: []map[string]int{{"a": 1}, {"a": 2}}, + expected: map[string]int{"a": 2}, + }, + } + + g := NewWithT(t) + t.Parallel() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + g.Expect(MergeMaps(tc.input...)).To(Equal(tc.expected)) + }) + } +} + +func TestKey(t *testing.T) { + testCases := []struct { + name string + namespaceOrName string + nameOpt []string + expected client.ObjectKey + }{ + { + name: "Namespace and name", + namespaceOrName: "test-namespace", + nameOpt: []string{"test-name"}, + expected: client.ObjectKey{Namespace: "test-namespace", Name: "test-name"}, + }, + { + name: "Only name", + namespaceOrName: "test-name", + nameOpt: nil, + expected: client.ObjectKey{Name: "test-name"}, + }, + } + + g := NewWithT(t) + t.Parallel() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + g.Expect(Key(tc.namespaceOrName, tc.nameOpt...)).To(Equal(tc.expected)) + }) + } +} + +func TestIsEmptyString(t *testing.T) { + testCases := []struct { + name string + input string + expected bool + }{ + { + name: "Empty string", + input: "", + expected: true, + }, + { + name: "Whitespace string", + input: " ", + expected: true, + }, + { + name: "Non-empty string", + input: "non-empty", + expected: false, + }, + } + + g := NewWithT(t) + t.Parallel() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + g.Expect(IsEmptyString(tc.input)).To(Equal(tc.expected)) + }) + } +} + +func TestIfConditionOr(t *testing.T) { + testCases := []struct { + name string + condition bool + trueVal string + falseVal string + expected string + }{ + { + name: "True condition", + condition: true, + trueVal: "trueVal", + falseVal: "falseVal", + expected: "trueVal", + }, + { + name: "False condition", + condition: false, + trueVal: "trueVal", + falseVal: "falseVal", + expected: "falseVal", + }, + } + + g := NewWithT(t) + t.Parallel() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + g.Expect(IfConditionOr(tc.condition, tc.trueVal, tc.falseVal)).To(Equal(tc.expected)) + }) + } +} + +func TestComputeScheduleInterval(t *testing.T) { + testCases := []struct { + name string + cronSchedule string + expected time.Duration + expectError bool + }{ + { + name: "Valid cron schedule", + cronSchedule: "0 0 * * *", + expected: 24 * time.Hour, + expectError: false, + }, + { + name: "Valid cron schedule", + cronSchedule: "0 0 * * 1", + expected: 24 * time.Hour * 7, + expectError: false, + }, + { + name: "Valid cron schedule", + cronSchedule: "*/1 * * * *", + expected: 1 * time.Minute, + expectError: false, + }, + { + name: "Valid cron schedule", + cronSchedule: "0 */1 * * *", + expected: 1 * time.Hour, + expectError: false, + }, + { + name: "Invalid cron schedule", + cronSchedule: "invalid-cron", + expected: 0, + expectError: true, + }, + } + + g := NewWithT(t) + t.Parallel() + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + duration, err := ComputeScheduleInterval(tc.cronSchedule) + if tc.expectError { + g.Expect(err).ToNot(BeNil()) + } else { + g.Expect(err).To(BeNil()) + g.Expect(duration).To(Equal(tc.expected)) + } + }) + } +} diff --git a/test/it/controller/etcd/reconciler_test.go b/test/it/controller/etcd/reconciler_test.go index b4c6f907e..1e2f892f5 100644 --- a/test/it/controller/etcd/reconciler_test.go +++ b/test/it/controller/etcd/reconciler_test.go @@ -118,12 +118,12 @@ func testAllManagedResourcesAreCreated(t *testing.T, testNs string, reconcilerTe // It is sufficient to test that the resources are created as part of the sync. The configuration of each // resource is now extensively covered in the unit tests for the respective component operator. assertAllComponentsExists(ctx, t, reconcilerTestEnv, etcdInstance, timeout, pollingInterval) - assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](1), 5*time.Second, 1*time.Second) expectedLastOperation := &druidv1alpha1.LastOperation{ Type: druidv1alpha1.LastOperationTypeReconcile, State: druidv1alpha1.LastOperationStateSucceeded, } assertETCDLastOperationAndLastErrorsUpdatedSuccessfully(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), expectedLastOperation, nil, 5*time.Second, 1*time.Second) + assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](1), 30*time.Second, 1*time.Second) assertETCDOperationAnnotation(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), false, 5*time.Second, 1*time.Second) } @@ -163,12 +163,12 @@ func testFailureToCreateAllResources(t *testing.T, testNs string, reconcilerTest component.StatefulSetKind, } assertComponentsDoNotExist(ctx, t, reconcilerTestEnv, etcdInstance, componentKindNotCreated, timeout, pollingInterval) - assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), nil, 5*time.Second, 1*time.Second) expectedLastOperation := &druidv1alpha1.LastOperation{ Type: druidv1alpha1.LastOperationTypeReconcile, State: druidv1alpha1.LastOperationStateError, } assertETCDLastOperationAndLastErrorsUpdatedSuccessfully(t, cl, client.ObjectKeyFromObject(etcdInstance), expectedLastOperation, []string{"ERR_SYNC_CLIENT_SERVICE"}, 5*time.Second, 1*time.Second) + assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), nil, 30*time.Second, 1*time.Second) } func testWhenReconciliationIsSuspended(t *testing.T, testNs string, reconcilerTestEnv ReconcilerTestEnv) { @@ -189,8 +189,8 @@ func testWhenReconciliationIsSuspended(t *testing.T, testNs string, reconcilerTe // ***************** test etcd spec reconciliation ***************** assertNoComponentsExist(ctx, t, reconcilerTestEnv, etcdInstance, 10*time.Second, 2*time.Second) assertReconcileSuspensionEventRecorded(ctx, t, cl, client.ObjectKeyFromObject(etcdInstance), 10*time.Second, 2*time.Second) - assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), nil, 5*time.Second, 1*time.Second) assertETCDLastOperationAndLastErrorsUpdatedSuccessfully(t, cl, client.ObjectKeyFromObject(etcdInstance), nil, nil, 5*time.Second, 1*time.Second) + assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), nil, 30*time.Second, 1*time.Second) assertETCDOperationAnnotation(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), true, 5*time.Second, 1*time.Second) } @@ -218,7 +218,7 @@ func testEtcdSpecUpdateWhenNoReconcileOperationAnnotationIsSet(t *testing.T, tes // ***************** test etcd spec reconciliation ***************** assertAllComponentsExists(ctx, t, reconcilerTestEnv, etcdInstance, 2*time.Second, 2*time.Second) _ = updateAndGetStsRevision(ctx, t, reconcilerTestEnv.itTestEnv.GetClient(), etcdInstance) - assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](1), 5*time.Second, 1*time.Second) + assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](1), 30*time.Second, 1*time.Second) // ensure that sts generation does not change, ie, it should remain 1, as sts is not updated after etcd spec change without reconcile operation annotation assertStatefulSetGeneration(ctx, t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), 1, 30*time.Second, 2*time.Second) } @@ -259,12 +259,12 @@ func testEtcdSpecUpdateWhenReconcileOperationAnnotationIsSet(t *testing.T, testN // ***************** test etcd spec reconciliation ***************** assertAllComponentsExists(ctx, t, reconcilerTestEnv, etcdInstance, 30*time.Minute, 2*time.Second) _ = updateAndGetStsRevision(ctx, t, reconcilerTestEnv.itTestEnv.GetClient(), etcdInstance) - assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](2), 2*time.Minute, 1*time.Second) expectedLastOperation := &druidv1alpha1.LastOperation{ Type: druidv1alpha1.LastOperationTypeReconcile, State: druidv1alpha1.LastOperationStateSucceeded, } assertETCDLastOperationAndLastErrorsUpdatedSuccessfully(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), expectedLastOperation, nil, 5*time.Second, 1*time.Second) + assertETCDObservedGeneration(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), ptr.To[int64](2), 30*time.Second, 1*time.Second) assertETCDOperationAnnotation(t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), false, 5*time.Second, 1*time.Second) // ensure that sts generation is updated to 2, since reconciliation of the etcd spec change causes an update of the sts spec assertStatefulSetGeneration(ctx, t, reconcilerTestEnv.itTestEnv.GetClient(), client.ObjectKeyFromObject(etcdInstance), 2, 30*time.Second, 2*time.Second)