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
|
||||
// before anything has been updated in the cluster.
|
||||
validator := &validation.Validator{Mapper: mapper}
|
||||
if err := validator.Validate(objects); err != nil {
|
||||
handleError(eventChannel, err)
|
||||
return
|
||||
vCollector := &validation.Collector{}
|
||||
validator := &validation.Validator{
|
||||
Collector: vCollector,
|
||||
Mapper: mapper,
|
||||
}
|
||||
validator.Validate(objects)
|
||||
|
||||
// Decide which objects to apply and which to prune
|
||||
applyObjs, pruneObjs, err := a.prepareObjects(invInfo, objects, options)
|
||||
if err != nil {
|
||||
handleError(eventChannel, err)
|
||||
|
|
@ -166,6 +168,7 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje
|
|||
Mapper: mapper,
|
||||
InvClient: a.invClient,
|
||||
Destroy: false,
|
||||
Collector: vCollector,
|
||||
}
|
||||
opts := solver.Options{
|
||||
ServerSideOptions: options.ServerSideOptions,
|
||||
|
|
@ -199,7 +202,6 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje
|
|||
},
|
||||
}
|
||||
// Build list of apply mutators.
|
||||
// Share a thread-safe cache with the status poller.
|
||||
resourceCache := cache.NewResourceCacheMap()
|
||||
applyMutators := []mutator.Interface{
|
||||
&mutator.ApplyTimeMutator{
|
||||
|
|
@ -208,17 +210,43 @@ func (a *Applier) Run(ctx context.Context, invInfo inventory.InventoryInfo, obje
|
|||
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).
|
||||
AppendApplyWaitTasks(applyObjs, applyFilters, applyMutators, opts).
|
||||
AppendPruneWaitTasks(pruneObjs, pruneFilters, opts).
|
||||
AppendInvSetTask(invInfo, options.DryRunStrategy).
|
||||
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
|
||||
}
|
||||
|
||||
// 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
|
||||
// will be applied/pruned.
|
||||
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...")
|
||||
allIds := object.UnstructuredSetToObjMetadataSet(append(applyObjs, pruneObjs...))
|
||||
runner := taskrunner.NewTaskStatusRunner(allIds, a.StatusPoller)
|
||||
taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache)
|
||||
klog.V(4).Infoln("applier running TaskStatusRunner...")
|
||||
err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{
|
||||
PollInterval: options.PollInterval,
|
||||
|
|
@ -283,6 +310,9 @@ type Options struct {
|
|||
|
||||
// InventoryPolicy defines the inventory policy of apply.
|
||||
InventoryPolicy inventory.InventoryPolicy
|
||||
|
||||
// ValidationPolicy defines how to handle invalid objects.
|
||||
ValidationPolicy validation.Policy
|
||||
}
|
||||
|
||||
// setDefaults set the options to the default values if they
|
||||
|
|
@ -322,3 +352,25 @@ func localNamespaces(localInv inventory.InventoryInfo, localObjs []object.ObjMet
|
|||
}
|
||||
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/require"
|
||||
"k8s.io/apimachinery/pkg/util/validation/field"
|
||||
"k8s.io/kubectl/pkg/scheme"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/inventory"
|
||||
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
|
||||
"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/validation"
|
||||
"sigs.k8s.io/cli-utils/pkg/testutil"
|
||||
)
|
||||
|
||||
|
|
@ -489,7 +492,7 @@ func TestApplier(t *testing.T) {
|
|||
Type: event.Started,
|
||||
},
|
||||
},
|
||||
// Secrets applied before Deployments (see pkg/ordering)
|
||||
// Apply Secrets before Deployments (see ordering.SortableMetas)
|
||||
{
|
||||
EventType: event.ApplyType,
|
||||
ApplyEvent: &testutil.ExpApplyEvent{
|
||||
|
|
@ -522,7 +525,7 @@ func TestApplier(t *testing.T) {
|
|||
Type: event.Started,
|
||||
},
|
||||
},
|
||||
// Secrets before Deployments (see pkg/ordering)
|
||||
// Apply Secrets before Deployments (see ordering.SortableMetas)
|
||||
{
|
||||
EventType: event.WaitType,
|
||||
WaitEvent: &testutil.ExpWaitEvent{
|
||||
|
|
@ -539,7 +542,7 @@ func TestApplier(t *testing.T) {
|
|||
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
|
||||
},
|
||||
},
|
||||
// Deployment before Secret (see statusEvents)
|
||||
// Wait Deployments before Secrets (see testutil.GroupedEventsByID)
|
||||
{
|
||||
EventType: event.WaitType,
|
||||
WaitEvent: &testutil.ExpWaitEvent{
|
||||
|
|
@ -695,7 +698,7 @@ func TestApplier(t *testing.T) {
|
|||
Type: event.Started,
|
||||
},
|
||||
},
|
||||
// Deployments deleted before Secrets (see pkg/ordering)
|
||||
// Prune Deployments before Secrets (see ordering.SortableMetas)
|
||||
{
|
||||
EventType: event.PruneType,
|
||||
PruneEvent: &testutil.ExpPruneEvent{
|
||||
|
|
@ -728,7 +731,7 @@ func TestApplier(t *testing.T) {
|
|||
Type: event.Started,
|
||||
},
|
||||
},
|
||||
// Deployments before Secrets (see pkg/ordering)
|
||||
// Prune Deployments before Secrets (see ordering.SortableMetas)
|
||||
{
|
||||
EventType: event.WaitType,
|
||||
WaitEvent: &testutil.ExpWaitEvent{
|
||||
|
|
@ -745,7 +748,7 @@ func TestApplier(t *testing.T) {
|
|||
Identifier: testutil.ToIdentifier(t, resources["secret"]),
|
||||
},
|
||||
},
|
||||
// Deployment before Secret (see statusEvents)
|
||||
// Wait Deployments before Secrets (see testutil.GroupedEventsByID)
|
||||
{
|
||||
EventType: event.WaitType,
|
||||
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 {
|
||||
t.Run(tn, func(t *testing.T) {
|
||||
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,
|
||||
tc.invInfo,
|
||||
tc.resources,
|
||||
validObjs,
|
||||
tc.clusterObjs,
|
||||
poller,
|
||||
)
|
||||
|
|
|
|||
|
|
@ -30,6 +30,7 @@ import (
|
|||
"sigs.k8s.io/cli-utils/pkg/apply/poller"
|
||||
"sigs.k8s.io/cli-utils/pkg/common"
|
||||
"sigs.k8s.io/cli-utils/pkg/inventory"
|
||||
"sigs.k8s.io/cli-utils/pkg/jsonpath"
|
||||
"sigs.k8s.io/cli-utils/pkg/kstatus/polling"
|
||||
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/object"
|
||||
|
|
@ -457,3 +458,15 @@ func toJSONBytes(t *testing.T, obj runtime.Object) []byte {
|
|||
}
|
||||
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/engine"
|
||||
"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
|
||||
|
|
@ -81,6 +82,9 @@ type DestroyerOptions struct {
|
|||
// PollInterval defines how often we should poll for the status
|
||||
// of resources.
|
||||
PollInterval time.Duration
|
||||
|
||||
// ValidationPolicy defines how to handle invalid objects.
|
||||
ValidationPolicy validation.Policy
|
||||
}
|
||||
|
||||
func setDestroyerDefaults(o *DestroyerOptions) {
|
||||
|
|
@ -115,6 +119,16 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option
|
|||
handleError(eventChannel, err)
|
||||
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...")
|
||||
taskBuilder := &solver.TaskQueueBuilder{
|
||||
Pruner: d.pruner,
|
||||
|
|
@ -122,6 +136,7 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option
|
|||
Mapper: mapper,
|
||||
InvClient: d.invClient,
|
||||
Destroy: true,
|
||||
Collector: vCollector,
|
||||
}
|
||||
opts := solver.Options{
|
||||
Prune: true,
|
||||
|
|
@ -136,15 +151,42 @@ func (d *Destroyer) Run(ctx context.Context, inv inventory.InventoryInfo, option
|
|||
InvPolicy: options.InventoryPolicy,
|
||||
},
|
||||
}
|
||||
|
||||
// Build the ordered set of tasks to execute.
|
||||
taskQueue, err := taskBuilder.
|
||||
taskQueue := taskBuilder.
|
||||
AppendPruneWaitTasks(deleteObjs, deleteFilters, opts).
|
||||
AppendDeleteInvTask(inv, options.DryRunStrategy).
|
||||
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
|
||||
}
|
||||
|
||||
// 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
|
||||
// will be pruned.
|
||||
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.
|
||||
klog.V(4).Infoln("destroyer building TaskStatusRunner...")
|
||||
deleteIds := object.UnstructuredSetToObjMetadataSet(deleteObjs)
|
||||
resourceCache := cache.NewResourceCacheMap()
|
||||
runner := taskrunner.NewTaskStatusRunner(deleteIds, d.StatusPoller)
|
||||
taskContext := taskrunner.NewTaskContext(eventChannel, resourceCache)
|
||||
klog.V(4).Infoln("destroyer running TaskStatusRunner...")
|
||||
err = runner.Run(ctx, taskContext, taskQueue.ToChannel(), taskrunner.Options{
|
||||
UseCache: true,
|
||||
|
|
|
|||
|
|
@ -25,6 +25,7 @@ const (
|
|||
PruneType
|
||||
DeleteType
|
||||
WaitType
|
||||
ValidationType
|
||||
)
|
||||
|
||||
// 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 WaitEvent
|
||||
|
||||
// ValidationEvent contains information about validation errors.
|
||||
ValidationEvent ValidationEvent
|
||||
}
|
||||
|
||||
// String returns a string suitable for logging
|
||||
|
|
@ -88,6 +92,8 @@ func (e Event) String() string {
|
|||
sb.WriteString(e.DeleteEvent.String())
|
||||
case WaitType:
|
||||
sb.WriteString(e.WaitEvent.String())
|
||||
case ValidationType:
|
||||
sb.WriteString(e.ValidationEvent.String())
|
||||
}
|
||||
sb.WriteString(" }")
|
||||
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 }",
|
||||
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[DeleteType-6]
|
||||
_ = 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 {
|
||||
if i < 0 || i >= Type(len(_Type_index)-1) {
|
||||
|
|
|
|||
|
|
@ -410,7 +410,7 @@ func TestMutate(t *testing.T) {
|
|||
reason: "",
|
||||
// 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): ` +
|
||||
`failed to parse apply-time-mutation annotation: ` +
|
||||
`invalid "config.kubernetes.io/apply-time-mutation" annotation: ` +
|
||||
`error unmarshaling JSON: ` +
|
||||
`while decoding JSON: ` +
|
||||
`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/object"
|
||||
"sigs.k8s.io/cli-utils/pkg/object/graph"
|
||||
"sigs.k8s.io/cli-utils/pkg/object/validation"
|
||||
)
|
||||
|
||||
type TaskQueueBuilder struct {
|
||||
|
|
@ -41,6 +42,9 @@ type TaskQueueBuilder struct {
|
|||
Factory util.Factory
|
||||
Mapper meta.RESTMapper
|
||||
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
|
||||
// as well (possibly) the inventory namespace.
|
||||
Destroy bool
|
||||
|
|
@ -52,7 +56,6 @@ type TaskQueueBuilder struct {
|
|||
waitCounter int
|
||||
pruneCounter int
|
||||
tasks []taskrunner.Task
|
||||
err error
|
||||
}
|
||||
|
||||
type TaskQueue struct {
|
||||
|
|
@ -90,23 +93,17 @@ type Options struct {
|
|||
InventoryPolicy inventory.InventoryPolicy
|
||||
}
|
||||
|
||||
// Build returns the queue of tasks that have been created.
|
||||
// TODO(seans): Now that we're reporting errors, we probably
|
||||
// want to move away from the Builder patter for the TaskBuilder.
|
||||
func (t *TaskQueueBuilder) Build() (*TaskQueue, error) {
|
||||
if t.err != nil {
|
||||
return nil, t.err
|
||||
}
|
||||
return &TaskQueue{
|
||||
tasks: t.tasks,
|
||||
}, nil
|
||||
// Build returns the queue of tasks that have been created
|
||||
func (t *TaskQueueBuilder) Build() *TaskQueue {
|
||||
return &TaskQueue{tasks: t.tasks}
|
||||
}
|
||||
|
||||
// AppendInvAddTask appends an inventory add task to the task queue.
|
||||
// Returns a pointer to the Builder to chain function calls.
|
||||
func (t *TaskQueueBuilder) AppendInvAddTask(inv inventory.InventoryInfo, applyObjs object.UnstructuredSet,
|
||||
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{
|
||||
TaskName: fmt.Sprintf("inventory-add-%d", t.invAddCounter),
|
||||
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.
|
||||
func (t *TaskQueueBuilder) AppendApplyTask(applyObjs object.UnstructuredSet,
|
||||
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))
|
||||
t.tasks = append(t.tasks, &task.ApplyTask{
|
||||
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.
|
||||
func (t *TaskQueueBuilder) AppendWaitTask(waitIds object.ObjMetadataSet, condition taskrunner.Condition,
|
||||
waitTimeout time.Duration) *TaskQueueBuilder {
|
||||
waitIds = t.Collector.FilterInvalidIds(waitIds)
|
||||
klog.V(2).Infoln("adding wait task")
|
||||
t.tasks = append(t.tasks, taskrunner.NewWaitTask(
|
||||
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.
|
||||
func (t *TaskQueueBuilder) AppendPruneTask(pruneObjs object.UnstructuredSet,
|
||||
pruneFilters []filter.ValidationFilter, o Options) *TaskQueueBuilder {
|
||||
pruneObjs = t.Collector.FilterInvalidObjects(pruneObjs)
|
||||
klog.V(2).Infof("adding prune task (%d objects)", len(pruneObjs))
|
||||
t.tasks = append(t.tasks,
|
||||
&task.PruneTask{
|
||||
|
|
@ -213,9 +213,13 @@ func (t *TaskQueueBuilder) AppendApplyWaitTasks(applyObjs object.UnstructuredSet
|
|||
// objects to apply into sets using a topological sort.
|
||||
applySets, err := graph.SortObjs(applyObjs)
|
||||
if err != nil {
|
||||
t.err = err
|
||||
t.Collector.Collect(err)
|
||||
}
|
||||
for _, applySet := range applySets {
|
||||
applySet = t.Collector.FilterInvalidObjects(applySet)
|
||||
if len(applySet) == 0 {
|
||||
continue
|
||||
}
|
||||
t.AppendApplyTask(applySet, applyFilters, applyMutators, o)
|
||||
// dry-run skips wait tasks
|
||||
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.
|
||||
pruneSets, err := graph.ReverseSortObjs(pruneObjs)
|
||||
if err != nil {
|
||||
t.err = err
|
||||
t.Collector.Collect(err)
|
||||
}
|
||||
for _, pruneSet := range pruneSets {
|
||||
pruneSet = t.Collector.FilterInvalidObjects(pruneSet)
|
||||
if len(pruneSet) == 0 {
|
||||
continue
|
||||
}
|
||||
t.AppendPruneTask(pruneSet, pruneFilters, o)
|
||||
// dry-run skips wait tasks
|
||||
if !o.DryRunStrategy.ClientOrServerDryRun() {
|
||||
|
|
|
|||
|
|
@ -422,19 +422,22 @@ func TestTaskQueueBuilder_AppendApplyWaitTasks(t *testing.T) {
|
|||
|
||||
applyIds := object.UnstructuredSetToObjMetadataSet(tc.applyObjs)
|
||||
fakeInvClient := inventory.NewFakeInventoryClient(applyIds)
|
||||
vCollector := &validation.Collector{}
|
||||
tqb := TaskQueueBuilder{
|
||||
Pruner: pruner,
|
||||
Mapper: mapper,
|
||||
InvClient: fakeInvClient,
|
||||
Collector: vCollector,
|
||||
}
|
||||
var filters []filter.ValidationFilter
|
||||
var mutators []mutator.Interface
|
||||
tq, err := tqb.AppendApplyWaitTasks(
|
||||
tq := tqb.AppendApplyWaitTasks(
|
||||
tc.applyObjs,
|
||||
filters,
|
||||
mutators,
|
||||
tc.options,
|
||||
).Build()
|
||||
err := vCollector.ToError()
|
||||
if tc.expectedError != nil {
|
||||
assert.EqualError(t, err, tc.expectedError.Error())
|
||||
return
|
||||
|
|
@ -699,7 +702,7 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
"cyclic dependency returns error": {
|
||||
"cyclic dependency": {
|
||||
pruneObjs: []*unstructured.Unstructured{
|
||||
testutil.Unstructured(t, resources["deployment"],
|
||||
testutil.AddDependsOn(t, testutil.ToIdentifier(t, resources["secret"]))),
|
||||
|
|
@ -725,6 +728,48 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) {
|
|||
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 {
|
||||
|
|
@ -742,13 +787,16 @@ func TestTaskQueueBuilder_AppendPruneWaitTasks(t *testing.T) {
|
|||
|
||||
pruneIds := object.UnstructuredSetToObjMetadataSet(tc.pruneObjs)
|
||||
fakeInvClient := inventory.NewFakeInventoryClient(pruneIds)
|
||||
vCollector := &validation.Collector{}
|
||||
tqb := TaskQueueBuilder{
|
||||
Pruner: pruner,
|
||||
Mapper: mapper,
|
||||
InvClient: fakeInvClient,
|
||||
Collector: vCollector,
|
||||
}
|
||||
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 {
|
||||
assert.EqualError(t, err, tc.expectedError.Error())
|
||||
return
|
||||
|
|
|
|||
|
|
@ -105,6 +105,12 @@ func (i *InvSetTask) Start(taskContext *taskrunner.TaskContext) {
|
|||
klog.V(4).Infof("remove from inventory %d abandoned objects", len(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))
|
||||
err := i.InvClient.Replace(i.InvInfo, invObjs, i.DryRun)
|
||||
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package task
|
|||
import (
|
||||
"testing"
|
||||
|
||||
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/cache"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/taskrunner"
|
||||
|
|
@ -14,10 +15,18 @@ import (
|
|||
"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) {
|
||||
id1 := object.UnstructuredToObjMetadata(obj1)
|
||||
id2 := object.UnstructuredToObjMetadata(obj2)
|
||||
id3 := object.UnstructuredToObjMetadata(obj3)
|
||||
idInvalid := object.UnstructuredToObjMetadata(objInvalid)
|
||||
|
||||
tests := map[string]struct {
|
||||
prevInventory object.ObjMetadataSet
|
||||
|
|
@ -27,28 +36,23 @@ func TestInvSetTask(t *testing.T) {
|
|||
skippedApplies object.ObjMetadataSet
|
||||
skippedDeletes object.ObjMetadataSet
|
||||
abandonedObjs object.ObjMetadataSet
|
||||
invalidObjs object.ObjMetadataSet
|
||||
expectedObjs object.ObjMetadataSet
|
||||
}{
|
||||
"no apply objs, no prune failures; no inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
"one apply objs, no prune failures; one inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{id1},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id1},
|
||||
appliedObjs: object.ObjMetadataSet{id1},
|
||||
expectedObjs: object.ObjMetadataSet{id1},
|
||||
},
|
||||
"no apply objs, one prune failure, in prev inventory; one inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{id1},
|
||||
expectedObjs: object.ObjMetadataSet{id1},
|
||||
},
|
||||
"no apply objs, one prune failure, not in prev inventory; no inventory": {
|
||||
// aritifical use case: prunes come from the inventory
|
||||
prevInventory: object.ObjMetadataSet{},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{id1},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
|
|
@ -66,129 +70,95 @@ func TestInvSetTask(t *testing.T) {
|
|||
expectedObjs: object.ObjMetadataSet{id1, id2, id3},
|
||||
},
|
||||
"no apply objs, no apply failures, no prune failures; no inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{id3},
|
||||
prevInventory: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
"one apply failure not in prev inventory; no inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{id3},
|
||||
prevInventory: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
"one apply obj, one apply failure not in prev inventory; one inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{id2},
|
||||
failedApplies: object.ObjMetadataSet{id3},
|
||||
prevInventory: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id2},
|
||||
},
|
||||
"one apply obj, one apply failure in prev inventory; one inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{id2},
|
||||
failedApplies: object.ObjMetadataSet{id3},
|
||||
prevInventory: object.ObjMetadataSet{id3},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id2, id3},
|
||||
},
|
||||
"one apply obj, two apply failures with one in prev inventory; two inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{id2},
|
||||
failedApplies: object.ObjMetadataSet{id1, id3},
|
||||
prevInventory: object.ObjMetadataSet{id3},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id2, id3},
|
||||
},
|
||||
"three apply failures with two in prev inventory; two inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{id1, id2, id3},
|
||||
prevInventory: object.ObjMetadataSet{id2, id3},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id2, id3},
|
||||
},
|
||||
"three apply failures with three in prev inventory; three inventory": {
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{id1, id2, id3},
|
||||
prevInventory: object.ObjMetadataSet{id2, id3, id1},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id2, id1, id3},
|
||||
},
|
||||
"one skipped apply from prev inventory; one inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedApplies: object.ObjMetadataSet{id1},
|
||||
skippedDeletes: object.ObjMetadataSet{},
|
||||
abandonedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id1},
|
||||
},
|
||||
"one skipped apply, no prev inventory; no inventory": {
|
||||
prevInventory: object.ObjMetadataSet{},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedApplies: object.ObjMetadataSet{id1},
|
||||
skippedDeletes: object.ObjMetadataSet{},
|
||||
abandonedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
"one apply obj, one skipped apply, two prev inventory; two inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1, id2},
|
||||
appliedObjs: object.ObjMetadataSet{id2},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedApplies: object.ObjMetadataSet{id1},
|
||||
skippedDeletes: object.ObjMetadataSet{},
|
||||
abandonedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id1, id2},
|
||||
},
|
||||
"one skipped delete from prev inventory; one inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedApplies: object.ObjMetadataSet{},
|
||||
skippedDeletes: object.ObjMetadataSet{id1},
|
||||
abandonedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id1},
|
||||
},
|
||||
"one apply obj, one skipped delete, two prev inventory; two inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1, id2},
|
||||
appliedObjs: object.ObjMetadataSet{id2},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedApplies: object.ObjMetadataSet{},
|
||||
skippedDeletes: object.ObjMetadataSet{id1},
|
||||
abandonedObjs: object.ObjMetadataSet{},
|
||||
expectedObjs: object.ObjMetadataSet{id1, id2},
|
||||
},
|
||||
"two apply obj, one abandoned, three in prev inventory; two inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id1, id2, id3},
|
||||
appliedObjs: object.ObjMetadataSet{id1, id2},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
abandonedObjs: object.ObjMetadataSet{id3},
|
||||
expectedObjs: object.ObjMetadataSet{id1, id2},
|
||||
},
|
||||
"two abandoned, two in prev inventory; no inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id2, id3},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
abandonedObjs: object.ObjMetadataSet{id2, id3},
|
||||
expectedObjs: object.ObjMetadataSet{},
|
||||
},
|
||||
"same obj skipped delete and abandoned, one in prev inventory; no inventory": {
|
||||
prevInventory: object.ObjMetadataSet{id3},
|
||||
appliedObjs: object.ObjMetadataSet{},
|
||||
failedApplies: object.ObjMetadataSet{},
|
||||
failedDeletes: object.ObjMetadataSet{},
|
||||
skippedDeletes: object.ObjMetadataSet{id3},
|
||||
abandonedObjs: object.ObjMetadataSet{id3},
|
||||
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 {
|
||||
|
|
@ -222,6 +192,9 @@ func TestInvSetTask(t *testing.T) {
|
|||
for _, abandonedObj := range tc.abandonedObjs {
|
||||
context.AddAbandonedObject(abandonedObj)
|
||||
}
|
||||
for _, invalidObj := range tc.invalidObjs {
|
||||
context.AddInvalidObject(invalidObj)
|
||||
}
|
||||
if 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{}),
|
||||
skippedDeletes: 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{}
|
||||
skippedDeletes map[object.ObjMetadata]struct{}
|
||||
abandonedObjects map[object.ObjMetadata]struct{}
|
||||
invalidObjects map[object.ObjMetadata]struct{}
|
||||
}
|
||||
|
||||
func (tc *TaskContext) TaskChannel() chan TaskResult {
|
||||
|
|
@ -211,6 +213,22 @@ func (tc *TaskContext) AbandonedObjects() object.ObjMetadataSet {
|
|||
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
|
||||
// applied. This is captured in the TaskContext so other tasks
|
||||
// running later might use this information.
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import (
|
|||
|
||||
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
|
||||
"k8s.io/klog/v2"
|
||||
"sigs.k8s.io/cli-utils/pkg/object"
|
||||
)
|
||||
|
||||
const (
|
||||
|
|
@ -42,7 +43,10 @@ func ReadAnnotation(u *unstructured.Unstructured) (DependencySet, error) {
|
|||
|
||||
depSet, err := ParseDependencySet(depSetStr)
|
||||
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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -54,10 +54,10 @@ func FormatDependencySet(depSet DependencySet) (string, error) {
|
|||
// Returns the parsed DependencySet or an error if unable to parse.
|
||||
func ParseDependencySet(depsStr string) (DependencySet, error) {
|
||||
objs := DependencySet{}
|
||||
for _, objStr := range strings.Split(depsStr, annotationSeparator) {
|
||||
for i, objStr := range strings.Split(depsStr, annotationSeparator) {
|
||||
obj, err := ParseObjMetadata(objStr)
|
||||
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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
|
||||
"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.
|
||||
// Waiting for external dependencies isn't implemented (yet).
|
||||
if !ids.Contains(dep) {
|
||||
err := fmt.Errorf("invalid %q annotation: dependency not in object set: %s",
|
||||
mutation.Annotation, sub.SourceRef)
|
||||
err := object.InvalidAnnotationError{
|
||||
Annotation: mutation.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: id,
|
||||
To: dep,
|
||||
},
|
||||
},
|
||||
}
|
||||
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
|
||||
}
|
||||
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.
|
||||
// Having duplicates won't break the graph, but skip it anyway.
|
||||
if _, found := seen[dep]; found {
|
||||
// Won't error - already passed validation
|
||||
depStr, _ := dependson.FormatObjMetadata(dep)
|
||||
err := fmt.Errorf("invalid %q annotation: duplicate reference: %s",
|
||||
dependson.Annotation, depStr)
|
||||
err := object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: DuplicateDependencyError{
|
||||
Edge: Edge{
|
||||
From: id,
|
||||
To: dep,
|
||||
},
|
||||
},
|
||||
}
|
||||
objErrors = append(objErrors, err)
|
||||
klog.V(3).Infof("failed to add edges from: %s: %v", id, err)
|
||||
continue
|
||||
|
|
@ -188,10 +199,17 @@ func addDependsOnEdges(g *Graph, objs object.UnstructuredSet, ids object.ObjMeta
|
|||
// Require dependencies to be in the same resource group.
|
||||
// Waiting for external dependencies isn't implemented (yet).
|
||||
if !ids.Contains(dep) {
|
||||
err := fmt.Errorf("invalid %q annotation: dependency not in object set: %s",
|
||||
dependson.Annotation, mutation.ResourceReferenceFromObjMetadata(dep))
|
||||
err := object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: id,
|
||||
To: dep,
|
||||
},
|
||||
},
|
||||
}
|
||||
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
|
||||
}
|
||||
klog.V(3).Infof("adding edge from: %s, to: %s", id, dep)
|
||||
|
|
|
|||
|
|
@ -521,10 +521,12 @@ func TestApplyTimeMutationEdges(t *testing.T) {
|
|||
},
|
||||
expected: []Edge{},
|
||||
expectedError: validation.NewError(
|
||||
errors.New("failed to parse apply-time-mutation annotation: "+
|
||||
"error unmarshaling JSON: "+
|
||||
"while decoding JSON: json: "+
|
||||
"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: mutation.Annotation,
|
||||
Cause: errors.New("error unmarshaling JSON: " +
|
||||
"while decoding JSON: json: " +
|
||||
"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"),
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "apps",
|
||||
|
|
@ -549,9 +551,15 @@ func TestApplyTimeMutationEdges(t *testing.T) {
|
|||
},
|
||||
expected: []Edge{},
|
||||
expectedError: validation.NewError(
|
||||
errors.New(`invalid "config.kubernetes.io/apply-time-mutation" annotation: `+
|
||||
"dependency not in object set: "+
|
||||
"apps/namespaces/test-namespace/Deployment/foo"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: mutation.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
To: testutil.ToIdentifier(t, resources["deployment"]),
|
||||
},
|
||||
},
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "",
|
||||
|
|
@ -590,10 +598,12 @@ func TestApplyTimeMutationEdges(t *testing.T) {
|
|||
expected: []Edge{},
|
||||
expectedError: multierror.New(
|
||||
validation.NewError(
|
||||
errors.New("failed to parse apply-time-mutation annotation: "+
|
||||
"error unmarshaling JSON: "+
|
||||
"while decoding JSON: json: "+
|
||||
"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: mutation.Annotation,
|
||||
Cause: errors.New("error unmarshaling JSON: " +
|
||||
"while decoding JSON: json: " +
|
||||
"cannot unmarshal string into Go value of type mutation.ApplyTimeMutation"),
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "apps",
|
||||
|
|
@ -604,9 +614,15 @@ func TestApplyTimeMutationEdges(t *testing.T) {
|
|||
},
|
||||
),
|
||||
validation.NewError(
|
||||
errors.New(`invalid "config.kubernetes.io/apply-time-mutation" annotation: `+
|
||||
"dependency not in object set: "+
|
||||
"/namespaces/test-namespace/Secret/secret"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: mutation.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
To: testutil.ToIdentifier(t, resources["secret"]),
|
||||
},
|
||||
},
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "",
|
||||
|
|
@ -731,10 +747,11 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
},
|
||||
expected: []Edge{},
|
||||
expectedError: validation.NewError(
|
||||
errors.New("failed to parse depends-on annotation: "+
|
||||
"failed to parse object metadata: "+
|
||||
"expected 3 or 5 fields, found 1: "+
|
||||
`"invalid-obj-ref"`),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: errors.New("failed to parse object reference (index: 0): " +
|
||||
`expected 3 or 5 fields, found 1: "invalid-obj-ref"`),
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "apps",
|
||||
|
|
@ -762,9 +779,15 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
},
|
||||
},
|
||||
expectedError: validation.NewError(
|
||||
errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+
|
||||
"duplicate reference: "+
|
||||
"apps/namespaces/test-namespace/Deployment/foo"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: DuplicateDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
To: testutil.ToIdentifier(t, resources["deployment"]),
|
||||
},
|
||||
},
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "",
|
||||
|
|
@ -775,7 +798,7 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
},
|
||||
),
|
||||
},
|
||||
"error: dependency not in object set": {
|
||||
"error: external dependency": {
|
||||
objs: []*unstructured.Unstructured{
|
||||
testutil.Unstructured(t, resources["pod"],
|
||||
testutil.AddDependsOn(t,
|
||||
|
|
@ -785,9 +808,15 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
},
|
||||
expected: []Edge{},
|
||||
expectedError: validation.NewError(
|
||||
errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+
|
||||
"dependency not in object set: "+
|
||||
"apps/namespaces/test-namespace/Deployment/foo"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
To: testutil.ToIdentifier(t, resources["deployment"]),
|
||||
},
|
||||
},
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "",
|
||||
|
|
@ -822,10 +851,11 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
expected: []Edge{},
|
||||
expectedError: multierror.New(
|
||||
validation.NewError(
|
||||
errors.New("failed to parse depends-on annotation: "+
|
||||
"failed to parse object metadata: "+
|
||||
"expected 3 or 5 fields, found 1: "+
|
||||
`"invalid-obj-ref"`),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: errors.New("failed to parse object reference (index: 0): " +
|
||||
`expected 3 or 5 fields, found 1: "invalid-obj-ref"`),
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "apps",
|
||||
|
|
@ -836,9 +866,15 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
},
|
||||
),
|
||||
validation.NewError(
|
||||
errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+
|
||||
"dependency not in object set: "+
|
||||
"/namespaces/test-namespace/Secret/secret"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
To: testutil.ToIdentifier(t, resources["secret"]),
|
||||
},
|
||||
},
|
||||
},
|
||||
object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "",
|
||||
|
|
@ -862,12 +898,24 @@ func TestAddDependsOnEdges(t *testing.T) {
|
|||
expected: []Edge{},
|
||||
expectedError: validation.NewError(
|
||||
multierror.New(
|
||||
errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+
|
||||
"dependency not in object set: "+
|
||||
"apps/namespaces/test-namespace/Deployment/foo"),
|
||||
errors.New(`invalid "config.kubernetes.io/depends-on" annotation: `+
|
||||
"duplicate reference: "+
|
||||
"apps/namespaces/test-namespace/Deployment/foo"),
|
||||
object.InvalidAnnotationError{
|
||||
Annotation: dependson.Annotation,
|
||||
Cause: ExternalDependencyError{
|
||||
Edge: Edge{
|
||||
From: testutil.ToIdentifier(t, resources["pod"]),
|
||||
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{
|
||||
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
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"sort"
|
||||
|
||||
"sigs.k8s.io/cli-utils/pkg/multierror"
|
||||
"sigs.k8s.io/cli-utils/pkg/object"
|
||||
"sigs.k8s.io/cli-utils/pkg/object/validation"
|
||||
"sigs.k8s.io/cli-utils/pkg/ordering"
|
||||
|
|
@ -25,13 +22,6 @@ type Graph struct {
|
|||
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.
|
||||
func New() *Graph {
|
||||
g := &Graph{}
|
||||
|
|
@ -150,47 +140,3 @@ func (g *Graph) Sort() ([]object.ObjMetadataSet, error) {
|
|||
}
|
||||
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
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
"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/klog/v2"
|
||||
"sigs.k8s.io/cli-utils/pkg/object"
|
||||
"sigs.k8s.io/yaml"
|
||||
)
|
||||
|
||||
|
|
@ -41,7 +42,10 @@ func ReadAnnotation(obj *unstructured.Unstructured) (ApplyTimeMutation, error) {
|
|||
|
||||
err := yaml.Unmarshal([]byte(mutationYaml), &mutation)
|
||||
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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -89,8 +89,18 @@ func (setA ObjMetadataSet) Intersection(setB ObjMetadataSet) ObjMetadataSet {
|
|||
}
|
||||
}
|
||||
intersection := make(ObjMetadataSet, 0, len(mapI))
|
||||
for o := range mapI {
|
||||
intersection = append(intersection, o)
|
||||
// Iterate over setA & setB to retain input order and have stable output
|
||||
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
|
||||
}
|
||||
|
|
@ -105,8 +115,18 @@ func (setA ObjMetadataSet) Union(setB ObjMetadataSet) ObjMetadataSet {
|
|||
m[b] = struct{}{}
|
||||
}
|
||||
union := make(ObjMetadataSet, 0, len(m))
|
||||
for u := range m {
|
||||
union = append(union, u)
|
||||
// Iterate over setA & setB to retain input order and have stable output
|
||||
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
|
||||
}
|
||||
|
|
@ -124,12 +144,22 @@ func (setA ObjMetadataSet) Diff(setB ObjMetadataSet) ObjMetadataSet {
|
|||
}
|
||||
// Create/return slice from the map of remaining items
|
||||
diff := make(ObjMetadataSet, 0, len(m))
|
||||
for r := range m {
|
||||
diff = append(diff, r)
|
||||
// Iterate over setA to retain input order and have stable output
|
||||
for _, id := range setA {
|
||||
if _, ok := m[id]; ok {
|
||||
diff = append(diff, id)
|
||||
delete(m, id)
|
||||
}
|
||||
}
|
||||
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
|
||||
// hashing the result with the 32-bit FNV-1a algorithm.
|
||||
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
|
||||
// always required, such as namespaced resources must have the namespace set.
|
||||
type Validator struct {
|
||||
Mapper meta.RESTMapper
|
||||
Mapper meta.RESTMapper
|
||||
Collector *Collector
|
||||
}
|
||||
|
||||
// Validate validates the provided resources. A RESTMapper will be used
|
||||
// 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)
|
||||
var errs []error
|
||||
for _, obj := range objs {
|
||||
var objErrors []error
|
||||
if err := v.validateKind(obj); err != nil {
|
||||
|
|
@ -35,14 +35,13 @@ func (v *Validator) Validate(objs []*unstructured.Unstructured) error {
|
|||
objErrors = append(objErrors, err)
|
||||
}
|
||||
if len(objErrors) > 0 {
|
||||
errs = append(errs, NewError(multierror.Wrap(objErrors...),
|
||||
object.UnstructuredToObjMetadata(obj)))
|
||||
// one error per object
|
||||
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
|
||||
|
|
|
|||
|
|
@ -260,10 +260,13 @@ metadata:
|
|||
crdGV.WithResource("customresourcedefinition"), meta.RESTScopeRoot)
|
||||
mapper = meta.MultiRESTMapper([]meta.RESTMapper{mapper, crdMapper})
|
||||
|
||||
vCollector := &validation.Collector{}
|
||||
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 {
|
||||
assert.NoError(t, err)
|
||||
return
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import (
|
|||
)
|
||||
|
||||
type Formatter interface {
|
||||
FormatValidationEvent(ve event.ValidationEvent) error
|
||||
FormatApplyEvent(ae event.ApplyEvent) error
|
||||
FormatStatusEvent(se event.StatusEvent) error
|
||||
FormatPruneEvent(pe event.PruneEvent) error
|
||||
|
|
@ -68,6 +69,10 @@ func (b *BaseListPrinter) Print(ch <-chan event.Event, previewStrategy common.Dr
|
|||
case event.ErrorType:
|
||||
_ = formatter.FormatErrorEvent(e.ErrorEvent)
|
||||
return e.ErrorEvent.Err
|
||||
case event.ValidationType:
|
||||
if err := formatter.FormatValidationEvent(e.ValidationEvent); err != nil {
|
||||
return err
|
||||
}
|
||||
case event.ApplyType:
|
||||
if err := formatter.FormatApplyEvent(e.ApplyEvent); err != nil {
|
||||
return err
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ func newCountingFormatter() *countingFormatter {
|
|||
}
|
||||
|
||||
type countingFormatter struct {
|
||||
validationEvent []event.ValidationEvent
|
||||
applyEvents []event.ApplyEvent
|
||||
statusEvents []event.StatusEvent
|
||||
pruneEvents []event.PruneEvent
|
||||
|
|
@ -37,6 +38,11 @@ type countingFormatter struct {
|
|||
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 {
|
||||
c.applyEvents = append(c.applyEvents, e)
|
||||
return nil
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import (
|
|||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/common"
|
||||
"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/stats"
|
||||
)
|
||||
|
|
@ -27,6 +28,36 @@ type formatter struct {
|
|||
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 {
|
||||
gk := ae.Identifier.GroupKind
|
||||
name := ae.Identifier.Name
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package events
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
|
|
@ -11,12 +12,15 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
|
||||
"k8s.io/apimachinery/pkg/runtime/schema"
|
||||
"k8s.io/apimachinery/pkg/util/validation/field"
|
||||
"k8s.io/cli-runtime/pkg/genericclioptions"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/common"
|
||||
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
|
||||
"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"
|
||||
)
|
||||
|
||||
|
|
@ -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 {
|
||||
return &unstructured.Unstructured{
|
||||
Object: map[string]interface{}{
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import (
|
|||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/common"
|
||||
"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/stats"
|
||||
)
|
||||
|
|
@ -27,6 +28,26 @@ type formatter struct {
|
|||
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 {
|
||||
eventInfo := jf.baseResourceEvent(ae.Identifier)
|
||||
if ae.Error != nil {
|
||||
|
|
|
|||
|
|
@ -11,12 +11,15 @@ import (
|
|||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/apimachinery/pkg/runtime/schema"
|
||||
"k8s.io/apimachinery/pkg/util/validation/field"
|
||||
"k8s.io/cli-runtime/pkg/genericclioptions"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/common"
|
||||
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
|
||||
"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/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
|
||||
func assertOutput(t *testing.T, expectedMap map[string]interface{}, actual string) bool {
|
||||
if len(expectedMap) == 0 {
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package table
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
|
|
@ -12,10 +13,13 @@ import (
|
|||
pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
|
||||
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
|
||||
"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/table"
|
||||
)
|
||||
|
||||
const InvalidStatus status.Status = "Invalid"
|
||||
|
||||
func newResourceStateCollector(resourceGroups []event.ActionGroup) *ResourceStateCollector {
|
||||
resourceInfos := make(map[object.ObjMetadata]*ResourceInfo)
|
||||
for _, group := range resourceGroups {
|
||||
|
|
@ -181,6 +185,8 @@ func (r *ResourceStateCollector) processEvent(ev event.Event) error {
|
|||
r.mux.Lock()
|
||||
defer r.mux.Unlock()
|
||||
switch ev.Type {
|
||||
case event.ValidationType:
|
||||
return r.processValidationEvent(ev.ValidationEvent)
|
||||
case event.StatusType:
|
||||
r.processStatusEvent(ev.StatusEvent)
|
||||
case event.ApplyType:
|
||||
|
|
@ -195,6 +201,34 @@ func (r *ResourceStateCollector) processEvent(ev event.Event) error {
|
|||
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
|
||||
// update for a resource.
|
||||
func (r *ResourceStateCollector) processStatusEvent(e event.StatusEvent) {
|
||||
|
|
|
|||
|
|
@ -4,13 +4,17 @@
|
|||
package table
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"k8s.io/apimachinery/pkg/runtime/schema"
|
||||
"k8s.io/apimachinery/pkg/util/validation/field"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/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/graph"
|
||||
"sigs.k8s.io/cli-utils/pkg/object/validation"
|
||||
)
|
||||
|
||||
var (
|
||||
|
|
@ -19,8 +23,16 @@ var (
|
|||
Group: "apps",
|
||||
Kind: "Deployment",
|
||||
},
|
||||
Name: "Foo",
|
||||
Namespace: "Bar",
|
||||
Name: "foo",
|
||||
Namespace: "default",
|
||||
}
|
||||
depID2 = object.ObjMetadata{
|
||||
GroupKind: schema.GroupKind{
|
||||
Group: "apps",
|
||||
Kind: "Deployment",
|
||||
},
|
||||
Name: "bar",
|
||||
Namespace: "default",
|
||||
}
|
||||
customID = object.ObjMetadata{
|
||||
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) {
|
||||
if e.Resource == nil {
|
||||
return object.ObjMetadata{}, false
|
||||
|
|
|
|||
|
|
@ -24,6 +24,7 @@ type ExpEvent struct {
|
|||
PruneEvent *ExpPruneEvent
|
||||
DeleteEvent *ExpDeleteEvent
|
||||
WaitEvent *ExpWaitEvent
|
||||
ValidationEvent *ExpValidationEvent
|
||||
}
|
||||
|
||||
type ExpInitEvent struct {
|
||||
|
|
@ -74,6 +75,11 @@ type ExpWaitEvent struct {
|
|||
Identifier object.ObjMetadata
|
||||
}
|
||||
|
||||
type ExpValidationEvent struct {
|
||||
Identifiers object.ObjMetadataSet
|
||||
Error error
|
||||
}
|
||||
|
||||
func VerifyEvents(expEvents []ExpEvent, events []event.Event) error {
|
||||
if len(expEvents) == 0 && len(events) == 0 {
|
||||
return nil
|
||||
|
|
@ -270,6 +276,24 @@ func isMatch(ee ExpEvent, e event.Event) bool {
|
|||
}
|
||||
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:
|
||||
return true
|
||||
}
|
||||
|
|
@ -364,6 +388,15 @@ func EventToExpEvent(e event.Event) ExpEvent {
|
|||
Operation: e.WaitEvent.Operation,
|
||||
},
|
||||
}
|
||||
|
||||
case event.ValidationType:
|
||||
return ExpEvent{
|
||||
EventType: event.ValidationType,
|
||||
ValidationEvent: &ExpValidationEvent{
|
||||
Identifiers: e.ValidationEvent.Identifiers,
|
||||
Error: e.ValidationEvent.Error,
|
||||
},
|
||||
}
|
||||
}
|
||||
return ExpEvent{}
|
||||
}
|
||||
|
|
@ -433,6 +466,8 @@ func (ape GroupedEventsByID) Less(i, j int) bool {
|
|||
return false
|
||||
}
|
||||
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:
|
||||
// don't change order if not ApplyType, PruneType, or DeleteType
|
||||
return false
|
||||
|
|
|
|||
|
|
@ -143,3 +143,52 @@ spec:
|
|||
- name: tcp
|
||||
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"
|
||||
"context"
|
||||
"fmt"
|
||||
"html/template"
|
||||
"strings"
|
||||
"text/template"
|
||||
"time"
|
||||
|
||||
. "github.com/onsi/ginkgo"
|
||||
|
|
@ -22,6 +22,7 @@ import (
|
|||
"k8s.io/apimachinery/pkg/util/yaml"
|
||||
"sigs.k8s.io/cli-utils/pkg/apply/event"
|
||||
"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/mutation"
|
||||
"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)
|
||||
}
|
||||
|
||||
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 {
|
||||
randomSuffix := common.RandomStr()
|
||||
return fmt.Sprintf("%s%s", prefix, randomSuffix)
|
||||
|
|
@ -280,7 +315,7 @@ func manifestToUnstructured(manifest []byte) *unstructured.Unstructured {
|
|||
u := make(map[string]interface{})
|
||||
err := yaml.Unmarshal(manifest, &u)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
panic(fmt.Errorf("failed to parse manifest yaml: %w", err))
|
||||
}
|
||||
return &unstructured.Unstructured{
|
||||
Object: u,
|
||||
|
|
|
|||
|
|
@ -198,6 +198,14 @@ var _ = Describe("Applier", func() {
|
|||
It("Reconciliation timeout", func() {
|
||||
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() {
|
||||
|
|
|
|||
|
|
@ -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