From 12baa1658717c532393a8c649d0083742cabd775 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 11 Aug 2025 13:24:43 -0400 Subject: [PATCH 01/28] clarify doc Signed-off-by: Alexandre Gaudreault --- pkg/sync/doc.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sync/doc.go b/pkg/sync/doc.go index f4f5d8725..e3a28e7ed 100644 --- a/pkg/sync/doc.go +++ b/pkg/sync/doc.go @@ -61,8 +61,8 @@ Hooks can be deleted in an automatic fashion using the annotation: argocd.argopr The following policies define when the hook will be deleted. - - HookSucceeded - the hook resource is deleted after the hook succeeded (e.g. Job/Workflow completed successfully). - - HookFailed - the hook resource is deleted after the hook failed. + - HookSucceeded - the hook resource is deleted if the hook's phase succeeded (e.g. Job/Workflow completed successfully). + - HookFailed - the hook resource is deleted if the hook's phase failed. - BeforeHookCreation - any existing hook resource is deleted before the new one is created # Sync Waves From 03d77eb9ee788d020cffd4ad0662ba006edffa58 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 11 Aug 2025 15:02:52 -0400 Subject: [PATCH 02/28] do not preemptively fail sync if failure hooks fail to apply Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 35 ++++++++++++++++++++--------------- 1 file changed, 20 insertions(+), 15 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 8f4d51e4f..e58a96f09 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -534,7 +534,7 @@ func (sc *syncContext) Sync() { // syncFailTasks only run during failure, so separate them from regular tasks syncFailTasks, tasks := tasks.Split(func(t *syncTask) bool { return t.phase == common.SyncPhaseSyncFail }) - syncFailedTasks, _ := tasks.Split(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) // if there are any completed but unsuccessful tasks, sync is a failure. if tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { @@ -590,7 +590,7 @@ func (sc *syncContext) Sync() { switch runState { case failed: - syncFailedTasks, _ := tasks.Split(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) sc.deleteHooks(hooksPendingDeletionFailed) sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") case successful: @@ -733,23 +733,28 @@ func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTas errorMessage := errorMessageFactory(syncFailedTasks, message) - if len(syncFailTasks) > 0 { - // if all the failure hooks are completed, don't run them again, and mark the sync as failed - if syncFailTasks.All(func(task *syncTask) bool { return task.completed() }) { - sc.setOperationPhase(common.OperationFailed, errorMessage) - return - } - // otherwise, we need to start the failure hooks, and then return without setting - // the phase, so we make sure we have at least one more sync - sc.log.WithValues("syncFailTasks", syncFailTasks).V(1).Info("Running sync fail tasks") - if sc.runTasks(syncFailTasks, false) == failed { - failedSyncFailTasks := syncFailTasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + // if there is no failure hook, there is nothing more to do and we can fail + if len(syncFailTasks) == 0 { + sc.setOperationPhase(common.OperationFailed, errorMessage) + return + } + + // if all the failure hooks are completed, mark the sync as failed + if syncFailTasks.All(func(task *syncTask) bool { return task.completed() }) { + failedSyncFailTasks := syncFailTasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + if len(failedSyncFailTasks) > 0 { syncFailTasksMessage := errorMessageFactory(failedSyncFailTasks, "one or more SyncFail hooks failed") - sc.setOperationPhase(common.OperationFailed, fmt.Sprintf("%s\n%s", errorMessage, syncFailTasksMessage)) + errorMessage = fmt.Sprintf("%s\n%s", errorMessage, syncFailTasksMessage) } - } else { sc.setOperationPhase(common.OperationFailed, errorMessage) + return } + + // otherwise, we need to start the pending failure hooks, and then return WITHOUT setting + // the phase to failed, since we want the failure hooks to complete their running state before failing + pendingSyncFailTasks := syncFailTasks.Filter(func(task *syncTask) bool { return task.pending() }) + sc.log.WithValues("syncFailTasks", pendingSyncFailTasks).V(1).Info("Running sync fail tasks") + sc.runTasks(pendingSyncFailTasks, false) } func (sc *syncContext) started() bool { From 60c298f3d3200a464557b79cb3ccfc6620cb3394 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 11 Aug 2025 15:15:28 -0400 Subject: [PATCH 03/28] make sure finalizer is removed before deleting BeforeHookCreation hooks Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index e58a96f09..47cd5ac14 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -1425,6 +1425,11 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { ss.Go(func(state runState) runState { sc.log.WithValues("dryRun", dryRun, "task", t).V(1).Info("Deleting") if !dryRun { + // the hook might still have a finalizer, but we explicitly want to recreate it + if err := sc.removeHookFinalizer(t); err != nil { + state = failed + sc.setResourceResult(t, "", common.OperationError, fmt.Sprintf("failed to remove hook finalizer: %v", err)) + } err := sc.deleteResource(t) if err != nil { // it is possible to get a race condition here, such that the resource does not exist when From 6654e513fd9e1f3cb04046e563baba989e89104b Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 12 Aug 2025 14:46:59 -0400 Subject: [PATCH 04/28] terminate hooks on error/failures Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 203 +++++++++++++++++------------ pkg/sync/sync_context_test.go | 56 ++++---- pkg/utils/testing/api_resources.go | 81 ++++++++++++ 3 files changed, 233 insertions(+), 107 deletions(-) create mode 100644 pkg/utils/testing/api_resources.go diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 47cd5ac14..2cdf653a4 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -402,6 +402,9 @@ func (sc *syncContext) setRunningPhase(tasks []*syncTask, isPendingDeletion bool } if isPendingDeletion { waitingFor = "deletion of" + if firstTask.isHook() { + waitingFor += " hook" + } } message := fmt.Sprintf("waiting for %s %s/%s/%s", waitingFor, firstTask.group(), firstTask.kind(), firstTask.name()) @@ -536,16 +539,19 @@ func (sc *syncContext) Sync() { syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + terminatingTasks := tasks.Filter(func(t *syncTask) bool { return t.operationState == common.OperationTerminating }) + // if there are any completed but unsuccessful tasks, sync is a failure. - if tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { + // we already know tasks do not contain running tasks, but it may contain terminating tasks waiting for pruning/cleanup + if len(terminatingTasks) == 0 && tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { sc.deleteHooks(hooksPendingDeletionFailed) sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more synchronization tasks completed unsuccessfully") return } sc.log.WithValues("tasks", tasks).V(1).Info("Filtering out non-pending tasks") - // remove tasks that are completed, we can assume that there are no running tasks - tasks = tasks.Filter(func(t *syncTask) bool { return t.pending() }) + // remove tasks that are completed, we should only have pending and terminating tasks left + tasks = tasks.Filter(func(t *syncTask) bool { return !t.completed() }) if sc.applyOutOfSyncOnly { tasks = sc.filterOutOfSyncTasks(tasks) @@ -581,8 +587,11 @@ func (sc *syncContext) Sync() { if sc.syncWaveHook != nil && runState != failed { err := sc.syncWaveHook(phase, wave, finalWave) if err != nil { + // Since this is an unexpected error and is not related to a specific task, terminate the sync with error + // without triggering the syncFailTasks + sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) sc.deleteHooks(hooksPendingDeletionFailed) - sc.setOperationPhase(common.OperationFailed, fmt.Sprintf("SyncWaveHook failed: %v", err)) + sc.setOperationPhase(common.OperationError, fmt.Sprintf("SyncWaveHook failed: %v", err)) sc.log.Error(err, "SyncWaveHook failed") return } @@ -591,6 +600,7 @@ func (sc *syncContext) Sync() { switch runState { case failed: syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) + sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) sc.deleteHooks(hooksPendingDeletionFailed) sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") case successful: @@ -602,10 +612,31 @@ func (sc *syncContext) Sync() { sc.setRunningPhase(remainingTasks, false) } default: - sc.setRunningPhase(tasks.Filter(func(task *syncTask) bool { - return task.deleteOnPhaseCompletion() - }), true) + sc.setRunningPhase(tasks, true) + } +} + +// terminate looks for any running jobs/workflow hooks and deletes the resource +func (sc *syncContext) Terminate() { + sc.log.V(1).Info("terminating") + tasks, _ := sc.getSyncTasks() + terminateSuccessful := sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() && task.liveObj != nil })) + if terminateSuccessful { + sc.setOperationPhase(common.OperationFailed, "Operation terminated") + } else { + sc.setOperationPhase(common.OperationError, "Operation termination had errors") + } +} + +func (sc *syncContext) GetState() (common.OperationPhase, string, []common.ResourceSyncResult) { + var resourceRes []common.ResourceSyncResult + for _, v := range sc.syncRes { + resourceRes = append(resourceRes, v) } + sort.Slice(resourceRes, func(i, j int) bool { + return resourceRes[i].Order < resourceRes[j].Order + }) + return sc.phase, sc.message, resourceRes } // filter out out-of-sync tasks @@ -635,6 +666,61 @@ func (sc *syncContext) getNamespaceCreationTask(tasks syncTasks) *syncTask { return nil } +// terminateHooksPreemptively terminates ongoing hook tasks +func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { + terminateSuccessful := true + for _, task := range tasks { + if !task.isHook() { + continue + } + + if task.running() && task.liveObj == nil { + // if we terminate preemtively after the task was run, it will not contain the live object yet + liveObj, err := sc.getResource(task) + if err != nil && !apierrors.IsNotFound(err) { + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to get live resource: %v", err)) + terminateSuccessful = false + continue + } + task.liveObj = liveObj + } + + if task.liveObj == nil { + // if the live object does not exist, the resource has been deleted already + sc.setResourceResult(task, "", common.OperationSucceeded, "Terminated") + continue + } + + phase, msg, err := sc.getOperationPhase(task.liveObj) + if err != nil { + sc.setResourceResult(task, "", common.OperationError, fmt.Sprintf("Failed to get hook health: %v", err)) + phase = common.OperationRunning + } + + if err := sc.removeHookFinalizer(task); err != nil { + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to remove hook finalizer: %v", err)) + terminateSuccessful = false + continue + } + + if phase.Running() || task.deleteOnPhaseFailed() { + err := sc.deleteResource(task) + if err != nil && !apierrors.IsNotFound(err) { + sc.setResourceResult(task, "", common.OperationFailed, fmt.Sprintf("Failed to delete: %v", err)) + terminateSuccessful = false + continue + } + } + + if phase.Completed() { + sc.setResourceResult(task, "", phase, msg) + } else { + sc.setResourceResult(task, "", common.OperationSucceeded, "Terminated") + } + } + return terminateSuccessful +} + func (sc *syncContext) removeHookFinalizer(task *syncTask) error { if task.liveObj == nil { return nil @@ -663,11 +749,7 @@ func (sc *syncContext) removeHookFinalizer(task *syncTask) error { updateErr := sc.updateResource(task) if apierrors.IsConflict(updateErr) { sc.log.WithValues("task", task).V(1).Info("Retrying hook finalizer removal due to conflict on update") - resIf, err := sc.getResourceIf(task, "get") - if err != nil { - return fmt.Errorf("failed to get resource interface: %w", err) - } - liveObj, err := resIf.Get(context.TODO(), task.liveObj.GetName(), metav1.GetOptions{}) + liveObj, err := sc.getResource(task) if apierrors.IsNotFound(err) { sc.log.WithValues("task", task).V(1).Info("Resource is already deleted") return nil @@ -687,19 +769,6 @@ func (sc *syncContext) removeHookFinalizer(task *syncTask) error { }) } -func (sc *syncContext) updateResource(task *syncTask) error { - sc.log.WithValues("task", task).V(1).Info("Updating resource") - resIf, err := sc.getResourceIf(task, "update") - if err != nil { - return err - } - _, err = resIf.Update(context.TODO(), task.liveObj, metav1.UpdateOptions{}) - if err != nil { - return fmt.Errorf("failed to update resource: %w", err) - } - return nil -} - func (sc *syncContext) deleteHooks(hooksPendingDeletion syncTasks) { for _, task := range hooksPendingDeletion { err := sc.deleteResource(task) @@ -709,17 +778,6 @@ func (sc *syncContext) deleteHooks(hooksPendingDeletion syncTasks) { } } -func (sc *syncContext) GetState() (common.OperationPhase, string, []common.ResourceSyncResult) { - var resourceRes []common.ResourceSyncResult - for _, v := range sc.syncRes { - resourceRes = append(resourceRes, v) - } - sort.Slice(resourceRes, func(i, j int) bool { - return resourceRes[i].Order < resourceRes[j].Order - }) - return sc.phase, sc.message, resourceRes -} - func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTasks, message string) { errorMessageFactory := func(tasks syncTasks, message string) string { messages := tasks.Map(func(task *syncTask) string { @@ -752,7 +810,7 @@ func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTas // otherwise, we need to start the pending failure hooks, and then return WITHOUT setting // the phase to failed, since we want the failure hooks to complete their running state before failing - pendingSyncFailTasks := syncFailTasks.Filter(func(task *syncTask) bool { return task.pending() }) + pendingSyncFailTasks := syncFailTasks.Filter(func(task *syncTask) bool { return !task.completed() && !task.running() }) sc.log.WithValues("syncFailTasks", pendingSyncFailTasks).V(1).Info("Running sync fail tasks") sc.runTasks(pendingSyncFailTasks, false) } @@ -1279,42 +1337,30 @@ func (sc *syncContext) hasCRDOfGroupKind(group string, kind string) bool { return false } -// terminate looks for any running jobs/workflow hooks and deletes the resource -func (sc *syncContext) Terminate() { - terminateSuccessful := true - sc.log.V(1).Info("terminating") - tasks, _ := sc.getSyncTasks() - for _, task := range tasks { - if !task.isHook() || task.liveObj == nil { - continue - } - if err := sc.removeHookFinalizer(task); err != nil { - sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to remove hook finalizer: %v", err)) - terminateSuccessful = false - continue - } - phase, msg, err := sc.getOperationPhase(task.liveObj) - if err != nil { - sc.setOperationPhase(common.OperationError, fmt.Sprintf("Failed to get hook health: %v", err)) - return - } - if phase == common.OperationRunning { - err := sc.deleteResource(task) - if err != nil && !apierrors.IsNotFound(err) { - sc.setResourceResult(task, "", common.OperationFailed, fmt.Sprintf("Failed to delete: %v", err)) - terminateSuccessful = false - } else { - sc.setResourceResult(task, "", common.OperationSucceeded, "Deleted") - } - } else { - sc.setResourceResult(task, "", phase, msg) - } +func (sc *syncContext) getResource(task *syncTask) (*unstructured.Unstructured, error) { + sc.log.WithValues("task", task).V(1).Info("Getting resource") + resIf, err := sc.getResourceIf(task, "get") + if err != nil { + return nil, err } - if terminateSuccessful { - sc.setOperationPhase(common.OperationFailed, "Operation terminated") - } else { - sc.setOperationPhase(common.OperationError, "Operation termination had errors") + liveObj, err := resIf.Get(context.TODO(), task.name(), metav1.GetOptions{}) + if err != nil { + return nil, fmt.Errorf("failed to get resource: %w", err) + } + return liveObj, nil +} + +func (sc *syncContext) updateResource(task *syncTask) error { + sc.log.WithValues("task", task).V(1).Info("Updating resource") + resIf, err := sc.getResourceIf(task, "update") + if err != nil { + return err + } + _, err = resIf.Update(context.TODO(), task.liveObj, metav1.UpdateOptions{}) + if err != nil { + return fmt.Errorf("failed to update resource: %w", err) } + return nil } func (sc *syncContext) deleteResource(task *syncTask) error { @@ -1375,20 +1421,14 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { // prune first { if !sc.pruneConfirmed { - var resources []string for _, task := range pruneTasks { if resourceutil.HasAnnotationOption(task.liveObj, common.AnnotationSyncOptions, common.SyncOptionPruneRequireConfirm) { - resources = append(resources, fmt.Sprintf("%s/%s/%s", task.obj().GetAPIVersion(), task.obj().GetKind(), task.name())) + state = pending + sc.setResourceResult(task, "", common.OperationTerminating, "waiting for prune confirmation") } } - if len(resources) > 0 { - sc.log.WithValues("resources", resources).Info("Prune requires confirmation") - andMessage := "" - if len(resources) > 1 { - andMessage = fmt.Sprintf(" and %d more resources", len(resources)-1) - } - sc.message = fmt.Sprintf("Waiting for pruning confirmation of %s%s", resources[0], andMessage) - return pending + if state == pending { + return state } } @@ -1441,6 +1481,7 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { } else { // if there is anything that needs deleting, we are at best now in pending and // want to return and wait for sync to be invoked again + sc.setResourceResult(t, "", common.OperationTerminating, "waiting for deletion") state = pending } } diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 0e8d01ebb..0efc3bd50 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -41,28 +41,12 @@ import ( var standardVerbs = metav1.Verbs{"create", "delete", "deletecollection", "get", "list", "patch", "update", "watch"} func newTestSyncCtx(getResourceFunc *func(ctx context.Context, config *rest.Config, gvk schema.GroupVersionKind, name string, namespace string) (*unstructured.Unstructured, error), opts ...SyncOpt) *syncContext { - fakeDisco := &fakedisco.FakeDiscovery{Fake: &testcore.Fake{}} - fakeDisco.Resources = append(make([]*metav1.APIResourceList, 0), - &metav1.APIResourceList{ - GroupVersion: "v1", - APIResources: []metav1.APIResource{ - {Kind: "Pod", Group: "", Version: "v1", Namespaced: true, Verbs: standardVerbs}, - {Kind: "Service", Group: "", Version: "v1", Namespaced: true, Verbs: standardVerbs}, - {Kind: "Namespace", Group: "", Version: "v1", Namespaced: false, Verbs: standardVerbs}, - }, - }, - &metav1.APIResourceList{ - GroupVersion: "apps/v1", - APIResources: []metav1.APIResource{ - {Kind: "Deployment", Group: "apps", Version: "v1", Namespaced: true, Verbs: standardVerbs}, - }, - }) sc := syncContext{ config: &rest.Config{}, rawConfig: &rest.Config{}, namespace: testingutils.FakeArgoCDNamespace, revision: "FooBarBaz", - disco: fakeDisco, + disco: &fakedisco.FakeDiscovery{Fake: &testcore.Fake{Resources: testingutils.StaticAPIResources}}, log: textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app"), resources: map[kube.ResourceKey]reconciledResource{}, syncRes: map[string]synccommon.ResourceSyncResult{}, @@ -1285,22 +1269,42 @@ func TestSyncFailureHookWithFailedSync(t *testing.T) { } func TestBeforeHookCreation(t *testing.T) { + finalizerRemoved := false syncCtx := newTestSyncCtx(nil) - hook := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") - hook.SetNamespace(testingutils.FakeArgoCDNamespace) + hookObj := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") + hookObj.SetFinalizers([]string{hook.HookFinalizer}) + hookObj.SetNamespace(testingutils.FakeArgoCDNamespace) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{hook}, + Live: []*unstructured.Unstructured{hookObj}, Target: []*unstructured.Unstructured{nil}, }) - syncCtx.hooks = []*unstructured.Unstructured{hook} - syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) + syncCtx.hooks = []*unstructured.Unstructured{hookObj} + client := fake.NewSimpleDynamicClient(runtime.NewScheme(), hookObj) + client.PrependReactor("update", "pods", func(_ testcore.Action) (bool, runtime.Object, error) { + finalizerRemoved = true + return false, nil, nil + }) + syncCtx.dynamicIf = client + // First sync will delete the existing hook syncCtx.Sync() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for deletion of hook /Pod/my-pod", message) + assert.Len(t, resources, 1) + assert.Equal(t, synccommon.OperationTerminating, resources[0].HookPhase) + assert.True(t, finalizerRemoved) - _, _, resources := syncCtx.GetState() + // Make sure the sync is started, so dry-run is not applied twice + require.True(t, syncCtx.started()) + + // Second sync will create the hook + syncCtx.Sync() + phase, message, resources = syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) assert.Len(t, resources, 1) - assert.Empty(t, resources[0].Message) - assert.Equal(t, "waiting for completion of hook /Pod/my-pod", syncCtx.message) + assert.Equal(t, synccommon.OperationRunning, resources[0].HookPhase) + assert.Equal(t, "waiting for completion of hook /Pod/my-pod", message) } func TestRunSyncFailHooksFailed(t *testing.T) { @@ -1702,7 +1706,7 @@ func TestSyncWaveHookFail(t *testing.T) { syncCtx.Sync() assert.True(t, called) phase, msg, results := syncCtx.GetState() - assert.Equal(t, synccommon.OperationFailed, phase) + assert.Equal(t, synccommon.OperationError, phase) assert.Equal(t, "SyncWaveHook failed: intentional error", msg) assert.Equal(t, synccommon.OperationRunning, results[0].HookPhase) } diff --git a/pkg/utils/testing/api_resources.go b/pkg/utils/testing/api_resources.go new file mode 100644 index 000000000..ebfc53543 --- /dev/null +++ b/pkg/utils/testing/api_resources.go @@ -0,0 +1,81 @@ +package testing + +import metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + +var ( + commonVerbs = []string{"create", "get", "list", "watch", "update", "patch", "delete", "deletecollection"} + subresourceVerbs = []string{"get", "update", "patch"} +) + +// StaticAPIResources defines the common Kubernetes API resources that are usually returned by a DiscoveryClient +var StaticAPIResources = []*metav1.APIResourceList{ + { + GroupVersion: "v1", + APIResources: []metav1.APIResource{ + {Name: "pods", SingularName: "pod", Namespaced: true, Kind: "Pod", Verbs: commonVerbs}, + {Name: "pods/status", SingularName: "", Namespaced: true, Kind: "Pod", Verbs: subresourceVerbs}, + {Name: "pods/log", SingularName: "", Namespaced: true, Kind: "Pod", Verbs: []string{"get"}}, + {Name: "pods/exec", SingularName: "", Namespaced: true, Kind: "Pod", Verbs: []string{"create"}}, + {Name: "services", SingularName: "service", Namespaced: true, Kind: "Service", Verbs: commonVerbs}, + {Name: "services/status", SingularName: "", Namespaced: true, Kind: "Service", Verbs: subresourceVerbs}, + {Name: "configmaps", SingularName: "configmap", Namespaced: true, Kind: "ConfigMap", Verbs: commonVerbs}, + {Name: "secrets", SingularName: "secret", Namespaced: true, Kind: "Secret", Verbs: commonVerbs}, + {Name: "namespaces", SingularName: "namespace", Namespaced: false, Kind: "Namespace", Verbs: commonVerbs}, + {Name: "namespaces/status", SingularName: "", Namespaced: false, Kind: "Namespace", Verbs: subresourceVerbs}, + {Name: "nodes", SingularName: "node", Namespaced: false, Kind: "Node", Verbs: []string{"get", "list", "watch"}}, + {Name: "persistentvolumes", SingularName: "persistentvolume", Namespaced: false, Kind: "PersistentVolume", Verbs: commonVerbs}, + {Name: "persistentvolumeclaims", SingularName: "persistentvolumeclaim", Namespaced: true, Kind: "PersistentVolumeClaim", Verbs: commonVerbs}, + {Name: "persistentvolumeclaims/status", SingularName: "", Namespaced: true, Kind: "PersistentVolumeClaim", Verbs: subresourceVerbs}, + {Name: "events", SingularName: "event", Namespaced: true, Kind: "Event", Verbs: []string{"create", "get", "list", "watch"}}, + {Name: "serviceaccounts", SingularName: "serviceaccount", Namespaced: true, Kind: "ServiceAccount", Verbs: commonVerbs}, + }, + }, + { + GroupVersion: "apps/v1", + APIResources: []metav1.APIResource{ + {Name: "deployments", SingularName: "deployment", Namespaced: true, Kind: "Deployment", Verbs: commonVerbs}, + {Name: "deployments/status", SingularName: "", Namespaced: true, Kind: "Deployment", Verbs: subresourceVerbs}, + {Name: "deployments/scale", SingularName: "", Namespaced: true, Kind: "Scale", Verbs: subresourceVerbs}, + {Name: "statefulsets", SingularName: "statefulset", Namespaced: true, Kind: "StatefulSet", Verbs: commonVerbs}, + {Name: "statefulsets/status", SingularName: "", Namespaced: true, Kind: "StatefulSet", Verbs: subresourceVerbs}, + {Name: "statefulsets/scale", SingularName: "", Namespaced: true, Kind: "Scale", Verbs: subresourceVerbs}, + {Name: "daemonsets", SingularName: "daemonset", Namespaced: true, Kind: "DaemonSet", Verbs: commonVerbs}, + {Name: "daemonsets/status", SingularName: "", Namespaced: true, Kind: "DaemonSet", Verbs: subresourceVerbs}, + {Name: "replicasets", SingularName: "replicaset", Namespaced: true, Kind: "ReplicaSet", Verbs: commonVerbs}, + {Name: "replicasets/status", SingularName: "", Namespaced: true, Kind: "ReplicaSet", Verbs: subresourceVerbs}, + }, + }, + { + GroupVersion: "batch/v1", + APIResources: []metav1.APIResource{ + {Name: "jobs", SingularName: "job", Namespaced: true, Kind: "Job", Verbs: commonVerbs}, + {Name: "jobs/status", SingularName: "", Namespaced: true, Kind: "Job", Verbs: subresourceVerbs}, + {Name: "cronjobs", SingularName: "cronjob", Namespaced: true, Kind: "CronJob", Verbs: commonVerbs}, + {Name: "cronjobs/status", SingularName: "", Namespaced: true, Kind: "CronJob", Verbs: subresourceVerbs}, + }, + }, + { + GroupVersion: "rbac.authorization.k8s.io/v1", + APIResources: []metav1.APIResource{ + {Name: "roles", SingularName: "role", Namespaced: true, Kind: "Role", Verbs: commonVerbs}, + {Name: "rolebindings", SingularName: "rolebinding", Namespaced: true, Kind: "RoleBinding", Verbs: commonVerbs}, + {Name: "clusterroles", SingularName: "clusterrole", Namespaced: false, Kind: "ClusterRole", Verbs: commonVerbs}, + {Name: "clusterrolebindings", SingularName: "clusterrolebinding", Namespaced: false, Kind: "ClusterRoleBinding", Verbs: commonVerbs}, + }, + }, + { + GroupVersion: "networking.k8s.io/v1", + APIResources: []metav1.APIResource{ + {Name: "ingresses", SingularName: "ingress", Namespaced: true, Kind: "Ingress", Verbs: commonVerbs}, + {Name: "ingresses/status", SingularName: "", Namespaced: true, Kind: "Ingress", Verbs: subresourceVerbs}, + {Name: "networkpolicies", SingularName: "networkpolicy", Namespaced: true, Kind: "NetworkPolicy", Verbs: commonVerbs}, + }, + }, + { + GroupVersion: "policy/v1", + APIResources: []metav1.APIResource{ + {Name: "poddisruptionbudgets", SingularName: "poddisruptionbudget", Namespaced: true, Kind: "PodDisruptionBudget", Verbs: commonVerbs}, + {Name: "poddisruptionbudgets/status", SingularName: "", Namespaced: true, Kind: "PodDisruptionBudget", Verbs: subresourceVerbs}, + }, + }, +} From 9f490be34fd16bb6c06444c9b06a1ffa18fc1313 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 12 Aug 2025 17:56:25 -0400 Subject: [PATCH 05/28] always remove finalizers on existing task before creation Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 32 +++++++++++++++++++++++++------- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 2cdf653a4..b9e71761a 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -1418,6 +1418,31 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { createTasks = append(createTasks, task) } } + + // remove finalizers from previous sync on existing hooks to make sure the operation is idempotent + { + ss := newStateSync(state) + existingHooks := tasks.Filter(func(t *syncTask) bool { return t.isHook() && t.pending() && t.liveObj != nil }) + for _, task := range existingHooks { + t := task + ss.Go(func(state runState) runState { + logCtx := sc.log.WithValues("dryRun", dryRun, "task", t) + logCtx.V(1).Info("Removing finalizers") + if !dryRun { + if err := sc.removeHookFinalizer(t); err != nil { + state = failed + sc.setResourceResult(t, "", common.OperationError, fmt.Sprintf("failed to remove hook finalizer: %v", err)) + } + } + return state + }) + } + state = ss.Wait() + } + if state != successful { + return state + } + // prune first { if !sc.pruneConfirmed { @@ -1451,7 +1476,6 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { } state = ss.Wait() } - if state != successful { return state } @@ -1465,11 +1489,6 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { ss.Go(func(state runState) runState { sc.log.WithValues("dryRun", dryRun, "task", t).V(1).Info("Deleting") if !dryRun { - // the hook might still have a finalizer, but we explicitly want to recreate it - if err := sc.removeHookFinalizer(t); err != nil { - state = failed - sc.setResourceResult(t, "", common.OperationError, fmt.Sprintf("failed to remove hook finalizer: %v", err)) - } err := sc.deleteResource(t) if err != nil { // it is possible to get a race condition here, such that the resource does not exist when @@ -1490,7 +1509,6 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { } state = ss.Wait() } - if state != successful { return state } From 412d96b57d70fa86de380a71108af1da7c9b7850 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 13 Aug 2025 16:13:32 -0400 Subject: [PATCH 06/28] cleanup status update on terminate Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 36 +++++++++++++++++++----------------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index b9e71761a..cceb3d0ee 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -539,18 +539,16 @@ func (sc *syncContext) Sync() { syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) - terminatingTasks := tasks.Filter(func(t *syncTask) bool { return t.operationState == common.OperationTerminating }) - // if there are any completed but unsuccessful tasks, sync is a failure. // we already know tasks do not contain running tasks, but it may contain terminating tasks waiting for pruning/cleanup - if len(terminatingTasks) == 0 && tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { + if tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { sc.deleteHooks(hooksPendingDeletionFailed) sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more synchronization tasks completed unsuccessfully") return } sc.log.WithValues("tasks", tasks).V(1).Info("Filtering out non-pending tasks") - // remove tasks that are completed, we should only have pending and terminating tasks left + // remove tasks that are completed, we should only have pending tasks left tasks = tasks.Filter(func(t *syncTask) bool { return !t.completed() }) if sc.applyOutOfSyncOnly { @@ -670,11 +668,11 @@ func (sc *syncContext) getNamespaceCreationTask(tasks syncTasks) *syncTask { func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { terminateSuccessful := true for _, task := range tasks { - if !task.isHook() { + if !task.isHook() || !task.running() { continue } - if task.running() && task.liveObj == nil { + if task.liveObj == nil { // if we terminate preemtively after the task was run, it will not contain the live object yet liveObj, err := sc.getResource(task) if err != nil && !apierrors.IsNotFound(err) { @@ -686,14 +684,13 @@ func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { } if task.liveObj == nil { - // if the live object does not exist, the resource has been deleted already - sc.setResourceResult(task, "", common.OperationSucceeded, "Terminated") + sc.setResourceResult(task, task.syncStatus, common.OperationSucceeded, "Terminated") continue } phase, msg, err := sc.getOperationPhase(task.liveObj) if err != nil { - sc.setResourceResult(task, "", common.OperationError, fmt.Sprintf("Failed to get hook health: %v", err)) + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to get hook health: %v", err)) phase = common.OperationRunning } @@ -706,16 +703,16 @@ func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { if phase.Running() || task.deleteOnPhaseFailed() { err := sc.deleteResource(task) if err != nil && !apierrors.IsNotFound(err) { - sc.setResourceResult(task, "", common.OperationFailed, fmt.Sprintf("Failed to delete: %v", err)) + sc.setResourceResult(task, task.syncStatus, common.OperationFailed, fmt.Sprintf("Failed to delete: %v", err)) terminateSuccessful = false continue } } if phase.Completed() { - sc.setResourceResult(task, "", phase, msg) + sc.setResourceResult(task, task.syncStatus, phase, msg) } else { - sc.setResourceResult(task, "", common.OperationSucceeded, "Terminated") + sc.setResourceResult(task, task.syncStatus, common.OperationFailed, "Terminated") } } return terminateSuccessful @@ -1446,14 +1443,20 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { // prune first { if !sc.pruneConfirmed { + var resources []string for _, task := range pruneTasks { if resourceutil.HasAnnotationOption(task.liveObj, common.AnnotationSyncOptions, common.SyncOptionPruneRequireConfirm) { - state = pending - sc.setResourceResult(task, "", common.OperationTerminating, "waiting for prune confirmation") + resources = append(resources, fmt.Sprintf("%s/%s/%s", task.obj().GetAPIVersion(), task.obj().GetKind(), task.name())) } } - if state == pending { - return state + if len(resources) > 0 { + sc.log.WithValues("resources", resources).Info("Prune requires confirmation") + andMessage := "" + if len(resources) > 1 { + andMessage = fmt.Sprintf(" and %d more resources", len(resources)-1) + } + sc.message = fmt.Sprintf("Waiting for pruning confirmation of %s%s", resources[0], andMessage) + return pending } } @@ -1500,7 +1503,6 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { } else { // if there is anything that needs deleting, we are at best now in pending and // want to return and wait for sync to be invoked again - sc.setResourceResult(t, "", common.OperationTerminating, "waiting for deletion") state = pending } } From 230a1c0510258ef1fb2ab77ba7f720482c7921f4 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 18 Aug 2025 11:10:26 -0400 Subject: [PATCH 07/28] handle sync failure on manifest apply Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 44 +++++++++++++++++++++++------------ pkg/sync/sync_context_test.go | 12 +++++----- 2 files changed, 35 insertions(+), 21 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index cceb3d0ee..4e6b65906 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -442,7 +442,7 @@ func (sc *syncContext) Sync() { // No need to perform a dry-run on the namespace creation, because if it fails we stop anyway sc.log.WithValues("task", nsCreateTask).Info("Creating namespace") if sc.runTasks(nsSyncTasks, false) == failed { - sc.setOperationFailed(syncTasks{}, nsSyncTasks, "the namespace failed to apply") + sc.executeSyncFailPhase(syncTasks{}, nsSyncTasks, "the namespace failed to apply") return } @@ -470,9 +470,9 @@ func (sc *syncContext) Sync() { // update the hook's result operationState, message, err := sc.getOperationPhase(task.liveObj) if err != nil { - sc.setResourceResult(task, "", common.OperationError, fmt.Sprintf("failed to get resource health: %v", err)) + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("failed to get resource health: %v", err)) } else { - sc.setResourceResult(task, "", operationState, message) + sc.setResourceResult(task, task.syncStatus, operationState, message) } } else { // this must be calculated on the live object @@ -540,10 +540,10 @@ func (sc *syncContext) Sync() { syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) // if there are any completed but unsuccessful tasks, sync is a failure. - // we already know tasks do not contain running tasks, but it may contain terminating tasks waiting for pruning/cleanup + // we already know tasks do not contain running tasks if tasks.Any(func(t *syncTask) bool { return t.completed() && !t.successful() }) { sc.deleteHooks(hooksPendingDeletionFailed) - sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more synchronization tasks completed unsuccessfully") + sc.executeSyncFailPhase(syncFailTasks, syncFailedTasks, "one or more synchronization tasks completed unsuccessfully") return } @@ -588,7 +588,6 @@ func (sc *syncContext) Sync() { // Since this is an unexpected error and is not related to a specific task, terminate the sync with error // without triggering the syncFailTasks sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) - sc.deleteHooks(hooksPendingDeletionFailed) sc.setOperationPhase(common.OperationError, fmt.Sprintf("SyncWaveHook failed: %v", err)) sc.log.Error(err, "SyncWaveHook failed") return @@ -597,10 +596,24 @@ func (sc *syncContext) Sync() { switch runState { case failed: + // If we failed to apply at least one resource, we need to start the syncFailTasks and wait + // for the completion of any running hooks. In this case, the operation should be running. syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) - sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) - sc.deleteHooks(hooksPendingDeletionFailed) - sc.setOperationFailed(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") + runningHooks := tasks.Filter(func(t *syncTask) bool { return t.isHook() && !t.completed() }) + if len(runningHooks) > 0 { + if len(syncFailTasks) > 0 { + completed := sc.executeSyncFailPhase(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") + if !completed { + runningHooks = append(runningHooks, syncFailTasks...) + } + } + sc.setRunningPhase(runningHooks, false) + } else { + completed := sc.executeSyncFailPhase(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") + if completed { + sc.deleteHooks(hooksPendingDeletionFailed) + } + } case successful: if remainingTasks.Len() == 0 { // delete all completed hooks which have appropriate delete policy @@ -770,12 +783,12 @@ func (sc *syncContext) deleteHooks(hooksPendingDeletion syncTasks) { for _, task := range hooksPendingDeletion { err := sc.deleteResource(task) if err != nil && !apierrors.IsNotFound(err) { - sc.setResourceResult(task, "", common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) } } } -func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTasks, message string) { +func (sc *syncContext) executeSyncFailPhase(syncFailTasks, syncFailedTasks syncTasks, message string) (completed bool) { errorMessageFactory := func(tasks syncTasks, message string) string { messages := tasks.Map(func(task *syncTask) string { return task.message @@ -791,7 +804,7 @@ func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTas // if there is no failure hook, there is nothing more to do and we can fail if len(syncFailTasks) == 0 { sc.setOperationPhase(common.OperationFailed, errorMessage) - return + return true } // if all the failure hooks are completed, mark the sync as failed @@ -802,7 +815,7 @@ func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTas errorMessage = fmt.Sprintf("%s\n%s", errorMessage, syncFailTasksMessage) } sc.setOperationPhase(common.OperationFailed, errorMessage) - return + return true } // otherwise, we need to start the pending failure hooks, and then return WITHOUT setting @@ -810,6 +823,7 @@ func (sc *syncContext) setOperationFailed(syncFailTasks, syncFailedTasks syncTas pendingSyncFailTasks := syncFailTasks.Filter(func(task *syncTask) bool { return !task.completed() && !task.running() }) sc.log.WithValues("syncFailTasks", pendingSyncFailTasks).V(1).Info("Running sync fail tasks") sc.runTasks(pendingSyncFailTasks, false) + return false } func (sc *syncContext) started() bool { @@ -1428,7 +1442,7 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { if !dryRun { if err := sc.removeHookFinalizer(t); err != nil { state = failed - sc.setResourceResult(t, "", common.OperationError, fmt.Sprintf("failed to remove hook finalizer: %v", err)) + sc.setResourceResult(t, t.syncStatus, common.OperationError, fmt.Sprintf("failed to remove hook finalizer: %v", err)) } } return state @@ -1498,7 +1512,7 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { // delete is requested, we treat this as a nop if !apierrors.IsNotFound(err) { state = failed - sc.setResourceResult(t, "", common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) + sc.setResourceResult(t, t.syncStatus, common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) } } else { // if there is anything that needs deleting, we are at best now in pending and diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 0efc3bd50..f43ae45cf 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1814,19 +1814,19 @@ func TestSyncContext_GetDeleteOptions_WithPrunePropagationPolicy(t *testing.T) { assert.Equal(t, metav1.DeletePropagationBackground, *opts.PropagationPolicy) } -func TestSetOperationFailed(t *testing.T) { +func TestExecuteSyncFailPhase(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") tasks := make([]*syncTask, 0) tasks = append(tasks, &syncTask{message: "namespace not found"}) - sc.setOperationFailed(nil, tasks, "one or more objects failed to apply") + sc.executeSyncFailPhase(nil, tasks, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply, reason: namespace not found", sc.message) } -func TestSetOperationFailedDuplicatedMessages(t *testing.T) { +func TestExecuteSyncFailPhaseDuplicatedMessages(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") @@ -1834,16 +1834,16 @@ func TestSetOperationFailedDuplicatedMessages(t *testing.T) { tasks = append(tasks, &syncTask{message: "namespace not found"}) tasks = append(tasks, &syncTask{message: "namespace not found"}) - sc.setOperationFailed(nil, tasks, "one or more objects failed to apply") + sc.executeSyncFailPhase(nil, tasks, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply, reason: namespace not found", sc.message) } -func TestSetOperationFailedNoTasks(t *testing.T) { +func TestExecuteSyncFailPhaseNoTasks(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") - sc.setOperationFailed(nil, nil, "one or more objects failed to apply") + sc.executeSyncFailPhase(nil, nil, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply", sc.message) } From cccd178bfb03c3b9beb5a0bad215cb97cd3ce5c1 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 18 Aug 2025 11:17:14 -0400 Subject: [PATCH 08/28] fix test Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index f43ae45cf..69b876187 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1288,19 +1288,13 @@ func TestBeforeHookCreation(t *testing.T) { // First sync will delete the existing hook syncCtx.Sync() - phase, message, resources := syncCtx.GetState() + phase, _, _ := syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) - assert.Equal(t, "waiting for deletion of hook /Pod/my-pod", message) - assert.Len(t, resources, 1) - assert.Equal(t, synccommon.OperationTerminating, resources[0].HookPhase) assert.True(t, finalizerRemoved) - // Make sure the sync is started, so dry-run is not applied twice - require.True(t, syncCtx.started()) - // Second sync will create the hook syncCtx.Sync() - phase, message, resources = syncCtx.GetState() + phase, message, resources := syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) assert.Len(t, resources, 1) assert.Equal(t, synccommon.OperationRunning, resources[0].HookPhase) From 8c43f140a2a8f5a3194bde30a3f61adfba25d7c1 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 18 Aug 2025 11:19:56 -0400 Subject: [PATCH 09/28] clarify doc Signed-off-by: Alexandre Gaudreault --- pkg/sync/doc.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sync/doc.go b/pkg/sync/doc.go index e3a28e7ed..944b846fe 100644 --- a/pkg/sync/doc.go +++ b/pkg/sync/doc.go @@ -61,8 +61,8 @@ Hooks can be deleted in an automatic fashion using the annotation: argocd.argopr The following policies define when the hook will be deleted. - - HookSucceeded - the hook resource is deleted if the hook's phase succeeded (e.g. Job/Workflow completed successfully). - - HookFailed - the hook resource is deleted if the hook's phase failed. + - HookSucceeded - the hook resource is deleted if the sync succeed (e.g. Job/Workflow completed successfully). + - HookFailed - the hook resource is deleted if the sync fails. - BeforeHookCreation - any existing hook resource is deleted before the new one is created # Sync Waves From 5399bd1fa45bf45b3553f021bf1ada3b9a278d71 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Mon, 18 Aug 2025 11:22:23 -0400 Subject: [PATCH 10/28] typo Signed-off-by: Alexandre Gaudreault --- pkg/sync/doc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sync/doc.go b/pkg/sync/doc.go index 944b846fe..c025ed9fe 100644 --- a/pkg/sync/doc.go +++ b/pkg/sync/doc.go @@ -61,7 +61,7 @@ Hooks can be deleted in an automatic fashion using the annotation: argocd.argopr The following policies define when the hook will be deleted. - - HookSucceeded - the hook resource is deleted if the sync succeed (e.g. Job/Workflow completed successfully). + - HookSucceeded - the hook resource is deleted if the sync succeeds (e.g. Job/Workflow completed successfully). - HookFailed - the hook resource is deleted if the sync fails. - BeforeHookCreation - any existing hook resource is deleted before the new one is created From dc9311d85b8c4a85f163b23a8f22beae4f48dc58 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 13:13:53 -0400 Subject: [PATCH 11/28] race condition on BeforeHookCreation and replace Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 4e6b65906..a4dfd2f0d 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -1509,10 +1509,12 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { err := sc.deleteResource(t) if err != nil { // it is possible to get a race condition here, such that the resource does not exist when - // delete is requested, we treat this as a nop + // delete is requested, we treat this as a nop and remove the liveObj if !apierrors.IsNotFound(err) { state = failed sc.setResourceResult(t, t.syncStatus, common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) + } else { + t.liveObj = nil } } else { // if there is anything that needs deleting, we are at best now in pending and From 4416a7e0a1071b0c87f2c545490c279fef32c05e Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 13:23:02 -0400 Subject: [PATCH 12/28] log Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index a4dfd2f0d..10c7d0e5b 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -1504,7 +1504,8 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { for _, task := range hooksPendingDeletion { t := task ss.Go(func(state runState) runState { - sc.log.WithValues("dryRun", dryRun, "task", t).V(1).Info("Deleting") + log := sc.log.WithValues("dryRun", dryRun, "task", t).V(1) + log.Info("Deleting") if !dryRun { err := sc.deleteResource(t) if err != nil { @@ -1514,6 +1515,7 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { state = failed sc.setResourceResult(t, t.syncStatus, common.OperationError, fmt.Sprintf("failed to delete resource: %v", err)) } else { + log.Info("Resource not found, treating as no-op and removing liveObj") t.liveObj = nil } } else { From fa0bdc35d7a0c8638331b3c3fc027d1ded2f8809 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 15:17:00 -0400 Subject: [PATCH 13/28] review Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 10c7d0e5b..77612fabd 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -548,8 +548,8 @@ func (sc *syncContext) Sync() { } sc.log.WithValues("tasks", tasks).V(1).Info("Filtering out non-pending tasks") - // remove tasks that are completed, we should only have pending tasks left - tasks = tasks.Filter(func(t *syncTask) bool { return !t.completed() }) + // remove tasks that are completed, we can assume that there are no running tasks + tasks = tasks.Filter(func(t *syncTask) bool { return t.pending() }) if sc.applyOutOfSyncOnly { tasks = sc.filterOutOfSyncTasks(tasks) From b652e65ddda07b7de7a98615a781183d1024b2e2 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 19:43:13 -0400 Subject: [PATCH 14/28] unit tests Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 2 +- pkg/sync/sync_context_test.go | 456 +++++++++++++++++++++++++++++++++- pkg/utils/testing/testdata.go | 13 + 3 files changed, 458 insertions(+), 13 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 77612fabd..d87584d26 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -631,7 +631,7 @@ func (sc *syncContext) Sync() { func (sc *syncContext) Terminate() { sc.log.V(1).Info("terminating") tasks, _ := sc.getSyncTasks() - terminateSuccessful := sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() && task.liveObj != nil })) + terminateSuccessful := sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) if terminateSuccessful { sc.setOperationPhase(common.OperationFailed, "Operation terminated") } else { diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 69b876187..964a55ee8 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -823,6 +823,39 @@ func TestSync_Replace(t *testing.T) { } } +func TestSync_HookWithReplaceAndBeforeHookCreation_AlreadyDeleted(t *testing.T) { + // This test a race condition when Delete is called on an already deleted object + // LiveObj is set, but then the resource is deleted asynchronously in kubernetes + syncCtx := newTestSyncCtx(nil) + + target := withReplaceAnnotation(testingutils.NewPod()) + target.SetNamespace(testingutils.FakeArgoCDNamespace) + target = testingutils.Annotate(target, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyBeforeHookCreation)) + target = testingutils.Annotate(target, synccommon.AnnotationKeyHook, string(synccommon.SyncPhasePreSync)) + live := target.DeepCopy() + + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{live}, + Target: []*unstructured.Unstructured{target}, + }) + syncCtx.hooks = []*unstructured.Unstructured{live} + + client := fake.NewSimpleDynamicClient(runtime.NewScheme()) + deleted := false + client.PrependReactor("delete", "pods", func(_ testcore.Action) (bool, runtime.Object, error) { + deleted = true + // simulate the race conditions where liveObj was not null, but is now deleted in k8s + return true, nil, apierrors.NewNotFound(corev1.Resource("pods"), live.GetName()) + }) + syncCtx.dynamicIf = client + + syncCtx.Sync() + + resourceOps, _ := syncCtx.resourceOps.(*kubetest.MockResourceOps) + assert.Equal(t, "create", resourceOps.GetLastResourceCommand(kube.GetResourceKey(target))) + assert.True(t, deleted) +} + func withServerSideApplyAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionServerSideApply}) return un @@ -1358,7 +1391,54 @@ func (r resourceNameHealthOverride) GetResourceHealth(obj *unstructured.Unstruct return nil, nil } -func TestRunSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { +func TestSync_ExistingHooksWithFinalizer(t *testing.T) { + hook1 := newHook(synccommon.HookTypePreSync) + hook1.SetName("existing-hook-1") + hook1.SetNamespace(testingutils.FakeArgoCDNamespace) + _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyBeforeHookCreation)) + existingHook := hook1.DeepCopy() + existingHook.SetFinalizers(append(existingHook.GetFinalizers(), hook.HookFinalizer)) + + hook2 := newHook(synccommon.HookTypePreSync) + hook2.SetName("existing-hook-2") + hook2.SetNamespace(testingutils.FakeArgoCDNamespace) + _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) + existingHook2 := hook2.DeepCopy() + existingHook2.SetFinalizers(append(existingHook2.GetFinalizers(), hook.HookFinalizer)) + + syncCtx := newTestSyncCtx(nil) + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), existingHook, existingHook2) + syncCtx.dynamicIf = fakeDynamicClient + updatedCount := 0 + fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + // Removing the finalizers + updatedCount++ + return true, nil, nil + }) + deletedCount := 0 + fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + // because of HookDeletePolicyBeforeHookCreation + deletedCount++ + return true, nil, nil + }) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{existingHook, existingHook2}, + Target: []*unstructured.Unstructured{nil, nil}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2} + + syncCtx.kubectl = &kubetest.MockKubectlCmd{ + Commands: map[string]kubetest.KubectlOutput{}, + } + + syncCtx.Sync() + + assert.Equal(t, synccommon.OperationRunning, syncCtx.phase) + assert.Equal(t, 2, updatedCount) + assert.Equal(t, 1, deletedCount) +} + +func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { hook1 := newHook(synccommon.HookTypePreSync) hook1.SetName("completed-hook") hook1.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -1418,7 +1498,7 @@ func TestRunSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { assert.Equal(t, 0, deletedCount) } -func TestRunSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { +func TestSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { hook1 := newHook(synccommon.HookTypePreSync) hook1.SetName("completed-hook1") hook1.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -1429,7 +1509,7 @@ func TestRunSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { hook2 := newHook(synccommon.HookTypePreSync) hook2.SetNamespace(testingutils.FakeArgoCDNamespace) hook2.SetName("completed-hook2") - _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) + _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) completedHook2 := hook2.DeepCopy() completedHook2.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) @@ -1472,14 +1552,14 @@ func TestRunSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { assert.Equal(t, synccommon.OperationSucceeded, syncCtx.phase) assert.Equal(t, 2, updatedCount) - assert.Equal(t, 2, deletedCount) + assert.Equal(t, 1, deletedCount) } -func TestRunSync_HooksDeletedAfterPhaseCompletedFailed(t *testing.T) { +func TestSync_HooksDeletedAfterPhaseCompletedFailed(t *testing.T) { hook1 := newHook(synccommon.HookTypeSync) hook1.SetName("completed-hook1") hook1.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) + _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) completedHook1 := hook1.DeepCopy() completedHook1.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) @@ -1529,7 +1609,7 @@ func TestRunSync_HooksDeletedAfterPhaseCompletedFailed(t *testing.T) { assert.Equal(t, synccommon.OperationFailed, syncCtx.phase) assert.Equal(t, 2, updatedCount) - assert.Equal(t, 2, deletedCount) + assert.Equal(t, 1, deletedCount) } func Test_syncContext_liveObj(t *testing.T) { @@ -1682,15 +1762,20 @@ func TestSyncWaveHook(t *testing.T) { assert.True(t, called) } -func TestSyncWaveHookFail(t *testing.T) { +func TestSync_SyncWaveHookError(t *testing.T) { syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, false, false, false)) pod1 := testingutils.NewPod() pod1.SetName("pod-1") + syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, Target: []*unstructured.Unstructured{pod1}, }) + syncCtx.hooks = []*unstructured.Unstructured{ + newHook(synccommon.HookTypeSync), + newHook(synccommon.HookTypeSyncFail), + } called := false syncCtx.syncWaveHook = func(_ synccommon.SyncPhase, _ int, _ bool) error { @@ -1702,7 +1787,11 @@ func TestSyncWaveHookFail(t *testing.T) { phase, msg, results := syncCtx.GetState() assert.Equal(t, synccommon.OperationError, phase) assert.Equal(t, "SyncWaveHook failed: intentional error", msg) - assert.Equal(t, synccommon.OperationRunning, results[0].HookPhase) + require.Len(t, results, 2) + assert.Equal(t, kube.ResourceKey{Group: "", Kind: "Pod", Namespace: "fake-argocd-ns", Name: "my-pod"}, results[0].ResourceKey) + assert.Equal(t, "Terminated", results[0].Message) + assert.Equal(t, kube.ResourceKey{Group: "", Kind: "Pod", Namespace: "fake-argocd-ns", Name: "pod-1"}, results[1].ResourceKey) + assert.Equal(t, synccommon.OperationRunning, results[1].HookPhase) } func TestPruneLast(t *testing.T) { @@ -1808,7 +1897,7 @@ func TestSyncContext_GetDeleteOptions_WithPrunePropagationPolicy(t *testing.T) { assert.Equal(t, metav1.DeletePropagationBackground, *opts.PropagationPolicy) } -func TestExecuteSyncFailPhase(t *testing.T) { +func Test_executeSyncFailPhase(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") @@ -1818,9 +1907,10 @@ func TestExecuteSyncFailPhase(t *testing.T) { sc.executeSyncFailPhase(nil, tasks, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply, reason: namespace not found", sc.message) + assert.Equal(t, synccommon.OperationFailed, sc.phase) } -func TestExecuteSyncFailPhaseDuplicatedMessages(t *testing.T) { +func Test_executeSyncFailPhase_DuplicatedMessages(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") @@ -1831,15 +1921,50 @@ func TestExecuteSyncFailPhaseDuplicatedMessages(t *testing.T) { sc.executeSyncFailPhase(nil, tasks, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply, reason: namespace not found", sc.message) + assert.Equal(t, synccommon.OperationFailed, sc.phase) } -func TestExecuteSyncFailPhaseNoTasks(t *testing.T) { +func Test_executeSyncFailPhase_NoTasks(t *testing.T) { sc := syncContext{} sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") sc.executeSyncFailPhase(nil, nil, "one or more objects failed to apply") assert.Equal(t, "one or more objects failed to apply", sc.message) + assert.Equal(t, synccommon.OperationFailed, sc.phase) +} + +func Test_executeSyncFailPhase_RunningHooks(t *testing.T) { + sc := syncContext{ + phase: synccommon.OperationRunning, + } + sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") + + tasks := make([]*syncTask, 0) + tasks = append(tasks, &syncTask{operationState: synccommon.OperationRunning}) + + sc.executeSyncFailPhase(tasks, nil, "one or more objects failed to apply") + + assert.Equal(t, synccommon.OperationRunning, sc.phase) +} + +func Test_executeSyncFailPhase_CompletedHooks(t *testing.T) { + sc := syncContext{ + phase: synccommon.OperationRunning, + } + sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") + + failed := make([]*syncTask, 0) + failed = append(failed, &syncTask{message: "task in error"}) + + tasks := make([]*syncTask, 0) + tasks = append(tasks, &syncTask{operationState: synccommon.OperationSucceeded}) + tasks = append(tasks, &syncTask{operationState: synccommon.OperationFailed, syncStatus: synccommon.ResultCodeSyncFailed, message: "failed to apply"}) + + sc.executeSyncFailPhase(tasks, failed, "one or more objects failed to apply") + + assert.Equal(t, "one or more objects failed to apply, reason: task in error\none or more SyncFail hooks failed, reason: failed to apply", sc.message) + assert.Equal(t, synccommon.OperationFailed, sc.phase) } func TestWaveReorderingOfPruneTasks(t *testing.T) { @@ -2260,3 +2385,310 @@ func TestNeedsClientSideApplyMigration(t *testing.T) { }) } } + +func TestTerminate(t *testing.T) { + obj := testingutils.NewPod() + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, + WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ + ResourceKey: kube.GetResourceKey(obj), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + }}, + metav1.Now(), + )) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{obj}, + Target: []*unstructured.Unstructured{obj}, + }) + syncCtx.hooks = []*unstructured.Unstructured{} + + syncCtx.Terminate() + assert.Equal(t, synccommon.OperationFailed, syncCtx.phase) + assert.Equal(t, "Operation terminated", syncCtx.message) +} + +func TestTerminate_Hooks_Running(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + + obj := testingutils.NewPod() + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusProgressing, + hook2.GetName(): health.HealthStatusProgressing, + hook3.GetName(): health.HealthStatusProgressing, + })), + WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 0, + }, { + ResourceKey: kube.GetResourceKey(hook2), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 2, + }, { + ResourceKey: kube.GetResourceKey(obj), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 3, + }}, + metav1.Now(), + )) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{obj}, + Target: []*unstructured.Unstructured{obj}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) + syncCtx.dynamicIf = fakeDynamicClient + updatedCount := 0 + fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + // Removing the finalizers + updatedCount++ + return true, nil, nil + }) + deletedCount := 0 + fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + deletedCount++ + return true, nil, nil + }) + + syncCtx.Terminate() + phase, message, results := syncCtx.GetState() + assert.Equal(t, synccommon.OperationFailed, phase) + assert.Equal(t, "Operation terminated", message) + require.Len(t, results, 4) + assert.Equal(t, kube.GetResourceKey(hook1), results[0].ResourceKey) + assert.Equal(t, synccommon.OperationFailed, results[0].HookPhase) + assert.Equal(t, "Terminated", results[0].Message) + assert.Equal(t, kube.GetResourceKey(hook2), results[1].ResourceKey) + assert.Equal(t, synccommon.OperationFailed, results[1].HookPhase) + assert.Equal(t, "Terminated", results[1].Message) + assert.Equal(t, kube.GetResourceKey(hook3), results[2].ResourceKey) + assert.Equal(t, synccommon.OperationFailed, results[2].HookPhase) + assert.Equal(t, "Terminated", results[2].Message) + assert.Equal(t, 3, updatedCount) + assert.Equal(t, 3, deletedCount) +} + +func TestTerminate_Hooks_Running_Healthy(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + + obj := testingutils.NewPod() + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusHealthy, + hook2.GetName(): health.HealthStatusHealthy, + hook3.GetName(): health.HealthStatusHealthy, + })), + WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 0, + }, { + ResourceKey: kube.GetResourceKey(hook2), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 2, + }, { + ResourceKey: kube.GetResourceKey(obj), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 3, + }}, + metav1.Now(), + )) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{obj}, + Target: []*unstructured.Unstructured{obj}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) + syncCtx.dynamicIf = fakeDynamicClient + updatedCount := 0 + fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + // Removing the finalizers + updatedCount++ + return true, nil, nil + }) + deletedCount := 0 + fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + deletedCount++ + return true, nil, nil + }) + + syncCtx.Terminate() + phase, message, results := syncCtx.GetState() + assert.Equal(t, synccommon.OperationFailed, phase) + assert.Equal(t, "Operation terminated", message) + require.Len(t, results, 4) + assert.Equal(t, kube.GetResourceKey(hook1), results[0].ResourceKey) + assert.Equal(t, synccommon.OperationSucceeded, results[0].HookPhase) + assert.Equal(t, "test", results[0].Message) + assert.Equal(t, kube.GetResourceKey(hook2), results[1].ResourceKey) + assert.Equal(t, synccommon.OperationSucceeded, results[1].HookPhase) + assert.Equal(t, "test", results[1].Message) + assert.Equal(t, kube.GetResourceKey(hook3), results[2].ResourceKey) + assert.Equal(t, synccommon.OperationSucceeded, results[2].HookPhase) + assert.Equal(t, "test", results[2].Message) + assert.Equal(t, 3, updatedCount) + assert.Equal(t, 1, deletedCount) +} + +func TestTerminate_Hooks_Completed(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + + obj := testingutils.NewPod() + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusHealthy, + hook2.GetName(): health.HealthStatusHealthy, + hook3.GetName(): health.HealthStatusHealthy, + })), + WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationSucceeded, + Message: "hook1 completed", + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 0, + }, { + ResourceKey: kube.GetResourceKey(hook2), + HookPhase: synccommon.OperationFailed, + Message: "hook2 failed", + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationError, + Message: "hook3 error", + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 2, + }, { + ResourceKey: kube.GetResourceKey(obj), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 3, + }}, + metav1.Now(), + )) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{obj}, + Target: []*unstructured.Unstructured{obj}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) + syncCtx.dynamicIf = fakeDynamicClient + updatedCount := 0 + fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + // Removing the finalizers + updatedCount++ + return true, nil, nil + }) + deletedCount := 0 + fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + deletedCount++ + return true, nil, nil + }) + + syncCtx.Terminate() + phase, message, results := syncCtx.GetState() + assert.Equal(t, synccommon.OperationFailed, phase) + assert.Equal(t, "Operation terminated", message) + require.Len(t, results, 4) + assert.Equal(t, kube.GetResourceKey(hook1), results[0].ResourceKey) + assert.Equal(t, synccommon.OperationSucceeded, results[0].HookPhase) + assert.Equal(t, "hook1 completed", results[0].Message) + assert.Equal(t, kube.GetResourceKey(hook2), results[1].ResourceKey) + assert.Equal(t, synccommon.OperationFailed, results[1].HookPhase) + assert.Equal(t, "hook2 failed", results[1].Message) + assert.Equal(t, kube.GetResourceKey(hook3), results[2].ResourceKey) + assert.Equal(t, synccommon.OperationError, results[2].HookPhase) + assert.Equal(t, "hook3 error", results[2].Message) + assert.Equal(t, 0, updatedCount) + assert.Equal(t, 0, deletedCount) +} + +func TestTerminate_Hooks_Error(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + + obj := testingutils.NewPod() + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusHealthy, + })), + WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 0, + }, { + ResourceKey: kube.GetResourceKey(obj), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhaseSync, + Order: 1, + }}, + metav1.Now(), + )) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{obj}, + Target: []*unstructured.Unstructured{obj}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hook1} + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1) + syncCtx.dynamicIf = fakeDynamicClient + fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + return true, nil, apierrors.NewInternalError(fmt.Errorf("update failed")) + }) + + syncCtx.Terminate() + phase, message, results := syncCtx.GetState() + assert.Equal(t, synccommon.OperationError, phase) + assert.Equal(t, "Operation termination had errors", message) + require.Len(t, results, 2) + assert.Equal(t, kube.GetResourceKey(hook1), results[0].ResourceKey) + assert.Equal(t, synccommon.OperationError, results[0].HookPhase) + assert.Contains(t, results[0].Message, "update failed") +} diff --git a/pkg/utils/testing/testdata.go b/pkg/utils/testing/testdata.go index ad446fdfb..cf46b4828 100644 --- a/pkg/utils/testing/testdata.go +++ b/pkg/utils/testing/testdata.go @@ -2,6 +2,9 @@ package testing import ( "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + + synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" + "github.com/argoproj/gitops-engine/pkg/sync/hook" ) const ( @@ -22,6 +25,16 @@ func Annotate(obj *unstructured.Unstructured, key, val string) *unstructured.Uns return obj } +func NewHook(name string, hookType synccommon.HookType, deletePolicy synccommon.HookDeletePolicy) *unstructured.Unstructured { + obj := NewPod() + obj.SetName(name) + obj.SetNamespace(FakeArgoCDNamespace) + Annotate(obj, synccommon.AnnotationKeyHook, string(hookType)) + Annotate(obj, synccommon.AnnotationKeyHookDeletePolicy, string(deletePolicy)) + obj.SetFinalizers([]string{hook.HookFinalizer}) + return obj +} + var PodManifest = ` { "apiVersion": "v1", From a0bf17e3debb283285a1242489f199060ee5a851 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 20:14:36 -0400 Subject: [PATCH 15/28] clean tests Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 192 ++++++++++++++++------------------ 1 file changed, 89 insertions(+), 103 deletions(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 964a55ee8..ed155bc91 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1392,22 +1392,12 @@ func (r resourceNameHealthOverride) GetResourceHealth(obj *unstructured.Unstruct } func TestSync_ExistingHooksWithFinalizer(t *testing.T) { - hook1 := newHook(synccommon.HookTypePreSync) - hook1.SetName("existing-hook-1") - hook1.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyBeforeHookCreation)) - existingHook := hook1.DeepCopy() - existingHook.SetFinalizers(append(existingHook.GetFinalizers(), hook.HookFinalizer)) - - hook2 := newHook(synccommon.HookTypePreSync) - hook2.SetName("existing-hook-2") - hook2.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) - existingHook2 := hook2.DeepCopy() - existingHook2.SetFinalizers(append(existingHook2.GetFinalizers(), hook.HookFinalizer)) + hook1 := testingutils.NewHook("existing-hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("existing-hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("existing-hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil) - fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), existingHook, existingHook2) + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) syncCtx.dynamicIf = fakeDynamicClient updatedCount := 0 fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { @@ -1422,53 +1412,51 @@ func TestSync_ExistingHooksWithFinalizer(t *testing.T) { return true, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{existingHook, existingHook2}, - Target: []*unstructured.Unstructured{nil, nil}, + Live: []*unstructured.Unstructured{hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{nil, nil, nil}, }) - syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2} - - syncCtx.kubectl = &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{}, - } + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() assert.Equal(t, synccommon.OperationRunning, syncCtx.phase) - assert.Equal(t, 2, updatedCount) + assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) } func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { - hook1 := newHook(synccommon.HookTypePreSync) - hook1.SetName("completed-hook") - hook1.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) - completedHook := hook1.DeepCopy() - completedHook.SetFinalizers(append(completedHook.GetFinalizers(), hook.HookFinalizer)) - - hook2 := newHook(synccommon.HookTypePreSync) - hook2.SetNamespace(testingutils.FakeArgoCDNamespace) - hook2.SetName("in-progress-hook") - _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) - inProgressHook := hook2.DeepCopy() - inProgressHook.SetFinalizers(append(inProgressHook.GetFinalizers(), hook.HookFinalizer)) + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ - inProgressHook.GetName(): health.HealthStatusProgressing, + hook1.GetName(): health.HealthStatusProgressing, // At least one is running + hook2.GetName(): health.HealthStatusHealthy, + hook3.GetName(): health.HealthStatusHealthy, })), WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ - ResourceKey: kube.GetResourceKey(completedHook), - HookPhase: synccommon.OperationSucceeded, + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, SyncPhase: synccommon.SyncPhasePreSync, + Order: 0, }, { - ResourceKey: kube.GetResourceKey(inProgressHook), + ResourceKey: kube.GetResourceKey(hook2), HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, SyncPhase: synccommon.SyncPhasePreSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhasePreSync, + Order: 2, }}, metav1.Now(), )) - fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme()) + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) syncCtx.dynamicIf = fakeDynamicClient updatedCount := 0 fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { @@ -1482,14 +1470,10 @@ func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { return true, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{completedHook, inProgressHook}, - Target: []*unstructured.Unstructured{nil, nil}, + Live: []*unstructured.Unstructured{hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{nil, nil, nil}, }) - syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2} - - syncCtx.kubectl = &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{}, - } + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() @@ -1498,34 +1482,39 @@ func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { assert.Equal(t, 0, deletedCount) } -func TestSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { - hook1 := newHook(synccommon.HookTypePreSync) - hook1.SetName("completed-hook1") - hook1.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) - completedHook1 := hook1.DeepCopy() - completedHook1.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) - - hook2 := newHook(synccommon.HookTypePreSync) - hook2.SetNamespace(testingutils.FakeArgoCDNamespace) - hook2.SetName("completed-hook2") - _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) - completedHook2 := hook2.DeepCopy() - completedHook2.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) +func TestSync_HooksDeletedAfterSyncSucceeded(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusHealthy, + hook2.GetName(): health.HealthStatusHealthy, + hook3.GetName(): health.HealthStatusHealthy, + })), WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ - ResourceKey: kube.GetResourceKey(completedHook1), - HookPhase: synccommon.OperationSucceeded, + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, SyncPhase: synccommon.SyncPhasePreSync, + Order: 0, }, { - ResourceKey: kube.GetResourceKey(completedHook2), - HookPhase: synccommon.OperationSucceeded, + ResourceKey: kube.GetResourceKey(hook2), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhasePreSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, SyncPhase: synccommon.SyncPhasePreSync, + Order: 2, }}, metav1.Now(), )) - fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme()) + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) syncCtx.dynamicIf = fakeDynamicClient updatedCount := 0 fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { @@ -1539,50 +1528,51 @@ func TestSync_HooksDeletedAfterPhaseCompleted(t *testing.T) { return true, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{completedHook1, completedHook2}, - Target: []*unstructured.Unstructured{nil, nil}, + Live: []*unstructured.Unstructured{hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{nil, nil, nil}, }) - syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2} - - syncCtx.kubectl = &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{}, - } + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() assert.Equal(t, synccommon.OperationSucceeded, syncCtx.phase) - assert.Equal(t, 2, updatedCount) + assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) } -func TestSync_HooksDeletedAfterPhaseCompletedFailed(t *testing.T) { - hook1 := newHook(synccommon.HookTypeSync) - hook1.SetName("completed-hook1") - hook1.SetNamespace(testingutils.FakeArgoCDNamespace) - _ = testingutils.Annotate(hook1, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookSucceeded)) - completedHook1 := hook1.DeepCopy() - completedHook1.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) - - hook2 := newHook(synccommon.HookTypeSync) - hook2.SetNamespace(testingutils.FakeArgoCDNamespace) - hook2.SetName("completed-hook2") - _ = testingutils.Annotate(hook2, synccommon.AnnotationKeyHookDeletePolicy, string(synccommon.HookDeletePolicyHookFailed)) - completedHook2 := hook2.DeepCopy() - completedHook2.SetFinalizers(append(completedHook1.GetFinalizers(), hook.HookFinalizer)) +func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { + hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + hook1.GetName(): health.HealthStatusDegraded, // At least one has failed + hook2.GetName(): health.HealthStatusHealthy, + hook3.GetName(): health.HealthStatusHealthy, + })), WithInitialState(synccommon.OperationRunning, "", []synccommon.ResourceSyncResult{{ - ResourceKey: kube.GetResourceKey(completedHook1), - HookPhase: synccommon.OperationSucceeded, - SyncPhase: synccommon.SyncPhaseSync, + ResourceKey: kube.GetResourceKey(hook1), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhasePreSync, + Order: 0, }, { - ResourceKey: kube.GetResourceKey(completedHook2), - HookPhase: synccommon.OperationFailed, - SyncPhase: synccommon.SyncPhaseSync, + ResourceKey: kube.GetResourceKey(hook2), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhasePreSync, + Order: 1, + }, { + ResourceKey: kube.GetResourceKey(hook3), + HookPhase: synccommon.OperationRunning, + Status: synccommon.ResultCodeSynced, + SyncPhase: synccommon.SyncPhasePreSync, + Order: 2, }}, metav1.Now(), )) - fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme()) + fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) syncCtx.dynamicIf = fakeDynamicClient updatedCount := 0 fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { @@ -1596,19 +1586,15 @@ func TestSync_HooksDeletedAfterPhaseCompletedFailed(t *testing.T) { return true, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{completedHook1, completedHook2}, - Target: []*unstructured.Unstructured{nil, nil}, + Live: []*unstructured.Unstructured{hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{nil, nil, nil}, }) - syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2} - - syncCtx.kubectl = &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{}, - } + syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() assert.Equal(t, synccommon.OperationFailed, syncCtx.phase) - assert.Equal(t, 2, updatedCount) + assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) } From 3286e062c7c50adc9d8582f71bb46b93d9f1a318 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 19 Aug 2025 20:38:31 -0400 Subject: [PATCH 16/28] check deleted Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 63 ++++++++++++++++++++++++----------- 1 file changed, 44 insertions(+), 19 deletions(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index ed155bc91..c3f2e78cc 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1403,13 +1403,13 @@ func TestSync_ExistingHooksWithFinalizer(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // because of HookDeletePolicyBeforeHookCreation deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hook1, hook2, hook3}, @@ -1418,10 +1418,16 @@ func TestSync_ExistingHooksWithFinalizer(t *testing.T) { syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() + phase, message, _ := syncCtx.GetState() - assert.Equal(t, synccommon.OperationRunning, syncCtx.phase) + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for deletion of hook /Pod/existing-hook-1 and 2 more hooks", message) assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) + + _, err := syncCtx.getResource(&syncTask{liveObj: hook1}) + require.Error(t, err, "Expected resource to be deleted") + assert.True(t, apierrors.IsNotFound(err)) } func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { @@ -1462,12 +1468,12 @@ func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hook1, hook2, hook3}, @@ -1476,8 +1482,10 @@ func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() + phase, message, _ := syncCtx.GetState() - assert.Equal(t, synccommon.OperationRunning, syncCtx.phase) + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for completion of hook /Pod/hook-1", message) assert.Equal(t, 0, updatedCount) assert.Equal(t, 0, deletedCount) } @@ -1520,12 +1528,12 @@ func TestSync_HooksDeletedAfterSyncSucceeded(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hook1, hook2, hook3}, @@ -1534,10 +1542,16 @@ func TestSync_HooksDeletedAfterSyncSucceeded(t *testing.T) { syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() + phase, message, _ := syncCtx.GetState() - assert.Equal(t, synccommon.OperationSucceeded, syncCtx.phase) + assert.Equal(t, synccommon.OperationSucceeded, phase) + assert.Equal(t, "successfully synced (no more tasks)", message) assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) + + _, err := syncCtx.getResource(&syncTask{liveObj: hook3}) + require.Error(t, err, "Expected resource to be deleted") + assert.True(t, apierrors.IsNotFound(err)) } func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { @@ -1578,13 +1592,14 @@ func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 - fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { + fakeDynamicClient.PrependReactor("delete", "*", func(action testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) + syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hook1, hook2, hook3}, Target: []*unstructured.Unstructured{nil, nil, nil}, @@ -1592,10 +1607,16 @@ func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} syncCtx.Sync() + phase, message, _ := syncCtx.GetState() - assert.Equal(t, synccommon.OperationFailed, syncCtx.phase) + assert.Equal(t, synccommon.OperationFailed, phase) + assert.Equal(t, "one or more synchronization tasks completed unsuccessfully", message) assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) + + _, err := syncCtx.getResource(&syncTask{liveObj: hook2}) + require.Error(t, err, "Expected resource to be deleted") + assert.True(t, apierrors.IsNotFound(err)) } func Test_syncContext_liveObj(t *testing.T) { @@ -2448,12 +2469,12 @@ func TestTerminate_Hooks_Running(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.Terminate() @@ -2526,12 +2547,12 @@ func TestTerminate_Hooks_Running_Healthy(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.Terminate() @@ -2550,6 +2571,10 @@ func TestTerminate_Hooks_Running_Healthy(t *testing.T) { assert.Equal(t, "test", results[2].Message) assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) + + _, err := syncCtx.getResource(&syncTask{liveObj: hook2}) + require.Error(t, err, "Expected resource to be deleted") + assert.True(t, apierrors.IsNotFound(err)) } func TestTerminate_Hooks_Completed(t *testing.T) { @@ -2607,12 +2632,12 @@ func TestTerminate_Hooks_Completed(t *testing.T) { fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { // Removing the finalizers updatedCount++ - return true, nil, nil + return false, nil, nil }) deletedCount := 0 fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ - return true, nil, nil + return false, nil, nil }) syncCtx.Terminate() From f633932ff5c80b118ff1a8dda4a4e83f4e2c66bb Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 13:40:23 -0400 Subject: [PATCH 17/28] some tests Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 76 +++--- pkg/sync/sync_context_test.go | 220 ++++++++++++++---- .../kube/kubetest/mock_resource_operations.go | 27 ++- pkg/utils/testing/utils.go | 15 ++ 4 files changed, 252 insertions(+), 86 deletions(-) create mode 100644 pkg/utils/testing/utils.go diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index d87584d26..352c936ca 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -391,28 +391,44 @@ type syncContext struct { modificationResult map[kubeutil.ResourceKey]bool } -func (sc *syncContext) setRunningPhase(tasks []*syncTask, isPendingDeletion bool) { - if len(tasks) > 0 { - firstTask := tasks[0] - waitingFor := "completion of hook" - andMore := "hooks" - if !firstTask.isHook() { - waitingFor = "healthy state of" - andMore = "resources" - } - if isPendingDeletion { - waitingFor = "deletion of" - if firstTask.isHook() { - waitingFor += " hook" - } - } - message := fmt.Sprintf("waiting for %s %s/%s/%s", - waitingFor, firstTask.group(), firstTask.kind(), firstTask.name()) - if moreTasks := len(tasks) - 1; moreTasks > 0 { - message = fmt.Sprintf("%s and %d more %s", message, moreTasks, andMore) +func (sc *syncContext) setRunningPhase(tasks syncTasks, isPendingDeletion bool) { + if tasks.Len() == 0 { + sc.setOperationPhase(common.OperationRunning, "") + return + } + + hooks, resources := tasks.Split(func(task *syncTask) bool { return task.isHook() }) + + waitingFor := "completion of hook" + andMore := "resources" + + if hooks.Len() == 0 { + waitingFor = "healthy state of" + } + if resources.Len() == 0 { + andMore = "hooks" + } + + if isPendingDeletion { + waitingFor = "deletion of" + if hooks.Len() != 0 { + waitingFor += " hook" } - sc.setOperationPhase(common.OperationRunning, message) } + + var firstTask *syncTask + if hooks.Len() != 0 { + firstTask = hooks[0] + } else { + firstTask = resources[0] + } + + message := fmt.Sprintf("waiting for %s %s/%s/%s", waitingFor, firstTask.group(), firstTask.kind(), firstTask.name()) + if moreTasks := len(tasks) - 1; moreTasks > 0 { + message = fmt.Sprintf("%s and %d more %s", message, moreTasks, andMore) + } + sc.setOperationPhase(common.OperationRunning, message) + } // sync has performs the actual apply or hook based sync @@ -564,18 +580,12 @@ func (sc *syncContext) Sync() { return } - // remove any tasks not in this wave phase := tasks.phase() wave := tasks.wave() finalWave := phase == tasks.lastPhase() && wave == tasks.lastWave() - // if it is the last phase/wave and the only remaining tasks are non-hooks, the we are successful - // EVEN if those objects subsequently degraded - // This handles the common case where neither hooks or waves are used and a sync equates to simply an (asynchronous) kubectl apply of manifests, which succeeds immediately. - remainingTasks := tasks.Filter(func(t *syncTask) bool { return t.phase != phase || wave != t.wave() || t.isHook() }) - sc.log.WithValues("phase", phase, "wave", wave, "tasks", tasks, "syncFailTasks", syncFailTasks).V(1).Info("Filtering tasks in correct phase and wave") - tasks = tasks.Filter(func(t *syncTask) bool { return t.phase == phase && t.wave() == wave }) + tasks, remainingTasks := tasks.Split(func(t *syncTask) bool { return t.phase == phase && t.wave() == wave }) sc.setOperationPhase(common.OperationRunning, "one or more tasks are running") @@ -599,7 +609,7 @@ func (sc *syncContext) Sync() { // If we failed to apply at least one resource, we need to start the syncFailTasks and wait // for the completion of any running hooks. In this case, the operation should be running. syncFailedTasks := tasks.Filter(func(t *syncTask) bool { return t.syncStatus == common.ResultCodeSyncFailed }) - runningHooks := tasks.Filter(func(t *syncTask) bool { return t.isHook() && !t.completed() }) + runningHooks := tasks.Filter(func(t *syncTask) bool { return t.running() }) if len(runningHooks) > 0 { if len(syncFailTasks) > 0 { completed := sc.executeSyncFailPhase(syncFailTasks, syncFailedTasks, "one or more objects failed to apply") @@ -615,12 +625,17 @@ func (sc *syncContext) Sync() { } } case successful: - if remainingTasks.Len() == 0 { + if remainingTasks.Len() == 0 && !tasks.Any(func(task *syncTask) bool { return task.isHook() }) { + // if it is the last phase/wave and the only running tasks are non-hooks, then we are successful + // EVEN if those objects subsequently degrades + // This handles the common case where neither hooks or waves are used and a sync equates to simply + // an (asynchronous) kubectl apply of manifests, which succeeds immediately. + // delete all completed hooks which have appropriate delete policy sc.deleteHooks(hooksPendingDeletionSuccessful) sc.setOperationPhase(common.OperationSucceeded, "successfully synced (all tasks run)") } else { - sc.setRunningPhase(remainingTasks, false) + sc.setRunningPhase(tasks, false) } default: sc.setRunningPhase(tasks, true) @@ -823,6 +838,7 @@ func (sc *syncContext) executeSyncFailPhase(syncFailTasks, syncFailedTasks syncT pendingSyncFailTasks := syncFailTasks.Filter(func(task *syncTask) bool { return !task.completed() && !task.running() }) sc.log.WithValues("syncFailTasks", pendingSyncFailTasks).V(1).Info("Running sync fail tasks") sc.runTasks(pendingSyncFailTasks, false) + sc.setRunningPhase(pendingSyncFailTasks, false) return false } diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index c3f2e78cc..b6624892f 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -281,19 +281,27 @@ func TestSyncCustomResources(t *testing.T) { } func TestSyncSuccessfully(t *testing.T) { - syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, true, false, false)) + newSvc := testingutils.NewService() pod := testingutils.NewPod() pod.SetNamespace(testingutils.FakeArgoCDNamespace) + + syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, true, false, false), + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + newSvc.GetName(): health.HealthStatusDegraded, // Always failed + }))) + syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil, pod}, - Target: []*unstructured.Unstructured{testingutils.NewService(), nil}, + Target: []*unstructured.Unstructured{newSvc, nil}, }) + // Since we only have one step, we consider the sync successful if the manifest were applied correctly. + // In this case, we do not need to run the sync again to evaluate the health syncCtx.Sync() phase, _, resources := syncCtx.GetState() - assert.Equal(t, synccommon.OperationSucceeded, phase) assert.Len(t, resources, 2) + for i := range resources { result := resources[i] switch result.ResourceKey.Kind { @@ -309,6 +317,48 @@ func TestSyncSuccessfully(t *testing.T) { } } +func TestSyncSuccessfully_Multistep(t *testing.T) { + newSvc := testingutils.NewService() + newSvc.SetNamespace(testingutils.FakeArgoCDNamespace) + testingutils.Annotate(newSvc, synccommon.AnnotationSyncWave, "0") + + newSvc2 := testingutils.NewService() + newSvc.SetNamespace(testingutils.FakeArgoCDNamespace) + newSvc2.SetName("new-svc-2") + testingutils.Annotate(newSvc2, synccommon.AnnotationSyncWave, "5") + + syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, true, false, false), + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + newSvc.GetName(): health.HealthStatusHealthy, + }))) + + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{nil, nil}, + Target: []*unstructured.Unstructured{newSvc, newSvc2}, + }) + + // Since we have multiple step, we need to run the sync again to evaluate the health of current phase + // (wave 0) and start the new phase (wave 5). + syncCtx.Sync() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for healthy state of /Service/my-service", message) + assert.Len(t, resources, 1) + assert.Equal(t, kube.GetResourceKey(newSvc), resources[0].ResourceKey) + + // Update the live resources for the next sync + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{newSvc, nil}, + Target: []*unstructured.Unstructured{newSvc, newSvc2}, + }) + + syncCtx.Sync() + phase, message, resources = syncCtx.GetState() + assert.Equal(t, synccommon.OperationSucceeded, phase) + assert.Equal(t, "successfully synced (all tasks run)", message) + assert.Len(t, resources, 2) +} + func TestSyncDeleteSuccessfully(t *testing.T) { syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, true, false, false)) svc := testingutils.NewService() @@ -1261,44 +1311,97 @@ func createNamespaceTask(namespace string) (*syncTask, error) { return task, nil } -func TestSyncFailureHookWithSuccessfulSync(t *testing.T) { - syncCtx := newTestSyncCtx(nil) +func TestSync_SuccessfulSyncWithSyncFailHook(t *testing.T) { + hook := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + pod := testingutils.NewPod() + syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + pod.GetName(): health.HealthStatusHealthy, + hook.GetName(): health.HealthStatusHealthy, + }))) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, - Target: []*unstructured.Unstructured{testingutils.NewPod()}, + Target: []*unstructured.Unstructured{pod}, }) - syncCtx.hooks = []*unstructured.Unstructured{newHook(synccommon.HookTypeSyncFail)} + syncCtx.hooks = []*unstructured.Unstructured{hook, syncFailHook} + syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) + // First sync does dry-run and starts hooks syncCtx.Sync() - phase, _, resources := syncCtx.GetState() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for completion of hook /Pod/hook-1 and 1 more resources", message) + assert.Len(t, resources, 2) + + // Update the live resources for the next sync + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{pod, hook}, + Target: []*unstructured.Unstructured{pod, nil}, + }) + + // Second sync completes hooks + syncCtx.Sync() + phase, message, resources = syncCtx.GetState() assert.Equal(t, synccommon.OperationSucceeded, phase) - // only one result, we did not run the failure failureHook - assert.Len(t, resources, 1) + assert.Equal(t, "successfully synced (no more tasks)", message) + assert.Len(t, resources, 2) } -func TestSyncFailureHookWithFailedSync(t *testing.T) { - syncCtx := newTestSyncCtx(nil) +func TestSync_FailedSyncWithSyncFailHook_HookFailed(t *testing.T) { + hook := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) pod := testingutils.NewPod() + syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) + + syncCtx := newTestSyncCtx(nil, + WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ + pod.GetName(): health.HealthStatusHealthy, + hook.GetName(): health.HealthStatusDegraded, // Hook failure + syncFailHook.GetName(): health.HealthStatusHealthy, + }))) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, Target: []*unstructured.Unstructured{pod}, }) - syncCtx.hooks = []*unstructured.Unstructured{newHook(synccommon.HookTypeSyncFail)} - mockKubectl := &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{pod.GetName(): {Err: errors.New("")}}, - } - syncCtx.kubectl = mockKubectl - mockResourceOps := kubetest.MockResourceOps{ - Commands: map[string]kubetest.KubectlOutput{pod.GetName(): {Err: errors.New("")}}, - } - syncCtx.resourceOps = &mockResourceOps + syncCtx.hooks = []*unstructured.Unstructured{hook, syncFailHook} + syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) + // First sync does dry-run and starts hooks syncCtx.Sync() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for completion of hook /Pod/hook-1 and 1 more resources", message) + assert.Len(t, resources, 2) + + // Update the live resources for the next sync + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{pod, hook}, + Target: []*unstructured.Unstructured{pod, nil}, + }) + + // Second sync fails the sync and starts the SyncFail hooks syncCtx.Sync() + phase, message, resources = syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for completion of hook /Pod/sync-fail-hook", message) + assert.Len(t, resources, 3) - phase, _, resources := syncCtx.GetState() + // Update the live resources for the next sync + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{pod, hook, syncFailHook}, + Target: []*unstructured.Unstructured{pod, nil, nil}, + }) + + // Third sync completes the SyncFail hooks + syncCtx.Sync() + phase, message, resources = syncCtx.GetState() assert.Equal(t, synccommon.OperationFailed, phase) - assert.Len(t, resources, 2) + assert.Equal(t, "one or more synchronization tasks completed unsuccessfully", message) + assert.Len(t, resources, 3) +} + +func TestSync_FailedSyncWithSyncFailHook_ApplyFailed(t *testing.T) { } func TestBeforeHookCreation(t *testing.T) { @@ -1336,50 +1439,69 @@ func TestBeforeHookCreation(t *testing.T) { func TestRunSyncFailHooksFailed(t *testing.T) { // Tests that other SyncFail Hooks run even if one of them fail. - - syncCtx := newTestSyncCtx(nil) pod := testingutils.NewPod() - successfulSyncFailHook := newHook(synccommon.HookTypeSyncFail) - successfulSyncFailHook.SetName("successful-sync-fail-hook") - failedSyncFailHook := newHook(synccommon.HookTypeSyncFail) - failedSyncFailHook.SetName("failed-sync-fail-hook") + pod.SetNamespace(testingutils.FakeArgoCDNamespace) + successfulSyncFailHook := testingutils.NewHook("successful-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + failedSyncFailHook := testingutils.NewHook("failed-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + + syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride{ + successfulSyncFailHook.GetName(): health.HealthStatusHealthy, + })) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, Target: []*unstructured.Unstructured{pod}, }) syncCtx.hooks = []*unstructured.Unstructured{successfulSyncFailHook, failedSyncFailHook} + syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) - mockKubectl := &kubetest.MockKubectlCmd{ - Commands: map[string]kubetest.KubectlOutput{ - // Fail operation - pod.GetName(): {Err: errors.New("")}, - // Fail a single SyncFail hook - failedSyncFailHook.GetName(): {Err: errors.New("")}, - }, - } - syncCtx.kubectl = mockKubectl mockResourceOps := kubetest.MockResourceOps{ Commands: map[string]kubetest.KubectlOutput{ // Fail operation - pod.GetName(): {Err: errors.New("")}, + pod.GetName(): {Err: errors.New("fake pod failure")}, // Fail a single SyncFail hook - failedSyncFailHook.GetName(): {Err: errors.New("")}, + failedSyncFailHook.GetName(): {Err: errors.New("fake hook failure")}, }, } syncCtx.resourceOps = &mockResourceOps + // First sync triggers the SyncFail hooks on failure syncCtx.Sync() - syncCtx.Sync() - phase, _, resources := syncCtx.GetState() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Equal(t, "waiting for completion of hook /Pod/failed-sync-fail-hook and 1 more hooks", message) + assert.Len(t, resources, 3) + podResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(pod)) + require.NotNil(t, podResult, "%s not found", kube.GetResourceKey(pod)) + assert.Equal(t, synccommon.OperationFailed, podResult.HookPhase) + assert.Equal(t, synccommon.ResultCodeSyncFailed, podResult.Status) + + failedSyncFailHookResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(failedSyncFailHook)) + require.NotNil(t, failedSyncFailHookResult, "%s not found", kube.GetResourceKey(failedSyncFailHook)) + assert.Equal(t, synccommon.OperationFailed, failedSyncFailHookResult.HookPhase) + assert.Equal(t, synccommon.ResultCodeSyncFailed, failedSyncFailHookResult.Status) + + successfulSyncFailHookResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) + require.NotNil(t, successfulSyncFailHookResult, "%s not found", kube.GetResourceKey(successfulSyncFailHook)) + assert.Equal(t, synccommon.OperationRunning, successfulSyncFailHookResult.HookPhase) + assert.Equal(t, synccommon.ResultCodeSynced, successfulSyncFailHookResult.Status) + + // Update the live state for the next run + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{nil, successfulSyncFailHook}, + Target: []*unstructured.Unstructured{pod, nil}, + }) - // Operation as a whole should fail + // Second sync completes when the SyncFail hooks are done + syncCtx.Sync() + phase, message, resources = syncCtx.GetState() assert.Equal(t, synccommon.OperationFailed, phase) - // failedSyncFailHook should fail - assert.Equal(t, synccommon.OperationFailed, resources[1].HookPhase) - assert.Equal(t, synccommon.ResultCodeSyncFailed, resources[1].Status) - // successfulSyncFailHook should be synced running (it is an nginx pod) - assert.Equal(t, synccommon.OperationRunning, resources[2].HookPhase) - assert.Equal(t, synccommon.ResultCodeSynced, resources[2].Status) + assert.Equal(t, "one or more synchronization tasks completed unsuccessfully, reason: fake pod failure\none or more SyncFail hooks failed, reason: fake hook failure", message) + assert.Len(t, resources, 3) + + successfulSyncFailHookResult = testingutils.GetResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) + require.NotNil(t, successfulSyncFailHookResult, "%s not found", kube.GetResourceKey(successfulSyncFailHook)) + assert.Equal(t, synccommon.OperationSucceeded, successfulSyncFailHookResult.HookPhase) + assert.Equal(t, synccommon.ResultCodeSynced, successfulSyncFailHookResult.Status) } type resourceNameHealthOverride map[string]health.HealthStatusCode diff --git a/pkg/utils/kube/kubetest/mock_resource_operations.go b/pkg/utils/kube/kubetest/mock_resource_operations.go index 8f4242821..ae93c7a95 100644 --- a/pkg/utils/kube/kubetest/mock_resource_operations.go +++ b/pkg/utils/kube/kubetest/mock_resource_operations.go @@ -15,9 +15,10 @@ import ( ) type MockResourceOps struct { - Commands map[string]KubectlOutput - Events chan watch.Event - DynamicClient dynamic.Interface + ExecuteForDryRun bool + Commands map[string]KubectlOutput + Events chan watch.Event + DynamicClient dynamic.Interface lastCommandPerResource map[kube.ResourceKey]string lastValidate bool @@ -106,7 +107,10 @@ func (r *MockResourceOps) GetLastResourceCommand(key kube.ResourceKey) string { return r.lastCommandPerResource[key] } -func (r *MockResourceOps) ApplyResource(_ context.Context, obj *unstructured.Unstructured, _ cmdutil.DryRunStrategy, force, validate, serverSideApply bool, manager string) (string, error) { +func (r *MockResourceOps) ApplyResource(_ context.Context, obj *unstructured.Unstructured, dryRun cmdutil.DryRunStrategy, force, validate, serverSideApply bool, manager string) (string, error) { + if dryRun != cmdutil.DryRunNone && !r.ExecuteForDryRun { + return "", nil + } r.SetLastValidate(validate) r.SetLastServerSideApply(serverSideApply) r.SetLastServerSideApplyManager(manager) @@ -120,7 +124,10 @@ func (r *MockResourceOps) ApplyResource(_ context.Context, obj *unstructured.Uns return command.Output, command.Err } -func (r *MockResourceOps) ReplaceResource(_ context.Context, obj *unstructured.Unstructured, _ cmdutil.DryRunStrategy, force bool) (string, error) { +func (r *MockResourceOps) ReplaceResource(_ context.Context, obj *unstructured.Unstructured, dryRun cmdutil.DryRunStrategy, force bool) (string, error) { + if dryRun != cmdutil.DryRunNone && !r.ExecuteForDryRun { + return "", nil + } r.SetLastForce(force) command, ok := r.Commands[obj.GetName()] r.SetLastResourceCommand(kube.GetResourceKey(obj), "replace") @@ -131,7 +138,10 @@ func (r *MockResourceOps) ReplaceResource(_ context.Context, obj *unstructured.U return command.Output, command.Err } -func (r *MockResourceOps) UpdateResource(_ context.Context, obj *unstructured.Unstructured, _ cmdutil.DryRunStrategy) (*unstructured.Unstructured, error) { +func (r *MockResourceOps) UpdateResource(_ context.Context, obj *unstructured.Unstructured, dryRun cmdutil.DryRunStrategy) (*unstructured.Unstructured, error) { + if dryRun != cmdutil.DryRunNone && !r.ExecuteForDryRun { + return obj, nil + } r.SetLastResourceCommand(kube.GetResourceKey(obj), "update") command, ok := r.Commands[obj.GetName()] if !ok { @@ -140,7 +150,10 @@ func (r *MockResourceOps) UpdateResource(_ context.Context, obj *unstructured.Un return obj, command.Err } -func (r *MockResourceOps) CreateResource(_ context.Context, obj *unstructured.Unstructured, _ cmdutil.DryRunStrategy, _ bool) (string, error) { +func (r *MockResourceOps) CreateResource(_ context.Context, obj *unstructured.Unstructured, dryRun cmdutil.DryRunStrategy, _ bool) (string, error) { + if dryRun != cmdutil.DryRunNone && !r.ExecuteForDryRun { + return "", nil + } r.SetLastResourceCommand(kube.GetResourceKey(obj), "create") command, ok := r.Commands[obj.GetName()] if !ok { diff --git a/pkg/utils/testing/utils.go b/pkg/utils/testing/utils.go new file mode 100644 index 000000000..7e114ac5c --- /dev/null +++ b/pkg/utils/testing/utils.go @@ -0,0 +1,15 @@ +package testing + +import ( + synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" + "github.com/argoproj/gitops-engine/pkg/utils/kube" +) + +func GetResourceResult(resources []synccommon.ResourceSyncResult, resourceKey kube.ResourceKey) *synccommon.ResourceSyncResult { + for _, res := range resources { + if res.ResourceKey == resourceKey { + return &res + } + } + return nil +} From 0201c5ad4215b3488fab475c00c255e2f52cc252 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 13:48:33 -0400 Subject: [PATCH 18/28] move tests Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 69 +++++++++++++++++------------------ 1 file changed, 33 insertions(+), 36 deletions(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index b6624892f..72d49e1f1 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1402,42 +1402,6 @@ func TestSync_FailedSyncWithSyncFailHook_HookFailed(t *testing.T) { } func TestSync_FailedSyncWithSyncFailHook_ApplyFailed(t *testing.T) { -} - -func TestBeforeHookCreation(t *testing.T) { - finalizerRemoved := false - syncCtx := newTestSyncCtx(nil) - hookObj := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") - hookObj.SetFinalizers([]string{hook.HookFinalizer}) - hookObj.SetNamespace(testingutils.FakeArgoCDNamespace) - syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{hookObj}, - Target: []*unstructured.Unstructured{nil}, - }) - syncCtx.hooks = []*unstructured.Unstructured{hookObj} - client := fake.NewSimpleDynamicClient(runtime.NewScheme(), hookObj) - client.PrependReactor("update", "pods", func(_ testcore.Action) (bool, runtime.Object, error) { - finalizerRemoved = true - return false, nil, nil - }) - syncCtx.dynamicIf = client - - // First sync will delete the existing hook - syncCtx.Sync() - phase, _, _ := syncCtx.GetState() - assert.Equal(t, synccommon.OperationRunning, phase) - assert.True(t, finalizerRemoved) - - // Second sync will create the hook - syncCtx.Sync() - phase, message, resources := syncCtx.GetState() - assert.Equal(t, synccommon.OperationRunning, phase) - assert.Len(t, resources, 1) - assert.Equal(t, synccommon.OperationRunning, resources[0].HookPhase) - assert.Equal(t, "waiting for completion of hook /Pod/my-pod", message) -} - -func TestRunSyncFailHooksFailed(t *testing.T) { // Tests that other SyncFail Hooks run even if one of them fail. pod := testingutils.NewPod() pod.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -1504,6 +1468,39 @@ func TestRunSyncFailHooksFailed(t *testing.T) { assert.Equal(t, synccommon.ResultCodeSynced, successfulSyncFailHookResult.Status) } +func TestBeforeHookCreation(t *testing.T) { + finalizerRemoved := false + syncCtx := newTestSyncCtx(nil) + hookObj := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") + hookObj.SetFinalizers([]string{hook.HookFinalizer}) + hookObj.SetNamespace(testingutils.FakeArgoCDNamespace) + syncCtx.resources = groupResources(ReconciliationResult{ + Live: []*unstructured.Unstructured{hookObj}, + Target: []*unstructured.Unstructured{nil}, + }) + syncCtx.hooks = []*unstructured.Unstructured{hookObj} + client := fake.NewSimpleDynamicClient(runtime.NewScheme(), hookObj) + client.PrependReactor("update", "pods", func(_ testcore.Action) (bool, runtime.Object, error) { + finalizerRemoved = true + return false, nil, nil + }) + syncCtx.dynamicIf = client + + // First sync will delete the existing hook + syncCtx.Sync() + phase, _, _ := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.True(t, finalizerRemoved) + + // Second sync will create the hook + syncCtx.Sync() + phase, message, resources := syncCtx.GetState() + assert.Equal(t, synccommon.OperationRunning, phase) + assert.Len(t, resources, 1) + assert.Equal(t, synccommon.OperationRunning, resources[0].HookPhase) + assert.Equal(t, "waiting for completion of hook /Pod/my-pod", message) +} + type resourceNameHealthOverride map[string]health.HealthStatusCode func (r resourceNameHealthOverride) GetResourceHealth(obj *unstructured.Unstructured) (*health.HealthStatus, error) { From 3d2aa95a2eeac2a55f01bbe8e1f7ab7051f27dc4 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 14:07:26 -0400 Subject: [PATCH 19/28] cleanup tests Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 62 +++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 72d49e1f1..1c1fe0b59 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1130,6 +1130,7 @@ func TestDeDupingTasks(t *testing.T) { func TestObjectsGetANamespace(t *testing.T) { syncCtx := newTestSyncCtx(nil) pod := testingutils.NewPod() + pod.SetNamespace("") syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, Target: []*unstructured.Unstructured{pod}, @@ -1817,7 +1818,7 @@ func Test_setRunningPhase_pendingDeletion(t *testing.T) { assert.Equal(t, "waiting for deletion of /Pod/my-pod and 2 more resources", sc.message) } -func TestSyncWaveHook(t *testing.T) { +func TestSync_SyncWaveHook(t *testing.T) { syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, false, false, false)) pod1 := testingutils.NewPod() pod1.SetName("pod-1") @@ -1891,17 +1892,18 @@ func TestSyncWaveHook(t *testing.T) { func TestSync_SyncWaveHookError(t *testing.T) { syncCtx := newTestSyncCtx(nil, WithOperationSettings(false, false, false, false)) pod1 := testingutils.NewPod() + pod1.SetNamespace(testingutils.FakeArgoCDNamespace) pod1.SetName("pod-1") + syncHook := testingutils.NewHook("sync-hook", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{nil}, Target: []*unstructured.Unstructured{pod1}, }) - syncCtx.hooks = []*unstructured.Unstructured{ - newHook(synccommon.HookTypeSync), - newHook(synccommon.HookTypeSyncFail), - } + syncCtx.hooks = []*unstructured.Unstructured{syncHook, syncFailHook} called := false syncCtx.syncWaveHook = func(_ synccommon.SyncPhase, _ int, _ bool) error { @@ -1914,10 +1916,14 @@ func TestSync_SyncWaveHookError(t *testing.T) { assert.Equal(t, synccommon.OperationError, phase) assert.Equal(t, "SyncWaveHook failed: intentional error", msg) require.Len(t, results, 2) - assert.Equal(t, kube.ResourceKey{Group: "", Kind: "Pod", Namespace: "fake-argocd-ns", Name: "my-pod"}, results[0].ResourceKey) - assert.Equal(t, "Terminated", results[0].Message) - assert.Equal(t, kube.ResourceKey{Group: "", Kind: "Pod", Namespace: "fake-argocd-ns", Name: "pod-1"}, results[1].ResourceKey) - assert.Equal(t, synccommon.OperationRunning, results[1].HookPhase) + + podResult := testingutils.GetResourceResult(results, kube.GetResourceKey(pod1)) + require.NotNil(t, podResult, "%s not found", kube.GetResourceKey(pod1)) + assert.Equal(t, synccommon.OperationRunning, podResult.HookPhase) + + hookResult := testingutils.GetResourceResult(results, kube.GetResourceKey(syncHook)) + require.NotNil(t, hookResult, "%s not found", kube.GetResourceKey(syncHook)) + assert.Equal(t, "Terminated", hookResult.Message) } func TestPruneLast(t *testing.T) { @@ -2331,15 +2337,15 @@ func TestWaitForCleanUpBeforeNextWave(t *testing.T) { var phase synccommon.OperationPhase var msg string - var result []synccommon.ResourceSyncResult + var results []synccommon.ResourceSyncResult // 1st sync should prune only pod3 syncCtx.Sync() - phase, _, result = syncCtx.GetState() + phase, _, results = syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) - assert.Len(t, result, 1) - assert.Equal(t, "pod-3", result[0].ResourceKey.Name) - assert.Equal(t, synccommon.ResultCodePruned, result[0].Status) + assert.Len(t, results, 1) + assert.Equal(t, "pod-3", results[0].ResourceKey.Name) + assert.Equal(t, synccommon.ResultCodePruned, results[0].Status) // simulate successful delete of pod3 syncCtx.resources = groupResources(ReconciliationResult{ @@ -2349,11 +2355,11 @@ func TestWaitForCleanUpBeforeNextWave(t *testing.T) { // next sync should prune only pod2 syncCtx.Sync() - phase, _, result = syncCtx.GetState() + phase, _, results = syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) - assert.Len(t, result, 2) - assert.Equal(t, "pod-2", result[1].ResourceKey.Name) - assert.Equal(t, synccommon.ResultCodePruned, result[1].Status) + assert.Len(t, results, 2) + assert.Equal(t, "pod-2", results[1].ResourceKey.Name) + assert.Equal(t, synccommon.ResultCodePruned, results[1].Status) // add delete timestamp on pod2 to simulate pending delete pod2.SetDeletionTimestamp(&metav1.Time{Time: time.Now()}) @@ -2361,10 +2367,10 @@ func TestWaitForCleanUpBeforeNextWave(t *testing.T) { // next sync should wait for deletion of pod2 from cluster, // it should not move to next wave and prune pod1 syncCtx.Sync() - phase, msg, result = syncCtx.GetState() + phase, msg, results = syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) assert.Equal(t, "waiting for deletion of /Pod/pod-2", msg) - assert.Len(t, result, 2) + assert.Len(t, results, 2) // simulate successful delete of pod2 syncCtx.resources = groupResources(ReconciliationResult{ @@ -2375,15 +2381,15 @@ func TestWaitForCleanUpBeforeNextWave(t *testing.T) { // next sync should proceed with next wave // i.e deletion of pod1 syncCtx.Sync() - phase, _, result = syncCtx.GetState() + phase, _, results = syncCtx.GetState() assert.Equal(t, synccommon.OperationSucceeded, phase) - assert.Len(t, result, 3) - assert.Equal(t, "pod-3", result[0].ResourceKey.Name) - assert.Equal(t, "pod-2", result[1].ResourceKey.Name) - assert.Equal(t, "pod-1", result[2].ResourceKey.Name) - assert.Equal(t, synccommon.ResultCodePruned, result[0].Status) - assert.Equal(t, synccommon.ResultCodePruned, result[1].Status) - assert.Equal(t, synccommon.ResultCodePruned, result[2].Status) + assert.Len(t, results, 3) + assert.Equal(t, "pod-3", results[0].ResourceKey.Name) + assert.Equal(t, "pod-2", results[1].ResourceKey.Name) + assert.Equal(t, "pod-1", results[2].ResourceKey.Name) + assert.Equal(t, synccommon.ResultCodePruned, results[0].Status) + assert.Equal(t, synccommon.ResultCodePruned, results[1].Status) + assert.Equal(t, synccommon.ResultCodePruned, results[2].Status) } func BenchmarkSync(b *testing.B) { From b39a9c8150b72199cb606e520adf8a31ed16ea6b Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 14:08:43 -0400 Subject: [PATCH 20/28] docs Signed-off-by: Alexandre Gaudreault --- pkg/sync/doc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sync/doc.go b/pkg/sync/doc.go index c025ed9fe..66a469adf 100644 --- a/pkg/sync/doc.go +++ b/pkg/sync/doc.go @@ -61,7 +61,7 @@ Hooks can be deleted in an automatic fashion using the annotation: argocd.argopr The following policies define when the hook will be deleted. - - HookSucceeded - the hook resource is deleted if the sync succeeds (e.g. Job/Workflow completed successfully). + - HookSucceeded - the hook resource is deleted if the sync succeeds - HookFailed - the hook resource is deleted if the sync fails. - BeforeHookCreation - any existing hook resource is deleted before the new one is created From 9da34cc3beb562fa91888c4edca888f0c4682094 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 14:54:33 -0400 Subject: [PATCH 21/28] import cycle Signed-off-by: Alexandre Gaudreault --- pkg/sync/common/types.go | 3 +++ pkg/sync/hook/hook.go | 7 +------ pkg/sync/sync_context.go | 4 ++-- pkg/sync/sync_context_test.go | 3 +-- pkg/utils/kube/kube_test.go | 2 +- .../{testing => kube/kubetest}/unstructured.go | 2 +- pkg/utils/testing/testdata.go | 13 +++++++------ 7 files changed, 16 insertions(+), 18 deletions(-) rename pkg/utils/{testing => kube/kubetest}/unstructured.go (97%) diff --git a/pkg/sync/common/types.go b/pkg/sync/common/types.go index 00976ff5d..19578433a 100644 --- a/pkg/sync/common/types.go +++ b/pkg/sync/common/types.go @@ -53,6 +53,9 @@ const ( // Default field manager for client-side apply migration DefaultClientSideApplyMigrationManager = "kubectl-client-side-apply" + + // HookFinalizer is the finalizer added to hooks to ensure they are deleted only after the sync phase is completed. + HookFinalizer = "argocd.argoproj.io/hook-finalizer" ) type PermissionValidator func(un *unstructured.Unstructured, res *metav1.APIResource) error diff --git a/pkg/sync/hook/hook.go b/pkg/sync/hook/hook.go index 66dfc26e5..252e15456 100644 --- a/pkg/sync/hook/hook.go +++ b/pkg/sync/hook/hook.go @@ -8,15 +8,10 @@ import ( resourceutil "github.com/argoproj/gitops-engine/pkg/sync/resource" ) -const ( - // HookFinalizer is the finalizer added to hooks to ensure they are deleted only after the sync phase is completed. - HookFinalizer = "argocd.argoproj.io/hook-finalizer" -) - func HasHookFinalizer(obj *unstructured.Unstructured) bool { finalizers := obj.GetFinalizers() for _, finalizer := range finalizers { - if finalizer == HookFinalizer { + if finalizer == common.HookFinalizer { return true } } diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 352c936ca..0de5895d7 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -753,7 +753,7 @@ func (sc *syncContext) removeHookFinalizer(task *syncTask) error { removeFinalizerMutation := func(obj *unstructured.Unstructured) bool { finalizers := obj.GetFinalizers() for i, finalizer := range finalizers { - if finalizer == hook.HookFinalizer { + if finalizer == common.HookFinalizer { obj.SetFinalizers(append(finalizers[:i], finalizers[i+1:]...)) return true } @@ -896,7 +896,7 @@ func (sc *syncContext) getSyncTasks() (_ syncTasks, successful bool) { targetObj.SetName(fmt.Sprintf("%s%s", generateName, postfix)) } if !hook.HasHookFinalizer(targetObj) { - targetObj.SetFinalizers(append(targetObj.GetFinalizers(), hook.HookFinalizer)) + targetObj.SetFinalizers(append(targetObj.GetFinalizers(), common.HookFinalizer)) } hookTasks = append(hookTasks, &syncTask{phase: phase, targetObj: targetObj}) } diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 1c1fe0b59..4690ad947 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -32,7 +32,6 @@ import ( "github.com/argoproj/gitops-engine/pkg/diff" "github.com/argoproj/gitops-engine/pkg/health" synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" - "github.com/argoproj/gitops-engine/pkg/sync/hook" "github.com/argoproj/gitops-engine/pkg/utils/kube" "github.com/argoproj/gitops-engine/pkg/utils/kube/kubetest" testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" @@ -1473,7 +1472,7 @@ func TestBeforeHookCreation(t *testing.T) { finalizerRemoved := false syncCtx := newTestSyncCtx(nil) hookObj := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") - hookObj.SetFinalizers([]string{hook.HookFinalizer}) + hookObj.SetFinalizers([]string{synccommon.HookFinalizer}) hookObj.SetNamespace(testingutils.FakeArgoCDNamespace) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hookObj}, diff --git a/pkg/utils/kube/kube_test.go b/pkg/utils/kube/kube_test.go index e280bd391..f25276388 100644 --- a/pkg/utils/kube/kube_test.go +++ b/pkg/utils/kube/kube_test.go @@ -150,7 +150,7 @@ spec: - name: nginx image: nginx:1.7.9 ports: - - containerPort: 80 + - containerPort: 80 `) deployment := unstructured.Unstructured{} err := yaml.Unmarshal(manifest, &deployment) diff --git a/pkg/utils/testing/unstructured.go b/pkg/utils/kube/kubetest/unstructured.go similarity index 97% rename from pkg/utils/testing/unstructured.go rename to pkg/utils/kube/kubetest/unstructured.go index f6c292001..0c4a38ad5 100644 --- a/pkg/utils/testing/unstructured.go +++ b/pkg/utils/kube/kubetest/unstructured.go @@ -1,4 +1,4 @@ -package testing +package kubetest import ( "encoding/json" diff --git a/pkg/utils/testing/testdata.go b/pkg/utils/testing/testdata.go index cf46b4828..ef84deff5 100644 --- a/pkg/utils/testing/testdata.go +++ b/pkg/utils/testing/testdata.go @@ -4,7 +4,8 @@ import ( "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" - "github.com/argoproj/gitops-engine/pkg/sync/hook" + + "github.com/argoproj/gitops-engine/pkg/utils/kube/kubetest" ) const ( @@ -31,7 +32,7 @@ func NewHook(name string, hookType synccommon.HookType, deletePolicy synccommon. obj.SetNamespace(FakeArgoCDNamespace) Annotate(obj, synccommon.AnnotationKeyHook, string(hookType)) Annotate(obj, synccommon.AnnotationKeyHookDeletePolicy, string(deletePolicy)) - obj.SetFinalizers([]string{hook.HookFinalizer}) + obj.SetFinalizers([]string{synccommon.HookFinalizer}) return obj } @@ -59,7 +60,7 @@ var PodManifest = ` ` func NewPod() *unstructured.Unstructured { - return Unstructured(PodManifest) + return kubetest.Unstructured(PodManifest) } var ServiceManifest = ` @@ -86,11 +87,11 @@ var ServiceManifest = ` ` func NewService() *unstructured.Unstructured { - return Unstructured(ServiceManifest) + return kubetest.Unstructured(ServiceManifest) } func NewCRD() *unstructured.Unstructured { - return Unstructured(`apiVersion: apiextensions.k8s.io/v1beta1 + return kubetest.Unstructured(`apiVersion: apiextensions.k8s.io/v1beta1 kind: CustomResourceDefinition metadata: name: testcrds.argoproj.io @@ -104,7 +105,7 @@ spec: } func NewNamespace() *unstructured.Unstructured { - return Unstructured(`apiVersion: v1 + return kubetest.Unstructured(`apiVersion: v1 kind: Namespace metadata: name: testnamespace From 13fa1f32532d164391d5ba568136de464c4ab7f0 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 15:46:10 -0400 Subject: [PATCH 22/28] fix cycle in import Signed-off-by: Alexandre Gaudreault --- pkg/sync/helper_test.go | 83 +++++++++++ pkg/sync/sync_context.go | 1 - pkg/sync/sync_context_test.go | 129 ++++++------------ pkg/sync/sync_task_test.go | 10 +- pkg/utils/kube/convert_test.go | 4 +- pkg/utils/kube/ctl_test.go | 3 +- pkg/utils/testing/testdata.go | 22 +-- .../kubetest => testing}/unstructured.go | 2 +- pkg/utils/testing/utils.go | 15 -- 9 files changed, 134 insertions(+), 135 deletions(-) create mode 100644 pkg/sync/helper_test.go rename pkg/utils/{kube/kubetest => testing}/unstructured.go (97%) delete mode 100644 pkg/utils/testing/utils.go diff --git a/pkg/sync/helper_test.go b/pkg/sync/helper_test.go new file mode 100644 index 000000000..e119fc07d --- /dev/null +++ b/pkg/sync/helper_test.go @@ -0,0 +1,83 @@ +package sync + +import ( + "fmt" + + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" + + "github.com/argoproj/gitops-engine/pkg/health" + synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" + "github.com/argoproj/gitops-engine/pkg/utils/kube" + testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" +) + +type resourceNameHealthOverride map[string]health.HealthStatusCode + +func (r resourceNameHealthOverride) GetResourceHealth(obj *unstructured.Unstructured) (*health.HealthStatus, error) { + if status, ok := r[obj.GetName()]; ok { + return &health.HealthStatus{Status: status, Message: "test"}, nil + } + return nil, nil +} + +func getResourceResult(resources []synccommon.ResourceSyncResult, resourceKey kube.ResourceKey) *synccommon.ResourceSyncResult { + for _, res := range resources { + if res.ResourceKey == resourceKey { + return &res + } + } + return nil +} + +func newHook(name string, hookType synccommon.HookType, deletePolicy synccommon.HookDeletePolicy) *unstructured.Unstructured { + obj := testingutils.NewPod() + obj.SetName(name) + obj.SetNamespace(testingutils.FakeArgoCDNamespace) + testingutils.Annotate(obj, synccommon.AnnotationKeyHook, string(hookType)) + testingutils.Annotate(obj, synccommon.AnnotationKeyHookDeletePolicy, string(deletePolicy)) + obj.SetFinalizers([]string{synccommon.HookFinalizer}) + return obj +} + +func withReplaceAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionReplace}) + return un +} + +func withServerSideApplyAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionServerSideApply}) + return un +} + +func withDisableServerSideApplyAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionDisableServerSideApply}) + return un +} + +func withReplaceAndServerSideApplyAnnotations(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: "Replace=true,ServerSideApply=true"}) + return un +} + +func withForceAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionForce}) + return un +} + +func withForceAndReplaceAnnotations(un *unstructured.Unstructured) *unstructured.Unstructured { + un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: "Force=true,Replace=true"}) + return un +} + +func createNamespaceTask(namespace string) (*syncTask, error) { + nsSpec := &corev1.Namespace{TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: kube.NamespaceKind}, ObjectMeta: metav1.ObjectMeta{Name: namespace}} + unstructuredObj, err := kube.ToUnstructured(nsSpec) + + task := &syncTask{phase: synccommon.SyncPhasePreSync, targetObj: unstructuredObj} + if err != nil { + return task, fmt.Errorf("failed to convert namespace spec to unstructured: %w", err) + } + return task, nil +} diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 0de5895d7..90a1cdc93 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -428,7 +428,6 @@ func (sc *syncContext) setRunningPhase(tasks syncTasks, isPendingDeletion bool) message = fmt.Sprintf("%s and %d more %s", message, moreTasks, andMore) } sc.setOperationPhase(common.OperationRunning, message) - } // sync has performs the actual apply or hook based sync diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 4690ad947..cce48e081 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -4,7 +4,6 @@ import ( "context" "encoding/json" "errors" - "fmt" "net/http" "net/http/httptest" "reflect" @@ -833,11 +832,6 @@ func TestSyncOptionValidate(t *testing.T) { } } -func withReplaceAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionReplace}) - return un -} - func TestSync_Replace(t *testing.T) { testCases := []struct { name string @@ -905,21 +899,6 @@ func TestSync_HookWithReplaceAndBeforeHookCreation_AlreadyDeleted(t *testing.T) assert.True(t, deleted) } -func withServerSideApplyAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionServerSideApply}) - return un -} - -func withDisableServerSideApplyAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionDisableServerSideApply}) - return un -} - -func withReplaceAndServerSideApplyAnnotations(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: "Replace=true,ServerSideApply=true"}) - return un -} - func TestSync_ServerSideApply(t *testing.T) { testCases := []struct { name string @@ -992,16 +971,6 @@ func TestSyncContext_ServerSideApplyWithDryRun(t *testing.T) { } } -func withForceAnnotation(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: synccommon.SyncOptionForce}) - return un -} - -func withForceAndReplaceAnnotations(un *unstructured.Unstructured) *unstructured.Unstructured { - un.SetAnnotations(map[string]string{synccommon.AnnotationSyncOptions: "Force=true,Replace=true"}) - return un -} - func TestSync_Force(t *testing.T) { testCases := []struct { name string @@ -1300,21 +1269,10 @@ func TestNamespaceAutoCreationForNonExistingNs(t *testing.T) { }) } -func createNamespaceTask(namespace string) (*syncTask, error) { - nsSpec := &corev1.Namespace{TypeMeta: metav1.TypeMeta{APIVersion: "v1", Kind: kube.NamespaceKind}, ObjectMeta: metav1.ObjectMeta{Name: namespace}} - unstructuredObj, err := kube.ToUnstructured(nsSpec) - - task := &syncTask{phase: synccommon.SyncPhasePreSync, targetObj: unstructuredObj} - if err != nil { - return task, fmt.Errorf("failed to convert namespace spec to unstructured: %w", err) - } - return task, nil -} - func TestSync_SuccessfulSyncWithSyncFailHook(t *testing.T) { - hook := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) pod := testingutils.NewPod() - syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) + syncFailHook := newHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ @@ -1350,9 +1308,9 @@ func TestSync_SuccessfulSyncWithSyncFailHook(t *testing.T) { } func TestSync_FailedSyncWithSyncFailHook_HookFailed(t *testing.T) { - hook := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) pod := testingutils.NewPod() - syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) + syncFailHook := newHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ @@ -1405,8 +1363,8 @@ func TestSync_FailedSyncWithSyncFailHook_ApplyFailed(t *testing.T) { // Tests that other SyncFail Hooks run even if one of them fail. pod := testingutils.NewPod() pod.SetNamespace(testingutils.FakeArgoCDNamespace) - successfulSyncFailHook := testingutils.NewHook("successful-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) - failedSyncFailHook := testingutils.NewHook("failed-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + successfulSyncFailHook := newHook("successful-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + failedSyncFailHook := newHook("failed-sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride{ successfulSyncFailHook.GetName(): health.HealthStatusHealthy, @@ -1434,17 +1392,17 @@ func TestSync_FailedSyncWithSyncFailHook_ApplyFailed(t *testing.T) { assert.Equal(t, synccommon.OperationRunning, phase) assert.Equal(t, "waiting for completion of hook /Pod/failed-sync-fail-hook and 1 more hooks", message) assert.Len(t, resources, 3) - podResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(pod)) + podResult := getResourceResult(resources, kube.GetResourceKey(pod)) require.NotNil(t, podResult, "%s not found", kube.GetResourceKey(pod)) assert.Equal(t, synccommon.OperationFailed, podResult.HookPhase) assert.Equal(t, synccommon.ResultCodeSyncFailed, podResult.Status) - failedSyncFailHookResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(failedSyncFailHook)) + failedSyncFailHookResult := getResourceResult(resources, kube.GetResourceKey(failedSyncFailHook)) require.NotNil(t, failedSyncFailHookResult, "%s not found", kube.GetResourceKey(failedSyncFailHook)) assert.Equal(t, synccommon.OperationFailed, failedSyncFailHookResult.HookPhase) assert.Equal(t, synccommon.ResultCodeSyncFailed, failedSyncFailHookResult.Status) - successfulSyncFailHookResult := testingutils.GetResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) + successfulSyncFailHookResult := getResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) require.NotNil(t, successfulSyncFailHookResult, "%s not found", kube.GetResourceKey(successfulSyncFailHook)) assert.Equal(t, synccommon.OperationRunning, successfulSyncFailHookResult.HookPhase) assert.Equal(t, synccommon.ResultCodeSynced, successfulSyncFailHookResult.Status) @@ -1462,7 +1420,7 @@ func TestSync_FailedSyncWithSyncFailHook_ApplyFailed(t *testing.T) { assert.Equal(t, "one or more synchronization tasks completed unsuccessfully, reason: fake pod failure\none or more SyncFail hooks failed, reason: fake hook failure", message) assert.Len(t, resources, 3) - successfulSyncFailHookResult = testingutils.GetResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) + successfulSyncFailHookResult = getResourceResult(resources, kube.GetResourceKey(successfulSyncFailHook)) require.NotNil(t, successfulSyncFailHookResult, "%s not found", kube.GetResourceKey(successfulSyncFailHook)) assert.Equal(t, synccommon.OperationSucceeded, successfulSyncFailHookResult.HookPhase) assert.Equal(t, synccommon.ResultCodeSynced, successfulSyncFailHookResult.Status) @@ -1501,19 +1459,10 @@ func TestBeforeHookCreation(t *testing.T) { assert.Equal(t, "waiting for completion of hook /Pod/my-pod", message) } -type resourceNameHealthOverride map[string]health.HealthStatusCode - -func (r resourceNameHealthOverride) GetResourceHealth(obj *unstructured.Unstructured) (*health.HealthStatus, error) { - if status, ok := r[obj.GetName()]; ok { - return &health.HealthStatus{Status: status, Message: "test"}, nil - } - return nil, nil -} - func TestSync_ExistingHooksWithFinalizer(t *testing.T) { - hook1 := testingutils.NewHook("existing-hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("existing-hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("existing-hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("existing-hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("existing-hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("existing-hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil) fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) @@ -1550,9 +1499,9 @@ func TestSync_ExistingHooksWithFinalizer(t *testing.T) { } func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ @@ -1610,9 +1559,9 @@ func TestSync_HooksNotDeletedIfPhaseNotCompleted(t *testing.T) { } func TestSync_HooksDeletedAfterSyncSucceeded(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ @@ -1674,9 +1623,9 @@ func TestSync_HooksDeletedAfterSyncSucceeded(t *testing.T) { } func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypePreSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypePreSync, synccommon.HookDeletePolicyHookSucceeded) syncCtx := newTestSyncCtx(nil, WithHealthOverride(resourceNameHealthOverride(map[string]health.HealthStatusCode{ @@ -1803,9 +1752,9 @@ func Test_setRunningPhase_runningHooks(t *testing.T) { var sc syncContext sc.log = textlogger.NewLogger(textlogger.NewConfig()).WithValues("application", "fake-app") - sc.setRunningPhase([]*syncTask{{targetObj: newHook(synccommon.HookTypeSyncFail)}}, false) + sc.setRunningPhase([]*syncTask{{targetObj: newHook("my-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation)}}, false) - assert.Equal(t, "waiting for completion of hook /Pod/my-pod", sc.message) + assert.Equal(t, "waiting for completion of hook /Pod/my-hook", sc.message) } func Test_setRunningPhase_pendingDeletion(t *testing.T) { @@ -1894,8 +1843,8 @@ func TestSync_SyncWaveHookError(t *testing.T) { pod1.SetNamespace(testingutils.FakeArgoCDNamespace) pod1.SetName("pod-1") - syncHook := testingutils.NewHook("sync-hook", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) - syncFailHook := testingutils.NewHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) + syncHook := newHook("sync-hook", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + syncFailHook := newHook("sync-fail-hook", synccommon.HookTypeSyncFail, synccommon.HookDeletePolicyBeforeHookCreation) syncCtx.dynamicIf = fake.NewSimpleDynamicClient(runtime.NewScheme()) syncCtx.resources = groupResources(ReconciliationResult{ @@ -1916,11 +1865,11 @@ func TestSync_SyncWaveHookError(t *testing.T) { assert.Equal(t, "SyncWaveHook failed: intentional error", msg) require.Len(t, results, 2) - podResult := testingutils.GetResourceResult(results, kube.GetResourceKey(pod1)) + podResult := getResourceResult(results, kube.GetResourceKey(pod1)) require.NotNil(t, podResult, "%s not found", kube.GetResourceKey(pod1)) assert.Equal(t, synccommon.OperationRunning, podResult.HookPhase) - hookResult := testingutils.GetResourceResult(results, kube.GetResourceKey(syncHook)) + hookResult := getResourceResult(results, kube.GetResourceKey(syncHook)) require.NotNil(t, hookResult, "%s not found", kube.GetResourceKey(syncHook)) assert.Equal(t, "Terminated", hookResult.Message) } @@ -2542,9 +2491,9 @@ func TestTerminate(t *testing.T) { } func TestTerminate_Hooks_Running(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) obj := testingutils.NewPod() obj.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -2620,9 +2569,9 @@ func TestTerminate_Hooks_Running(t *testing.T) { } func TestTerminate_Hooks_Running_Healthy(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) obj := testingutils.NewPod() obj.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -2702,9 +2651,9 @@ func TestTerminate_Hooks_Running_Healthy(t *testing.T) { } func TestTerminate_Hooks_Completed(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) - hook2 := testingutils.NewHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) - hook3 := testingutils.NewHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) + hook1 := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook2 := newHook("hook-2", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookFailed) + hook3 := newHook("hook-3", synccommon.HookTypeSync, synccommon.HookDeletePolicyHookSucceeded) obj := testingutils.NewPod() obj.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -2783,7 +2732,7 @@ func TestTerminate_Hooks_Completed(t *testing.T) { } func TestTerminate_Hooks_Error(t *testing.T) { - hook1 := testingutils.NewHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) + hook1 := newHook("hook-1", synccommon.HookTypeSync, synccommon.HookDeletePolicyBeforeHookCreation) obj := testingutils.NewPod() obj.SetNamespace(testingutils.FakeArgoCDNamespace) @@ -2815,7 +2764,7 @@ func TestTerminate_Hooks_Error(t *testing.T) { fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1) syncCtx.dynamicIf = fakeDynamicClient fakeDynamicClient.PrependReactor("update", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { - return true, nil, apierrors.NewInternalError(fmt.Errorf("update failed")) + return true, nil, apierrors.NewInternalError(errors.New("update failed")) }) syncCtx.Terminate() diff --git a/pkg/sync/sync_task_test.go b/pkg/sync/sync_task_test.go index 16825d0b4..de946f833 100644 --- a/pkg/sync/sync_task_test.go +++ b/pkg/sync/sync_task_test.go @@ -10,10 +10,6 @@ import ( testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" ) -func newHook(hookType common.HookType) *unstructured.Unstructured { - return testingutils.Annotate(testingutils.NewPod(), "argocd.argoproj.io/hook", string(hookType)) -} - func Test_syncTask_hookType(t *testing.T) { type fields struct { phase common.SyncPhase @@ -25,9 +21,9 @@ func Test_syncTask_hookType(t *testing.T) { want common.HookType }{ {"Empty", fields{common.SyncPhaseSync, testingutils.NewPod()}, ""}, - {"PreSyncHook", fields{common.SyncPhasePreSync, newHook(common.HookTypePreSync)}, common.HookTypePreSync}, - {"SyncHook", fields{common.SyncPhaseSync, newHook(common.HookTypeSync)}, common.HookTypeSync}, - {"PostSyncHook", fields{common.SyncPhasePostSync, newHook(common.HookTypePostSync)}, common.HookTypePostSync}, + {"PreSyncHook", fields{common.SyncPhasePreSync, newHook("hook", common.HookTypePreSync, common.HookDeletePolicyBeforeHookCreation)}, common.HookTypePreSync}, + {"SyncHook", fields{common.SyncPhaseSync, newHook("hook", common.HookTypeSync, common.HookDeletePolicyBeforeHookCreation)}, common.HookTypeSync}, + {"PostSyncHook", fields{common.SyncPhasePostSync, newHook("hook", common.HookTypePostSync, common.HookDeletePolicyBeforeHookCreation)}, common.HookTypePostSync}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { diff --git a/pkg/utils/kube/convert_test.go b/pkg/utils/kube/convert_test.go index 43e6ebe38..436761e16 100644 --- a/pkg/utils/kube/convert_test.go +++ b/pkg/utils/kube/convert_test.go @@ -3,12 +3,12 @@ package kube import ( "testing" - testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "k8s.io/apimachinery/pkg/runtime/schema" "sigs.k8s.io/yaml" + + testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" ) type testcase struct { diff --git a/pkg/utils/kube/ctl_test.go b/pkg/utils/kube/ctl_test.go index 11bb6c786..552339234 100644 --- a/pkg/utils/kube/ctl_test.go +++ b/pkg/utils/kube/ctl_test.go @@ -12,8 +12,9 @@ import ( "github.com/stretchr/testify/assert" "k8s.io/klog/v2/textlogger" - testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" "github.com/argoproj/gitops-engine/pkg/utils/tracing" + + testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" ) var _ Kubectl = &KubectlCmd{} diff --git a/pkg/utils/testing/testdata.go b/pkg/utils/testing/testdata.go index ef84deff5..ad446fdfb 100644 --- a/pkg/utils/testing/testdata.go +++ b/pkg/utils/testing/testdata.go @@ -2,10 +2,6 @@ package testing import ( "k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" - - synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" - - "github.com/argoproj/gitops-engine/pkg/utils/kube/kubetest" ) const ( @@ -26,16 +22,6 @@ func Annotate(obj *unstructured.Unstructured, key, val string) *unstructured.Uns return obj } -func NewHook(name string, hookType synccommon.HookType, deletePolicy synccommon.HookDeletePolicy) *unstructured.Unstructured { - obj := NewPod() - obj.SetName(name) - obj.SetNamespace(FakeArgoCDNamespace) - Annotate(obj, synccommon.AnnotationKeyHook, string(hookType)) - Annotate(obj, synccommon.AnnotationKeyHookDeletePolicy, string(deletePolicy)) - obj.SetFinalizers([]string{synccommon.HookFinalizer}) - return obj -} - var PodManifest = ` { "apiVersion": "v1", @@ -60,7 +46,7 @@ var PodManifest = ` ` func NewPod() *unstructured.Unstructured { - return kubetest.Unstructured(PodManifest) + return Unstructured(PodManifest) } var ServiceManifest = ` @@ -87,11 +73,11 @@ var ServiceManifest = ` ` func NewService() *unstructured.Unstructured { - return kubetest.Unstructured(ServiceManifest) + return Unstructured(ServiceManifest) } func NewCRD() *unstructured.Unstructured { - return kubetest.Unstructured(`apiVersion: apiextensions.k8s.io/v1beta1 + return Unstructured(`apiVersion: apiextensions.k8s.io/v1beta1 kind: CustomResourceDefinition metadata: name: testcrds.argoproj.io @@ -105,7 +91,7 @@ spec: } func NewNamespace() *unstructured.Unstructured { - return kubetest.Unstructured(`apiVersion: v1 + return Unstructured(`apiVersion: v1 kind: Namespace metadata: name: testnamespace diff --git a/pkg/utils/kube/kubetest/unstructured.go b/pkg/utils/testing/unstructured.go similarity index 97% rename from pkg/utils/kube/kubetest/unstructured.go rename to pkg/utils/testing/unstructured.go index 0c4a38ad5..f6c292001 100644 --- a/pkg/utils/kube/kubetest/unstructured.go +++ b/pkg/utils/testing/unstructured.go @@ -1,4 +1,4 @@ -package kubetest +package testing import ( "encoding/json" diff --git a/pkg/utils/testing/utils.go b/pkg/utils/testing/utils.go deleted file mode 100644 index 7e114ac5c..000000000 --- a/pkg/utils/testing/utils.go +++ /dev/null @@ -1,15 +0,0 @@ -package testing - -import ( - synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" - "github.com/argoproj/gitops-engine/pkg/utils/kube" -) - -func GetResourceResult(resources []synccommon.ResourceSyncResult, resourceKey kube.ResourceKey) *synccommon.ResourceSyncResult { - for _, res := range resources { - if res.ResourceKey == resourceKey { - return &res - } - } - return nil -} From 9c694edfc7d570cfa9b4df3e528ce6658c7ea4f6 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 16:15:01 -0400 Subject: [PATCH 23/28] fix unit test Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index cce48e081..682067ff2 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -127,6 +127,7 @@ func TestSyncNamespaceCreatedBeforeDryRunWithFailure(t *testing.T) { return true, nil }), func(ctx *syncContext) { resourceOps := ctx.resourceOps.(*kubetest.MockResourceOps) + resourceOps.ExecuteForDryRun = true resourceOps.Commands = map[string]kubetest.KubectlOutput{} resourceOps.Commands[pod.GetName()] = kubetest.KubectlOutput{ Output: "should not be returned", @@ -1663,7 +1664,7 @@ func TestSync_HooksDeletedAfterSyncFailed(t *testing.T) { return false, nil, nil }) deletedCount := 0 - fakeDynamicClient.PrependReactor("delete", "*", func(action testcore.Action) (handled bool, ret runtime.Object, err error) { + fakeDynamicClient.PrependReactor("delete", "*", func(_ testcore.Action) (handled bool, ret runtime.Object, err error) { deletedCount++ return false, nil, nil }) From 1cec50b8b094f410bba83be6f76c1423fece65e4 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 16:24:25 -0400 Subject: [PATCH 24/28] revert hook finalizer const move Signed-off-by: Alexandre Gaudreault --- pkg/sync/common/types.go | 3 --- pkg/sync/helper_test.go | 3 ++- pkg/sync/hook/hook.go | 7 ++++++- pkg/sync/sync_context.go | 4 ++-- pkg/sync/sync_context_test.go | 3 ++- 5 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/sync/common/types.go b/pkg/sync/common/types.go index 19578433a..00976ff5d 100644 --- a/pkg/sync/common/types.go +++ b/pkg/sync/common/types.go @@ -53,9 +53,6 @@ const ( // Default field manager for client-side apply migration DefaultClientSideApplyMigrationManager = "kubectl-client-side-apply" - - // HookFinalizer is the finalizer added to hooks to ensure they are deleted only after the sync phase is completed. - HookFinalizer = "argocd.argoproj.io/hook-finalizer" ) type PermissionValidator func(un *unstructured.Unstructured, res *metav1.APIResource) error diff --git a/pkg/sync/helper_test.go b/pkg/sync/helper_test.go index e119fc07d..f1938cf2a 100644 --- a/pkg/sync/helper_test.go +++ b/pkg/sync/helper_test.go @@ -9,6 +9,7 @@ import ( "github.com/argoproj/gitops-engine/pkg/health" synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" + "github.com/argoproj/gitops-engine/pkg/sync/hook" "github.com/argoproj/gitops-engine/pkg/utils/kube" testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" ) @@ -37,7 +38,7 @@ func newHook(name string, hookType synccommon.HookType, deletePolicy synccommon. obj.SetNamespace(testingutils.FakeArgoCDNamespace) testingutils.Annotate(obj, synccommon.AnnotationKeyHook, string(hookType)) testingutils.Annotate(obj, synccommon.AnnotationKeyHookDeletePolicy, string(deletePolicy)) - obj.SetFinalizers([]string{synccommon.HookFinalizer}) + obj.SetFinalizers([]string{hook.HookFinalizer}) return obj } diff --git a/pkg/sync/hook/hook.go b/pkg/sync/hook/hook.go index 252e15456..66dfc26e5 100644 --- a/pkg/sync/hook/hook.go +++ b/pkg/sync/hook/hook.go @@ -8,10 +8,15 @@ import ( resourceutil "github.com/argoproj/gitops-engine/pkg/sync/resource" ) +const ( + // HookFinalizer is the finalizer added to hooks to ensure they are deleted only after the sync phase is completed. + HookFinalizer = "argocd.argoproj.io/hook-finalizer" +) + func HasHookFinalizer(obj *unstructured.Unstructured) bool { finalizers := obj.GetFinalizers() for _, finalizer := range finalizers { - if finalizer == common.HookFinalizer { + if finalizer == HookFinalizer { return true } } diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 90a1cdc93..45b1ab590 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -752,7 +752,7 @@ func (sc *syncContext) removeHookFinalizer(task *syncTask) error { removeFinalizerMutation := func(obj *unstructured.Unstructured) bool { finalizers := obj.GetFinalizers() for i, finalizer := range finalizers { - if finalizer == common.HookFinalizer { + if finalizer == hook.HookFinalizer { obj.SetFinalizers(append(finalizers[:i], finalizers[i+1:]...)) return true } @@ -895,7 +895,7 @@ func (sc *syncContext) getSyncTasks() (_ syncTasks, successful bool) { targetObj.SetName(fmt.Sprintf("%s%s", generateName, postfix)) } if !hook.HasHookFinalizer(targetObj) { - targetObj.SetFinalizers(append(targetObj.GetFinalizers(), common.HookFinalizer)) + targetObj.SetFinalizers(append(targetObj.GetFinalizers(), hook.HookFinalizer)) } hookTasks = append(hookTasks, &syncTask{phase: phase, targetObj: targetObj}) } diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 682067ff2..555ae2d92 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -31,6 +31,7 @@ import ( "github.com/argoproj/gitops-engine/pkg/diff" "github.com/argoproj/gitops-engine/pkg/health" synccommon "github.com/argoproj/gitops-engine/pkg/sync/common" + "github.com/argoproj/gitops-engine/pkg/sync/hook" "github.com/argoproj/gitops-engine/pkg/utils/kube" "github.com/argoproj/gitops-engine/pkg/utils/kube/kubetest" testingutils "github.com/argoproj/gitops-engine/pkg/utils/testing" @@ -1431,7 +1432,7 @@ func TestBeforeHookCreation(t *testing.T) { finalizerRemoved := false syncCtx := newTestSyncCtx(nil) hookObj := testingutils.Annotate(testingutils.Annotate(testingutils.NewPod(), synccommon.AnnotationKeyHook, "Sync"), synccommon.AnnotationKeyHookDeletePolicy, "BeforeHookCreation") - hookObj.SetFinalizers([]string{synccommon.HookFinalizer}) + hookObj.SetFinalizers([]string{hook.HookFinalizer}) hookObj.SetNamespace(testingutils.FakeArgoCDNamespace) syncCtx.resources = groupResources(ReconciliationResult{ Live: []*unstructured.Unstructured{hookObj}, From bc80f5356cbf74a32657d429c9899d919221eda7 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 16:45:37 -0400 Subject: [PATCH 25/28] fix pending msg Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 15 ++++----------- pkg/sync/sync_context_test.go | 2 +- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 45b1ab590..6f37a194c 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -637,7 +637,9 @@ func (sc *syncContext) Sync() { sc.setRunningPhase(tasks, false) } default: - sc.setRunningPhase(tasks, true) + sc.setRunningPhase(tasks.Filter(func(task *syncTask) bool { + return task.deleteBeforeCreation() || (task.isPrune() && task.pending()) + }), true) } } @@ -1434,16 +1436,7 @@ func (sc *syncContext) runTasks(tasks syncTasks, dryRun bool) runState { sc.log.WithValues("numTasks", len(tasks), "dryRun", dryRun).V(1).Info("Running tasks") state := successful - var createTasks syncTasks - var pruneTasks syncTasks - - for _, task := range tasks { - if task.isPrune() { - pruneTasks = append(pruneTasks, task) - } else { - createTasks = append(createTasks, task) - } - } + pruneTasks, createTasks := tasks.Split(func(task *syncTask) bool { return task.isPrune() }) // remove finalizers from previous sync on existing hooks to make sure the operation is idempotent { diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 555ae2d92..7785a0040 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -1491,7 +1491,7 @@ func TestSync_ExistingHooksWithFinalizer(t *testing.T) { phase, message, _ := syncCtx.GetState() assert.Equal(t, synccommon.OperationRunning, phase) - assert.Equal(t, "waiting for deletion of hook /Pod/existing-hook-1 and 2 more hooks", message) + assert.Equal(t, "waiting for deletion of hook /Pod/existing-hook-1", message) assert.Equal(t, 3, updatedCount) assert.Equal(t, 1, deletedCount) From 9bd0a633494b9a7f7083f49f15c4e89347bc3c24 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Wed, 20 Aug 2025 16:59:06 -0400 Subject: [PATCH 26/28] more comments Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 6f37a194c..0047a6d65 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -717,19 +717,22 @@ func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { continue } - phase, msg, err := sc.getOperationPhase(task.liveObj) - if err != nil { - sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to get hook health: %v", err)) - phase = common.OperationRunning + // get the latest status of the running hook. Perhaps it already completed + phase, msg, statusErr := sc.getOperationPhase(task.liveObj) + if statusErr != nil { + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to get hook health: %v", statusErr)) } + // Now that we have the latest status, we can remove the finalizer. if err := sc.removeHookFinalizer(task); err != nil { sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to remove hook finalizer: %v", err)) terminateSuccessful = false continue } - if phase.Running() || task.deleteOnPhaseFailed() { + // delete the hook if it is running, if we dont know that it is running, + // or if it has just completed and is meant to be deleted on sync failed + if statusErr != nil || phase.Running() || task.deleteOnPhaseFailed() { err := sc.deleteResource(task) if err != nil && !apierrors.IsNotFound(err) { sc.setResourceResult(task, task.syncStatus, common.OperationFailed, fmt.Sprintf("Failed to delete: %v", err)) @@ -739,8 +742,10 @@ func (sc *syncContext) terminateHooksPreemptively(tasks syncTasks) bool { } if phase.Completed() { + // If the hook has completed, we can update it to it's real status sc.setResourceResult(task, task.syncStatus, phase, msg) - } else { + } else if task.operationState != common.OperationError { + // update the status if the resource is not in error to preserve the error message sc.setResourceResult(task, task.syncStatus, common.OperationFailed, "Terminated") } } From 8af49beba37e3cffb94b9ff1956976d15f9e3de0 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Fri, 22 Aug 2025 14:20:10 -0400 Subject: [PATCH 27/28] Update pkg/sync/doc.go Co-authored-by: Dan Garfield --- pkg/sync/doc.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/sync/doc.go b/pkg/sync/doc.go index 66a469adf..dbc033653 100644 --- a/pkg/sync/doc.go +++ b/pkg/sync/doc.go @@ -59,6 +59,7 @@ Hooks can be deleted in an automatic fashion using the annotation: argocd.argopr argocd.argoproj.io/hook: PostSync argocd.argoproj.io/hook-delete-policy: HookSucceeded +Hook deletion policies are governed by sync success and failure. A successful sync operation requires all hooks to complete successfully. A sync will fail if _any_ hooks fail. The following policies define when the hook will be deleted. - HookSucceeded - the hook resource is deleted if the sync succeeds From ef00d914b5e381485d71293b29688e7331c93747 Mon Sep 17 00:00:00 2001 From: Alexandre Gaudreault Date: Tue, 2 Sep 2025 13:11:36 -0400 Subject: [PATCH 28/28] remove finalizer from completed hooks on terminate Signed-off-by: Alexandre Gaudreault --- pkg/sync/sync_context.go | 12 ++++++++++++ pkg/sync/sync_context_test.go | 18 +++++++++--------- 2 files changed, 21 insertions(+), 9 deletions(-) diff --git a/pkg/sync/sync_context.go b/pkg/sync/sync_context.go index 0047a6d65..2ded02738 100644 --- a/pkg/sync/sync_context.go +++ b/pkg/sync/sync_context.go @@ -647,6 +647,18 @@ func (sc *syncContext) Sync() { func (sc *syncContext) Terminate() { sc.log.V(1).Info("terminating") tasks, _ := sc.getSyncTasks() + + // Remove completed hook finalizers + hooksCompleted := tasks.Filter(func(task *syncTask) bool { + return task.isHook() && task.completed() + }) + for _, task := range hooksCompleted { + if err := sc.removeHookFinalizer(task); err != nil { + sc.setResourceResult(task, task.syncStatus, common.OperationError, fmt.Sprintf("Failed to remove hook finalizer: %v", err)) + } + } + + // Terminate running hooks terminateSuccessful := sc.terminateHooksPreemptively(tasks.Filter(func(task *syncTask) bool { return task.isHook() })) if terminateSuccessful { sc.setOperationPhase(common.OperationFailed, "Operation terminated") diff --git a/pkg/sync/sync_context_test.go b/pkg/sync/sync_context_test.go index 7785a0040..64712d798 100644 --- a/pkg/sync/sync_context_test.go +++ b/pkg/sync/sync_context_test.go @@ -2534,8 +2534,8 @@ func TestTerminate_Hooks_Running(t *testing.T) { metav1.Now(), )) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{obj}, - Target: []*unstructured.Unstructured{obj}, + Live: []*unstructured.Unstructured{obj, hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{obj, nil, nil, nil}, }) syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) @@ -2612,8 +2612,8 @@ func TestTerminate_Hooks_Running_Healthy(t *testing.T) { metav1.Now(), )) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{obj}, - Target: []*unstructured.Unstructured{obj}, + Live: []*unstructured.Unstructured{obj, hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{obj, nil, nil, nil}, }) syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) @@ -2697,8 +2697,8 @@ func TestTerminate_Hooks_Completed(t *testing.T) { metav1.Now(), )) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{obj}, - Target: []*unstructured.Unstructured{obj}, + Live: []*unstructured.Unstructured{obj, hook1, hook2, hook3}, + Target: []*unstructured.Unstructured{obj, nil, nil, nil}, }) syncCtx.hooks = []*unstructured.Unstructured{hook1, hook2, hook3} fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1, hook2, hook3) @@ -2729,7 +2729,7 @@ func TestTerminate_Hooks_Completed(t *testing.T) { assert.Equal(t, kube.GetResourceKey(hook3), results[2].ResourceKey) assert.Equal(t, synccommon.OperationError, results[2].HookPhase) assert.Equal(t, "hook3 error", results[2].Message) - assert.Equal(t, 0, updatedCount) + assert.Equal(t, 3, updatedCount) assert.Equal(t, 0, deletedCount) } @@ -2759,8 +2759,8 @@ func TestTerminate_Hooks_Error(t *testing.T) { metav1.Now(), )) syncCtx.resources = groupResources(ReconciliationResult{ - Live: []*unstructured.Unstructured{obj}, - Target: []*unstructured.Unstructured{obj}, + Live: []*unstructured.Unstructured{obj, hook1}, + Target: []*unstructured.Unstructured{obj, nil}, }) syncCtx.hooks = []*unstructured.Unstructured{hook1} fakeDynamicClient := fake.NewSimpleDynamicClient(runtime.NewScheme(), hook1)