mirror of https://github.com/fluxcd/cli-utils.git
				
				
				
			Merge pull request #488 from karlkfi/karl-validation-event
feat: Add ValidationPolicy & ValidationEvent
This commit is contained in:
		
						commit
						5fb19a18af
					
				|  | @ -144,12 +144,14 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje | ||||||
| 
 | 
 | ||||||
| 		// Validate the resources to make sure we catch those problems early
 | 		// Validate the resources to make sure we catch those problems early
 | ||||||
| 		// before anything has been updated in the cluster.
 | 		// before anything has been updated in the cluster.
 | ||||||
| 		validator := &validation.Validator{Mapper: mapper} | 		vCollector := &validation.Collector{} | ||||||
| 		if err := validator.Validate(objects); err != nil { | 		validator := &validation.Validator{ | ||||||
| 			handleError(eventChannel, err) | 			Collector: vCollector, | ||||||
| 			return | 			Mapper:    mapper, | ||||||
| 		} | 		} | ||||||
|  | 		validator.Validate(objects) | ||||||
| 
 | 
 | ||||||
|  | 		// Decide which objects to apply and which to prune
 | ||||||
| 		applyObjs, pruneObjs, err := a.prepareObjects(invInfo, objects, options) | 		applyObjs, pruneObjs, err := a.prepareObjects(invInfo, objects, options) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			handleError(eventChannel, err) | 			handleError(eventChannel, err) | ||||||
|  | @ -166,6 +168,7 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje | ||||||
| 			Mapper:     mapper, | 			Mapper:     mapper, | ||||||
| 			InvClient:  a.invClient, | 			InvClient:  a.invClient, | ||||||
| 			Destroy:    false, | 			Destroy:    false, | ||||||
|  | 			Collector:  vCollector, | ||||||
| 		} | 		} | ||||||
| 		opts := solver.Options{ | 		opts := solver.Options{ | ||||||
| 			ServerSideOptions:      options.ServerSideOptions, | 			ServerSideOptions:      options.ServerSideOptions, | ||||||
|  | @ -199,7 +202,6 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje | ||||||
| 			}, | 			}, | ||||||
| 		} | 		} | ||||||
| 		// Build list of apply mutators.
 | 		// Build list of apply mutators.
 | ||||||
| 		// Share a thread-safe cache with the status poller.
 |  | ||||||
| 		resourceCache := cache.NewResourceCacheMap() | 		resourceCache := cache.NewResourceCacheMap() | ||||||
| 		applyMutators := []mutator.Interface{ | 		applyMutators := []mutator.Interface{ | ||||||
| 			&mutator.ApplyTimeMutator{ | 			&mutator.ApplyTimeMutator{ | ||||||
|  | @ -208,17 +210,43 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje | ||||||
| 				ResourceCache: resourceCache, | 				ResourceCache: resourceCache, | ||||||
| 			}, | 			}, | ||||||
| 		} | 		} | ||||||
| 		// Build the task queue by appending tasks in the proper order.
 | 
 | ||||||
| 		taskQueue, err := taskBuilder. | 		// Build the ordered set of tasks to execute.
 | ||||||
|  | 		taskQueue := taskBuilder. | ||||||
| 			AppendInvAddTask(invInfo, applyObjs, options.DryRunStrategy). | 			AppendInvAddTask(invInfo, applyObjs, options.DryRunStrategy). | ||||||
| 			AppendApplyWaitTasks(applyObjs, applyFilters, applyMutators, opts). | 			AppendApplyWaitTasks(applyObjs, applyFilters, applyMutators, opts). | ||||||
| 			AppendPruneWaitTasks(pruneObjs, pruneFilters, opts). | 			AppendPruneWaitTasks(pruneObjs, pruneFilters, opts). | ||||||
| 			AppendInvSetTask(invInfo, options.DryRunStrategy). | 			AppendInvSetTask(invInfo, options.DryRunStrategy). | ||||||
| 			Build() | 			Build() | ||||||
| 		if err != nil { | 
 | ||||||
| 			handleError(eventChannel, err) | 		klog.V(4).Infof("validation errors: %d", len(vCollector.Errors)) | ||||||
|  | 		klog.V(4).Infof("invalid objects: %d", len(vCollector.InvalidIds)) | ||||||
|  | 
 | ||||||
|  | 		// Handle validation errors
 | ||||||
|  | 		switch options.ValidationPolicy { | ||||||
|  | 		case validation.ExitEarly: | ||||||
|  | 			err = vCollector.ToError() | ||||||
|  | 			if err != nil { | ||||||
|  | 				handleError(eventChannel, err) | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 		case validation.SkipInvalid: | ||||||
|  | 			for _, err := range vCollector.Errors { | ||||||
|  | 				handleValidationError(eventChannel, err) | ||||||
|  | 			} | ||||||
|  | 		default: | ||||||
|  | 			handleError(eventChannel, fmt.Errorf("invalid ValidationPolicy: %q", options.ValidationPolicy)) | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
|  | 
 | ||||||
|  | 		// Build a TaskContext for passing info between tasks
 | ||||||
|  | 		taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache) | ||||||
|  | 
 | ||||||
|  | 		// Register invalid objects to be retained in the inventory, if present.
 | ||||||
|  | 		for _, id := range vCollector.InvalidIds { | ||||||
|  | 			taskContext.AddInvalidObject(id) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
| 		// Send event to inform the caller about the resources that
 | 		// Send event to inform the caller about the resources that
 | ||||||
| 		// will be applied/pruned.
 | 		// will be applied/pruned.
 | ||||||
| 		eventChannel <- event.Event{ | 		eventChannel <- event.Event{ | ||||||
|  | @ -231,7 +259,6 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje | ||||||
| 		klog.V(4).Infoln("applier building TaskStatusRunner...") | 		klog.V(4).Infoln("applier building TaskStatusRunner...") | ||||||
| 		allIds := object.UnstructuredSetToObjMetadataSet(append(applyObjs, pruneObjs...)) | 		allIds := object.UnstructuredSetToObjMetadataSet(append(applyObjs, pruneObjs...)) | ||||||
| 		runner := taskrunner.NewTaskStatusRunner(allIds, a.StatusPoller) | 		runner := taskrunner.NewTaskStatusRunner(allIds, a.StatusPoller) | ||||||
| 		taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache) |  | ||||||
| 		klog.V(4).Infoln("applier running TaskStatusRunner...") | 		klog.V(4).Infoln("applier running TaskStatusRunner...") | ||||||
| 		err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{ | 		err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{ | ||||||
| 			PollInterval:     options.PollInterval, | 			PollInterval:     options.PollInterval, | ||||||
|  | @ -283,6 +310,9 @@ type Options struct { | ||||||
| 
 | 
 | ||||||
| 	// InventoryPolicy defines the inventory policy of apply.
 | 	// InventoryPolicy defines the inventory policy of apply.
 | ||||||
| 	InventoryPolicy inventory.InventoryPolicy | 	InventoryPolicy inventory.InventoryPolicy | ||||||
|  | 
 | ||||||
|  | 	// ValidationPolicy defines how to handle invalid objects.
 | ||||||
|  | 	ValidationPolicy validation.Policy | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // setDefaults set the options to the default values if they
 | // setDefaults set the options to the default values if they
 | ||||||
|  | @ -322,3 +352,25 @@ func localNamespaces(localInv inventory.InventoryInfo, localObjs []object.ObjMet | ||||||
| 	} | 	} | ||||||
| 	return namespaces | 	return namespaces | ||||||
| } | } | ||||||
|  | 
 | ||||||
|  | func handleValidationError(eventChannel chan<- event.Event, err error) { | ||||||
|  | 	switch tErr := err.(type) { | ||||||
|  | 	case *validation.Error: | ||||||
|  | 		// handle validation error about one or more specific objects
 | ||||||
|  | 		eventChannel <- event.Event{ | ||||||
|  | 			Type: event.ValidationType, | ||||||
|  | 			ValidationEvent: event.ValidationEvent{ | ||||||
|  | 				Identifiers: tErr.Identifiers(), | ||||||
|  | 				Error:       tErr, | ||||||
|  | 			}, | ||||||
|  | 		} | ||||||
|  | 	default: | ||||||
|  | 		// handle general validation error (no specific object)
 | ||||||
|  | 		eventChannel <- event.Event{ | ||||||
|  | 			Type: event.ValidationType, | ||||||
|  | 			ValidationEvent: event.ValidationEvent{ | ||||||
|  | 				Error: tErr, | ||||||
|  | 			}, | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | @ -13,12 +13,15 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| 	"github.com/stretchr/testify/require" | 	"github.com/stretchr/testify/require" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
| 	"k8s.io/kubectl/pkg/scheme" | 	"k8s.io/kubectl/pkg/scheme" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/inventory" | 	"sigs.k8s.io/cli-utils/pkg/inventory" | ||||||
| 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/multierror" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/testutil" | 	"sigs.k8s.io/cli-utils/pkg/testutil" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | @ -489,7 +492,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Type:      event.Started, | 						Type:      event.Started, | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Secrets applied before Deployments (see pkg/ordering)
 | 				// Apply Secrets before Deployments (see ordering.SortableMetas)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.ApplyType, | 					EventType: event.ApplyType, | ||||||
| 					ApplyEvent: &testutil.ExpApplyEvent{ | 					ApplyEvent: &testutil.ExpApplyEvent{ | ||||||
|  | @ -522,7 +525,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Type:      event.Started, | 						Type:      event.Started, | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Secrets before Deployments (see pkg/ordering)
 | 				// Apply Secrets before Deployments (see ordering.SortableMetas)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.WaitType, | 					EventType: event.WaitType, | ||||||
| 					WaitEvent: &testutil.ExpWaitEvent{ | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | @ -539,7 +542,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Identifier: testutil.ToIdentifier(t, resources["deployment"]), | 						Identifier: testutil.ToIdentifier(t, resources["deployment"]), | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Deployment before Secret (see statusEvents)
 | 				// Wait Deployments before Secrets (see testutil.GroupedEventsByID)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.WaitType, | 					EventType: event.WaitType, | ||||||
| 					WaitEvent: &testutil.ExpWaitEvent{ | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | @ -695,7 +698,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Type:      event.Started, | 						Type:      event.Started, | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Deployments deleted before Secrets (see pkg/ordering)
 | 				// Prune Deployments before Secrets (see ordering.SortableMetas)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.PruneType, | 					EventType: event.PruneType, | ||||||
| 					PruneEvent: &testutil.ExpPruneEvent{ | 					PruneEvent: &testutil.ExpPruneEvent{ | ||||||
|  | @ -728,7 +731,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Type:      event.Started, | 						Type:      event.Started, | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Deployments before Secrets (see pkg/ordering)
 | 				// Prune Deployments before Secrets (see ordering.SortableMetas)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.WaitType, | 					EventType: event.WaitType, | ||||||
| 					WaitEvent: &testutil.ExpWaitEvent{ | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | @ -745,7 +748,7 @@ func TestApplier(t *testing.T) { | ||||||
| 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
| 					}, | 					}, | ||||||
| 				}, | 				}, | ||||||
| 				// Deployment before Secret (see statusEvents)
 | 				// Wait Deployments before Secrets (see testutil.GroupedEventsByID)
 | ||||||
| 				{ | 				{ | ||||||
| 					EventType: event.WaitType, | 					EventType: event.WaitType, | ||||||
| 					WaitEvent: &testutil.ExpWaitEvent{ | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | @ -1179,15 +1182,258 @@ func TestApplier(t *testing.T) { | ||||||
| 				}, | 				}, | ||||||
| 			}, | 			}, | ||||||
| 		}, | 		}, | ||||||
|  | 		"SkipInvalid - skip invalid objects and apply valid objects": { | ||||||
|  | 			namespace: "default", | ||||||
|  | 			resources: object.UnstructuredSet{ | ||||||
|  | 				testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 					"$.metadata.name", "", | ||||||
|  | 				}), | ||||||
|  | 				testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 					"$.kind", "", | ||||||
|  | 				}), | ||||||
|  | 				testutil.Unstructured(t, resources["secret"]), | ||||||
|  | 			}, | ||||||
|  | 			invInfo: inventoryInfo{ | ||||||
|  | 				name:      "inv-123", | ||||||
|  | 				namespace: "default", | ||||||
|  | 				id:        "test", | ||||||
|  | 			}, | ||||||
|  | 			clusterObjs: object.UnstructuredSet{}, | ||||||
|  | 			options: Options{ | ||||||
|  | 				ReconcileTimeout: time.Minute, | ||||||
|  | 				InventoryPolicy:  inventory.AdoptIfNoInventory, | ||||||
|  | 				EmitStatusEvents: true, | ||||||
|  | 				ValidationPolicy: validation.SkipInvalid, | ||||||
|  | 			}, | ||||||
|  | 			statusEvents: []pollevent.Event{ | ||||||
|  | 				{ | ||||||
|  | 					EventType: pollevent.ResourceUpdateEvent, | ||||||
|  | 					Resource: &pollevent.ResourceStatus{ | ||||||
|  | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 						Status:     status.CurrentStatus, | ||||||
|  | 						Resource:   testutil.Unstructured(t, resources["secret"]), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedStatusEvents: []testutil.ExpEvent{ | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.StatusType, | ||||||
|  | 					StatusEvent: &testutil.ExpStatusEvent{ | ||||||
|  | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 						Status:     status.CurrentStatus, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedEvents: []testutil.ExpEvent{ | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ValidationType, | ||||||
|  | 					ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 						Identifiers: object.ObjMetadataSet{ | ||||||
|  | 							object.UnstructuredToObjMetadata( | ||||||
|  | 								testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 									"$.kind", "", | ||||||
|  | 								}), | ||||||
|  | 							), | ||||||
|  | 						}, | ||||||
|  | 						Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 							field.Required(field.NewPath("kind"), "kind is required"), | ||||||
|  | 							object.UnstructuredToObjMetadata( | ||||||
|  | 								testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 									"$.kind", "", | ||||||
|  | 								}), | ||||||
|  | 							), | ||||||
|  | 						).Error()), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ValidationType, | ||||||
|  | 					ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 						Identifiers: object.ObjMetadataSet{ | ||||||
|  | 							object.UnstructuredToObjMetadata( | ||||||
|  | 								testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 									"$.metadata.name", "", | ||||||
|  | 								}), | ||||||
|  | 							), | ||||||
|  | 						}, | ||||||
|  | 						Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 							field.Required(field.NewPath("metadata", "name"), "name is required"), | ||||||
|  | 							object.UnstructuredToObjMetadata( | ||||||
|  | 								testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 									"$.metadata.name", "", | ||||||
|  | 								}), | ||||||
|  | 							), | ||||||
|  | 						).Error()), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.InitType, | ||||||
|  | 					InitEvent: &testutil.ExpInitEvent{}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "inventory-add-0", | ||||||
|  | 						Action:    event.InventoryAction, | ||||||
|  | 						Type:      event.Started, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "inventory-add-0", | ||||||
|  | 						Action:    event.InventoryAction, | ||||||
|  | 						Type:      event.Finished, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 
 | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "apply-0", | ||||||
|  | 						Action:    event.ApplyAction, | ||||||
|  | 						Type:      event.Started, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				// Secret applied
 | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ApplyType, | ||||||
|  | 					ApplyEvent: &testutil.ExpApplyEvent{ | ||||||
|  | 						GroupName:  "apply-0", | ||||||
|  | 						Operation:  event.Created, // Create new
 | ||||||
|  | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "apply-0", | ||||||
|  | 						Action:    event.ApplyAction, | ||||||
|  | 						Type:      event.Finished, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "wait-0", | ||||||
|  | 						Action:    event.WaitAction, | ||||||
|  | 						Type:      event.Started, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				// Secret pending
 | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.WaitType, | ||||||
|  | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 						GroupName:  "wait-0", | ||||||
|  | 						Operation:  event.ReconcilePending, | ||||||
|  | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				// Secret reconciled
 | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.WaitType, | ||||||
|  | 					WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 						GroupName:  "wait-0", | ||||||
|  | 						Operation:  event.Reconciled, | ||||||
|  | 						Identifier: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "wait-0", | ||||||
|  | 						Action:    event.WaitAction, | ||||||
|  | 						Type:      event.Finished, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "inventory-set-0", | ||||||
|  | 						Action:    event.InventoryAction, | ||||||
|  | 						Type:      event.Started, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ActionGroupType, | ||||||
|  | 					ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 						GroupName: "inventory-set-0", | ||||||
|  | 						Action:    event.InventoryAction, | ||||||
|  | 						Type:      event.Finished, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"ExitEarly - exit early on invalid objects and skip valid objects": { | ||||||
|  | 			namespace: "default", | ||||||
|  | 			resources: object.UnstructuredSet{ | ||||||
|  | 				testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 					"$.metadata.name", "", | ||||||
|  | 				}), | ||||||
|  | 				testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 					"$.kind", "", | ||||||
|  | 				}), | ||||||
|  | 				testutil.Unstructured(t, resources["secret"]), | ||||||
|  | 			}, | ||||||
|  | 			invInfo: inventoryInfo{ | ||||||
|  | 				name:      "inv-123", | ||||||
|  | 				namespace: "default", | ||||||
|  | 				id:        "test", | ||||||
|  | 			}, | ||||||
|  | 			clusterObjs: object.UnstructuredSet{}, | ||||||
|  | 			options: Options{ | ||||||
|  | 				ReconcileTimeout: time.Minute, | ||||||
|  | 				InventoryPolicy:  inventory.AdoptIfNoInventory, | ||||||
|  | 				EmitStatusEvents: true, | ||||||
|  | 				ValidationPolicy: validation.ExitEarly, | ||||||
|  | 			}, | ||||||
|  | 			statusEvents:         []pollevent.Event{}, | ||||||
|  | 			expectedStatusEvents: []testutil.ExpEvent{}, | ||||||
|  | 			expectedEvents: []testutil.ExpEvent{ | ||||||
|  | 				{ | ||||||
|  | 					EventType: event.ErrorType, | ||||||
|  | 					ErrorEvent: &testutil.ExpErrorEvent{ | ||||||
|  | 						Err: testutil.EqualErrorString(multierror.New( | ||||||
|  | 							validation.NewError( | ||||||
|  | 								field.Required(field.NewPath("metadata", "name"), "name is required"), | ||||||
|  | 								object.UnstructuredToObjMetadata( | ||||||
|  | 									testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 										"$.metadata.name", "", | ||||||
|  | 									}), | ||||||
|  | 								), | ||||||
|  | 							), | ||||||
|  | 							validation.NewError( | ||||||
|  | 								field.Required(field.NewPath("kind"), "kind is required"), | ||||||
|  | 								object.UnstructuredToObjMetadata( | ||||||
|  | 									testutil.Unstructured(t, resources["deployment"], JSONPathSetter{ | ||||||
|  | 										"$.kind", "", | ||||||
|  | 									}), | ||||||
|  | 								), | ||||||
|  | 							), | ||||||
|  | 						).Error()), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	for tn, tc := range testCases { | 	for tn, tc := range testCases { | ||||||
| 		t.Run(tn, func(t *testing.T) { | 		t.Run(tn, func(t *testing.T) { | ||||||
| 			poller := newFakePoller(tc.statusEvents) | 			poller := newFakePoller(tc.statusEvents) | ||||||
| 
 | 
 | ||||||
|  | 			// Only feed valid objects into the TestApplier.
 | ||||||
|  | 			// Invalid objects should not generate API requests.
 | ||||||
|  | 			validObjs := object.UnstructuredSet{} | ||||||
|  | 			for _, obj := range tc.resources { | ||||||
|  | 				id := object.UnstructuredToObjMetadata(obj) | ||||||
|  | 				if id.GroupKind.Kind == "" || id.Name == "" { | ||||||
|  | 					continue | ||||||
|  | 				} | ||||||
|  | 				validObjs = append(validObjs, obj) | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
| 			applier := newTestApplier(t, | 			applier := newTestApplier(t, | ||||||
| 				tc.invInfo, | 				tc.invInfo, | ||||||
| 				tc.resources, | 				validObjs, | ||||||
| 				tc.clusterObjs, | 				tc.clusterObjs, | ||||||
| 				poller, | 				poller, | ||||||
| 			) | 			) | ||||||
|  |  | ||||||
|  | @ -30,6 +30,7 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/poller" | 	"sigs.k8s.io/cli-utils/pkg/apply/poller" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/inventory" | 	"sigs.k8s.io/cli-utils/pkg/inventory" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/jsonpath" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling" | ||||||
| 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | @ -457,3 +458,15 @@ func toJSONBytes(t *testing.T, obj runtime.Object) []byte { | ||||||
| 	} | 	} | ||||||
| 	return objBytes | 	return objBytes | ||||||
| } | } | ||||||
|  | 
 | ||||||
|  | type JSONPathSetter struct { | ||||||
|  | 	Path  string | ||||||
|  | 	Value interface{} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (jps JSONPathSetter) Mutate(u *unstructured.Unstructured) { | ||||||
|  | 	_, err := jsonpath.Set(u.Object, jps.Path, jps.Value) | ||||||
|  | 	if err != nil { | ||||||
|  | 		panic(fmt.Sprintf("failed to mutate unstructured object: %v", err)) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | @ -23,6 +23,7 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling/engine" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/polling/engine" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| // NewDestroyer returns a new destroyer. It will set up the ApplyOptions and
 | // NewDestroyer returns a new destroyer. It will set up the ApplyOptions and
 | ||||||
|  | @ -81,6 +82,9 @@ type DestroyerOptions struct { | ||||||
| 	// PollInterval defines how often we should poll for the status
 | 	// PollInterval defines how often we should poll for the status
 | ||||||
| 	// of resources.
 | 	// of resources.
 | ||||||
| 	PollInterval time.Duration | 	PollInterval time.Duration | ||||||
|  | 
 | ||||||
|  | 	// ValidationPolicy defines how to handle invalid objects.
 | ||||||
|  | 	ValidationPolicy validation.Policy | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func setDestroyerDefaults(o *DestroyerOptions) { | func setDestroyerDefaults(o *DestroyerOptions) { | ||||||
|  | @ -115,6 +119,16 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option | ||||||
| 			handleError(eventChannel, err) | 			handleError(eventChannel, err) | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
|  | 
 | ||||||
|  | 		// Validate the resources to make sure we catch those problems early
 | ||||||
|  | 		// before anything has been updated in the cluster.
 | ||||||
|  | 		vCollector := &validation.Collector{} | ||||||
|  | 		validator := &validation.Validator{ | ||||||
|  | 			Collector: vCollector, | ||||||
|  | 			Mapper:    mapper, | ||||||
|  | 		} | ||||||
|  | 		validator.Validate(deleteObjs) | ||||||
|  | 
 | ||||||
| 		klog.V(4).Infoln("destroyer building task queue...") | 		klog.V(4).Infoln("destroyer building task queue...") | ||||||
| 		taskBuilder := &solver.TaskQueueBuilder{ | 		taskBuilder := &solver.TaskQueueBuilder{ | ||||||
| 			Pruner:    d.pruner, | 			Pruner:    d.pruner, | ||||||
|  | @ -122,6 +136,7 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option | ||||||
| 			Mapper:    mapper, | 			Mapper:    mapper, | ||||||
| 			InvClient: d.invClient, | 			InvClient: d.invClient, | ||||||
| 			Destroy:   true, | 			Destroy:   true, | ||||||
|  | 			Collector: vCollector, | ||||||
| 		} | 		} | ||||||
| 		opts := solver.Options{ | 		opts := solver.Options{ | ||||||
| 			Prune:                  true, | 			Prune:                  true, | ||||||
|  | @ -136,15 +151,42 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option | ||||||
| 				InvPolicy: options.InventoryPolicy, | 				InvPolicy: options.InventoryPolicy, | ||||||
| 			}, | 			}, | ||||||
| 		} | 		} | ||||||
|  | 
 | ||||||
| 		// Build the ordered set of tasks to execute.
 | 		// Build the ordered set of tasks to execute.
 | ||||||
| 		taskQueue, err := taskBuilder. | 		taskQueue := taskBuilder. | ||||||
| 			AppendPruneWaitTasks(deleteObjs, deleteFilters, opts). | 			AppendPruneWaitTasks(deleteObjs, deleteFilters, opts). | ||||||
| 			AppendDeleteInvTask(inv, options.DryRunStrategy). | 			AppendDeleteInvTask(inv, options.DryRunStrategy). | ||||||
| 			Build() | 			Build() | ||||||
| 		if err != nil { | 
 | ||||||
| 			handleError(eventChannel, err) | 		klog.V(4).Infof("validation errors: %d", len(vCollector.Errors)) | ||||||
|  | 		klog.V(4).Infof("invalid objects: %d", len(vCollector.InvalidIds)) | ||||||
|  | 
 | ||||||
|  | 		// Handle validation errors
 | ||||||
|  | 		switch options.ValidationPolicy { | ||||||
|  | 		case validation.ExitEarly: | ||||||
|  | 			err = vCollector.ToError() | ||||||
|  | 			if err != nil { | ||||||
|  | 				handleError(eventChannel, err) | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 		case validation.SkipInvalid: | ||||||
|  | 			for _, err := range vCollector.Errors { | ||||||
|  | 				handleValidationError(eventChannel, err) | ||||||
|  | 			} | ||||||
|  | 		default: | ||||||
|  | 			handleError(eventChannel, fmt.Errorf("invalid ValidationPolicy: %q", options.ValidationPolicy)) | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
|  | 
 | ||||||
|  | 		// Build a TaskContext for passing info between tasks
 | ||||||
|  | 		resourceCache := cache.NewResourceCacheMap() | ||||||
|  | 		taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache) | ||||||
|  | 
 | ||||||
|  | 		// Register invalid objects to be retained in the inventory, if present.
 | ||||||
|  | 		for _, id := range vCollector.InvalidIds { | ||||||
|  | 			taskContext.AddInvalidObject(id) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
| 		// Send event to inform the caller about the resources that
 | 		// Send event to inform the caller about the resources that
 | ||||||
| 		// will be pruned.
 | 		// will be pruned.
 | ||||||
| 		eventChannel <- event.Event{ | 		eventChannel <- event.Event{ | ||||||
|  | @ -156,9 +198,7 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option | ||||||
| 		// Create a new TaskStatusRunner to execute the taskQueue.
 | 		// Create a new TaskStatusRunner to execute the taskQueue.
 | ||||||
| 		klog.V(4).Infoln("destroyer building TaskStatusRunner...") | 		klog.V(4).Infoln("destroyer building TaskStatusRunner...") | ||||||
| 		deleteIds := object.UnstructuredSetToObjMetadataSet(deleteObjs) | 		deleteIds := object.UnstructuredSetToObjMetadataSet(deleteObjs) | ||||||
| 		resourceCache := cache.NewResourceCacheMap() |  | ||||||
| 		runner := taskrunner.NewTaskStatusRunner(deleteIds, d.StatusPoller) | 		runner := taskrunner.NewTaskStatusRunner(deleteIds, d.StatusPoller) | ||||||
| 		taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache) |  | ||||||
| 		klog.V(4).Infoln("destroyer running TaskStatusRunner...") | 		klog.V(4).Infoln("destroyer running TaskStatusRunner...") | ||||||
| 		err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{ | 		err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{ | ||||||
| 			UseCache:         true, | 			UseCache:         true, | ||||||
|  |  | ||||||
|  | @ -25,6 +25,7 @@ const ( | ||||||
| 	PruneType | 	PruneType | ||||||
| 	DeleteType | 	DeleteType | ||||||
| 	WaitType | 	WaitType | ||||||
|  | 	ValidationType | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| // Event is the type of the objects that will be returned through
 | // Event is the type of the objects that will be returned through
 | ||||||
|  | @ -65,6 +66,9 @@ type Event struct { | ||||||
| 
 | 
 | ||||||
| 	// WaitEvent contains information about any errors encountered in a WaitTask.
 | 	// WaitEvent contains information about any errors encountered in a WaitTask.
 | ||||||
| 	WaitEvent WaitEvent | 	WaitEvent WaitEvent | ||||||
|  | 
 | ||||||
|  | 	// ValidationEvent contains information about validation errors.
 | ||||||
|  | 	ValidationEvent ValidationEvent | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // String returns a string suitable for logging
 | // String returns a string suitable for logging
 | ||||||
|  | @ -88,6 +92,8 @@ func (e Event) String() string { | ||||||
| 		sb.WriteString(e.DeleteEvent.String()) | 		sb.WriteString(e.DeleteEvent.String()) | ||||||
| 	case WaitType: | 	case WaitType: | ||||||
| 		sb.WriteString(e.WaitEvent.String()) | 		sb.WriteString(e.WaitEvent.String()) | ||||||
|  | 	case ValidationType: | ||||||
|  | 		sb.WriteString(e.ValidationEvent.String()) | ||||||
| 	} | 	} | ||||||
| 	sb.WriteString(" }") | 	sb.WriteString(" }") | ||||||
| 	return sb.String() | 	return sb.String() | ||||||
|  | @ -290,3 +296,14 @@ func (de DeleteEvent) String() string { | ||||||
| 	return fmt.Sprintf("DeleteEvent{ GroupName: %q, Operation: %q, Identifier: %q, Reason: %q, Error: %q }", | 	return fmt.Sprintf("DeleteEvent{ GroupName: %q, Operation: %q, Identifier: %q, Reason: %q, Error: %q }", | ||||||
| 		de.GroupName, de.Operation, de.Identifier, de.Reason, de.Error) | 		de.GroupName, de.Operation, de.Identifier, de.Reason, de.Error) | ||||||
| } | } | ||||||
|  | 
 | ||||||
|  | type ValidationEvent struct { | ||||||
|  | 	Identifiers object.ObjMetadataSet | ||||||
|  | 	Error       error | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // String returns a string suitable for logging
 | ||||||
|  | func (ve ValidationEvent) String() string { | ||||||
|  | 	return fmt.Sprintf("ValidationEvent{ Identifiers: %+v, Error: %q }", | ||||||
|  | 		ve.Identifiers, ve.Error) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | @ -16,11 +16,12 @@ func _() { | ||||||
| 	_ = x[PruneType-5] | 	_ = x[PruneType-5] | ||||||
| 	_ = x[DeleteType-6] | 	_ = x[DeleteType-6] | ||||||
| 	_ = x[WaitType-7] | 	_ = x[WaitType-7] | ||||||
|  | 	_ = x[ValidationType-8] | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| const _Type_name = "InitTypeErrorTypeActionGroupTypeApplyTypeStatusTypePruneTypeDeleteTypeWaitType" | const _Type_name = "InitTypeErrorTypeActionGroupTypeApplyTypeStatusTypePruneTypeDeleteTypeWaitTypeValidationType" | ||||||
| 
 | 
 | ||||||
| var _Type_index = [...]uint8{0, 8, 17, 32, 41, 51, 60, 70, 78} | var _Type_index = [...]uint8{0, 8, 17, 32, 41, 51, 60, 70, 78, 92} | ||||||
| 
 | 
 | ||||||
| func (i Type) String() string { | func (i Type) String() string { | ||||||
| 	if i < 0 || i >= Type(len(_Type_index)-1) { | 	if i < 0 || i >= Type(len(_Type_index)-1) { | ||||||
|  |  | ||||||
|  | @ -410,7 +410,7 @@ func TestMutate(t *testing.T) { | ||||||
| 			reason:  "", | 			reason:  "", | ||||||
| 			// exact error message isn't very important. Feel free to update if the error text changes.
 | 			// exact error message isn't very important. Feel free to update if the error text changes.
 | ||||||
| 			errMsg: `failed to read annotation in resource (v1/namespaces/map-namespace/ConfigMap/map3-name): ` + | 			errMsg: `failed to read annotation in resource (v1/namespaces/map-namespace/ConfigMap/map3-name): ` + | ||||||
| 				`failed to parse apply-time-mutation annotation: ` + | 				`invalid "config.kubernetes.io/apply-time-mutation" annotation: ` + | ||||||
| 				`error unmarshaling JSON: ` + | 				`error unmarshaling JSON: ` + | ||||||
| 				`while decoding JSON: ` + | 				`while decoding JSON: ` + | ||||||
| 				`json: cannot unmarshal string into Go value of type mutation.ApplyTimeMutation`, | 				`json: cannot unmarshal string into Go value of type mutation.ApplyTimeMutation`, | ||||||
|  |  | ||||||
|  | @ -33,6 +33,7 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/inventory" | 	"sigs.k8s.io/cli-utils/pkg/inventory" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object/graph" | 	"sigs.k8s.io/cli-utils/pkg/object/graph" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| type TaskQueueBuilder struct { | type TaskQueueBuilder struct { | ||||||
|  | @ -41,6 +42,9 @@ type TaskQueueBuilder struct { | ||||||
| 	Factory    util.Factory | 	Factory    util.Factory | ||||||
| 	Mapper     meta.RESTMapper | 	Mapper     meta.RESTMapper | ||||||
| 	InvClient  inventory.InventoryClient | 	InvClient  inventory.InventoryClient | ||||||
|  | 	// Collector is used to collect validation errors and invalid objects.
 | ||||||
|  | 	// Invalid objects will be filtered and not be injected into tasks.
 | ||||||
|  | 	Collector *validation.Collector | ||||||
| 	// True if we are destroying, which deletes the inventory object
 | 	// True if we are destroying, which deletes the inventory object
 | ||||||
| 	// as well (possibly) the inventory namespace.
 | 	// as well (possibly) the inventory namespace.
 | ||||||
| 	Destroy bool | 	Destroy bool | ||||||
|  | @ -52,7 +56,6 @@ type TaskQueueBuilder struct { | ||||||
| 	waitCounter      int | 	waitCounter      int | ||||||
| 	pruneCounter     int | 	pruneCounter     int | ||||||
| 	tasks            []taskrunner.Task | 	tasks            []taskrunner.Task | ||||||
| 	err              error |  | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| type TaskQueue struct { | type TaskQueue struct { | ||||||
|  | @ -90,23 +93,17 @@ type Options struct { | ||||||
| 	InventoryPolicy        inventory.InventoryPolicy | 	InventoryPolicy        inventory.InventoryPolicy | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Build returns the queue of tasks that have been created.
 | // Build returns the queue of tasks that have been created
 | ||||||
| // TODO(seans): Now that we're reporting errors, we probably
 | func (t *TaskQueueBuilder) Build() *TaskQueue { | ||||||
| // want to move away from the Builder patter for the TaskBuilder.
 | 	return &TaskQueue{tasks: t.tasks} | ||||||
| func (t *TaskQueueBuilder) Build() (*TaskQueue, error) { |  | ||||||
| 	if t.err != nil { |  | ||||||
| 		return nil, t.err |  | ||||||
| 	} |  | ||||||
| 	return &TaskQueue{ |  | ||||||
| 		tasks: t.tasks, |  | ||||||
| 	}, nil |  | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // AppendInvAddTask appends an inventory add task to the task queue.
 | // AppendInvAddTask appends an inventory add task to the task queue.
 | ||||||
| // Returns a pointer to the Builder to chain function calls.
 | // Returns a pointer to the Builder to chain function calls.
 | ||||||
| func (t *TaskQueueBuilder) AppendInvAddTask(inv inventory.InventoryInfo, applyObjs object.UnstructuredSet, | func (t *TaskQueueBuilder) AppendInvAddTask(inv inventory.InventoryInfo, applyObjs object.UnstructuredSet, | ||||||
| 	dryRun common.DryRunStrategy) *TaskQueueBuilder { | 	dryRun common.DryRunStrategy) *TaskQueueBuilder { | ||||||
| 	klog.V(2).Infoln("adding inventory add task") | 	applyObjs = t.Collector.FilterInvalidObjects(applyObjs) | ||||||
|  | 	klog.V(2).Infoln("adding inventory add task (%d objects)", len(applyObjs)) | ||||||
| 	t.tasks = append(t.tasks, &task.InvAddTask{ | 	t.tasks = append(t.tasks, &task.InvAddTask{ | ||||||
| 		TaskName:  fmt.Sprintf("inventory-add-%d", t.invAddCounter), | 		TaskName:  fmt.Sprintf("inventory-add-%d", t.invAddCounter), | ||||||
| 		InvClient: t.InvClient, | 		InvClient: t.InvClient, | ||||||
|  | @ -152,6 +149,7 @@ func (t *TaskQueueBuilder) AppendDeleteInvTask(inv inventory.InventoryInfo, dryR | ||||||
| // to the cluster. Returns a pointer to the Builder to chain function calls.
 | // to the cluster. Returns a pointer to the Builder to chain function calls.
 | ||||||
| func (t *TaskQueueBuilder) AppendApplyTask(applyObjs object.UnstructuredSet, | func (t *TaskQueueBuilder) AppendApplyTask(applyObjs object.UnstructuredSet, | ||||||
| 	applyFilters []filter.ValidationFilter, applyMutators []mutator.Interface, o Options) *TaskQueueBuilder { | 	applyFilters []filter.ValidationFilter, applyMutators []mutator.Interface, o Options) *TaskQueueBuilder { | ||||||
|  | 	applyObjs = t.Collector.FilterInvalidObjects(applyObjs) | ||||||
| 	klog.V(2).Infof("adding apply task (%d objects)", len(applyObjs)) | 	klog.V(2).Infof("adding apply task (%d objects)", len(applyObjs)) | ||||||
| 	t.tasks = append(t.tasks, &task.ApplyTask{ | 	t.tasks = append(t.tasks, &task.ApplyTask{ | ||||||
| 		TaskName:          fmt.Sprintf("apply-%d", t.applyCounter), | 		TaskName:          fmt.Sprintf("apply-%d", t.applyCounter), | ||||||
|  | @ -172,6 +170,7 @@ func (t *TaskQueueBuilder) AppendApplyTask(applyObjs object.UnstructuredSet, | ||||||
| // Returns a pointer to the Builder to chain function calls.
 | // Returns a pointer to the Builder to chain function calls.
 | ||||||
| func (t *TaskQueueBuilder) AppendWaitTask(waitIds object.ObjMetadataSet, condition taskrunner.Condition, | func (t *TaskQueueBuilder) AppendWaitTask(waitIds object.ObjMetadataSet, condition taskrunner.Condition, | ||||||
| 	waitTimeout time.Duration) *TaskQueueBuilder { | 	waitTimeout time.Duration) *TaskQueueBuilder { | ||||||
|  | 	waitIds = t.Collector.FilterInvalidIds(waitIds) | ||||||
| 	klog.V(2).Infoln("adding wait task") | 	klog.V(2).Infoln("adding wait task") | ||||||
| 	t.tasks = append(t.tasks, taskrunner.NewWaitTask( | 	t.tasks = append(t.tasks, taskrunner.NewWaitTask( | ||||||
| 		fmt.Sprintf("wait-%d", t.waitCounter), | 		fmt.Sprintf("wait-%d", t.waitCounter), | ||||||
|  | @ -188,6 +187,7 @@ func (t *TaskQueueBuilder) AppendWaitTask(waitIds object.ObjMetadataSet, conditi | ||||||
| // Returns a pointer to the Builder to chain function calls.
 | // Returns a pointer to the Builder to chain function calls.
 | ||||||
| func (t *TaskQueueBuilder) AppendPruneTask(pruneObjs object.UnstructuredSet, | func (t *TaskQueueBuilder) AppendPruneTask(pruneObjs object.UnstructuredSet, | ||||||
| 	pruneFilters []filter.ValidationFilter, o Options) *TaskQueueBuilder { | 	pruneFilters []filter.ValidationFilter, o Options) *TaskQueueBuilder { | ||||||
|  | 	pruneObjs = t.Collector.FilterInvalidObjects(pruneObjs) | ||||||
| 	klog.V(2).Infof("adding prune task (%d objects)", len(pruneObjs)) | 	klog.V(2).Infof("adding prune task (%d objects)", len(pruneObjs)) | ||||||
| 	t.tasks = append(t.tasks, | 	t.tasks = append(t.tasks, | ||||||
| 		&task.PruneTask{ | 		&task.PruneTask{ | ||||||
|  | @ -213,9 +213,13 @@ func (t *TaskQueueBuilder) AppendApplyWaitTasks(applyObjs object.UnstructuredSet | ||||||
| 	// objects to apply into sets using a topological sort.
 | 	// objects to apply into sets using a topological sort.
 | ||||||
| 	applySets, err := graph.SortObjs(applyObjs) | 	applySets, err := graph.SortObjs(applyObjs) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		t.err = err | 		t.Collector.Collect(err) | ||||||
| 	} | 	} | ||||||
| 	for _, applySet := range applySets { | 	for _, applySet := range applySets { | ||||||
|  | 		applySet = t.Collector.FilterInvalidObjects(applySet) | ||||||
|  | 		if len(applySet) == 0 { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
| 		t.AppendApplyTask(applySet, applyFilters, applyMutators, o) | 		t.AppendApplyTask(applySet, applyFilters, applyMutators, o) | ||||||
| 		// dry-run skips wait tasks
 | 		// dry-run skips wait tasks
 | ||||||
| 		if !o.DryRunStrategy.ClientOrServerDryRun() { | 		if !o.DryRunStrategy.ClientOrServerDryRun() { | ||||||
|  | @ -236,9 +240,13 @@ func (t *TaskQueueBuilder) AppendPruneWaitTasks(pruneObjs object.UnstructuredSet | ||||||
| 		// objects to prune into sets using a (reverse) topological sort.
 | 		// objects to prune into sets using a (reverse) topological sort.
 | ||||||
| 		pruneSets, err := graph.ReverseSortObjs(pruneObjs) | 		pruneSets, err := graph.ReverseSortObjs(pruneObjs) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			t.err = err | 			t.Collector.Collect(err) | ||||||
| 		} | 		} | ||||||
| 		for _, pruneSet := range pruneSets { | 		for _, pruneSet := range pruneSets { | ||||||
|  | 			pruneSet = t.Collector.FilterInvalidObjects(pruneSet) | ||||||
|  | 			if len(pruneSet) == 0 { | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
| 			t.AppendPruneTask(pruneSet, pruneFilters, o) | 			t.AppendPruneTask(pruneSet, pruneFilters, o) | ||||||
| 			// dry-run skips wait tasks
 | 			// dry-run skips wait tasks
 | ||||||
| 			if !o.DryRunStrategy.ClientOrServerDryRun() { | 			if !o.DryRunStrategy.ClientOrServerDryRun() { | ||||||
|  |  | ||||||
|  | @ -422,19 +422,22 @@ func TestTaskQueueBuilder_AppendApplyWaitTasks(t *testing.T) { | ||||||
| 
 | 
 | ||||||
| 			applyIds := object.UnstructuredSetToObjMetadataSet(tc.applyObjs) | 			applyIds := object.UnstructuredSetToObjMetadataSet(tc.applyObjs) | ||||||
| 			fakeInvClient := inventory.NewFakeInventoryClient(applyIds) | 			fakeInvClient := inventory.NewFakeInventoryClient(applyIds) | ||||||
|  | 			vCollector := &validation.Collector{} | ||||||
| 			tqb := TaskQueueBuilder{ | 			tqb := TaskQueueBuilder{ | ||||||
| 				Pruner:    pruner, | 				Pruner:    pruner, | ||||||
| 				Mapper:    mapper, | 				Mapper:    mapper, | ||||||
| 				InvClient: fakeInvClient, | 				InvClient: fakeInvClient, | ||||||
|  | 				Collector: vCollector, | ||||||
| 			} | 			} | ||||||
| 			var filters []filter.ValidationFilter | 			var filters []filter.ValidationFilter | ||||||
| 			var mutators []mutator.Interface | 			var mutators []mutator.Interface | ||||||
| 			tq, err := tqb.AppendApplyWaitTasks( | 			tq := tqb.AppendApplyWaitTasks( | ||||||
| 				tc.applyObjs, | 				tc.applyObjs, | ||||||
| 				filters, | 				filters, | ||||||
| 				mutators, | 				mutators, | ||||||
| 				tc.options, | 				tc.options, | ||||||
| 			).Build() | 			).Build() | ||||||
|  | 			err := vCollector.ToError() | ||||||
| 			if tc.expectedError != nil { | 			if tc.expectedError != nil { | ||||||
| 				assert.EqualError(t, err, tc.expectedError.Error()) | 				assert.EqualError(t, err, tc.expectedError.Error()) | ||||||
| 				return | 				return | ||||||
|  | @ -699,7 +702,7 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) { | ||||||
| 				}, | 				}, | ||||||
| 			}, | 			}, | ||||||
| 		}, | 		}, | ||||||
| 		"cyclic dependency returns error": { | 		"cyclic dependency": { | ||||||
| 			pruneObjs: []*unstructured.Unstructured{ | 			pruneObjs: []*unstructured.Unstructured{ | ||||||
| 				testutil.Unstructured(t, resources["deployment"], | 				testutil.Unstructured(t, resources["deployment"], | ||||||
| 					testutil.AddDependsOn(t, testutil.ToIdentifier(t, resources["secret"]))), | 					testutil.AddDependsOn(t, testutil.ToIdentifier(t, resources["secret"]))), | ||||||
|  | @ -725,6 +728,48 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) { | ||||||
| 				testutil.ToIdentifier(t, resources["deployment"]), | 				testutil.ToIdentifier(t, resources["deployment"]), | ||||||
| 			), | 			), | ||||||
| 		}, | 		}, | ||||||
|  | 		"cyclic dependency and valid": { | ||||||
|  | 			pruneObjs: []*unstructured.Unstructured{ | ||||||
|  | 				testutil.Unstructured(t, resources["deployment"], | ||||||
|  | 					testutil.AddDependsOn(t, testutil.ToIdentifier(t, resources["secret"]))), | ||||||
|  | 				testutil.Unstructured(t, resources["secret"], | ||||||
|  | 					testutil.AddDependsOn(t, testutil.ToIdentifier(t, resources["deployment"]))), | ||||||
|  | 				testutil.Unstructured(t, resources["pod"]), | ||||||
|  | 			}, | ||||||
|  | 			options: Options{Prune: true}, | ||||||
|  | 			expectedTasks: []taskrunner.Task{ | ||||||
|  | 				&task.PruneTask{ | ||||||
|  | 					TaskName: "prune-0", | ||||||
|  | 					Objects: []*unstructured.Unstructured{ | ||||||
|  | 						testutil.Unstructured(t, resources["pod"]), | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				taskrunner.NewWaitTask( | ||||||
|  | 					"wait-0", | ||||||
|  | 					object.ObjMetadataSet{ | ||||||
|  | 						testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 					}, | ||||||
|  | 					taskrunner.AllCurrent, 1*time.Second, | ||||||
|  | 					testutil.NewFakeRESTMapper(), | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 			expectedError: validation.NewError( | ||||||
|  | 				graph.CyclicDependencyError{ | ||||||
|  | 					Edges: []graph.Edge{ | ||||||
|  | 						{ | ||||||
|  | 							From: testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 							To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 						}, | ||||||
|  | 						{ | ||||||
|  | 							From: testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 							To:   testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 				testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 			), | ||||||
|  | 		}, | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	for tn, tc := range testCases { | 	for tn, tc := range testCases { | ||||||
|  | @ -742,13 +787,16 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) { | ||||||
| 
 | 
 | ||||||
| 			pruneIds := object.UnstructuredSetToObjMetadataSet(tc.pruneObjs) | 			pruneIds := object.UnstructuredSetToObjMetadataSet(tc.pruneObjs) | ||||||
| 			fakeInvClient := inventory.NewFakeInventoryClient(pruneIds) | 			fakeInvClient := inventory.NewFakeInventoryClient(pruneIds) | ||||||
|  | 			vCollector := &validation.Collector{} | ||||||
| 			tqb := TaskQueueBuilder{ | 			tqb := TaskQueueBuilder{ | ||||||
| 				Pruner:    pruner, | 				Pruner:    pruner, | ||||||
| 				Mapper:    mapper, | 				Mapper:    mapper, | ||||||
| 				InvClient: fakeInvClient, | 				InvClient: fakeInvClient, | ||||||
|  | 				Collector: vCollector, | ||||||
| 			} | 			} | ||||||
| 			var emptyPruneFilters []filter.ValidationFilter | 			var emptyPruneFilters []filter.ValidationFilter | ||||||
| 			tq, err := tqb.AppendPruneWaitTasks(tc.pruneObjs, emptyPruneFilters, tc.options).Build() | 			tq := tqb.AppendPruneWaitTasks(tc.pruneObjs, emptyPruneFilters, tc.options).Build() | ||||||
|  | 			err := vCollector.ToError() | ||||||
| 			if tc.expectedError != nil { | 			if tc.expectedError != nil { | ||||||
| 				assert.EqualError(t, err, tc.expectedError.Error()) | 				assert.EqualError(t, err, tc.expectedError.Error()) | ||||||
| 				return | 				return | ||||||
|  |  | ||||||
|  | @ -105,6 +105,12 @@ func (i *InvSetTask) Start(taskContext *taskrunner.TaskContext) { | ||||||
| 		klog.V(4).Infof("remove from inventory %d abandoned objects", len(abandonedObjects)) | 		klog.V(4).Infof("remove from inventory %d abandoned objects", len(abandonedObjects)) | ||||||
| 		invObjs = invObjs.Diff(abandonedObjects) | 		invObjs = invObjs.Diff(abandonedObjects) | ||||||
| 
 | 
 | ||||||
|  | 		// If an object is invalid and was previously stored in the inventory,
 | ||||||
|  | 		// then keep it in the inventory so it can be applied/pruned next time.
 | ||||||
|  | 		invalidObjects := i.PrevInventory.Intersection(taskContext.InvalidObjects()) | ||||||
|  | 		klog.V(4).Infof("keep in inventory %d invalid objects", len(invalidObjects)) | ||||||
|  | 		invObjs = invObjs.Union(invalidObjects) | ||||||
|  | 
 | ||||||
| 		klog.V(4).Infof("set inventory %d total objects", len(invObjs)) | 		klog.V(4).Infof("set inventory %d total objects", len(invObjs)) | ||||||
| 		err := i.InvClient.Replace(i.InvInfo, invObjs, i.DryRun) | 		err := i.InvClient.Replace(i.InvInfo, invObjs, i.DryRun) | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -6,6 +6,7 @@ package task | ||||||
| import ( | import ( | ||||||
| 	"testing" | 	"testing" | ||||||
| 
 | 
 | ||||||
|  | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/cache" | 	"sigs.k8s.io/cli-utils/pkg/apply/cache" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/taskrunner" | 	"sigs.k8s.io/cli-utils/pkg/apply/taskrunner" | ||||||
|  | @ -14,10 +15,18 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/testutil" | 	"sigs.k8s.io/cli-utils/pkg/testutil" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | var objInvalid = &unstructured.Unstructured{ | ||||||
|  | 	Object: map[string]interface{}{ | ||||||
|  | 		"apiVersion": "apps/v1", | ||||||
|  | 		"kind":       "Deployment", | ||||||
|  | 	}, | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func TestInvSetTask(t *testing.T) { | func TestInvSetTask(t *testing.T) { | ||||||
| 	id1 := object.UnstructuredToObjMetadata(obj1) | 	id1 := object.UnstructuredToObjMetadata(obj1) | ||||||
| 	id2 := object.UnstructuredToObjMetadata(obj2) | 	id2 := object.UnstructuredToObjMetadata(obj2) | ||||||
| 	id3 := object.UnstructuredToObjMetadata(obj3) | 	id3 := object.UnstructuredToObjMetadata(obj3) | ||||||
|  | 	idInvalid := object.UnstructuredToObjMetadata(objInvalid) | ||||||
| 
 | 
 | ||||||
| 	tests := map[string]struct { | 	tests := map[string]struct { | ||||||
| 		prevInventory  object.ObjMetadataSet | 		prevInventory  object.ObjMetadataSet | ||||||
|  | @ -27,28 +36,23 @@ func TestInvSetTask(t *testing.T) { | ||||||
| 		skippedApplies object.ObjMetadataSet | 		skippedApplies object.ObjMetadataSet | ||||||
| 		skippedDeletes object.ObjMetadataSet | 		skippedDeletes object.ObjMetadataSet | ||||||
| 		abandonedObjs  object.ObjMetadataSet | 		abandonedObjs  object.ObjMetadataSet | ||||||
|  | 		invalidObjs    object.ObjMetadataSet | ||||||
| 		expectedObjs   object.ObjMetadataSet | 		expectedObjs   object.ObjMetadataSet | ||||||
| 	}{ | 	}{ | ||||||
| 		"no apply objs, no prune failures; no inventory": { | 		"no apply objs, no prune failures; no inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, | 			expectedObjs: object.ObjMetadataSet{}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 		}, | 		}, | ||||||
| 		"one apply objs, no prune failures; one inventory": { | 		"one apply objs, no prune failures; one inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{id1}, | 			appliedObjs:  object.ObjMetadataSet{id1}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, | 			expectedObjs: object.ObjMetadataSet{id1}, | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id1}, |  | ||||||
| 		}, | 		}, | ||||||
| 		"no apply objs, one prune failure, in prev inventory; one inventory": { | 		"no apply objs, one prune failure, in prev inventory; one inventory": { | ||||||
| 			prevInventory: object.ObjMetadataSet{id1}, | 			prevInventory: object.ObjMetadataSet{id1}, | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{id1}, | 			failedDeletes: object.ObjMetadataSet{id1}, | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id1}, | 			expectedObjs:  object.ObjMetadataSet{id1}, | ||||||
| 		}, | 		}, | ||||||
| 		"no apply objs, one prune failure, not in prev inventory; no inventory": { | 		"no apply objs, one prune failure, not in prev inventory; no inventory": { | ||||||
| 			// aritifical use case: prunes come from the inventory
 | 			// aritifical use case: prunes come from the inventory
 | ||||||
| 			prevInventory: object.ObjMetadataSet{}, |  | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{id1}, | 			failedDeletes: object.ObjMetadataSet{id1}, | ||||||
| 			expectedObjs:  object.ObjMetadataSet{}, | 			expectedObjs:  object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
|  | @ -66,129 +70,95 @@ func TestInvSetTask(t *testing.T) { | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id1, id2, id3}, | 			expectedObjs:  object.ObjMetadataSet{id1, id2, id3}, | ||||||
| 		}, | 		}, | ||||||
| 		"no apply objs, no apply failures, no prune failures; no inventory": { | 		"no apply objs, no apply failures, no prune failures; no inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies: object.ObjMetadataSet{id3}, | 			failedApplies: object.ObjMetadataSet{id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{}, | 			expectedObjs:  object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply failure not in prev inventory; no inventory": { | 		"one apply failure not in prev inventory; no inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies: object.ObjMetadataSet{id3}, | 			failedApplies: object.ObjMetadataSet{id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{}, | 			expectedObjs:  object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply obj, one apply failure not in prev inventory; one inventory": { | 		"one apply obj, one apply failure not in prev inventory; one inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{id2}, | 			appliedObjs:   object.ObjMetadataSet{id2}, | ||||||
| 			failedApplies: object.ObjMetadataSet{id3}, | 			failedApplies: object.ObjMetadataSet{id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id2}, | 			expectedObjs:  object.ObjMetadataSet{id2}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply obj, one apply failure in prev inventory; one inventory": { | 		"one apply obj, one apply failure in prev inventory; one inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{id2}, | 			appliedObjs:   object.ObjMetadataSet{id2}, | ||||||
| 			failedApplies: object.ObjMetadataSet{id3}, | 			failedApplies: object.ObjMetadataSet{id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{id3}, | 			prevInventory: object.ObjMetadataSet{id3}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply obj, two apply failures with one in prev inventory; two inventory": { | 		"one apply obj, two apply failures with one in prev inventory; two inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{id2}, | 			appliedObjs:   object.ObjMetadataSet{id2}, | ||||||
| 			failedApplies: object.ObjMetadataSet{id1, id3}, | 			failedApplies: object.ObjMetadataSet{id1, id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{id3}, | 			prevInventory: object.ObjMetadataSet{id3}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | ||||||
| 		}, | 		}, | ||||||
| 		"three apply failures with two in prev inventory; two inventory": { | 		"three apply failures with two in prev inventory; two inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies: object.ObjMetadataSet{id1, id2, id3}, | 			failedApplies: object.ObjMetadataSet{id1, id2, id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{id2, id3}, | 			prevInventory: object.ObjMetadataSet{id2, id3}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | 			expectedObjs:  object.ObjMetadataSet{id2, id3}, | ||||||
| 		}, | 		}, | ||||||
| 		"three apply failures with three in prev inventory; three inventory": { | 		"three apply failures with three in prev inventory; three inventory": { | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies: object.ObjMetadataSet{id1, id2, id3}, | 			failedApplies: object.ObjMetadataSet{id1, id2, id3}, | ||||||
| 			prevInventory: object.ObjMetadataSet{id2, id3, id1}, | 			prevInventory: object.ObjMetadataSet{id2, id3, id1}, | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id2, id1, id3}, | 			expectedObjs:  object.ObjMetadataSet{id2, id1, id3}, | ||||||
| 		}, | 		}, | ||||||
| 		"one skipped apply from prev inventory; one inventory": { | 		"one skipped apply from prev inventory; one inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{id1}, | 			prevInventory:  object.ObjMetadataSet{id1}, | ||||||
| 			appliedObjs:    object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedApplies: object.ObjMetadataSet{id1}, | 			skippedApplies: object.ObjMetadataSet{id1}, | ||||||
| 			skippedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:   object.ObjMetadataSet{id1}, | 			expectedObjs:   object.ObjMetadataSet{id1}, | ||||||
| 		}, | 		}, | ||||||
| 		"one skipped apply, no prev inventory; no inventory": { | 		"one skipped apply, no prev inventory; no inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{}, |  | ||||||
| 			appliedObjs:    object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedApplies: object.ObjMetadataSet{id1}, | 			skippedApplies: object.ObjMetadataSet{id1}, | ||||||
| 			skippedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:   object.ObjMetadataSet{}, | 			expectedObjs:   object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply obj, one skipped apply, two prev inventory; two inventory": { | 		"one apply obj, one skipped apply, two prev inventory; two inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{id1, id2}, | 			prevInventory:  object.ObjMetadataSet{id1, id2}, | ||||||
| 			appliedObjs:    object.ObjMetadataSet{id2}, | 			appliedObjs:    object.ObjMetadataSet{id2}, | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedApplies: object.ObjMetadataSet{id1}, | 			skippedApplies: object.ObjMetadataSet{id1}, | ||||||
| 			skippedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:   object.ObjMetadataSet{id1, id2}, | 			expectedObjs:   object.ObjMetadataSet{id1, id2}, | ||||||
| 		}, | 		}, | ||||||
| 		"one skipped delete from prev inventory; one inventory": { | 		"one skipped delete from prev inventory; one inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{id1}, | 			prevInventory:  object.ObjMetadataSet{id1}, | ||||||
| 			appliedObjs:    object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedApplies: object.ObjMetadataSet{}, |  | ||||||
| 			skippedDeletes: object.ObjMetadataSet{id1}, | 			skippedDeletes: object.ObjMetadataSet{id1}, | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:   object.ObjMetadataSet{id1}, | 			expectedObjs:   object.ObjMetadataSet{id1}, | ||||||
| 		}, | 		}, | ||||||
| 		"one apply obj, one skipped delete, two prev inventory; two inventory": { | 		"one apply obj, one skipped delete, two prev inventory; two inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{id1, id2}, | 			prevInventory:  object.ObjMetadataSet{id1, id2}, | ||||||
| 			appliedObjs:    object.ObjMetadataSet{id2}, | 			appliedObjs:    object.ObjMetadataSet{id2}, | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedApplies: object.ObjMetadataSet{}, |  | ||||||
| 			skippedDeletes: object.ObjMetadataSet{id1}, | 			skippedDeletes: object.ObjMetadataSet{id1}, | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{}, |  | ||||||
| 			expectedObjs:   object.ObjMetadataSet{id1, id2}, | 			expectedObjs:   object.ObjMetadataSet{id1, id2}, | ||||||
| 		}, | 		}, | ||||||
| 		"two apply obj, one abandoned, three in prev inventory; two inventory": { | 		"two apply obj, one abandoned, three in prev inventory; two inventory": { | ||||||
| 			prevInventory: object.ObjMetadataSet{id1, id2, id3}, | 			prevInventory: object.ObjMetadataSet{id1, id2, id3}, | ||||||
| 			appliedObjs:   object.ObjMetadataSet{id1, id2}, | 			appliedObjs:   object.ObjMetadataSet{id1, id2}, | ||||||
| 			failedApplies: object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			abandonedObjs: object.ObjMetadataSet{id3}, | 			abandonedObjs: object.ObjMetadataSet{id3}, | ||||||
| 			expectedObjs:  object.ObjMetadataSet{id1, id2}, | 			expectedObjs:  object.ObjMetadataSet{id1, id2}, | ||||||
| 		}, | 		}, | ||||||
| 		"two abandoned, two in prev inventory; no inventory": { | 		"two abandoned, two in prev inventory; no inventory": { | ||||||
| 			prevInventory: object.ObjMetadataSet{id2, id3}, | 			prevInventory: object.ObjMetadataSet{id2, id3}, | ||||||
| 			appliedObjs:   object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies: object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes: object.ObjMetadataSet{}, |  | ||||||
| 			abandonedObjs: object.ObjMetadataSet{id2, id3}, | 			abandonedObjs: object.ObjMetadataSet{id2, id3}, | ||||||
| 			expectedObjs:  object.ObjMetadataSet{}, | 			expectedObjs:  object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
| 		"same obj skipped delete and abandoned, one in prev inventory; no inventory": { | 		"same obj skipped delete and abandoned, one in prev inventory; no inventory": { | ||||||
| 			prevInventory:  object.ObjMetadataSet{id3}, | 			prevInventory:  object.ObjMetadataSet{id3}, | ||||||
| 			appliedObjs:    object.ObjMetadataSet{}, |  | ||||||
| 			failedApplies:  object.ObjMetadataSet{}, |  | ||||||
| 			failedDeletes:  object.ObjMetadataSet{}, |  | ||||||
| 			skippedDeletes: object.ObjMetadataSet{id3}, | 			skippedDeletes: object.ObjMetadataSet{id3}, | ||||||
| 			abandonedObjs:  object.ObjMetadataSet{id3}, | 			abandonedObjs:  object.ObjMetadataSet{id3}, | ||||||
| 			expectedObjs:   object.ObjMetadataSet{}, | 			expectedObjs:   object.ObjMetadataSet{}, | ||||||
| 		}, | 		}, | ||||||
|  | 		"preserve invalid objects in the inventory": { | ||||||
|  | 			prevInventory: object.ObjMetadataSet{id3, idInvalid}, | ||||||
|  | 			appliedObjs:   object.ObjMetadataSet{id3}, | ||||||
|  | 			invalidObjs:   object.ObjMetadataSet{idInvalid}, | ||||||
|  | 			expectedObjs:  object.ObjMetadataSet{id3, idInvalid}, | ||||||
|  | 		}, | ||||||
|  | 		"ignore invalid objects not in the inventory": { | ||||||
|  | 			prevInventory: object.ObjMetadataSet{id3}, | ||||||
|  | 			appliedObjs:   object.ObjMetadataSet{id3}, | ||||||
|  | 			invalidObjs:   object.ObjMetadataSet{idInvalid}, | ||||||
|  | 			expectedObjs:  object.ObjMetadataSet{id3}, | ||||||
|  | 		}, | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	for name, tc := range tests { | 	for name, tc := range tests { | ||||||
|  | @ -222,6 +192,9 @@ func TestInvSetTask(t *testing.T) { | ||||||
| 			for _, abandonedObj := range tc.abandonedObjs { | 			for _, abandonedObj := range tc.abandonedObjs { | ||||||
| 				context.AddAbandonedObject(abandonedObj) | 				context.AddAbandonedObject(abandonedObj) | ||||||
| 			} | 			} | ||||||
|  | 			for _, invalidObj := range tc.invalidObjs { | ||||||
|  | 				context.AddInvalidObject(invalidObj) | ||||||
|  | 			} | ||||||
| 			if taskName != task.Name() { | 			if taskName != task.Name() { | ||||||
| 				t.Errorf("expected task name (%s), got (%s)", taskName, task.Name()) | 				t.Errorf("expected task name (%s), got (%s)", taskName, task.Name()) | ||||||
| 			} | 			} | ||||||
|  |  | ||||||
|  | @ -24,6 +24,7 @@ func NewTaskContext(eventChannel chan event.Event, resourceCache cache.ResourceC | ||||||
| 		skippedApplies:    make(map[object.ObjMetadata]struct{}), | 		skippedApplies:    make(map[object.ObjMetadata]struct{}), | ||||||
| 		skippedDeletes:    make(map[object.ObjMetadata]struct{}), | 		skippedDeletes:    make(map[object.ObjMetadata]struct{}), | ||||||
| 		abandonedObjects:  make(map[object.ObjMetadata]struct{}), | 		abandonedObjects:  make(map[object.ObjMetadata]struct{}), | ||||||
|  | 		invalidObjects:    make(map[object.ObjMetadata]struct{}), | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | @ -39,6 +40,7 @@ type TaskContext struct { | ||||||
| 	skippedApplies    map[object.ObjMetadata]struct{} | 	skippedApplies    map[object.ObjMetadata]struct{} | ||||||
| 	skippedDeletes    map[object.ObjMetadata]struct{} | 	skippedDeletes    map[object.ObjMetadata]struct{} | ||||||
| 	abandonedObjects  map[object.ObjMetadata]struct{} | 	abandonedObjects  map[object.ObjMetadata]struct{} | ||||||
|  | 	invalidObjects    map[object.ObjMetadata]struct{} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func (tc *TaskContext) TaskChannel() chan TaskResult { | func (tc *TaskContext) TaskChannel() chan TaskResult { | ||||||
|  | @ -211,6 +213,22 @@ func (tc *TaskContext) AbandonedObjects() object.ObjMetadataSet { | ||||||
| 	return object.ObjMetadataSetFromMap(tc.abandonedObjects) | 	return object.ObjMetadataSetFromMap(tc.abandonedObjects) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | // IsInvalidObject returns true if the object is abandoned
 | ||||||
|  | func (tc *TaskContext) IsInvalidObject(id object.ObjMetadata) bool { | ||||||
|  | 	_, found := tc.invalidObjects[id] | ||||||
|  | 	return found | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // AddInvalidObject registers that the object is abandoned
 | ||||||
|  | func (tc *TaskContext) AddInvalidObject(id object.ObjMetadata) { | ||||||
|  | 	tc.invalidObjects[id] = struct{}{} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // InvalidObjects returns all the abandoned objects
 | ||||||
|  | func (tc *TaskContext) InvalidObjects() object.ObjMetadataSet { | ||||||
|  | 	return object.ObjMetadataSetFromMap(tc.invalidObjects) | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // applyInfo captures information about resources that have been
 | // applyInfo captures information about resources that have been
 | ||||||
| // applied. This is captured in the TaskContext so other tasks
 | // applied. This is captured in the TaskContext so other tasks
 | ||||||
| // running later might use this information.
 | // running later might use this information.
 | ||||||
|  |  | ||||||
|  | @ -10,6 +10,7 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
| 	"k8s.io/klog/v2" | 	"k8s.io/klog/v2" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| const ( | const ( | ||||||
|  | @ -42,7 +43,10 @@ func ReadAnnotation(u *unstructured.Unstructured) (DependencySet, error) { | ||||||
| 
 | 
 | ||||||
| 	depSet, err := ParseDependencySet(depSetStr) | 	depSet, err := ParseDependencySet(depSetStr) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return depSet, fmt.Errorf("failed to parse depends-on annotation: %w", err) | 		return depSet, object.InvalidAnnotationError{ | ||||||
|  | 			Annotation: Annotation, | ||||||
|  | 			Cause:      err, | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return depSet, nil | 	return depSet, nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | @ -54,10 +54,10 @@ func FormatDependencySet(depSet DependencySet) (string, error) { | ||||||
| // Returns the parsed DependencySet or an error if unable to parse.
 | // Returns the parsed DependencySet or an error if unable to parse.
 | ||||||
| func ParseDependencySet(depsStr string) (DependencySet, error) { | func ParseDependencySet(depsStr string) (DependencySet, error) { | ||||||
| 	objs := DependencySet{} | 	objs := DependencySet{} | ||||||
| 	for _, objStr := range strings.Split(depsStr, annotationSeparator) { | 	for i, objStr := range strings.Split(depsStr, annotationSeparator) { | ||||||
| 		obj, err := ParseObjMetadata(objStr) | 		obj, err := ParseObjMetadata(objStr) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			return objs, fmt.Errorf("failed to parse object metadata: %w", err) | 			return objs, fmt.Errorf("failed to parse object reference (index: %d): %w", i, err) | ||||||
| 		} | 		} | ||||||
| 		objs = append(objs, obj) | 		objs = append(objs, obj) | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
|  | @ -0,0 +1,24 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package object | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"fmt" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // InvalidAnnotationError represents an invalid annotation.
 | ||||||
|  | // Fields are exposed to allow callers to perform introspection.
 | ||||||
|  | type InvalidAnnotationError struct { | ||||||
|  | 	Annotation string | ||||||
|  | 	Cause      error | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (iae InvalidAnnotationError) Error() string { | ||||||
|  | 	return fmt.Sprintf("invalid %q annotation: %v", | ||||||
|  | 		iae.Annotation, iae.Cause) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (iae InvalidAnnotationError) Unwrap() error { | ||||||
|  | 	return iae.Cause | ||||||
|  | } | ||||||
|  | @ -0,0 +1,32 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package object | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"errors" | ||||||
|  | 	"testing" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestInvalidAnnotationErrorString(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		err            InvalidAnnotationError | ||||||
|  | 		expectedString string | ||||||
|  | 	}{ | ||||||
|  | 		"cluster-scoped": { | ||||||
|  | 			err: InvalidAnnotationError{ | ||||||
|  | 				Annotation: "example", | ||||||
|  | 				Cause:      errors.New("underlying error"), | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `invalid "example" annotation: underlying error`, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			assert.Equal(t, tc.expectedString, tc.err.Error()) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | @ -7,7 +7,6 @@ | ||||||
| package graph | package graph | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
| 	"fmt" |  | ||||||
| 	"sort" | 	"sort" | ||||||
| 
 | 
 | ||||||
| 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
|  | @ -133,10 +132,17 @@ func addApplyTimeMutationEdges(g *Graph, objs object.UnstructuredSet, ids object | ||||||
| 			// Require dependencies to be in the same resource group.
 | 			// Require dependencies to be in the same resource group.
 | ||||||
| 			// Waiting for external dependencies isn't implemented (yet).
 | 			// Waiting for external dependencies isn't implemented (yet).
 | ||||||
| 			if !ids.Contains(dep) { | 			if !ids.Contains(dep) { | ||||||
| 				err := fmt.Errorf("invalid %q annotation: dependency not in object set: %s", | 				err := object.InvalidAnnotationError{ | ||||||
| 					mutation.Annotation, sub.SourceRef) | 					Annotation: mutation.Annotation, | ||||||
|  | 					Cause: ExternalDependencyError{ | ||||||
|  | 						Edge: Edge{ | ||||||
|  | 							From: id, | ||||||
|  | 							To:   dep, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				} | ||||||
| 				objErrors = append(objErrors, err) | 				objErrors = append(objErrors, err) | ||||||
| 				klog.V(3).Infof("failed to add edges from: %s: %v", id, err) | 				klog.V(3).Infof("failed to add edges: %v", err) | ||||||
| 				continue | 				continue | ||||||
| 			} | 			} | ||||||
| 			klog.V(3).Infof("adding edge from: %s, to: %s", id, dep) | 			klog.V(3).Infof("adding edge from: %s, to: %s", id, dep) | ||||||
|  | @ -175,10 +181,15 @@ func addDependsOnEdges(g *Graph, objs object.UnstructuredSet, ids object.ObjMeta | ||||||
| 			// Duplicate dependencies in the same annotation are not allowed.
 | 			// Duplicate dependencies in the same annotation are not allowed.
 | ||||||
| 			// Having duplicates won't break the graph, but skip it anyway.
 | 			// Having duplicates won't break the graph, but skip it anyway.
 | ||||||
| 			if _, found := seen[dep]; found { | 			if _, found := seen[dep]; found { | ||||||
| 				// Won't error - already passed validation
 | 				err := object.InvalidAnnotationError{ | ||||||
| 				depStr, _ := dependson.FormatObjMetadata(dep) | 					Annotation: dependson.Annotation, | ||||||
| 				err := fmt.Errorf("invalid %q annotation: duplicate reference: %s", | 					Cause: DuplicateDependencyError{ | ||||||
| 					dependson.Annotation, depStr) | 						Edge: Edge{ | ||||||
|  | 							From: id, | ||||||
|  | 							To:   dep, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				} | ||||||
| 				objErrors = append(objErrors, err) | 				objErrors = append(objErrors, err) | ||||||
| 				klog.V(3).Infof("failed to add edges from: %s: %v", id, err) | 				klog.V(3).Infof("failed to add edges from: %s: %v", id, err) | ||||||
| 				continue | 				continue | ||||||
|  | @ -188,10 +199,17 @@ func addDependsOnEdges(g *Graph, objs object.UnstructuredSet, ids object.ObjMeta | ||||||
| 			// Require dependencies to be in the same resource group.
 | 			// Require dependencies to be in the same resource group.
 | ||||||
| 			// Waiting for external dependencies isn't implemented (yet).
 | 			// Waiting for external dependencies isn't implemented (yet).
 | ||||||
| 			if !ids.Contains(dep) { | 			if !ids.Contains(dep) { | ||||||
| 				err := fmt.Errorf("invalid %q annotation: dependency not in object set: %s", | 				err := object.InvalidAnnotationError{ | ||||||
| 					dependson.Annotation, mutation.ResourceReferenceFromObjMetadata(dep)) | 					Annotation: dependson.Annotation, | ||||||
|  | 					Cause: ExternalDependencyError{ | ||||||
|  | 						Edge: Edge{ | ||||||
|  | 							From: id, | ||||||
|  | 							To:   dep, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				} | ||||||
| 				objErrors = append(objErrors, err) | 				objErrors = append(objErrors, err) | ||||||
| 				klog.V(3).Infof("failed to add edges from: %s: %v", id, err) | 				klog.V(3).Infof("failed to add edges: %v", err) | ||||||
| 				continue | 				continue | ||||||
| 			} | 			} | ||||||
| 			klog.V(3).Infof("adding edge from: %s, to: %s", id, dep) | 			klog.V(3).Infof("adding edge from: %s, to: %s", id, dep) | ||||||
|  |  | ||||||
|  | @ -521,10 +521,12 @@ func TestApplyTimeMutationEdges(t *testing.T) { | ||||||
| 			}, | 			}, | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				errors.New("failed to parse apply-time-mutation annotation: "+ | 				object.InvalidAnnotationError{ | ||||||
| 					"error unmarshaling JSON: "+ | 					Annotation: mutation.Annotation, | ||||||
| 					"while decoding JSON: json: "+ | 					Cause: errors.New("error unmarshaling JSON: " + | ||||||
| 					"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"), | 						"while decoding JSON: json: " + | ||||||
|  | 						"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"), | ||||||
|  | 				}, | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
| 						Group: "apps", | 						Group: "apps", | ||||||
|  | @ -549,9 +551,15 @@ func TestApplyTimeMutationEdges(t *testing.T) { | ||||||
| 			}, | 			}, | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				errors.New(`invalid "config.kubernetes.io/apply-time-mutation" annotation: `+ | 				object.InvalidAnnotationError{ | ||||||
| 					"dependency not in object set: "+ | 					Annotation: mutation.Annotation, | ||||||
| 					"apps/namespaces/test-namespace/Deployment/foo"), | 					Cause: ExternalDependencyError{ | ||||||
|  | 						Edge: Edge{ | ||||||
|  | 							From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 							To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
| 						Group: "", | 						Group: "", | ||||||
|  | @ -590,10 +598,12 @@ func TestApplyTimeMutationEdges(t *testing.T) { | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: multierror.New( | 			expectedError: multierror.New( | ||||||
| 				validation.NewError( | 				validation.NewError( | ||||||
| 					errors.New("failed to parse apply-time-mutation annotation: "+ | 					object.InvalidAnnotationError{ | ||||||
| 						"error unmarshaling JSON: "+ | 						Annotation: mutation.Annotation, | ||||||
| 						"while decoding JSON: json: "+ | 						Cause: errors.New("error unmarshaling JSON: " + | ||||||
| 						"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"), | 							"while decoding JSON: json: " + | ||||||
|  | 							"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"), | ||||||
|  | 					}, | ||||||
| 					object.ObjMetadata{ | 					object.ObjMetadata{ | ||||||
| 						GroupKind: schema.GroupKind{ | 						GroupKind: schema.GroupKind{ | ||||||
| 							Group: "apps", | 							Group: "apps", | ||||||
|  | @ -604,9 +614,15 @@ func TestApplyTimeMutationEdges(t *testing.T) { | ||||||
| 					}, | 					}, | ||||||
| 				), | 				), | ||||||
| 				validation.NewError( | 				validation.NewError( | ||||||
| 					errors.New(`invalid "config.kubernetes.io/apply-time-mutation" annotation: `+ | 					object.InvalidAnnotationError{ | ||||||
| 						"dependency not in object set: "+ | 						Annotation: mutation.Annotation, | ||||||
| 						"/namespaces/test-namespace/Secret/secret"), | 						Cause: ExternalDependencyError{ | ||||||
|  | 							Edge: Edge{ | ||||||
|  | 								From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 								To:   testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
| 					object.ObjMetadata{ | 					object.ObjMetadata{ | ||||||
| 						GroupKind: schema.GroupKind{ | 						GroupKind: schema.GroupKind{ | ||||||
| 							Group: "", | 							Group: "", | ||||||
|  | @ -731,10 +747,11 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 			}, | 			}, | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				errors.New("failed to parse depends-on annotation: "+ | 				object.InvalidAnnotationError{ | ||||||
| 					"failed to parse object metadata: "+ | 					Annotation: dependson.Annotation, | ||||||
| 					"expected 3 or 5 fields, found 1: "+ | 					Cause: errors.New("failed to parse object reference (index: 0): " + | ||||||
| 					`"invalid-obj-ref"`), | 						`expected 3 or 5 fields, found 1: "invalid-obj-ref"`), | ||||||
|  | 				}, | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
| 						Group: "apps", | 						Group: "apps", | ||||||
|  | @ -762,9 +779,15 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 				}, | 				}, | ||||||
| 			}, | 			}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+ | 				object.InvalidAnnotationError{ | ||||||
| 					"duplicate reference: "+ | 					Annotation: dependson.Annotation, | ||||||
| 					"apps/namespaces/test-namespace/Deployment/foo"), | 					Cause: DuplicateDependencyError{ | ||||||
|  | 						Edge: Edge{ | ||||||
|  | 							From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 							To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
| 						Group: "", | 						Group: "", | ||||||
|  | @ -775,7 +798,7 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 				}, | 				}, | ||||||
| 			), | 			), | ||||||
| 		}, | 		}, | ||||||
| 		"error: dependency not in object set": { | 		"error: external dependency": { | ||||||
| 			objs: []*unstructured.Unstructured{ | 			objs: []*unstructured.Unstructured{ | ||||||
| 				testutil.Unstructured(t, resources["pod"], | 				testutil.Unstructured(t, resources["pod"], | ||||||
| 					testutil.AddDependsOn(t, | 					testutil.AddDependsOn(t, | ||||||
|  | @ -785,9 +808,15 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 			}, | 			}, | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+ | 				object.InvalidAnnotationError{ | ||||||
| 					"dependency not in object set: "+ | 					Annotation: dependson.Annotation, | ||||||
| 					"apps/namespaces/test-namespace/Deployment/foo"), | 					Cause: ExternalDependencyError{ | ||||||
|  | 						Edge: Edge{ | ||||||
|  | 							From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 							To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
| 						Group: "", | 						Group: "", | ||||||
|  | @ -822,10 +851,11 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: multierror.New( | 			expectedError: multierror.New( | ||||||
| 				validation.NewError( | 				validation.NewError( | ||||||
| 					errors.New("failed to parse depends-on annotation: "+ | 					object.InvalidAnnotationError{ | ||||||
| 						"failed to parse object metadata: "+ | 						Annotation: dependson.Annotation, | ||||||
| 						"expected 3 or 5 fields, found 1: "+ | 						Cause: errors.New("failed to parse object reference (index: 0): " + | ||||||
| 						`"invalid-obj-ref"`), | 							`expected 3 or 5 fields, found 1: "invalid-obj-ref"`), | ||||||
|  | 					}, | ||||||
| 					object.ObjMetadata{ | 					object.ObjMetadata{ | ||||||
| 						GroupKind: schema.GroupKind{ | 						GroupKind: schema.GroupKind{ | ||||||
| 							Group: "apps", | 							Group: "apps", | ||||||
|  | @ -836,9 +866,15 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 					}, | 					}, | ||||||
| 				), | 				), | ||||||
| 				validation.NewError( | 				validation.NewError( | ||||||
| 					errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+ | 					object.InvalidAnnotationError{ | ||||||
| 						"dependency not in object set: "+ | 						Annotation: dependson.Annotation, | ||||||
| 						"/namespaces/test-namespace/Secret/secret"), | 						Cause: ExternalDependencyError{ | ||||||
|  | 							Edge: Edge{ | ||||||
|  | 								From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 								To:   testutil.ToIdentifier(t, resources["secret"]), | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
| 					object.ObjMetadata{ | 					object.ObjMetadata{ | ||||||
| 						GroupKind: schema.GroupKind{ | 						GroupKind: schema.GroupKind{ | ||||||
| 							Group: "", | 							Group: "", | ||||||
|  | @ -862,12 +898,24 @@ func TestAddDependsOnEdges(t *testing.T) { | ||||||
| 			expected: []Edge{}, | 			expected: []Edge{}, | ||||||
| 			expectedError: validation.NewError( | 			expectedError: validation.NewError( | ||||||
| 				multierror.New( | 				multierror.New( | ||||||
| 					errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+ | 					object.InvalidAnnotationError{ | ||||||
| 						"dependency not in object set: "+ | 						Annotation: dependson.Annotation, | ||||||
| 						"apps/namespaces/test-namespace/Deployment/foo"), | 						Cause: ExternalDependencyError{ | ||||||
| 					errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+ | 							Edge: Edge{ | ||||||
| 						"duplicate reference: "+ | 								From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
| 						"apps/namespaces/test-namespace/Deployment/foo"), | 								To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.InvalidAnnotationError{ | ||||||
|  | 						Annotation: dependson.Annotation, | ||||||
|  | 						Cause: DuplicateDependencyError{ | ||||||
|  | 							Edge: Edge{ | ||||||
|  | 								From: testutil.ToIdentifier(t, resources["pod"]), | ||||||
|  | 								To:   testutil.ToIdentifier(t, resources["deployment"]), | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
| 				), | 				), | ||||||
| 				object.ObjMetadata{ | 				object.ObjMetadata{ | ||||||
| 					GroupKind: schema.GroupKind{ | 					GroupKind: schema.GroupKind{ | ||||||
|  |  | ||||||
|  | @ -0,0 +1,44 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package graph | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"sort" | ||||||
|  | 
 | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // Edge encapsulates a pair of vertices describing a
 | ||||||
|  | // directed edge.
 | ||||||
|  | type Edge struct { | ||||||
|  | 	From object.ObjMetadata | ||||||
|  | 	To   object.ObjMetadata | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // SortableEdges sorts a list of edges alphanumerically by From and then To.
 | ||||||
|  | type SortableEdges []Edge | ||||||
|  | 
 | ||||||
|  | var _ sort.Interface = SortableEdges{} | ||||||
|  | 
 | ||||||
|  | func (a SortableEdges) Len() int      { return len(a) } | ||||||
|  | func (a SortableEdges) Swap(i, j int) { a[i], a[j] = a[j], a[i] } | ||||||
|  | func (a SortableEdges) Less(i, j int) bool { | ||||||
|  | 	if a[i].From != a[j].From { | ||||||
|  | 		return metaIsLessThan(a[i].From, a[j].From) | ||||||
|  | 	} | ||||||
|  | 	return metaIsLessThan(a[i].To, a[j].To) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func metaIsLessThan(i, j object.ObjMetadata) bool { | ||||||
|  | 	if i.GroupKind.Group != j.GroupKind.Group { | ||||||
|  | 		return i.GroupKind.Group < j.GroupKind.Group | ||||||
|  | 	} | ||||||
|  | 	if i.GroupKind.Kind != j.GroupKind.Kind { | ||||||
|  | 		return i.GroupKind.Kind < j.GroupKind.Kind | ||||||
|  | 	} | ||||||
|  | 	if i.Namespace != j.Namespace { | ||||||
|  | 		return i.Namespace < j.Namespace | ||||||
|  | 	} | ||||||
|  | 	return i.Name < j.Name | ||||||
|  | } | ||||||
|  | @ -0,0 +1,212 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package graph | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"sort" | ||||||
|  | 	"testing" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestEdgeSort(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		edges    []Edge | ||||||
|  | 		expected []Edge | ||||||
|  | 	}{ | ||||||
|  | 		"one edge": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two edges no change": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two edges same from": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two edges": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two edges by name": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"three edges": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj4"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj4"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two edges cycle": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"three edges cycle": { | ||||||
|  | 			edges: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expected: []Edge{ | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj2"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 				}, | ||||||
|  | 				{ | ||||||
|  | 					From: object.ObjMetadata{Name: "obj3"}, | ||||||
|  | 					To:   object.ObjMetadata{Name: "obj1"}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			sort.Sort(SortableEdges(tc.edges)) | ||||||
|  | 			assert.Equal(t, tc.expected, tc.edges) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | @ -0,0 +1,53 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package graph | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"bytes" | ||||||
|  | 	"fmt" | ||||||
|  | 
 | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/multierror" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/mutation" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // ExternalDependencyError represents an invalid graph edge caused by an
 | ||||||
|  | // object that is not in the object set.
 | ||||||
|  | type ExternalDependencyError struct { | ||||||
|  | 	Edge Edge | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (ede ExternalDependencyError) Error() string { | ||||||
|  | 	return fmt.Sprintf("external dependency: %s -> %s", | ||||||
|  | 		mutation.ResourceReferenceFromObjMetadata(ede.Edge.From), | ||||||
|  | 		mutation.ResourceReferenceFromObjMetadata(ede.Edge.To)) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // CyclicDependencyError represents a cycle in the graph, making topological
 | ||||||
|  | // sort impossible.
 | ||||||
|  | type CyclicDependencyError struct { | ||||||
|  | 	Edges []Edge | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (cde CyclicDependencyError) Error() string { | ||||||
|  | 	var errorBuf bytes.Buffer | ||||||
|  | 	errorBuf.WriteString("cyclic dependency:") | ||||||
|  | 	for _, edge := range cde.Edges { | ||||||
|  | 		errorBuf.WriteString(fmt.Sprintf("\n%s%s -> %s", multierror.Prefix, | ||||||
|  | 			mutation.ResourceReferenceFromObjMetadata(edge.From), | ||||||
|  | 			mutation.ResourceReferenceFromObjMetadata(edge.To))) | ||||||
|  | 	} | ||||||
|  | 	return errorBuf.String() | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // DuplicateDependencyError represents an invalid depends-on annotation with
 | ||||||
|  | // duplicate references.
 | ||||||
|  | type DuplicateDependencyError struct { | ||||||
|  | 	Edge Edge | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (dde DuplicateDependencyError) Error() string { | ||||||
|  | 	return fmt.Sprintf("duplicate dependency: %s -> %s", | ||||||
|  | 		mutation.ResourceReferenceFromObjMetadata(dde.Edge.From), | ||||||
|  | 		mutation.ResourceReferenceFromObjMetadata(dde.Edge.To)) | ||||||
|  | } | ||||||
|  | @ -0,0 +1,134 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package graph | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"testing" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | var ( | ||||||
|  | 	on1 = object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Group: "test", Kind: "foo"}} | ||||||
|  | 	on2 = object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Group: "test", Kind: "foo"}} | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestExternalDependencyErrorString(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		err            ExternalDependencyError | ||||||
|  | 		expectedString string | ||||||
|  | 	}{ | ||||||
|  | 		"cluster-scoped": { | ||||||
|  | 			err: ExternalDependencyError{ | ||||||
|  | 				Edge: Edge{ | ||||||
|  | 					From: o1, | ||||||
|  | 					To:   o2, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `external dependency: test/foo/obj1 -> test/foo/obj2`, | ||||||
|  | 		}, | ||||||
|  | 		"namespace-scoped": { | ||||||
|  | 			err: ExternalDependencyError{ | ||||||
|  | 				Edge: Edge{ | ||||||
|  | 					From: on1, | ||||||
|  | 					To:   on2, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `external dependency: test/namespaces/ns1/foo/obj1 -> test/namespaces/ns1/foo/obj2`, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			assert.Equal(t, tc.expectedString, tc.err.Error()) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestCyclicDependencyErrorString(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		err            CyclicDependencyError | ||||||
|  | 		expectedString string | ||||||
|  | 	}{ | ||||||
|  | 		"two object cycle": { | ||||||
|  | 			err: CyclicDependencyError{ | ||||||
|  | 				Edges: []Edge{ | ||||||
|  | 					{ | ||||||
|  | 						From: o1, | ||||||
|  | 						To:   o2, | ||||||
|  | 					}, | ||||||
|  | 					{ | ||||||
|  | 						From: o2, | ||||||
|  | 						To:   o1, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `cyclic dependency: | ||||||
|  | - test/foo/obj1 -> test/foo/obj2 | ||||||
|  | - test/foo/obj2 -> test/foo/obj1`, | ||||||
|  | 		}, | ||||||
|  | 		"three object cycle": { | ||||||
|  | 			err: CyclicDependencyError{ | ||||||
|  | 				Edges: []Edge{ | ||||||
|  | 					{ | ||||||
|  | 						From: o1, | ||||||
|  | 						To:   o2, | ||||||
|  | 					}, | ||||||
|  | 					{ | ||||||
|  | 						From: o2, | ||||||
|  | 						To:   o3, | ||||||
|  | 					}, | ||||||
|  | 					{ | ||||||
|  | 						From: o3, | ||||||
|  | 						To:   o1, | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `cyclic dependency: | ||||||
|  | - test/foo/obj1 -> test/foo/obj2 | ||||||
|  | - test/foo/obj2 -> test/foo/obj3 | ||||||
|  | - test/foo/obj3 -> test/foo/obj1`, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			assert.Equal(t, tc.expectedString, tc.err.Error()) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestDuplicateDependencyErrorString(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		err            DuplicateDependencyError | ||||||
|  | 		expectedString string | ||||||
|  | 	}{ | ||||||
|  | 		"cluster-scoped": { | ||||||
|  | 			err: DuplicateDependencyError{ | ||||||
|  | 				Edge: Edge{ | ||||||
|  | 					From: o1, | ||||||
|  | 					To:   o2, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `duplicate dependency: test/foo/obj1 -> test/foo/obj2`, | ||||||
|  | 		}, | ||||||
|  | 		"namespace-scoped": { | ||||||
|  | 			err: DuplicateDependencyError{ | ||||||
|  | 				Edge: Edge{ | ||||||
|  | 					From: on1, | ||||||
|  | 					To:   on2, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			expectedString: `duplicate dependency: test/namespaces/ns1/foo/obj1 -> test/namespaces/ns1/foo/obj2`, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			assert.Equal(t, tc.expectedString, tc.err.Error()) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | @ -7,11 +7,8 @@ | ||||||
| package graph | package graph | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
| 	"bytes" |  | ||||||
| 	"fmt" |  | ||||||
| 	"sort" | 	"sort" | ||||||
| 
 | 
 | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/multierror" |  | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object/validation" | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/ordering" | 	"sigs.k8s.io/cli-utils/pkg/ordering" | ||||||
|  | @ -25,13 +22,6 @@ type Graph struct { | ||||||
| 	edges map[object.ObjMetadata]object.ObjMetadataSet | 	edges map[object.ObjMetadata]object.ObjMetadataSet | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Edge encapsulates a pair of vertices describing a
 |  | ||||||
| // directed edge.
 |  | ||||||
| type Edge struct { |  | ||||||
| 	From object.ObjMetadata |  | ||||||
| 	To   object.ObjMetadata |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // New returns a pointer to an empty Graph data structure.
 | // New returns a pointer to an empty Graph data structure.
 | ||||||
| func New() *Graph { | func New() *Graph { | ||||||
| 	g := &Graph{} | 	g := &Graph{} | ||||||
|  | @ -150,47 +140,3 @@ func (g *Graph) Sort() ([]object.ObjMetadataSet, error) { | ||||||
| 	} | 	} | ||||||
| 	return sorted, nil | 	return sorted, nil | ||||||
| } | } | ||||||
| 
 |  | ||||||
| // CyclicDependencyError when directed acyclic graph contains a cycle.
 |  | ||||||
| // The cycle makes it impossible to topological sort.
 |  | ||||||
| type CyclicDependencyError struct { |  | ||||||
| 	Edges []Edge |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (cde CyclicDependencyError) Error() string { |  | ||||||
| 	var errorBuf bytes.Buffer |  | ||||||
| 	errorBuf.WriteString("cyclic dependency:\n") |  | ||||||
| 	for _, edge := range cde.Edges { |  | ||||||
| 		from := fmt.Sprintf("%s/%s", edge.From.Namespace, edge.From.Name) |  | ||||||
| 		to := fmt.Sprintf("%s/%s", edge.To.Namespace, edge.To.Name) |  | ||||||
| 		errorBuf.WriteString(fmt.Sprintf("%s%s -> %s\n", multierror.Prefix, from, to)) |  | ||||||
| 	} |  | ||||||
| 	return errorBuf.String() |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // SortableEdges sorts a list of edges alphanumerically by From and then To.
 |  | ||||||
| type SortableEdges []Edge |  | ||||||
| 
 |  | ||||||
| var _ sort.Interface = SortableEdges{} |  | ||||||
| 
 |  | ||||||
| func (a SortableEdges) Len() int      { return len(a) } |  | ||||||
| func (a SortableEdges) Swap(i, j int) { a[i], a[j] = a[j], a[i] } |  | ||||||
| func (a SortableEdges) Less(i, j int) bool { |  | ||||||
| 	if a[i].From != a[j].From { |  | ||||||
| 		return metaIsLessThan(a[i].From, a[j].From) |  | ||||||
| 	} |  | ||||||
| 	return metaIsLessThan(a[i].To, a[j].To) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func metaIsLessThan(i, j object.ObjMetadata) bool { |  | ||||||
| 	if i.GroupKind.Group != j.GroupKind.Group { |  | ||||||
| 		return i.GroupKind.Group < j.GroupKind.Group |  | ||||||
| 	} |  | ||||||
| 	if i.GroupKind.Kind != j.GroupKind.Kind { |  | ||||||
| 		return i.GroupKind.Kind < j.GroupKind.Kind |  | ||||||
| 	} |  | ||||||
| 	if i.Namespace != j.Namespace { |  | ||||||
| 		return i.Namespace < j.Namespace |  | ||||||
| 	} |  | ||||||
| 	return i.Name < j.Name |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -6,7 +6,6 @@ | ||||||
| package graph | package graph | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
| 	"sort" |  | ||||||
| 	"testing" | 	"testing" | ||||||
| 
 | 
 | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
|  | @ -142,257 +141,3 @@ func TestObjectGraphSort(t *testing.T) { | ||||||
| 		}) | 		}) | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 |  | ||||||
| func TestEdgeSort(t *testing.T) { |  | ||||||
| 	testCases := map[string]struct { |  | ||||||
| 		edges    []Edge |  | ||||||
| 		expected []Edge |  | ||||||
| 	}{ |  | ||||||
| 		"one edge": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"two edges no change": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"two edges same from": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"two edges": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"two edges by name": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"three edges": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj4"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj4"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"two edges cycle": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj1", Namespace: "ns1", GroupKind: schema.GroupKind{Kind: "Pod"}}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		"three edges cycle": { |  | ||||||
| 			edges: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expected: []Edge{ |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj2"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 				}, |  | ||||||
| 				{ |  | ||||||
| 					From: object.ObjMetadata{Name: "obj3"}, |  | ||||||
| 					To:   object.ObjMetadata{Name: "obj1"}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	for tn, tc := range testCases { |  | ||||||
| 		t.Run(tn, func(t *testing.T) { |  | ||||||
| 			sort.Sort(SortableEdges(tc.edges)) |  | ||||||
| 			assert.Equal(t, tc.expected, tc.edges) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestCyclicDependencyErrorString(t *testing.T) { |  | ||||||
| 	testCases := map[string]struct { |  | ||||||
| 		cycle          CyclicDependencyError |  | ||||||
| 		expectedString string |  | ||||||
| 	}{ |  | ||||||
| 		"two object cycle": { |  | ||||||
| 			cycle: CyclicDependencyError{ |  | ||||||
| 				Edges: []Edge{ |  | ||||||
| 					{ |  | ||||||
| 						From: o1, |  | ||||||
| 						To:   o2, |  | ||||||
| 					}, |  | ||||||
| 					{ |  | ||||||
| 						From: o2, |  | ||||||
| 						To:   o1, |  | ||||||
| 					}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expectedString: `cyclic dependency: |  | ||||||
| - /obj1 -> /obj2 |  | ||||||
| - /obj2 -> /obj1 |  | ||||||
| `, |  | ||||||
| 		}, |  | ||||||
| 		"three object cycle": { |  | ||||||
| 			cycle: CyclicDependencyError{ |  | ||||||
| 				Edges: []Edge{ |  | ||||||
| 					{ |  | ||||||
| 						From: o1, |  | ||||||
| 						To:   o2, |  | ||||||
| 					}, |  | ||||||
| 					{ |  | ||||||
| 						From: o2, |  | ||||||
| 						To:   o3, |  | ||||||
| 					}, |  | ||||||
| 					{ |  | ||||||
| 						From: o3, |  | ||||||
| 						To:   o1, |  | ||||||
| 					}, |  | ||||||
| 				}, |  | ||||||
| 			}, |  | ||||||
| 			expectedString: `cyclic dependency: |  | ||||||
| - /obj1 -> /obj2 |  | ||||||
| - /obj2 -> /obj3 |  | ||||||
| - /obj3 -> /obj1 |  | ||||||
| `, |  | ||||||
| 		}, |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	for tn, tc := range testCases { |  | ||||||
| 		t.Run(tn, func(t *testing.T) { |  | ||||||
| 			assert.Equal(t, tc.expectedString, tc.cycle.Error()) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -9,6 +9,7 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
| 	"k8s.io/klog/v2" | 	"k8s.io/klog/v2" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
| 	"sigs.k8s.io/yaml" | 	"sigs.k8s.io/yaml" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | @ -41,7 +42,10 @@ func ReadAnnotation(obj *unstructured.Unstructured) (ApplyTimeMutation, error) { | ||||||
| 
 | 
 | ||||||
| 	err := yaml.Unmarshal([]byte(mutationYaml), &mutation) | 	err := yaml.Unmarshal([]byte(mutationYaml), &mutation) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return nil, fmt.Errorf("failed to parse apply-time-mutation annotation: %v", err) | 		return mutation, object.InvalidAnnotationError{ | ||||||
|  | 			Annotation: Annotation, | ||||||
|  | 			Cause:      err, | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return mutation, nil | 	return mutation, nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | @ -89,8 +89,18 @@ func (setA ObjMetadataSet) Intersection(setB ObjMetadataSet) ObjMetadataSet { | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| 	intersection := make(ObjMetadataSet, 0, len(mapI)) | 	intersection := make(ObjMetadataSet, 0, len(mapI)) | ||||||
| 	for o := range mapI { | 	// Iterate over setA & setB to retain input order and have stable output
 | ||||||
| 		intersection = append(intersection, o) | 	for _, id := range setA { | ||||||
|  | 		if _, ok := mapI[id]; ok { | ||||||
|  | 			intersection = append(intersection, id) | ||||||
|  | 			delete(mapI, id) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	for _, id := range setB { | ||||||
|  | 		if _, ok := mapI[id]; ok { | ||||||
|  | 			intersection = append(intersection, id) | ||||||
|  | 			delete(mapI, id) | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return intersection | 	return intersection | ||||||
| } | } | ||||||
|  | @ -105,8 +115,18 @@ func (setA ObjMetadataSet) Union(setB ObjMetadataSet) ObjMetadataSet { | ||||||
| 		m[b] = struct{}{} | 		m[b] = struct{}{} | ||||||
| 	} | 	} | ||||||
| 	union := make(ObjMetadataSet, 0, len(m)) | 	union := make(ObjMetadataSet, 0, len(m)) | ||||||
| 	for u := range m { | 	// Iterate over setA & setB to retain input order and have stable output
 | ||||||
| 		union = append(union, u) | 	for _, id := range setA { | ||||||
|  | 		if _, ok := m[id]; ok { | ||||||
|  | 			union = append(union, id) | ||||||
|  | 			delete(m, id) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	for _, id := range setB { | ||||||
|  | 		if _, ok := m[id]; ok { | ||||||
|  | 			union = append(union, id) | ||||||
|  | 			delete(m, id) | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return union | 	return union | ||||||
| } | } | ||||||
|  | @ -124,12 +144,22 @@ func (setA ObjMetadataSet) Diff(setB ObjMetadataSet) ObjMetadataSet { | ||||||
| 	} | 	} | ||||||
| 	// Create/return slice from the map of remaining items
 | 	// Create/return slice from the map of remaining items
 | ||||||
| 	diff := make(ObjMetadataSet, 0, len(m)) | 	diff := make(ObjMetadataSet, 0, len(m)) | ||||||
| 	for r := range m { | 	// Iterate over setA to retain input order and have stable output
 | ||||||
| 		diff = append(diff, r) | 	for _, id := range setA { | ||||||
|  | 		if _, ok := m[id]; ok { | ||||||
|  | 			diff = append(diff, id) | ||||||
|  | 			delete(m, id) | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return diff | 	return diff | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | // Unique returns the set with duplicates removed.
 | ||||||
|  | // Order may or may not remain consistent.
 | ||||||
|  | func (setA ObjMetadataSet) Unique() ObjMetadataSet { | ||||||
|  | 	return ObjMetadataSetFromMap(setA.ToMap()) | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // Hash the objects in the set by serializing, sorting, concatonating, and
 | // Hash the objects in the set by serializing, sorting, concatonating, and
 | ||||||
| // hashing the result with the 32-bit FNV-1a algorithm.
 | // hashing the result with the 32-bit FNV-1a algorithm.
 | ||||||
| func (setA ObjMetadataSet) Hash() string { | func (setA ObjMetadataSet) Hash() string { | ||||||
|  |  | ||||||
|  | @ -0,0 +1,63 @@ | ||||||
|  | // Copyright 2021 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package validation | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"errors" | ||||||
|  | 
 | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/multierror" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // Collector simplifies collecting validation errors from multiple sources and
 | ||||||
|  | // extracting the IDs of the invalid objects.
 | ||||||
|  | type Collector struct { | ||||||
|  | 	Errors     []error | ||||||
|  | 	InvalidIds object.ObjMetadataSet | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Collect unwraps MultiErrors, adds them to Errors, extracts invalid object
 | ||||||
|  | // IDs from validation.Error, and adds them to InvalidIds.
 | ||||||
|  | func (c *Collector) Collect(err error) { | ||||||
|  | 	errs := multierror.Unwrap(err) | ||||||
|  | 	c.InvalidIds = c.InvalidIds.Union(extractInvalidIds(errs)) | ||||||
|  | 	c.Errors = append(c.Errors, errs...) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // ToError returns the list of errors as a single error.
 | ||||||
|  | func (c *Collector) ToError() error { | ||||||
|  | 	return multierror.Wrap(c.Errors...) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // FilterInvalidObjects returns a set of objects that does not contain any
 | ||||||
|  | // invalid objects, based on the collected InvalidIds.
 | ||||||
|  | func (c *Collector) FilterInvalidObjects(objs object.UnstructuredSet) object.UnstructuredSet { | ||||||
|  | 	var diff object.UnstructuredSet | ||||||
|  | 	for _, obj := range objs { | ||||||
|  | 		if !c.InvalidIds.Contains(object.UnstructuredToObjMetadata(obj)) { | ||||||
|  | 			diff = append(diff, obj) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return diff | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // FilterInvalidIds returns a set of object ID that does not contain any
 | ||||||
|  | // invalid IDs, based on the collected InvalidIds.
 | ||||||
|  | func (c *Collector) FilterInvalidIds(ids object.ObjMetadataSet) object.ObjMetadataSet { | ||||||
|  | 	return ids.Diff(c.InvalidIds) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // extractInvalidIds extracts invalid object IDs from a list of possible
 | ||||||
|  | // validation.Error.
 | ||||||
|  | func extractInvalidIds(errs []error) object.ObjMetadataSet { | ||||||
|  | 	var invalidIds object.ObjMetadataSet | ||||||
|  | 	for _, err := range errs { | ||||||
|  | 		// unwrap recursively looking for a validation.Error
 | ||||||
|  | 		var vErr *Error | ||||||
|  | 		if errors.As(err, &vErr) { | ||||||
|  | 			invalidIds = invalidIds.Union(vErr.Identifiers()) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return invalidIds | ||||||
|  | } | ||||||
|  | @ -0,0 +1,16 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package validation | ||||||
|  | 
 | ||||||
|  | //go:generate stringer -type=Policy
 | ||||||
|  | type Policy int | ||||||
|  | 
 | ||||||
|  | const ( | ||||||
|  | 	// ExitEarly policy errors and exits if any objects are invalid, before
 | ||||||
|  | 	// apply/delete of any objects.
 | ||||||
|  | 	ExitEarly Policy = iota | ||||||
|  | 
 | ||||||
|  | 	// SkipInvalid policy skips the apply/delete of invalid objects.
 | ||||||
|  | 	SkipInvalid | ||||||
|  | ) | ||||||
|  | @ -0,0 +1,24 @@ | ||||||
|  | // Code generated by "stringer -type=Policy"; DO NOT EDIT.
 | ||||||
|  | 
 | ||||||
|  | package validation | ||||||
|  | 
 | ||||||
|  | import "strconv" | ||||||
|  | 
 | ||||||
|  | func _() { | ||||||
|  | 	// An "invalid array index" compiler error signifies that the constant values have changed.
 | ||||||
|  | 	// Re-run the stringer command to generate them again.
 | ||||||
|  | 	var x [1]struct{} | ||||||
|  | 	_ = x[ExitEarly-0] | ||||||
|  | 	_ = x[SkipInvalid-1] | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | const _Policy_name = "ExitEarlySkipInvalid" | ||||||
|  | 
 | ||||||
|  | var _Policy_index = [...]uint8{0, 9, 20} | ||||||
|  | 
 | ||||||
|  | func (i Policy) String() string { | ||||||
|  | 	if i < 0 || i >= Policy(len(_Policy_index)-1) { | ||||||
|  | 		return "Policy(" + strconv.FormatInt(int64(i), 10) + ")" | ||||||
|  | 	} | ||||||
|  | 	return _Policy_name[_Policy_index[i]:_Policy_index[i+1]] | ||||||
|  | } | ||||||
|  | @ -15,14 +15,14 @@ import ( | ||||||
| // to being used by the Apply functionality. This imposes some constraint not
 | // to being used by the Apply functionality. This imposes some constraint not
 | ||||||
| // always required, such as namespaced resources must have the namespace set.
 | // always required, such as namespaced resources must have the namespace set.
 | ||||||
| type Validator struct { | type Validator struct { | ||||||
| 	Mapper meta.RESTMapper | 	Mapper    meta.RESTMapper | ||||||
|  | 	Collector *Collector | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // Validate validates the provided resources. A RESTMapper will be used
 | // Validate validates the provided resources. A RESTMapper will be used
 | ||||||
| // to fetch type information from the live cluster.
 | // to fetch type information from the live cluster.
 | ||||||
| func (v *Validator) Validate(objs []*unstructured.Unstructured) error { | func (v *Validator) Validate(objs []*unstructured.Unstructured) { | ||||||
| 	crds := findCRDs(objs) | 	crds := findCRDs(objs) | ||||||
| 	var errs []error |  | ||||||
| 	for _, obj := range objs { | 	for _, obj := range objs { | ||||||
| 		var objErrors []error | 		var objErrors []error | ||||||
| 		if err := v.validateKind(obj); err != nil { | 		if err := v.validateKind(obj); err != nil { | ||||||
|  | @ -35,14 +35,13 @@ func (v *Validator) Validate(objs []*unstructured.Unstructured) error { | ||||||
| 			objErrors = append(objErrors, err) | 			objErrors = append(objErrors, err) | ||||||
| 		} | 		} | ||||||
| 		if len(objErrors) > 0 { | 		if len(objErrors) > 0 { | ||||||
| 			errs = append(errs, NewError(multierror.Wrap(objErrors...), | 			// one error per object
 | ||||||
| 				object.UnstructuredToObjMetadata(obj))) | 			v.Collector.Collect(NewError( | ||||||
|  | 				multierror.Wrap(objErrors...), | ||||||
|  | 				object.UnstructuredToObjMetadata(obj), | ||||||
|  | 			)) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| 	if len(errs) > 0 { |  | ||||||
| 		return multierror.Wrap(errs...) |  | ||||||
| 	} |  | ||||||
| 	return nil |  | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // findCRDs looks through the provided resources and returns a slice with
 | // findCRDs looks through the provided resources and returns a slice with
 | ||||||
|  |  | ||||||
|  | @ -260,10 +260,13 @@ metadata: | ||||||
| 				crdGV.WithResource("customresourcedefinition"), meta.RESTScopeRoot) | 				crdGV.WithResource("customresourcedefinition"), meta.RESTScopeRoot) | ||||||
| 			mapper = meta.MultiRESTMapper([]meta.RESTMapper{mapper, crdMapper}) | 			mapper = meta.MultiRESTMapper([]meta.RESTMapper{mapper, crdMapper}) | ||||||
| 
 | 
 | ||||||
|  | 			vCollector := &validation.Collector{} | ||||||
| 			validator := &validation.Validator{ | 			validator := &validation.Validator{ | ||||||
| 				Mapper: mapper, | 				Mapper:    mapper, | ||||||
|  | 				Collector: vCollector, | ||||||
| 			} | 			} | ||||||
| 			err = validator.Validate(tc.resources) | 			validator.Validate(tc.resources) | ||||||
|  | 			err = vCollector.ToError() | ||||||
| 			if tc.expectedError == nil { | 			if tc.expectedError == nil { | ||||||
| 				assert.NoError(t, err) | 				assert.NoError(t, err) | ||||||
| 				return | 				return | ||||||
|  |  | ||||||
|  | @ -12,6 +12,7 @@ import ( | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| type Formatter interface { | type Formatter interface { | ||||||
|  | 	FormatValidationEvent(ve event.ValidationEvent) error | ||||||
| 	FormatApplyEvent(ae event.ApplyEvent) error | 	FormatApplyEvent(ae event.ApplyEvent) error | ||||||
| 	FormatStatusEvent(se event.StatusEvent) error | 	FormatStatusEvent(se event.StatusEvent) error | ||||||
| 	FormatPruneEvent(pe event.PruneEvent) error | 	FormatPruneEvent(pe event.PruneEvent) error | ||||||
|  | @ -68,6 +69,10 @@ func (b *BaseListPrinter) Print(ch <-chan event.Event, previewStrategy common.Dr | ||||||
| 		case event.ErrorType: | 		case event.ErrorType: | ||||||
| 			_ = formatter.FormatErrorEvent(e.ErrorEvent) | 			_ = formatter.FormatErrorEvent(e.ErrorEvent) | ||||||
| 			return e.ErrorEvent.Err | 			return e.ErrorEvent.Err | ||||||
|  | 		case event.ValidationType: | ||||||
|  | 			if err := formatter.FormatValidationEvent(e.ValidationEvent); err != nil { | ||||||
|  | 				return err | ||||||
|  | 			} | ||||||
| 		case event.ApplyType: | 		case event.ApplyType: | ||||||
| 			if err := formatter.FormatApplyEvent(e.ApplyEvent); err != nil { | 			if err := formatter.FormatApplyEvent(e.ApplyEvent); err != nil { | ||||||
| 				return err | 				return err | ||||||
|  |  | ||||||
|  | @ -28,6 +28,7 @@ func newCountingFormatter() *countingFormatter { | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| type countingFormatter struct { | type countingFormatter struct { | ||||||
|  | 	validationEvent  []event.ValidationEvent | ||||||
| 	applyEvents      []event.ApplyEvent | 	applyEvents      []event.ApplyEvent | ||||||
| 	statusEvents     []event.StatusEvent | 	statusEvents     []event.StatusEvent | ||||||
| 	pruneEvents      []event.PruneEvent | 	pruneEvents      []event.PruneEvent | ||||||
|  | @ -37,6 +38,11 @@ type countingFormatter struct { | ||||||
| 	actionGroupEvent []event.ActionGroupEvent | 	actionGroupEvent []event.ActionGroupEvent | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func (c *countingFormatter) FormatValidationEvent(e event.ValidationEvent) error { | ||||||
|  | 	c.validationEvent = append(c.validationEvent, e) | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func (c *countingFormatter) FormatApplyEvent(e event.ApplyEvent) error { | func (c *countingFormatter) FormatApplyEvent(e event.ApplyEvent) error { | ||||||
| 	c.applyEvents = append(c.applyEvents, e) | 	c.applyEvents = append(c.applyEvents, e) | ||||||
| 	return nil | 	return nil | ||||||
|  |  | ||||||
|  | @ -12,6 +12,7 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/list" | 	"sigs.k8s.io/cli-utils/pkg/print/list" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/stats" | 	"sigs.k8s.io/cli-utils/pkg/print/stats" | ||||||
| ) | ) | ||||||
|  | @ -27,6 +28,36 @@ type formatter struct { | ||||||
| 	ioStreams genericclioptions.IOStreams | 	ioStreams genericclioptions.IOStreams | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func (ef *formatter) FormatValidationEvent(ve event.ValidationEvent) error { | ||||||
|  | 	// unwrap validation errors
 | ||||||
|  | 	err := ve.Error | ||||||
|  | 	if vErr, ok := err.(*validation.Error); ok { | ||||||
|  | 		err = vErr.Unwrap() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	switch { | ||||||
|  | 	case len(ve.Identifiers) == 0: | ||||||
|  | 		// no objects, invalid event
 | ||||||
|  | 		return fmt.Errorf("invalid validation event: no identifiers: %w", err) | ||||||
|  | 	case len(ve.Identifiers) == 1: | ||||||
|  | 		// only 1 object, unwrap for similarity with status event
 | ||||||
|  | 		id := ve.Identifiers[0] | ||||||
|  | 		ef.print("Invalid object (%s): %v", | ||||||
|  | 			resourceIDToString(id.GroupKind, id.Name), err.Error()) | ||||||
|  | 	default: | ||||||
|  | 		// more than 1 object, wrap list in brackets
 | ||||||
|  | 		var sb strings.Builder | ||||||
|  | 		id := ve.Identifiers[0] | ||||||
|  | 		_, _ = fmt.Fprintf(&sb, "Invalid objects (%s", resourceIDToString(id.GroupKind, id.Name)) | ||||||
|  | 		for _, id := range ve.Identifiers[1:] { | ||||||
|  | 			_, _ = fmt.Fprintf(&sb, ", %s", resourceIDToString(id.GroupKind, id.Name)) | ||||||
|  | 		} | ||||||
|  | 		_, _ = fmt.Fprintf(&sb, "): %v", err) | ||||||
|  | 		ef.print(sb.String()) | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func (ef *formatter) FormatApplyEvent(ae event.ApplyEvent) error { | func (ef *formatter) FormatApplyEvent(ae event.ApplyEvent) error { | ||||||
| 	gk := ae.Identifier.GroupKind | 	gk := ae.Identifier.GroupKind | ||||||
| 	name := ae.Identifier.Name | 	name := ae.Identifier.Name | ||||||
|  |  | ||||||
|  | @ -4,6 +4,7 @@ | ||||||
| package events | package events | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
|  | 	"errors" | ||||||
| 	"fmt" | 	"fmt" | ||||||
| 	"strings" | 	"strings" | ||||||
| 	"testing" | 	"testing" | ||||||
|  | @ -11,12 +12,15 @@ import ( | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
| 	"k8s.io/apimachinery/pkg/runtime/schema" | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
| 	"k8s.io/cli-runtime/pkg/genericclioptions" | 	"k8s.io/cli-runtime/pkg/genericclioptions" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
| 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/graph" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/list" | 	"sigs.k8s.io/cli-utils/pkg/print/list" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | @ -323,6 +327,150 @@ func TestFormatter_FormatWaitEvent(t *testing.T) { | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func TestFormatter_FormatValidationEvent(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		previewStrategy common.DryRunStrategy | ||||||
|  | 		event           event.ValidationEvent | ||||||
|  | 		statusCollector list.Collector | ||||||
|  | 		expected        string | ||||||
|  | 		expectedError   error | ||||||
|  | 	}{ | ||||||
|  | 		"zero objects, return error": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{}, | ||||||
|  | 				Error:       errors.New("unexpected"), | ||||||
|  | 			}, | ||||||
|  | 			expectedError: errors.New("invalid validation event: no identifiers: unexpected"), | ||||||
|  | 		}, | ||||||
|  | 		"one object, missing namespace": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "foo", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					field.Required(field.NewPath("metadata", "namespace"), "namespace is required"), | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "foo", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 			expected: "Invalid object (deployment.apps/bar): metadata.namespace: Required value: namespace is required", | ||||||
|  | 		}, | ||||||
|  | 		"two objects, cyclic dependency": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "foo", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					graph.CyclicDependencyError{ | ||||||
|  | 						Edges: []graph.Edge{ | ||||||
|  | 							{ | ||||||
|  | 								From: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "bar", | ||||||
|  | 								}, | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "foo", | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 							{ | ||||||
|  | 								From: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "foo", | ||||||
|  | 								}, | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "bar", | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "foo", | ||||||
|  | 					}, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 			expected: `Invalid objects (deployment.apps/bar, deployment.apps/foo): cyclic dependency: | ||||||
|  | - apps/namespaces/default/Deployment/bar -> apps/namespaces/default/Deployment/foo | ||||||
|  | - apps/namespaces/default/Deployment/foo -> apps/namespaces/default/Deployment/bar`, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			ioStreams, _, out, _ := genericclioptions.NewTestIOStreams() //nolint:dogsled
 | ||||||
|  | 			formatter := NewFormatter(ioStreams, tc.previewStrategy) | ||||||
|  | 			err := formatter.FormatValidationEvent(tc.event) | ||||||
|  | 			if tc.expectedError != nil { | ||||||
|  | 				assert.EqualError(t, err, tc.expectedError.Error()) | ||||||
|  | 			} else { | ||||||
|  | 				assert.NoError(t, err) | ||||||
|  | 			} | ||||||
|  | 			assert.Equal(t, tc.expected, strings.TrimSpace(out.String())) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func createObject(group, kind, namespace, name string) *unstructured.Unstructured { | func createObject(group, kind, namespace, name string) *unstructured.Unstructured { | ||||||
| 	return &unstructured.Unstructured{ | 	return &unstructured.Unstructured{ | ||||||
| 		Object: map[string]interface{}{ | 		Object: map[string]interface{}{ | ||||||
|  |  | ||||||
|  | @ -12,6 +12,7 @@ import ( | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/list" | 	"sigs.k8s.io/cli-utils/pkg/print/list" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/stats" | 	"sigs.k8s.io/cli-utils/pkg/print/stats" | ||||||
| ) | ) | ||||||
|  | @ -27,6 +28,26 @@ type formatter struct { | ||||||
| 	ioStreams genericclioptions.IOStreams | 	ioStreams genericclioptions.IOStreams | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func (jf *formatter) FormatValidationEvent(ve event.ValidationEvent) error { | ||||||
|  | 	// unwrap validation errors
 | ||||||
|  | 	err := ve.Error | ||||||
|  | 	if vErr, ok := err.(*validation.Error); ok { | ||||||
|  | 		err = vErr.Unwrap() | ||||||
|  | 	} | ||||||
|  | 	if len(ve.Identifiers) == 0 { | ||||||
|  | 		// no objects, invalid event
 | ||||||
|  | 		return fmt.Errorf("invalid validation event: no identifiers: %w", err) | ||||||
|  | 	} | ||||||
|  | 	objects := make([]interface{}, len(ve.Identifiers)) | ||||||
|  | 	for i, id := range ve.Identifiers { | ||||||
|  | 		objects[i] = jf.baseResourceEvent(id) | ||||||
|  | 	} | ||||||
|  | 	return jf.printEvent("validation", "validation", map[string]interface{}{ | ||||||
|  | 		"objects": objects, | ||||||
|  | 		"error":   err.Error(), | ||||||
|  | 	}) | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func (jf *formatter) FormatApplyEvent(ae event.ApplyEvent) error { | func (jf *formatter) FormatApplyEvent(ae event.ApplyEvent) error { | ||||||
| 	eventInfo := jf.baseResourceEvent(ae.Identifier) | 	eventInfo := jf.baseResourceEvent(ae.Identifier) | ||||||
| 	if ae.Error != nil { | 	if ae.Error != nil { | ||||||
|  |  | ||||||
|  | @ -11,12 +11,15 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| 	"k8s.io/apimachinery/pkg/runtime/schema" | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
| 	"k8s.io/cli-runtime/pkg/genericclioptions" | 	"k8s.io/cli-runtime/pkg/genericclioptions" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
| 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/graph" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/list" | 	"sigs.k8s.io/cli-utils/pkg/print/list" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/stats" | 	"sigs.k8s.io/cli-utils/pkg/print/stats" | ||||||
| ) | ) | ||||||
|  | @ -565,6 +568,181 @@ func TestFormatter_FormatActionGroupEvent(t *testing.T) { | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func TestFormatter_FormatValidationEvent(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		previewStrategy common.DryRunStrategy | ||||||
|  | 		event           event.ValidationEvent | ||||||
|  | 		expected        map[string]interface{} | ||||||
|  | 		expectedError   error | ||||||
|  | 	}{ | ||||||
|  | 		"zero objects, return error": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{}, | ||||||
|  | 				Error:       errors.New("unexpected"), | ||||||
|  | 			}, | ||||||
|  | 			expectedError: errors.New("invalid validation event: no identifiers: unexpected"), | ||||||
|  | 		}, | ||||||
|  | 		"one object, missing namespace": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "foo", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					field.Required(field.NewPath("metadata", "namespace"), "namespace is required"), | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "foo", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 			expected: map[string]interface{}{ | ||||||
|  | 				"eventType": "validation", | ||||||
|  | 				"type":      "validation", | ||||||
|  | 				"timestamp": "", | ||||||
|  | 				"objects": []interface{}{ | ||||||
|  | 					map[string]interface{}{ | ||||||
|  | 						"group":     "apps", | ||||||
|  | 						"kind":      "Deployment", | ||||||
|  | 						"name":      "bar", | ||||||
|  | 						"namespace": "foo", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				"error": "metadata.namespace: Required value: namespace is required", | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two objects, cyclic dependency": { | ||||||
|  | 			previewStrategy: common.DryRunNone, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 					{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "foo", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					graph.CyclicDependencyError{ | ||||||
|  | 						Edges: []graph.Edge{ | ||||||
|  | 							{ | ||||||
|  | 								From: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "bar", | ||||||
|  | 								}, | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "foo", | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 							{ | ||||||
|  | 								From: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "foo", | ||||||
|  | 								}, | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{ | ||||||
|  | 										Group: "apps", | ||||||
|  | 										Kind:  "Deployment", | ||||||
|  | 									}, | ||||||
|  | 									Namespace: "default", | ||||||
|  | 									Name:      "bar", | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "bar", | ||||||
|  | 					}, | ||||||
|  | 					object.ObjMetadata{ | ||||||
|  | 						GroupKind: schema.GroupKind{ | ||||||
|  | 							Group: "apps", | ||||||
|  | 							Kind:  "Deployment", | ||||||
|  | 						}, | ||||||
|  | 						Namespace: "default", | ||||||
|  | 						Name:      "foo", | ||||||
|  | 					}, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 			expected: map[string]interface{}{ | ||||||
|  | 				"eventType": "validation", | ||||||
|  | 				"type":      "validation", | ||||||
|  | 				"timestamp": "", | ||||||
|  | 				"objects": []interface{}{ | ||||||
|  | 					map[string]interface{}{ | ||||||
|  | 						"group":     "apps", | ||||||
|  | 						"kind":      "Deployment", | ||||||
|  | 						"name":      "bar", | ||||||
|  | 						"namespace": "default", | ||||||
|  | 					}, | ||||||
|  | 					map[string]interface{}{ | ||||||
|  | 						"group":     "apps", | ||||||
|  | 						"kind":      "Deployment", | ||||||
|  | 						"name":      "foo", | ||||||
|  | 						"namespace": "default", | ||||||
|  | 					}, | ||||||
|  | 				}, | ||||||
|  | 				"error": `cyclic dependency: | ||||||
|  | - apps/namespaces/default/Deployment/bar -> apps/namespaces/default/Deployment/foo | ||||||
|  | - apps/namespaces/default/Deployment/foo -> apps/namespaces/default/Deployment/bar`, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			ioStreams, _, out, _ := genericclioptions.NewTestIOStreams() //nolint:dogsled
 | ||||||
|  | 			formatter := NewFormatter(ioStreams, tc.previewStrategy) | ||||||
|  | 			err := formatter.FormatValidationEvent(tc.event) | ||||||
|  | 			if tc.expectedError != nil { | ||||||
|  | 				assert.EqualError(t, err, tc.expectedError.Error()) | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 			assert.NoError(t, err) | ||||||
|  | 			assertOutput(t, tc.expected, out.String()) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // nolint:unparam
 | // nolint:unparam
 | ||||||
| func assertOutput(t *testing.T, expectedMap map[string]interface{}, actual string) bool { | func assertOutput(t *testing.T, expectedMap map[string]interface{}, actual string) bool { | ||||||
| 	if len(expectedMap) == 0 { | 	if len(expectedMap) == 0 { | ||||||
|  |  | ||||||
|  | @ -4,6 +4,7 @@ | ||||||
| package table | package table | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
|  | 	"fmt" | ||||||
| 	"sort" | 	"sort" | ||||||
| 	"sync" | 	"sync" | ||||||
| 
 | 
 | ||||||
|  | @ -12,10 +13,13 @@ import ( | ||||||
| 	pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/stats" | 	"sigs.k8s.io/cli-utils/pkg/print/stats" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/print/table" | 	"sigs.k8s.io/cli-utils/pkg/print/table" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | const InvalidStatus status.Status = "Invalid" | ||||||
|  | 
 | ||||||
| func newResourceStateCollector(resourceGroups []event.ActionGroup) *ResourceStateCollector { | func newResourceStateCollector(resourceGroups []event.ActionGroup) *ResourceStateCollector { | ||||||
| 	resourceInfos := make(map[object.ObjMetadata]*ResourceInfo) | 	resourceInfos := make(map[object.ObjMetadata]*ResourceInfo) | ||||||
| 	for _, group := range resourceGroups { | 	for _, group := range resourceGroups { | ||||||
|  | @ -181,6 +185,8 @@ func (r *ResourceStateCollector) processEvent(ev event.Event) error { | ||||||
| 	r.mux.Lock() | 	r.mux.Lock() | ||||||
| 	defer r.mux.Unlock() | 	defer r.mux.Unlock() | ||||||
| 	switch ev.Type { | 	switch ev.Type { | ||||||
|  | 	case event.ValidationType: | ||||||
|  | 		return r.processValidationEvent(ev.ValidationEvent) | ||||||
| 	case event.StatusType: | 	case event.StatusType: | ||||||
| 		r.processStatusEvent(ev.StatusEvent) | 		r.processStatusEvent(ev.StatusEvent) | ||||||
| 	case event.ApplyType: | 	case event.ApplyType: | ||||||
|  | @ -195,6 +201,34 @@ func (r *ResourceStateCollector) processEvent(ev event.Event) error { | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | // processValidationEvent handles events pertaining to a validation error
 | ||||||
|  | // for a resource.
 | ||||||
|  | func (r *ResourceStateCollector) processValidationEvent(e event.ValidationEvent) error { | ||||||
|  | 	klog.V(7).Infoln("processing validation event") | ||||||
|  | 	// unwrap validation errors
 | ||||||
|  | 	err := e.Error | ||||||
|  | 	if vErr, ok := err.(*validation.Error); ok { | ||||||
|  | 		err = vErr.Unwrap() | ||||||
|  | 	} | ||||||
|  | 	if len(e.Identifiers) == 0 { | ||||||
|  | 		// no objects, invalid event
 | ||||||
|  | 		return fmt.Errorf("invalid validation event: no identifiers: %w", err) | ||||||
|  | 	} | ||||||
|  | 	for _, id := range e.Identifiers { | ||||||
|  | 		previous, found := r.resourceInfos[id] | ||||||
|  | 		if !found { | ||||||
|  | 			klog.V(4).Infof("%s status event not found in ResourceInfos; no processing", id) | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  | 		previous.resourceStatus = &pe.ResourceStatus{ | ||||||
|  | 			Identifier: id, | ||||||
|  | 			Status:     InvalidStatus, | ||||||
|  | 			Message:    e.Error.Error(), | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // processStatusEvent handles events pertaining to a status
 | // processStatusEvent handles events pertaining to a status
 | ||||||
| // update for a resource.
 | // update for a resource.
 | ||||||
| func (r *ResourceStateCollector) processStatusEvent(e event.StatusEvent) { | func (r *ResourceStateCollector) processStatusEvent(e event.StatusEvent) { | ||||||
|  |  | ||||||
|  | @ -4,13 +4,17 @@ | ||||||
| package table | package table | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
|  | 	"errors" | ||||||
| 	"testing" | 	"testing" | ||||||
| 
 | 
 | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| 	"k8s.io/apimachinery/pkg/runtime/schema" | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | 	pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object" | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/graph" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| var ( | var ( | ||||||
|  | @ -19,8 +23,16 @@ var ( | ||||||
| 			Group: "apps", | 			Group: "apps", | ||||||
| 			Kind:  "Deployment", | 			Kind:  "Deployment", | ||||||
| 		}, | 		}, | ||||||
| 		Name:      "Foo", | 		Name:      "foo", | ||||||
| 		Namespace: "Bar", | 		Namespace: "default", | ||||||
|  | 	} | ||||||
|  | 	depID2 = object.ObjMetadata{ | ||||||
|  | 		GroupKind: schema.GroupKind{ | ||||||
|  | 			Group: "apps", | ||||||
|  | 			Kind:  "Deployment", | ||||||
|  | 		}, | ||||||
|  | 		Name:      "bar", | ||||||
|  | 		Namespace: "default", | ||||||
| 	} | 	} | ||||||
| 	customID = object.ObjMetadata{ | 	customID = object.ObjMetadata{ | ||||||
| 		GroupKind: schema.GroupKind{ | 		GroupKind: schema.GroupKind{ | ||||||
|  | @ -184,6 +196,79 @@ func TestResourceStateCollector_ProcessStatusEvent(t *testing.T) { | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func TestResourceStateCollector_ProcessValidationEvent(t *testing.T) { | ||||||
|  | 	testCases := map[string]struct { | ||||||
|  | 		resourceGroups []event.ActionGroup | ||||||
|  | 		event          event.ValidationEvent | ||||||
|  | 		expectedError  error | ||||||
|  | 	}{ | ||||||
|  | 		"zero objects, return error": { | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{}, | ||||||
|  | 				Error:       errors.New("unexpected"), | ||||||
|  | 			}, | ||||||
|  | 			expectedError: errors.New("invalid validation event: no identifiers: unexpected"), | ||||||
|  | 		}, | ||||||
|  | 		"one object, missing namespace": { | ||||||
|  | 			resourceGroups: []event.ActionGroup{ | ||||||
|  | 				{ | ||||||
|  | 					Action:      event.ApplyAction, | ||||||
|  | 					Identifiers: object.ObjMetadataSet{depID}, | ||||||
|  | 				}, | ||||||
|  | 			}, | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{depID}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					field.Required(field.NewPath("metadata", "namespace"), "namespace is required"), | ||||||
|  | 					depID, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		"two objects, cyclic dependency": { | ||||||
|  | 			event: event.ValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{depID, depID2}, | ||||||
|  | 				Error: validation.NewError( | ||||||
|  | 					graph.CyclicDependencyError{ | ||||||
|  | 						Edges: []graph.Edge{ | ||||||
|  | 							{ | ||||||
|  | 								From: depID, | ||||||
|  | 								To:   depID2, | ||||||
|  | 							}, | ||||||
|  | 							{ | ||||||
|  | 								From: depID2, | ||||||
|  | 								To:   depID, | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					depID, | ||||||
|  | 					depID2, | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for tn, tc := range testCases { | ||||||
|  | 		t.Run(tn, func(t *testing.T) { | ||||||
|  | 			rsc := newResourceStateCollector(tc.resourceGroups) | ||||||
|  | 			err := rsc.processValidationEvent(tc.event) | ||||||
|  | 			if tc.expectedError != nil { | ||||||
|  | 				assert.EqualError(t, err, tc.expectedError.Error()) | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 			for _, id := range tc.event.Identifiers { | ||||||
|  | 				resourceInfo, found := rsc.resourceInfos[id] | ||||||
|  | 				if found { | ||||||
|  | 					assert.Equal(t, &pe.ResourceStatus{ | ||||||
|  | 						Identifier: id, | ||||||
|  | 						Status:     InvalidStatus, | ||||||
|  | 						Message:    tc.event.Error.Error(), | ||||||
|  | 					}, resourceInfo.resourceStatus) | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func getID(e event.StatusEvent) (object.ObjMetadata, bool) { | func getID(e event.StatusEvent) (object.ObjMetadata, bool) { | ||||||
| 	if e.Resource == nil { | 	if e.Resource == nil { | ||||||
| 		return object.ObjMetadata{}, false | 		return object.ObjMetadata{}, false | ||||||
|  |  | ||||||
|  | @ -24,6 +24,7 @@ type ExpEvent struct { | ||||||
| 	PruneEvent       *ExpPruneEvent | 	PruneEvent       *ExpPruneEvent | ||||||
| 	DeleteEvent      *ExpDeleteEvent | 	DeleteEvent      *ExpDeleteEvent | ||||||
| 	WaitEvent        *ExpWaitEvent | 	WaitEvent        *ExpWaitEvent | ||||||
|  | 	ValidationEvent  *ExpValidationEvent | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| type ExpInitEvent struct { | type ExpInitEvent struct { | ||||||
|  | @ -74,6 +75,11 @@ type ExpWaitEvent struct { | ||||||
| 	Identifier object.ObjMetadata | 	Identifier object.ObjMetadata | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | type ExpValidationEvent struct { | ||||||
|  | 	Identifiers object.ObjMetadataSet | ||||||
|  | 	Error       error | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func VerifyEvents(expEvents []ExpEvent, events []event.Event) error { | func VerifyEvents(expEvents []ExpEvent, events []event.Event) error { | ||||||
| 	if len(expEvents) == 0 && len(events) == 0 { | 	if len(expEvents) == 0 && len(events) == 0 { | ||||||
| 		return nil | 		return nil | ||||||
|  | @ -270,6 +276,24 @@ func isMatch(ee ExpEvent, e event.Event) bool { | ||||||
| 		} | 		} | ||||||
| 		return true | 		return true | ||||||
| 
 | 
 | ||||||
|  | 	case event.ValidationType: | ||||||
|  | 		vee := ee.ValidationEvent | ||||||
|  | 		if vee == nil { | ||||||
|  | 			return true | ||||||
|  | 		} | ||||||
|  | 		ve := e.ValidationEvent | ||||||
|  | 
 | ||||||
|  | 		if vee.Identifiers != nil { | ||||||
|  | 			if !vee.Identifiers.Equal(ve.Identifiers) { | ||||||
|  | 				return false | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		if vee.Error != nil { | ||||||
|  | 			return ve.Error != nil | ||||||
|  | 		} | ||||||
|  | 		return ve.Error == nil | ||||||
|  | 
 | ||||||
| 	default: | 	default: | ||||||
| 		return true | 		return true | ||||||
| 	} | 	} | ||||||
|  | @ -364,6 +388,15 @@ func EventToExpEvent(e event.Event) ExpEvent { | ||||||
| 				Operation:  e.WaitEvent.Operation, | 				Operation:  e.WaitEvent.Operation, | ||||||
| 			}, | 			}, | ||||||
| 		} | 		} | ||||||
|  | 
 | ||||||
|  | 	case event.ValidationType: | ||||||
|  | 		return ExpEvent{ | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &ExpValidationEvent{ | ||||||
|  | 				Identifiers: e.ValidationEvent.Identifiers, | ||||||
|  | 				Error:       e.ValidationEvent.Error, | ||||||
|  | 			}, | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return ExpEvent{} | 	return ExpEvent{} | ||||||
| } | } | ||||||
|  | @ -433,6 +466,8 @@ func (ape GroupedEventsByID) Less(i, j int) bool { | ||||||
| 			return false | 			return false | ||||||
| 		} | 		} | ||||||
| 		return ape[i].WaitEvent.Identifier.String() < ape[j].WaitEvent.Identifier.String() | 		return ape[i].WaitEvent.Identifier.String() < ape[j].WaitEvent.Identifier.String() | ||||||
|  | 	case event.ValidationType: | ||||||
|  | 		return ape[i].ValidationEvent.Identifiers.Hash() < ape[j].ValidationEvent.Identifiers.Hash() | ||||||
| 	default: | 	default: | ||||||
| 		// don't change order if not ApplyType, PruneType, or DeleteType
 | 		// don't change order if not ApplyType, PruneType, or DeleteType
 | ||||||
| 		return false | 		return false | ||||||
|  |  | ||||||
|  | @ -143,3 +143,52 @@ spec: | ||||||
|     - name: tcp |     - name: tcp | ||||||
|       containerPort: 80 |       containerPort: 80 | ||||||
| ` | ` | ||||||
|  | 
 | ||||||
|  | var invalidMutationPodBTemplate = ` | ||||||
|  | kind: Pod | ||||||
|  | apiVersion: v1 | ||||||
|  | metadata: | ||||||
|  |   name: pod-b | ||||||
|  |   namespace: {{.Namespace}} | ||||||
|  |   annotations: | ||||||
|  |     config.kubernetes.io/apply-time-mutation: | | ||||||
|  |       - sourceRef: | ||||||
|  |           kind: Pod | ||||||
|  |           name: pod-a # cyclic dependency | ||||||
|  |           namespace: {{.Namespace}} | ||||||
|  |         sourcePath: $.status.podIP | ||||||
|  |         targetPath: $.spec.containers[?(@.name=="nginx")].env[?(@.name=="SERVICE_HOST")].value | ||||||
|  |         token: ${pob-b-ip} | ||||||
|  |       - sourceRef: | ||||||
|  |           kind: Pod | ||||||
|  |           name: pod-a | ||||||
|  |           namespace: "" # empty namespace on a namespaced type | ||||||
|  |         sourcePath: $.spec.containers[?(@.name=="nginx")].ports[?(@.name=="tcp")].containerPort | ||||||
|  |         targetPath: $.spec.containers[?(@.name=="nginx")].env[?(@.name=="SERVICE_HOST")].value | ||||||
|  |         token: ${pob-b-port} | ||||||
|  | spec: | ||||||
|  |   containers: | ||||||
|  |   - name: nginx | ||||||
|  |     image: nginx:1.21 | ||||||
|  |     ports: | ||||||
|  |     - name: tcp | ||||||
|  |       containerPort: 80 | ||||||
|  |     env: | ||||||
|  |     - name: SERVICE_HOST | ||||||
|  |       value: "${pob-b-ip}:${pob-b-port}" | ||||||
|  | ` | ||||||
|  | 
 | ||||||
|  | var invalidPodTemplate = ` | ||||||
|  | kind: Pod | ||||||
|  | apiVersion: v1 | ||||||
|  | metadata: | ||||||
|  |   # missing name | ||||||
|  |   namespace: {{.Namespace}} | ||||||
|  | spec: | ||||||
|  |   containers: | ||||||
|  |   - name: nginx | ||||||
|  |     image: nginx:1.21 | ||||||
|  |     ports: | ||||||
|  |     - name: tcp | ||||||
|  |       containerPort: 80 | ||||||
|  | ` | ||||||
|  |  | ||||||
|  | @ -7,8 +7,8 @@ import ( | ||||||
| 	"bytes" | 	"bytes" | ||||||
| 	"context" | 	"context" | ||||||
| 	"fmt" | 	"fmt" | ||||||
| 	"html/template" |  | ||||||
| 	"strings" | 	"strings" | ||||||
|  | 	"text/template" | ||||||
| 	"time" | 	"time" | ||||||
| 
 | 
 | ||||||
| 	. "github.com/onsi/ginkgo" | 	. "github.com/onsi/ginkgo" | ||||||
|  | @ -22,6 +22,7 @@ import ( | ||||||
| 	"k8s.io/apimachinery/pkg/util/yaml" | 	"k8s.io/apimachinery/pkg/util/yaml" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/apply/event" | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/common" | 	"sigs.k8s.io/cli-utils/pkg/common" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/kstatus/status" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object/dependson" | 	"sigs.k8s.io/cli-utils/pkg/object/dependson" | ||||||
| 	"sigs.k8s.io/cli-utils/pkg/object/mutation" | 	"sigs.k8s.io/cli-utils/pkg/object/mutation" | ||||||
| 	"sigs.k8s.io/controller-runtime/pkg/client" | 	"sigs.k8s.io/controller-runtime/pkg/client" | ||||||
|  | @ -203,6 +204,40 @@ func assertUnstructuredDoesNotExist(ctx context.Context, c client.Client, obj *u | ||||||
| 		"expected GET to error with NotFound (%s): %s", ref, err) | 		"expected GET to error with NotFound (%s): %s", ref, err) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func applyUnstructured(ctx context.Context, c client.Client, obj *unstructured.Unstructured) { | ||||||
|  | 	ref := mutation.ResourceReferenceFromUnstructured(obj) | ||||||
|  | 	resultObj := ref.ToUnstructured() | ||||||
|  | 
 | ||||||
|  | 	err := c.Get(ctx, types.NamespacedName{ | ||||||
|  | 		Namespace: obj.GetNamespace(), | ||||||
|  | 		Name:      obj.GetName(), | ||||||
|  | 	}, resultObj) | ||||||
|  | 	Expect(err).NotTo(HaveOccurred(), | ||||||
|  | 		"expected GET not to error (%s)", ref) | ||||||
|  | 
 | ||||||
|  | 	err = c.Patch(ctx, obj, client.MergeFrom(resultObj)) | ||||||
|  | 	Expect(err).NotTo(HaveOccurred(), | ||||||
|  | 		"expected PATCH not to error (%s): %s", ref, err) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func assertUnstructuredAvailable(obj *unstructured.Unstructured) { | ||||||
|  | 	ref := mutation.ResourceReferenceFromUnstructured(obj) | ||||||
|  | 	objc, err := status.GetObjectWithConditions(obj.Object) | ||||||
|  | 	Expect(err).NotTo(HaveOccurred()) | ||||||
|  | 	available := false | ||||||
|  | 	for _, c := range objc.Status.Conditions { | ||||||
|  | 		switch c.Type { | ||||||
|  | 		case "Available": // appsv1.DeploymentAvailable
 | ||||||
|  | 			if c.Status == "True" { // corev1.ConditionTrue
 | ||||||
|  | 				available = true | ||||||
|  | 				break | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	Expect(available).To(BeTrue(), | ||||||
|  | 		"expected Available condition to be True (%s)", ref) | ||||||
|  | } | ||||||
|  | 
 | ||||||
| func randomString(prefix string) string { | func randomString(prefix string) string { | ||||||
| 	randomSuffix := common.RandomStr() | 	randomSuffix := common.RandomStr() | ||||||
| 	return fmt.Sprintf("%s%s", prefix, randomSuffix) | 	return fmt.Sprintf("%s%s", prefix, randomSuffix) | ||||||
|  | @ -280,7 +315,7 @@ func manifestToUnstructured(manifest []byte) *unstructured.Unstructured { | ||||||
| 	u := make(map[string]interface{}) | 	u := make(map[string]interface{}) | ||||||
| 	err := yaml.Unmarshal(manifest, &u) | 	err := yaml.Unmarshal(manifest, &u) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		panic(err) | 		panic(fmt.Errorf("failed to parse manifest yaml: %w", err)) | ||||||
| 	} | 	} | ||||||
| 	return &unstructured.Unstructured{ | 	return &unstructured.Unstructured{ | ||||||
| 		Object: u, | 		Object: u, | ||||||
|  |  | ||||||
|  | @ -198,6 +198,14 @@ var _ = Describe("Applier", func() { | ||||||
| 				It("Reconciliation timeout", func() { | 				It("Reconciliation timeout", func() { | ||||||
| 					reconciliationTimeout(ctx, invConfig, inventoryName, namespace.GetName()) | 					reconciliationTimeout(ctx, invConfig, inventoryName, namespace.GetName()) | ||||||
| 				}) | 				}) | ||||||
|  | 
 | ||||||
|  | 				It("SkipInvalid", func() { | ||||||
|  | 					skipInvalidTest(ctx, c, invConfig, inventoryName, namespace.GetName()) | ||||||
|  | 				}) | ||||||
|  | 
 | ||||||
|  | 				It("ExitEarly", func() { | ||||||
|  | 					exitEarlyTest(ctx, c, invConfig, inventoryName, namespace.GetName()) | ||||||
|  | 				}) | ||||||
| 			}) | 			}) | ||||||
| 
 | 
 | ||||||
| 			Context("Inventory policy", func() { | 			Context("Inventory policy", func() { | ||||||
|  |  | ||||||
|  | @ -0,0 +1,67 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package e2e | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 
 | ||||||
|  | 	. "github.com/onsi/ginkgo" | ||||||
|  | 	. "github.com/onsi/gomega" | ||||||
|  | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/apply" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/testutil" | ||||||
|  | 	"sigs.k8s.io/controller-runtime/pkg/client" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func exitEarlyTest(ctx context.Context, c client.Client, invConfig InventoryConfig, inventoryName, namespaceName string) { | ||||||
|  | 	By("exit early on invalid object") | ||||||
|  | 	applier := invConfig.ApplierFactoryFunc() | ||||||
|  | 
 | ||||||
|  | 	inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(inventoryName, namespaceName, "test")) | ||||||
|  | 
 | ||||||
|  | 	fields := struct{ Namespace string }{Namespace: namespaceName} | ||||||
|  | 	// valid pod
 | ||||||
|  | 	pod1Obj := withNamespace(manifestToUnstructured(pod1), namespaceName) | ||||||
|  | 	// valid deployment with dependency
 | ||||||
|  | 	deployment1Obj := withDependsOn(withNamespace(manifestToUnstructured(deployment1), namespaceName), | ||||||
|  | 		fmt.Sprintf("/namespaces/%s/Pod/%s", namespaceName, pod1Obj.GetName())) | ||||||
|  | 	// missing name
 | ||||||
|  | 	invalidPodObj := templateToUnstructured(invalidPodTemplate, fields) | ||||||
|  | 
 | ||||||
|  | 	resources := []*unstructured.Unstructured{ | ||||||
|  | 		pod1Obj, | ||||||
|  | 		deployment1Obj, | ||||||
|  | 		invalidPodObj, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	applierEvents := runCollect(applier.Run(ctx, inv, resources, apply.Options{ | ||||||
|  | 		EmitStatusEvents: false, | ||||||
|  | 		ValidationPolicy: validation.ExitEarly, | ||||||
|  | 	})) | ||||||
|  | 
 | ||||||
|  | 	expEvents := []testutil.ExpEvent{ | ||||||
|  | 		{ | ||||||
|  | 			// invalid pod validation error
 | ||||||
|  | 			EventType: event.ErrorType, | ||||||
|  | 			ErrorEvent: &testutil.ExpErrorEvent{ | ||||||
|  | 				Err: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 					field.Required(field.NewPath("metadata", "name"), "name is required"), | ||||||
|  | 					object.UnstructuredToObjMetadata(invalidPodObj), | ||||||
|  | 				).Error()), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 	Expect(testutil.EventsToExpEvents(applierEvents)).To(testutil.Equal(expEvents)) | ||||||
|  | 
 | ||||||
|  | 	By("verify pod1 not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, pod1Obj) | ||||||
|  | 
 | ||||||
|  | 	By("verify deployment1 not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, deployment1Obj) | ||||||
|  | } | ||||||
|  | @ -0,0 +1,470 @@ | ||||||
|  | // Copyright 2022 The Kubernetes Authors.
 | ||||||
|  | // SPDX-License-Identifier: Apache-2.0
 | ||||||
|  | 
 | ||||||
|  | package e2e | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 
 | ||||||
|  | 	. "github.com/onsi/ginkgo" | ||||||
|  | 	. "github.com/onsi/gomega" | ||||||
|  | 	"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured" | ||||||
|  | 	"k8s.io/apimachinery/pkg/runtime/schema" | ||||||
|  | 	"k8s.io/apimachinery/pkg/util/validation/field" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/apply" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/apply/event" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/inventory" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/dependson" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/graph" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/mutation" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/object/validation" | ||||||
|  | 	"sigs.k8s.io/cli-utils/pkg/testutil" | ||||||
|  | 	"sigs.k8s.io/controller-runtime/pkg/client" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func skipInvalidTest(ctx context.Context, c client.Client, invConfig InventoryConfig, inventoryName, namespaceName string) { | ||||||
|  | 	By("apply valid objects and skip invalid objects") | ||||||
|  | 	applier := invConfig.ApplierFactoryFunc() | ||||||
|  | 
 | ||||||
|  | 	inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(inventoryName, namespaceName, "test")) | ||||||
|  | 
 | ||||||
|  | 	fields := struct{ Namespace string }{Namespace: namespaceName} | ||||||
|  | 	// valid pod
 | ||||||
|  | 	pod1Obj := withNamespace(manifestToUnstructured(pod1), namespaceName) | ||||||
|  | 	// valid deployment with dependency
 | ||||||
|  | 	deployment1Obj := withDependsOn(withNamespace(manifestToUnstructured(deployment1), namespaceName), | ||||||
|  | 		fmt.Sprintf("/namespaces/%s/Pod/%s", namespaceName, pod1Obj.GetName())) | ||||||
|  | 	// external/missing dependency
 | ||||||
|  | 	pod3Obj := withDependsOn(withNamespace(manifestToUnstructured(pod3), namespaceName), | ||||||
|  | 		fmt.Sprintf("/namespaces/%s/Pod/pod0", namespaceName)) | ||||||
|  | 	// cyclic dependency (podB)
 | ||||||
|  | 	podAObj := templateToUnstructured(podATemplate, fields) | ||||||
|  | 	// cyclic dependency (podA) & invalid source reference (dependency not in object set)
 | ||||||
|  | 	podBObj := templateToUnstructured(invalidMutationPodBTemplate, fields) | ||||||
|  | 	// missing name
 | ||||||
|  | 	invalidPodObj := templateToUnstructured(invalidPodTemplate, fields) | ||||||
|  | 
 | ||||||
|  | 	resources := []*unstructured.Unstructured{ | ||||||
|  | 		pod1Obj, | ||||||
|  | 		deployment1Obj, | ||||||
|  | 		pod3Obj, | ||||||
|  | 		podAObj, | ||||||
|  | 		podBObj, | ||||||
|  | 		invalidPodObj, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	applierEvents := runCollect(applier.Run(ctx, inv, resources, apply.Options{ | ||||||
|  | 		EmitStatusEvents: false, | ||||||
|  | 		ValidationPolicy: validation.SkipInvalid, | ||||||
|  | 	})) | ||||||
|  | 
 | ||||||
|  | 	expEvents := []testutil.ExpEvent{ | ||||||
|  | 		{ | ||||||
|  | 			// invalid pod validation error
 | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					object.UnstructuredToObjMetadata(invalidPodObj), | ||||||
|  | 				}, | ||||||
|  | 				Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 					field.Required(field.NewPath("metadata", "name"), "name is required"), | ||||||
|  | 					object.UnstructuredToObjMetadata(invalidPodObj), | ||||||
|  | 				).Error()), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Pod3 validation error
 | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					object.UnstructuredToObjMetadata(pod3Obj), | ||||||
|  | 				}, | ||||||
|  | 				Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 					object.InvalidAnnotationError{ | ||||||
|  | 						Annotation: dependson.Annotation, | ||||||
|  | 						Cause: graph.ExternalDependencyError{ | ||||||
|  | 							Edge: graph.Edge{ | ||||||
|  | 								From: object.UnstructuredToObjMetadata(pod3Obj), | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{Kind: "Pod"}, | ||||||
|  | 									Name:      "pod0", | ||||||
|  | 									Namespace: namespaceName, | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.UnstructuredToObjMetadata(pod3Obj), | ||||||
|  | 				).Error()), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// PodB validation error
 | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 				}, | ||||||
|  | 				Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 					object.InvalidAnnotationError{ | ||||||
|  | 						Annotation: mutation.Annotation, | ||||||
|  | 						Cause: graph.ExternalDependencyError{ | ||||||
|  | 							Edge: graph.Edge{ | ||||||
|  | 								From: object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 								To: object.ObjMetadata{ | ||||||
|  | 									GroupKind: schema.GroupKind{Kind: "Pod"}, | ||||||
|  | 									Name:      "pod-a", | ||||||
|  | 								}, | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 				).Error()), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Cyclic Dependency validation error
 | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					object.UnstructuredToObjMetadata(podAObj), | ||||||
|  | 					object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 				}, | ||||||
|  | 				Error: testutil.EqualErrorString(validation.NewError( | ||||||
|  | 					graph.CyclicDependencyError{ | ||||||
|  | 						Edges: []graph.Edge{ | ||||||
|  | 							{ | ||||||
|  | 								From: object.UnstructuredToObjMetadata(podAObj), | ||||||
|  | 								To:   object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 							}, | ||||||
|  | 							{ | ||||||
|  | 								From: object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 								To:   object.UnstructuredToObjMetadata(podAObj), | ||||||
|  | 							}, | ||||||
|  | 						}, | ||||||
|  | 					}, | ||||||
|  | 					object.UnstructuredToObjMetadata(podAObj), | ||||||
|  | 					object.UnstructuredToObjMetadata(podBObj), | ||||||
|  | 				).Error()), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InitTask
 | ||||||
|  | 			EventType: event.InitType, | ||||||
|  | 			InitEvent: &testutil.ExpInitEvent{}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InvAddTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "inventory-add-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InvAddTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "inventory-add-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// ApplyTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.ApplyAction, | ||||||
|  | 				GroupName: "apply-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Apply Pod1
 | ||||||
|  | 			EventType: event.ApplyType, | ||||||
|  | 			ApplyEvent: &testutil.ExpApplyEvent{ | ||||||
|  | 				GroupName:  "apply-0", | ||||||
|  | 				Operation:  event.Created, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// ApplyTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.ApplyAction, | ||||||
|  | 				GroupName: "apply-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Pod1 reconcile Pending.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-0", | ||||||
|  | 				Operation:  event.ReconcilePending, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Pod1 confirmed Current.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-0", | ||||||
|  | 				Operation:  event.Reconciled, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// ApplyTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.ApplyAction, | ||||||
|  | 				GroupName: "apply-1", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Apply Deployment1
 | ||||||
|  | 			EventType: event.ApplyType, | ||||||
|  | 			ApplyEvent: &testutil.ExpApplyEvent{ | ||||||
|  | 				GroupName:  "apply-1", | ||||||
|  | 				Operation:  event.Created, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(deployment1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// ApplyTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.ApplyAction, | ||||||
|  | 				GroupName: "apply-1", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-1", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Deployment1 reconcile Pending.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-1", | ||||||
|  | 				Operation:  event.ReconcilePending, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(deployment1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Deployment1 confirmed Current.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-1", | ||||||
|  | 				Operation:  event.Reconciled, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(deployment1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-1", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InvSetTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "inventory-set-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InvSetTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "inventory-set-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 	Expect(testutil.EventsToExpEvents(applierEvents)).To(testutil.Equal(expEvents)) | ||||||
|  | 
 | ||||||
|  | 	By("verify pod1 created and ready") | ||||||
|  | 	result := assertUnstructuredExists(ctx, c, pod1Obj) | ||||||
|  | 	podIP, found, err := object.NestedField(result.Object, "status", "podIP") | ||||||
|  | 	Expect(err).NotTo(HaveOccurred()) | ||||||
|  | 	Expect(found).To(BeTrue()) | ||||||
|  | 	Expect(podIP).NotTo(BeEmpty()) // use podIP as proxy for readiness
 | ||||||
|  | 
 | ||||||
|  | 	By("verify deployment1 created and ready") | ||||||
|  | 	result = assertUnstructuredExists(ctx, c, deployment1Obj) | ||||||
|  | 	assertUnstructuredAvailable(result) | ||||||
|  | 
 | ||||||
|  | 	By("verify pod3 not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, pod3Obj) | ||||||
|  | 
 | ||||||
|  | 	By("verify podA not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, podAObj) | ||||||
|  | 
 | ||||||
|  | 	By("verify podB not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, podBObj) | ||||||
|  | 
 | ||||||
|  | 	By("modify deployment1 depends-on annotation to be invalid") | ||||||
|  | 	applyUnstructured(ctx, c, withDependsOn(deployment1Obj, "invalid")) | ||||||
|  | 
 | ||||||
|  | 	By("destroy valid objects and skip invalid objects") | ||||||
|  | 	destroyer := invConfig.DestroyerFactoryFunc() | ||||||
|  | 	destroyerEvents := runCollect(destroyer.Run(ctx, inv, apply.DestroyerOptions{ | ||||||
|  | 		InventoryPolicy:  inventory.AdoptIfNoInventory, | ||||||
|  | 		ValidationPolicy: validation.SkipInvalid, | ||||||
|  | 	})) | ||||||
|  | 
 | ||||||
|  | 	expEvents = []testutil.ExpEvent{ | ||||||
|  | 		{ | ||||||
|  | 			// Deployment1 validation error
 | ||||||
|  | 			EventType: event.ValidationType, | ||||||
|  | 			ValidationEvent: &testutil.ExpValidationEvent{ | ||||||
|  | 				Identifiers: object.ObjMetadataSet{ | ||||||
|  | 					object.UnstructuredToObjMetadata(deployment1Obj), | ||||||
|  | 				}, | ||||||
|  | 				Error: testutil.EqualErrorType( | ||||||
|  | 					validation.NewError(nil), // TODO: be more specific
 | ||||||
|  | 				), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// InitTask
 | ||||||
|  | 			EventType: event.InitType, | ||||||
|  | 			InitEvent: &testutil.ExpInitEvent{}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// PruneTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.DeleteAction, | ||||||
|  | 				GroupName: "prune-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		// TODO: Filter deletes so dependencies don't get deleted when the objects that used to depend on them are invalid?
 | ||||||
|  | 		{ | ||||||
|  | 			// Delete pod1
 | ||||||
|  | 			EventType: event.DeleteType, | ||||||
|  | 			DeleteEvent: &testutil.ExpDeleteEvent{ | ||||||
|  | 				GroupName:  "prune-0", | ||||||
|  | 				Operation:  event.Deleted, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// PruneTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.DeleteAction, | ||||||
|  | 				GroupName: "prune-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Pod1 reconcile Pending.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-0", | ||||||
|  | 				Operation:  event.ReconcilePending, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// Pod1 confirmed NotFound.
 | ||||||
|  | 			EventType: event.WaitType, | ||||||
|  | 			WaitEvent: &testutil.ExpWaitEvent{ | ||||||
|  | 				GroupName:  "wait-0", | ||||||
|  | 				Operation:  event.Reconciled, | ||||||
|  | 				Identifier: object.UnstructuredToObjMetadata(pod1Obj), | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// WaitTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.WaitAction, | ||||||
|  | 				GroupName: "wait-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// DeleteInvTask start
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "delete-inventory-0", | ||||||
|  | 				Type:      event.Started, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			// DeleteInvTask finished
 | ||||||
|  | 			EventType: event.ActionGroupType, | ||||||
|  | 			ActionGroupEvent: &testutil.ExpActionGroupEvent{ | ||||||
|  | 				Action:    event.InventoryAction, | ||||||
|  | 				GroupName: "delete-inventory-0", | ||||||
|  | 				Type:      event.Finished, | ||||||
|  | 			}, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 	Expect(testutil.EventsToExpEvents(destroyerEvents)).To(testutil.Equal(expEvents)) | ||||||
|  | 
 | ||||||
|  | 	By("verify pod1 deleted") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, pod1Obj) | ||||||
|  | 
 | ||||||
|  | 	By("verify deployment1 not deleted") | ||||||
|  | 	assertUnstructuredExists(ctx, c, deployment1Obj) | ||||||
|  | 	deleteUnstructuredIfExists(ctx, c, deployment1Obj) | ||||||
|  | 
 | ||||||
|  | 	By("verify pod3 not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, pod3Obj) | ||||||
|  | 
 | ||||||
|  | 	By("verify podA not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, podAObj) | ||||||
|  | 
 | ||||||
|  | 	By("verify podB not found") | ||||||
|  | 	assertUnstructuredDoesNotExist(ctx, c, podBObj) | ||||||
|  | } | ||||||
		Loading…
	
		Reference in New Issue