Merge pull request #463 from karlkfi/karl-wait-printer

feat: send WaitEvent for every resource
This commit is contained in:
Kubernetes Prow Robot 2021-11-04 13:02:23 -07:00 committed by GitHub
commit 8da7837865
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 1807 additions and 334 deletions

View File

@ -106,14 +106,15 @@ func (r *ApplyRunner) RunE(cmd *cobra.Command, args []string) error {
return err
}
// Only print status events if we are waiting for status.
//TODO: This is not the right way to do this. There are situations where
// we do need status events event if we are not waiting for status. The
// printers should be updated to handle this.
var printStatusEvents bool
// Print status events if a wait timeout is specified
if r.reconcileTimeout != time.Duration(0) || r.pruneTimeout != time.Duration(0) {
printStatusEvents = true
}
// Always enable status events for the table printer
if r.output == printers.TablePrinter {
printStatusEvents = true
}
// TODO: Fix DemandOneDirectory to no longer return FileNameFlags
// since we are no longer using them.

View File

@ -114,9 +114,19 @@ func (r *DestroyRunner) RunE(cmd *cobra.Command, args []string) error {
if err != nil {
return err
}
var printStatusEvents bool
// Print status events if a wait timeout is specified
if r.deleteTimeout != time.Duration(0) {
printStatusEvents = true
}
// Always enable status events for the table printer
if r.output == printers.TablePrinter {
printStatusEvents = true
}
// Run the destroyer. It will return a channel where we can receive updates
// to keep track of progress and any issues.
printStatusEvents := r.deleteTimeout != time.Duration(0)
ch := d.Run(ctx, inv, apply.DestroyerOptions{
DeleteTimeout: r.deleteTimeout,
DeletePropagationPolicy: deletePropPolicy,

View File

@ -6,6 +6,7 @@ package apply
import (
"context"
"fmt"
"sync"
"testing"
"time"
@ -514,6 +515,9 @@ func TestApplier(t *testing.T) {
eventChannel := applier.Run(ctx, tc.invInfo.toWrapped(), tc.resources, tc.options)
// only start sending events once
var once sync.Once
var events []event.Event
timer := time.NewTimer(10 * time.Second)
@ -526,12 +530,13 @@ func TestApplier(t *testing.T) {
}
if e.Type == event.ActionGroupType &&
e.ActionGroupEvent.Type == event.Finished {
// If we do not also check for PruneAction, then the tests
// hang, timeout, and fail.
// Send events after the first apply/prune task ends
if e.ActionGroupEvent.Action == event.ApplyAction ||
e.ActionGroupEvent.Action == event.PruneAction {
// start events
poller.Start()
once.Do(func() {
// start events
poller.Start()
})
}
}
events = append(events, e)
@ -679,8 +684,18 @@ func TestApplierCancel(t *testing.T) {
Type: event.Started,
},
},
{
// Deployment reconcile pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
Operation: event.ReconcilePending,
},
},
// Deployment never becomes Current.
// WaitTask is expected to be cancelled before ReconcileTimeout.
// Cancelled WaitTask do not sent individual timeout WaitEvents
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -834,7 +849,24 @@ func TestApplierCancel(t *testing.T) {
Type: event.Started,
},
},
// Deployment becomes Current.
{
// Deployment reconcile pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
Operation: event.ReconcilePending,
},
},
{
// Deployment becomes Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
Operation: event.Reconciled,
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -887,8 +919,8 @@ func TestApplierCancel(t *testing.T) {
eventChannel := applier.Run(runCtx, tc.invInfo.toWrapped(), tc.resources, tc.options)
// Start sending status events
poller.Start()
// only start sending events once
var once sync.Once
var events []event.Event
@ -908,6 +940,18 @@ func TestApplierCancel(t *testing.T) {
break loop
}
events = append(events, e)
if e.Type == event.ActionGroupType &&
e.ActionGroupEvent.Type == event.Finished {
// Send events after the first apply/prune task ends
if e.ActionGroupEvent.Action == event.ApplyAction ||
e.ActionGroupEvent.Action == event.PruneAction {
once.Do(func() {
// start events
poller.Start()
})
}
}
}
}

View File

@ -5,6 +5,7 @@ package apply
import (
"context"
"sync"
"testing"
"time"
@ -130,6 +131,15 @@ func TestDestroyerCancel(t *testing.T) {
Type: event.Started,
},
},
{
// Deployment reconcile pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
},
},
// Deployment never becomes NotFound.
// WaitTask is expected to be cancelled before DeleteTimeout.
{
@ -249,7 +259,24 @@ func TestDestroyerCancel(t *testing.T) {
Type: event.Started,
},
},
// Deployment becomes NotFound.
{
// Deployment reconcile pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
},
},
{
// Deployment confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: testutil.ToIdentifier(t, resources["deployment"]),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -304,9 +331,8 @@ func TestDestroyerCancel(t *testing.T) {
eventChannel := destroyer.Run(runCtx, invInfo, tc.options)
// Start sending status events
poller.Start()
// only start poller once per run
var once sync.Once
var events []event.Event
loop:
@ -325,6 +351,14 @@ func TestDestroyerCancel(t *testing.T) {
break loop
}
events = append(events, e)
if e.Type == event.ActionGroupType &&
e.ActionGroupEvent.Action == event.WaitAction {
once.Do(func() {
// Start sending status events after waiting starts
poller.Start()
})
}
}
}

View File

@ -89,9 +89,20 @@ type ErrorEvent struct {
Err error
}
//go:generate stringer -type=WaitEventOperation -linecomment
type WaitEventOperation int
const (
ReconcilePending WaitEventOperation = iota // Pending
Reconciled
ReconcileSkipped // Skipped
ReconcileTimeout // Timeout
)
type WaitEvent struct {
GroupName string
Error error
GroupName string
Identifier object.ObjMetadata
Operation WaitEventOperation
}
//go:generate stringer -type=ActionGroupEventType

View File

@ -0,0 +1,26 @@
// Code generated by "stringer -type=WaitEventOperation -linecomment"; DO NOT EDIT.
package event
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[ReconcilePending-0]
_ = x[Reconciled-1]
_ = x[ReconcileSkipped-2]
_ = x[ReconcileTimeout-3]
}
const _WaitEventOperation_name = "PendingReconciledSkippedTimeout"
var _WaitEventOperation_index = [...]uint8{0, 7, 17, 24, 31}
func (i WaitEventOperation) String() string {
if i < 0 || i >= WaitEventOperation(len(_WaitEventOperation_index)-1) {
return "WaitEventOperation(" + strconv.FormatInt(int64(i), 10) + ")"
}
return _WaitEventOperation_name[_WaitEventOperation_index[i]:_WaitEventOperation_index[i+1]]
}

View File

@ -235,8 +235,11 @@ func (a *ApplyTask) sendTaskResult(taskContext *taskrunner.TaskContext) {
taskContext.TaskChannel() <- taskrunner.TaskResult{}
}
// ClearTimeout is not supported by the ApplyTask.
func (a *ApplyTask) ClearTimeout() {}
// Cancel is not supported by the ApplyTask.
func (a *ApplyTask) Cancel(_ *taskrunner.TaskContext) {}
// StatusUpdate is not supported by the ApplyTask.
func (a *ApplyTask) StatusUpdate(_ *taskrunner.TaskContext, _ object.ObjMetadata) {}
// mutate loops through the mutator list and executes them on the object.
func (a *ApplyTask) mutate(ctx context.Context, obj *unstructured.Unstructured) error {

View File

@ -49,5 +49,8 @@ func (i *DeleteInvTask) Start(taskContext *taskrunner.TaskContext) {
}()
}
// ClearTimeout is not supported by the DeleteInvTask.
func (i *DeleteInvTask) ClearTimeout() {}
// Cancel is not supported by the DeleteInvTask.
func (i *DeleteInvTask) Cancel(_ *taskrunner.TaskContext) {}
// StatusUpdate is not supported by the DeleteInvTask.
func (i *DeleteInvTask) StatusUpdate(_ *taskrunner.TaskContext, _ object.ObjMetadata) {}

View File

@ -65,8 +65,11 @@ func (i *InvAddTask) Start(taskContext *taskrunner.TaskContext) {
}()
}
// ClearTimeout is not supported by the InvAddTask.
func (i *InvAddTask) ClearTimeout() {}
// Cancel is not supported by the InvAddTask.
func (i *InvAddTask) Cancel(_ *taskrunner.TaskContext) {}
// StatusUpdate is not supported by the InvAddTask.
func (i *InvAddTask) StatusUpdate(_ *taskrunner.TaskContext, _ object.ObjMetadata) {}
// inventoryNamespaceInSet returns the the namespace the passed inventory
// object will be applied to, or nil if this namespace object does not exist

View File

@ -113,5 +113,8 @@ func (i *InvSetTask) Start(taskContext *taskrunner.TaskContext) {
}()
}
// ClearTimeout is not supported by the InvSetTask.
func (i *InvSetTask) ClearTimeout() {}
// Cancel is not supported by the InvSetTask.
func (i *InvSetTask) Cancel(_ *taskrunner.TaskContext) {}
// StatusUpdate is not supported by the InvSetTask.
func (i *InvSetTask) StatusUpdate(_ *taskrunner.TaskContext, _ object.ObjMetadata) {}

View File

@ -76,5 +76,8 @@ func (p *PruneTask) Start(taskContext *taskrunner.TaskContext) {
}()
}
// ClearTimeout is not supported by the PruneTask.
func (p *PruneTask) ClearTimeout() {}
// Cancel is not supported by the PruneTask.
func (p *PruneTask) Cancel(_ *taskrunner.TaskContext) {}
// StatusUpdate is not supported by the PruneTask.
func (p *PruneTask) StatusUpdate(_ *taskrunner.TaskContext, _ object.ObjMetadata) {}

View File

@ -0,0 +1,19 @@
// Copyright 2021 The Kubernetes Authors.
// SPDX-License-Identifier: Apache-2.0
package taskrunner
import (
"os"
"testing"
"k8s.io/klog/v2"
)
// TestMain executes the tests for this package, with optional logging.
// To see all logs, use:
// go test sigs.k8s.io/cli-utils/pkg/apply/taskrunner -v -args -v=5
func TestMain(m *testing.M) {
klog.InitFlags(nil)
os.Exit(m.Run())
}

View File

@ -183,9 +183,7 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
abort = true
abortReason = fmt.Errorf("polling for status failed: %v",
statusEvent.Error)
// If the current task is a wait task, we just set it
// to complete so we can exit the loop as soon as possible.
completeIfWaitTask(currentTask, taskContext)
currentTask.Cancel(taskContext)
continue
}
@ -202,24 +200,23 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
}
}
id := statusEvent.Resource.Identifier
oldStatus := taskContext.ResourceCache().Get(id).Status
newStatus := statusEvent.Resource.Status
// Update the cache to track the latest resource spec & status.
// Status is computed from the resource on-demand.
// Warning: Resource may be nil!
taskContext.ResourceCache().Put(
statusEvent.Resource.Identifier,
cache.ResourceStatus{
Resource: statusEvent.Resource.Resource,
Status: statusEvent.Resource.Status,
StatusMessage: statusEvent.Resource.Message,
},
)
taskContext.ResourceCache().Put(id, cache.ResourceStatus{
Resource: statusEvent.Resource.Resource,
Status: statusEvent.Resource.Status,
StatusMessage: statusEvent.Resource.Message,
})
// If the current task is a wait task, we check whether
// the condition has been met. If so, we complete the task.
if wt, ok := currentTask.(*WaitTask); ok {
if wt.checkCondition(taskContext) {
completeIfWaitTask(currentTask, taskContext)
}
// send a status update to the running task, but only if the status
// has changed and the task is tracking the object.
if oldStatus != newStatus && currentTask.Identifiers().Contains(id) {
currentTask.StatusUpdate(taskContext, id)
}
// A message on the taskChannel means that the current task
// has either completed or failed.
@ -229,7 +226,6 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
// finish, we exit.
// If everything is ok, we fetch and start the next task.
case msg := <-taskContext.TaskChannel():
currentTask.ClearTimeout()
taskContext.EventChannel() <- event.Event{
Type: event.ActionGroupType,
ActionGroupEvent: event.ActionGroupEvent{
@ -257,19 +253,11 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
doneCh = nil // Set doneCh to nil so we don't enter a busy loop.
abort = true
abortReason = ctx.Err() // always non-nil when doneCh is closed
completeIfWaitTask(currentTask, taskContext)
currentTask.Cancel(taskContext)
}
}
}
// completeIfWaitTask checks if the current task is a wait task. If so,
// we invoke the complete function to complete it.
func completeIfWaitTask(currentTask Task, taskContext *TaskContext) {
if wt, ok := currentTask.(*WaitTask); ok {
wt.complete(taskContext)
}
}
// nextTask fetches the latest task from the taskQueue and
// starts it. If the taskQueue is empty, it the second
// return value will be true.
@ -295,20 +283,8 @@ func (b *baseRunner) nextTask(taskQueue chan Task,
},
}
switch st := tsk.(type) {
case *WaitTask:
// The wait tasks need to be handled specifically here. Before
// starting a new wait task, we check if the condition is already
// met. Without this check, a task might end up waiting for
// status events when the condition is in fact already met.
if st.checkCondition(taskContext) {
st.startAndComplete(taskContext)
} else {
st.Start(taskContext)
}
default:
tsk.Start(taskContext)
}
tsk.Start(taskContext)
return tsk, false
}

View File

@ -41,12 +41,11 @@ var (
func TestBaseRunner(t *testing.T) {
testCases := map[string]struct {
tasks []Task
statusEventsDelay time.Duration
statusEvents []pollevent.Event
expectedEventTypes []event.Type
expectedTimedOutResources []TimedOutResource
expectedTimeoutErrorMsg string
tasks []Task
statusEventsDelay time.Duration
statusEvents []pollevent.Event
expectedEventTypes []event.Type
expectedWaitEvents []event.WaitEvent
}{
"wait task runs until condition is met": {
tasks: []Task{
@ -87,13 +86,39 @@ func TestBaseRunner(t *testing.T) {
event.ApplyType,
event.ActionGroupType,
event.ActionGroupType,
event.WaitType, // deployment pending
event.WaitType, // configmap pending
event.StatusType,
event.WaitType, // configmap current
event.StatusType,
event.WaitType, // deployment current
event.ActionGroupType,
event.ActionGroupType,
event.PruneType,
event.ActionGroupType,
},
expectedWaitEvents: []event.WaitEvent{
{
GroupName: "wait",
Identifier: depID,
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.Reconciled,
},
{
GroupName: "wait",
Identifier: depID,
Operation: event.Reconciled,
},
},
},
"wait task times out eventually (Unknown)": {
tasks: []Task{
@ -112,18 +137,35 @@ func TestBaseRunner(t *testing.T) {
},
expectedEventTypes: []event.Type{
event.ActionGroupType,
event.WaitType, // configmap pending
event.WaitType, // deployment pending
event.StatusType,
event.WaitType,
event.WaitType, // configmap current
event.WaitType, // deployment timeout error
event.ActionGroupType,
},
expectedTimedOutResources: []TimedOutResource{
expectedWaitEvents: []event.WaitEvent{
{
GroupName: "wait",
Identifier: depID,
Status: status.UnknownStatus,
Message: "resource not cached",
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.Reconciled,
},
{
GroupName: "wait",
Identifier: depID,
Operation: event.ReconcileTimeout,
},
},
expectedTimeoutErrorMsg: "timeout after 2 seconds waiting for 2 resources ([default_cm__ConfigMap default_dep_apps_Deployment]) to reach condition AllCurrent",
},
"wait task times out eventually (InProgress)": {
tasks: []Task{
@ -149,18 +191,36 @@ func TestBaseRunner(t *testing.T) {
},
expectedEventTypes: []event.Type{
event.ActionGroupType,
event.WaitType, // configmap pending
event.WaitType, // deployment pending
event.StatusType,
event.WaitType, // configmap current
event.StatusType,
event.WaitType,
event.WaitType, // deployment timeout error
event.ActionGroupType,
},
expectedTimedOutResources: []TimedOutResource{
expectedWaitEvents: []event.WaitEvent{
{
GroupName: "wait",
Identifier: depID,
Status: status.InProgressStatus,
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.ReconcilePending,
},
{
GroupName: "wait",
Identifier: cmID,
Operation: event.Reconciled,
},
{
GroupName: "wait",
Identifier: depID,
Operation: event.ReconcileTimeout,
},
},
expectedTimeoutErrorMsg: "timeout after 2 seconds waiting for 2 resources ([default_cm__ConfigMap default_dep_apps_Deployment]) to reach condition AllCurrent",
},
"tasks run in order": {
tasks: []Task{
@ -255,6 +315,7 @@ func TestBaseRunner(t *testing.T) {
if want, got := len(tc.expectedEventTypes), len(events); want != got {
t.Errorf("expected %d events, but got %d", want, got)
}
var waitEvents []event.WaitEvent
for i, e := range events {
expectedEventType := tc.expectedEventTypes[i]
if want, got := expectedEventType, e.Type; want != got {
@ -262,14 +323,10 @@ func TestBaseRunner(t *testing.T) {
want, got)
}
if e.Type == event.WaitType {
err := e.WaitEvent.Error
if timeoutError, ok := err.(*TimeoutError); ok {
assert.ElementsMatch(t, tc.expectedTimedOutResources,
timeoutError.TimedOutResources)
assert.Equal(t, timeoutError.Error(), tc.expectedTimeoutErrorMsg)
}
waitEvents = append(waitEvents, e.WaitEvent)
}
}
assert.Equal(t, tc.expectedWaitEvents, waitEvents)
})
}
}
@ -323,6 +380,7 @@ func TestBaseRunnerCancellation(t *testing.T) {
expectedError: context.Canceled,
expectedEventTypes: []event.Type{
event.ActionGroupType,
event.WaitType, // pending
event.ActionGroupType,
},
},
@ -372,6 +430,7 @@ func TestBaseRunnerCancellation(t *testing.T) {
expectedError: testError,
expectedEventTypes: []event.Type{
event.ActionGroupType,
event.WaitType, // pending
event.ActionGroupType,
},
},
@ -471,4 +530,6 @@ func (f *fakeApplyTask) Start(taskContext *TaskContext) {
}()
}
func (f *fakeApplyTask) ClearTimeout() {}
func (f *fakeApplyTask) Cancel(_ *TaskContext) {}
func (f *fakeApplyTask) StatusUpdate(_ *TaskContext, _ object.ObjMetadata) {}

View File

@ -4,6 +4,7 @@
package taskrunner
import (
"context"
"fmt"
"reflect"
"time"
@ -26,25 +27,20 @@ type Task interface {
Name() string
Action() event.ResourceAction
Identifiers() object.ObjMetadataSet
Start(taskContext *TaskContext)
ClearTimeout()
Start(*TaskContext)
StatusUpdate(*TaskContext, object.ObjMetadata)
Cancel(*TaskContext)
}
// NewWaitTask creates a new wait task where we will wait until
// the resources specifies by ids all meet the specified condition.
func NewWaitTask(name string, ids object.ObjMetadataSet, cond Condition, timeout time.Duration, mapper meta.RESTMapper) *WaitTask {
// Create the token channel and only add one item.
tokenChannel := make(chan struct{}, 1)
tokenChannel <- struct{}{}
return &WaitTask{
name: name,
Ids: ids,
Condition: cond,
Timeout: timeout,
mapper: mapper,
token: tokenChannel,
mapper: mapper,
}
}
@ -71,14 +67,6 @@ type WaitTask struct {
// cancelFunc is a function that will cancel the timeout timer
// on the task.
cancelFunc func()
// token is a channel that is provided a single item when the
// task is created. Goroutines are only allowed to write to the
// taskChannel if they are able to get the item from the channel.
// This makes sure that the task only results in one message on the
// taskChannel, even if the condition is met and the task times out
// at the same time.
token chan struct{}
}
func (w *WaitTask) Name() string {
@ -96,131 +84,178 @@ func (w *WaitTask) Identifiers() object.ObjMetadataSet {
// Start kicks off the task. For the wait task, this just means
// setting up the timeout timer.
func (w *WaitTask) Start(taskContext *TaskContext) {
klog.V(2).Infof("starting wait task (%d objects)", len(w.Ids))
w.setTimer(taskContext)
}
klog.V(2).Infof("starting wait task (name: %q, objects: %d)", w.Name(), len(w.Ids))
// setTimer creates the timer with the timeout value taken from
// the WaitTask struct. Once the timer expires, it will send
// a message on the EventChannel provided in the taskContext.
func (w *WaitTask) setTimer(taskContext *TaskContext) {
timer := time.NewTimer(w.Timeout)
// TODO: inherit context from task runner, passed through the TaskContext
ctx := context.Background()
// use a context wrapper to handle complete/cancel/timeout
if w.Timeout > 0 {
ctx, w.cancelFunc = context.WithTimeout(ctx, w.Timeout)
} else {
ctx, w.cancelFunc = context.WithCancel(ctx)
}
// A goroutine to handle ending the WaitTask.
go func() {
// TODO(mortent): See if there is a better way to do this. This
// solution will cause the goroutine to hang forever if the
// Timeout is cancelled.
<-timer.C
select {
// We only send the TimeoutError to the eventChannel if no one has gotten
// to the token first.
case <-w.token:
err := &TimeoutError{
Identifiers: w.Ids,
Timeout: w.Timeout,
Condition: w.Condition,
// Block until complete/cancel/timeout
<-ctx.Done()
// Err is always non-nil when Done channel is closed.
err := ctx.Err()
klog.V(2).Infof("completing wait task (name: %q)", w.name)
// reset RESTMapper, if a CRD was applied/pruned
foundCRD := false
for _, obj := range w.Ids {
if obj.GroupKind == crdGK {
foundCRD = true
break
}
amendTimeoutError(taskContext, err)
taskContext.EventChannel() <- event.Event{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: w.Name(),
Error: err,
},
}
taskContext.TaskChannel() <- TaskResult{}
}
if foundCRD {
w.resetRESTMapper()
}
switch err {
case context.Canceled:
// happy path - cancelled or completed (not considered an error)
case context.DeadlineExceeded:
// timed out
w.sendTimeoutEvents(taskContext)
default:
return
// shouldn't happen, per context docs
klog.Errorf("wait task stopped with unexpected context error: %v", err)
}
// Done here. signal completion to the task runner
taskContext.TaskChannel() <- TaskResult{}
}()
w.cancelFunc = func() {
timer.Stop()
}
}
func amendTimeoutError(taskContext *TaskContext, err error) {
if timeoutErr, ok := err.(*TimeoutError); ok {
var timedOutResources []TimedOutResource
for _, id := range timeoutErr.Identifiers {
result := taskContext.ResourceCache().Get(id)
if timeoutErr.Condition.Meets(result.Status) {
continue
}
timedOutResources = append(timedOutResources, TimedOutResource{
Identifier: id,
Status: result.Status,
Message: result.StatusMessage,
})
// send initial events for all resources being waited on
for _, id := range w.Ids {
switch {
case w.skipped(taskContext, id):
w.sendEvent(taskContext, id, event.ReconcileSkipped)
case w.reconciledByID(taskContext, id):
w.sendEvent(taskContext, id, event.Reconciled)
default:
w.sendEvent(taskContext, id, event.ReconcilePending)
}
timeoutErr.TimedOutResources = timedOutResources
}
// exit early if all conditions are met
if w.reconciled(taskContext) {
w.cancelFunc()
}
}
// checkCondition checks whether the condition set in the task
// is currently met given the status of resources in the cache.
func (w *WaitTask) checkCondition(taskContext *TaskContext) bool {
func (w *WaitTask) sendEvent(taskContext *TaskContext, id object.ObjMetadata, op event.WaitEventOperation) {
taskContext.EventChannel() <- event.Event{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: w.Name(),
Identifier: id,
Operation: op,
},
}
}
// sendTimeoutEvents sends a timeout event for every pending object that isn't
// reconciled.
func (w *WaitTask) sendTimeoutEvents(taskContext *TaskContext) {
for _, id := range w.pending(taskContext) {
if !w.reconciledByID(taskContext, id) {
w.sendEvent(taskContext, id, event.ReconcileTimeout)
}
}
}
// reconciledByID checks whether the condition set in the task is currently met
// for the specified object given the status of resource in the cache.
func (w *WaitTask) reconciledByID(taskContext *TaskContext, id object.ObjMetadata) bool {
return conditionMet(taskContext, object.ObjMetadataSet{id}, w.Condition)
}
// reconciled checks whether the condition set in the task is currently met
// given the status of resources in the cache.
func (w *WaitTask) reconciled(taskContext *TaskContext) bool {
return conditionMet(taskContext, w.pending(taskContext), w.Condition)
}
// pending returns the set of resources being waited on excluding
// apply/delete failures. This includes resources which are skipped because of
// filtering.
// pending returns the set of resources being waited on (not skipped).
func (w *WaitTask) pending(taskContext *TaskContext) object.ObjMetadataSet {
var ids object.ObjMetadataSet
for _, id := range w.Ids {
if w.Condition == AllCurrent &&
taskContext.IsFailedApply(id) || taskContext.IsSkippedApply(id) {
continue
if !w.skipped(taskContext, id) {
ids = append(ids, id)
}
if w.Condition == AllNotFound &&
taskContext.IsFailedDelete(id) || taskContext.IsSkippedDelete(id) {
continue
}
ids = append(ids, id)
}
return ids
}
// startAndComplete is invoked when the condition is already
// met when the task should be started. In this case there is no
// need to start a timer. So it just sets the cancelFunc and then
// completes the task.
func (w *WaitTask) startAndComplete(taskContext *TaskContext) {
w.cancelFunc = func() {}
w.complete(taskContext)
// skipped returns true if the object failed or was skipped by a preceding
// apply/delete/prune task.
func (w *WaitTask) skipped(taskContext *TaskContext, id object.ObjMetadata) bool {
if w.Condition == AllCurrent &&
taskContext.IsFailedApply(id) || taskContext.IsSkippedApply(id) {
return true
}
if w.Condition == AllNotFound &&
taskContext.IsFailedDelete(id) || taskContext.IsSkippedDelete(id) {
return true
}
return false
}
// complete is invoked by the taskrunner when all the conditions
// for the task has been met, or something has failed so the task
// need to be stopped.
func (w *WaitTask) complete(taskContext *TaskContext) {
var err error
for _, obj := range w.Ids {
if obj.GroupKind == crdGK {
ddRESTMapper, err := extractDeferredDiscoveryRESTMapper(w.mapper)
if err == nil {
ddRESTMapper.Reset()
// We only need to reset once.
break
}
continue
}
// Cancel exits early with a timeout error
func (w *WaitTask) Cancel(_ *TaskContext) {
w.cancelFunc()
}
// StatusUpdate validates whether the update meets the conditions to stop
// the wait task. If the status is for a watched object and that object now
// meets the desired condition, a WaitEvent will be sent before exiting.
func (w *WaitTask) StatusUpdate(taskContext *TaskContext, id object.ObjMetadata) {
if klog.V(5).Enabled() {
status := taskContext.ResourceCache().Get(id).Status
klog.Errorf("status update (object: %q, status: %q)", id, status)
}
select {
// Only do something if we can get the token.
case <-w.token:
go func() {
taskContext.TaskChannel() <- TaskResult{
Err: err,
}
}()
default:
// ignored objects have already had skipped events sent at start
if w.skipped(taskContext, id) {
return
}
// if the condition is met for this object, send a wait event
if w.reconciledByID(taskContext, id) {
taskContext.EventChannel() <- event.Event{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: w.Name(),
Identifier: id,
Operation: event.Reconciled,
},
}
}
// if all conditions are met, complete the wait task
if w.reconciled(taskContext) {
w.cancelFunc()
}
}
// ClearTimeout cancels the timeout for the wait task.
func (w *WaitTask) ClearTimeout() {
w.cancelFunc()
// resetRESTMapper resets the RESTMapper so it can pick up new CRDs.
func (w *WaitTask) resetRESTMapper() {
// TODO: find a way to add/remove mappers without resetting the entire mapper
// Resetting the mapper requires all CRDs to be queried again.
ddRESTMapper, err := extractDeferredDiscoveryRESTMapper(w.mapper)
if err != nil {
if klog.V(4).Enabled() {
klog.Errorf("error resetting RESTMapper: %v", err)
}
}
ddRESTMapper.Reset()
}
// extractDeferredDiscoveryRESTMapper unwraps the provided RESTMapper

View File

@ -4,98 +4,541 @@
package taskrunner
import (
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"sigs.k8s.io/cli-utils/pkg/apply/cache"
"sigs.k8s.io/cli-utils/pkg/apply/event"
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
"sigs.k8s.io/cli-utils/pkg/object"
"sigs.k8s.io/cli-utils/pkg/testutil"
)
func TestWaitTask_TimeoutTriggered(t *testing.T) {
taskName := "wait"
task := NewWaitTask(taskName, object.ObjMetadataSet{}, AllCurrent,
2*time.Second, testutil.NewFakeRESTMapper())
var testDeployment1YAML = `
apiVersion: apps/v1
kind: Deployment
metadata:
name: a
namespace: default
uid: dep-uid-a
generation: 1
spec:
replicas: 1
`
var testDeployment2YAML = `
apiVersion: v1
kind: Deployment
metadata:
name: b
namespace: default
uid: dep-uid-b
generation: 1
spec:
replicas: 2
`
var testDeployment3YAML = `
apiVersion: v1
kind: Deployment
metadata:
name: c
namespace: default
uid: dep-uid-c
generation: 1
spec:
replicas: 3
`
var testDeployment4YAML = `
apiVersion: v1
kind: Deployment
metadata:
name: d
namespace: default
uid: dep-uid-d
generation: 1
spec:
replicas: 4
`
func TestWaitTask_CompleteEventually(t *testing.T) {
testDeployment1ID := testutil.ToIdentifier(t, testDeployment1YAML)
testDeployment1 := testutil.Unstructured(t, testDeployment1YAML)
testDeployment2ID := testutil.ToIdentifier(t, testDeployment2YAML)
testDeployment2 := testutil.Unstructured(t, testDeployment2YAML)
testDeployment3ID := testutil.ToIdentifier(t, testDeployment3YAML)
testDeployment4ID := testutil.ToIdentifier(t, testDeployment4YAML)
ids := object.ObjMetadataSet{
testDeployment1ID,
testDeployment2ID,
testDeployment3ID,
testDeployment4ID,
}
waitTimeout := 2 * time.Second
taskName := "wait-1"
task := NewWaitTask(taskName, ids, AllCurrent,
waitTimeout, testutil.NewFakeRESTMapper())
eventChannel := make(chan event.Event)
resourceCache := cache.NewResourceCacheMap()
taskContext := NewTaskContext(eventChannel, resourceCache)
defer close(eventChannel)
task.Start(taskContext)
// mark deployment 1 & 2 as applied
taskContext.AddSuccessfulApply(testDeployment1ID, "unused", 1)
taskContext.AddSuccessfulApply(testDeployment2ID, "unused", 1)
timer := time.NewTimer(3 * time.Second)
// mark deployment 3 as failed
taskContext.AddFailedApply(testDeployment3ID)
select {
case e := <-taskContext.EventChannel():
if e.Type != event.WaitType {
t.Errorf("expected a WaitType event, but got a %v event", e.Type)
// mark deployment 4 as skipped
taskContext.AddSkippedApply(testDeployment4ID)
// run task async, to let the test collect events
go func() {
// start the task
task.Start(taskContext)
// mark deployment1 as Current
resourceCache.Put(testDeployment1ID, cache.ResourceStatus{
Resource: testDeployment1,
Status: status.CurrentStatus,
})
// tell the WaitTask deployment1 has new status
task.StatusUpdate(taskContext, testDeployment1ID)
// mark deployment2 as InProgress
resourceCache.Put(testDeployment2ID, cache.ResourceStatus{
Resource: testDeployment2,
Status: status.InProgressStatus,
})
// tell the WaitTask deployment2 has new status
task.StatusUpdate(taskContext, testDeployment2ID)
// mark deployment2 as Current
resourceCache.Put(testDeployment2ID, cache.ResourceStatus{
Resource: testDeployment2,
Status: status.CurrentStatus,
})
// tell the WaitTask deployment2 has new status
task.StatusUpdate(taskContext, testDeployment2ID)
}()
// wait for task result
timer := time.NewTimer(5 * time.Second)
receivedEvents := []event.Event{}
loop:
for {
select {
case e := <-taskContext.EventChannel():
receivedEvents = append(receivedEvents, e)
case res := <-taskContext.TaskChannel():
timer.Stop()
assert.NoError(t, res.Err)
break loop
case <-timer.C:
t.Fatalf("timed out waiting for TaskResult")
}
if e.WaitEvent.GroupName != taskName {
t.Errorf("expected WaitEvent.GroupName = %q, but got %q", taskName, e.WaitEvent.GroupName)
}
err := e.WaitEvent.Error
if _, ok := IsTimeoutError(err); !ok {
t.Errorf("expected timeout error, but got %v", err)
}
return
case <-timer.C:
t.Errorf("expected timeout to trigger, but it didn't")
}
// Expect an event for every object (sorted).
expectedEvents := []event.Event{
// skipped/reconciled/pending events first, in the order provided to the WaitTask
// deployment1 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment1ID,
Operation: event.ReconcilePending,
},
},
// deployment2 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment2ID,
Operation: event.ReconcilePending,
},
},
// deployment3 skipped
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment3ID,
Operation: event.ReconcileSkipped,
},
},
// deployment4 skipped
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment4ID,
Operation: event.ReconcileSkipped,
},
},
// current events next, in the order of status updates
// deployment1 current
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment1ID,
Operation: event.Reconciled,
},
},
// deployment2 current
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment2ID,
Operation: event.Reconciled,
},
},
}
testutil.AssertEqual(t, receivedEvents, expectedEvents)
}
func TestWaitTask_TimeoutCancelled(t *testing.T) {
task := NewWaitTask("wait", object.ObjMetadataSet{}, AllCurrent,
2*time.Second, testutil.NewFakeRESTMapper())
func TestWaitTask_Timeout(t *testing.T) {
testDeployment1ID := testutil.ToIdentifier(t, testDeployment1YAML)
testDeployment1 := testutil.Unstructured(t, testDeployment1YAML)
testDeployment2ID := testutil.ToIdentifier(t, testDeployment2YAML)
testDeployment3ID := testutil.ToIdentifier(t, testDeployment3YAML)
testDeployment4ID := testutil.ToIdentifier(t, testDeployment4YAML)
ids := object.ObjMetadataSet{
testDeployment1ID,
testDeployment2ID,
testDeployment3ID,
testDeployment4ID,
}
waitTimeout := 2 * time.Second
taskName := "wait-2"
task := NewWaitTask(taskName, ids, AllCurrent,
waitTimeout, testutil.NewFakeRESTMapper())
eventChannel := make(chan event.Event)
resourceCache := cache.NewResourceCacheMap()
taskContext := NewTaskContext(eventChannel, resourceCache)
defer close(eventChannel)
task.Start(taskContext)
task.ClearTimeout()
timer := time.NewTimer(3 * time.Second)
// mark deployment 1 & 2 as applied
taskContext.AddSuccessfulApply(testDeployment1ID, "unused", 1)
taskContext.AddSuccessfulApply(testDeployment2ID, "unused", 1)
select {
case res := <-taskContext.TaskChannel():
t.Errorf("didn't expect timeout error, but got %v", res.Err)
case <-timer.C:
return
// mark deployment 3 as failed
taskContext.AddFailedApply(testDeployment3ID)
// mark deployment 4 as skipped
taskContext.AddSkippedApply(testDeployment4ID)
// run task async, to let the test collect events
go func() {
// start the task
task.Start(taskContext)
// mark deployment1 as Current
resourceCache.Put(testDeployment1ID, cache.ResourceStatus{
Resource: testDeployment1,
Status: status.CurrentStatus,
})
// tell the WaitTask deployment1 has new status
task.StatusUpdate(taskContext, testDeployment1ID)
}()
// wait for task result
timer := time.NewTimer(5 * time.Second)
receivedEvents := []event.Event{}
loop:
for {
select {
case e := <-taskContext.EventChannel():
receivedEvents = append(receivedEvents, e)
case res := <-taskContext.TaskChannel():
timer.Stop()
assert.NoError(t, res.Err)
break loop
case <-timer.C:
t.Fatalf("timed out waiting for TaskResult")
}
}
// Expect an event for every object (sorted).
expectedEvents := []event.Event{
// skipped/reconciled/pending events first, in the order provided to the WaitTask
// deployment1 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment1ID,
Operation: event.ReconcilePending,
},
},
// deployment2 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment2ID,
Operation: event.ReconcilePending,
},
},
// deployment3 skipped
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment3ID,
Operation: event.ReconcileSkipped,
},
},
// deployment4 skipped
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment4ID,
Operation: event.ReconcileSkipped,
},
},
// current events next, in the order of status updates
// deployment1 current
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment1ID,
Operation: event.Reconciled,
},
},
// timeout events last, in the order provided to the WaitTask
// deployment2 timeout
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeployment2ID,
Operation: event.ReconcileTimeout,
},
},
}
testutil.AssertEqual(t, receivedEvents, expectedEvents)
}
func TestWaitTask_StartAndComplete(t *testing.T) {
testDeploymentID := testutil.ToIdentifier(t, testDeployment1YAML)
testDeployment := testutil.Unstructured(t, testDeployment1YAML)
ids := object.ObjMetadataSet{
testDeploymentID,
}
waitTimeout := 2 * time.Second
taskName := "wait-3"
task := NewWaitTask(taskName, ids, AllCurrent,
waitTimeout, testutil.NewFakeRESTMapper())
eventChannel := make(chan event.Event)
resourceCache := cache.NewResourceCacheMap()
taskContext := NewTaskContext(eventChannel, resourceCache)
defer close(eventChannel)
// mark the deployment as applied
appliedGeneration := int64(1)
taskContext.AddSuccessfulApply(testDeploymentID, "unused", appliedGeneration)
// mark the deployment as Current before starting
resourceCache.Put(testDeploymentID, cache.ResourceStatus{
Resource: testDeployment,
Status: status.CurrentStatus,
})
// run task async, to let the test collect events
go func() {
// start the task
task.Start(taskContext)
}()
// wait for first task result
timer := time.NewTimer(5 * time.Second)
receivedEvents := []event.Event{}
loop:
for {
select {
case e := <-taskContext.EventChannel():
receivedEvents = append(receivedEvents, e)
case res := <-taskContext.TaskChannel():
timer.Stop()
assert.NoError(t, res.Err)
break loop
case <-timer.C:
t.Fatalf("timed out waiting for TaskResult")
}
}
expectedEvents := []event.Event{
// deployment1 current (no pending event when Current before start)
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeploymentID,
Operation: event.Reconciled,
},
},
}
testutil.AssertEqual(t, receivedEvents, expectedEvents)
}
func TestWaitTask_Cancel(t *testing.T) {
testDeploymentID := testutil.ToIdentifier(t, testDeployment1YAML)
ids := object.ObjMetadataSet{
testDeploymentID,
}
waitTimeout := 5 * time.Second
taskName := "wait-4"
task := NewWaitTask(taskName, ids, AllCurrent,
waitTimeout, testutil.NewFakeRESTMapper())
eventChannel := make(chan event.Event)
resourceCache := cache.NewResourceCacheMap()
taskContext := NewTaskContext(eventChannel, resourceCache)
defer close(eventChannel)
// run task async, to let the test collect events
go func() {
// start the task
task.Start(taskContext)
// wait a bit
time.Sleep(1 * time.Second)
// cancel immediately (simulate context cancel from baseRunner)
task.Cancel(taskContext)
}()
// wait for first task result
timer := time.NewTimer(10 * time.Second)
receivedEvents := []event.Event{}
loop:
for {
select {
case e := <-taskContext.EventChannel():
receivedEvents = append(receivedEvents, e)
case res := <-taskContext.TaskChannel():
timer.Stop()
assert.NoError(t, res.Err)
break loop
case <-timer.C:
t.Fatalf("timed out waiting for TaskResult")
}
}
// no timeout events sent on cancel
expectedEvents := []event.Event{
// skipped/reconciled/pending events first, in the order provided to the WaitTask
// deployment1 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeploymentID,
Operation: event.ReconcilePending,
},
},
}
testutil.AssertEqual(t, receivedEvents, expectedEvents)
}
func TestWaitTask_SingleTaskResult(t *testing.T) {
task := NewWaitTask("wait", object.ObjMetadataSet{}, AllCurrent,
2*time.Second, testutil.NewFakeRESTMapper())
testDeploymentID := testutil.ToIdentifier(t, testDeployment1YAML)
testDeployment := testutil.Unstructured(t, testDeployment1YAML)
ids := object.ObjMetadataSet{
testDeploymentID,
}
waitTimeout := 3 * time.Second
taskName := "wait-5"
task := NewWaitTask(taskName, ids, AllCurrent,
waitTimeout, testutil.NewFakeRESTMapper())
eventChannel := make(chan event.Event)
// buffer events, because they're sent by StatusUpdate
eventChannel := make(chan event.Event, 10)
resourceCache := cache.NewResourceCacheMap()
taskContext := NewTaskContext(eventChannel, resourceCache)
taskContext.taskChannel = make(chan TaskResult, 10)
defer close(eventChannel)
var completeWg sync.WaitGroup
// mark the deployment as applied
appliedGeneration := int64(1)
taskContext.AddSuccessfulApply(testDeploymentID, "unused", appliedGeneration)
// run task async, to let the test collect events
go func() {
// start the task
task.Start(taskContext)
// wait a bit
time.Sleep(1 * time.Second)
// mark the deployment as Current
resourceCache.Put(testDeploymentID, cache.ResourceStatus{
Resource: withGeneration(testDeployment, appliedGeneration),
Status: status.CurrentStatus,
})
// send multiple status updates
for i := 0; i < 10; i++ {
task.StatusUpdate(taskContext, testDeploymentID)
}
}()
// wait for timeout
timer := time.NewTimer(5 * time.Second)
receivedEvents := []event.Event{}
receivedResults := []TaskResult{}
loop:
for {
select {
case e := <-taskContext.EventChannel():
receivedEvents = append(receivedEvents, e)
case res := <-taskContext.TaskChannel():
receivedResults = append(receivedResults, res)
case <-timer.C:
break loop
}
}
expectedEvents := []event.Event{
// skipped/reconciled/pending events first, in the order provided to the WaitTask
// deployment1 pending
{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeploymentID,
Operation: event.ReconcilePending,
},
},
}
// Expect an event for every call to StatusUpdate,
// because the object is already Current.
for i := 0; i < 10; i++ {
completeWg.Add(1)
go func() {
defer completeWg.Done()
task.complete(taskContext)
}()
expectedEvents = append(expectedEvents, event.Event{
Type: event.WaitType,
WaitEvent: event.WaitEvent{
GroupName: taskName,
Identifier: testDeploymentID,
Operation: event.Reconciled,
},
})
}
completeWg.Wait()
assert.Equal(t, expectedEvents, receivedEvents)
<-taskContext.TaskChannel()
timer := time.NewTimer(4 * time.Second)
select {
case <-taskContext.TaskChannel():
t.Errorf("expected only one result on taskChannel, but got more")
case <-timer.C:
return
expectedResults := []TaskResult{
{}, // Empty result means success
}
assert.Equal(t, expectedResults, receivedResults)
}

View File

@ -16,8 +16,17 @@ type Formatter interface {
FormatStatusEvent(se event.StatusEvent) error
FormatPruneEvent(pe event.PruneEvent) error
FormatDeleteEvent(de event.DeleteEvent) error
FormatWaitEvent(we event.WaitEvent) error
FormatErrorEvent(ee event.ErrorEvent) error
FormatActionGroupEvent(age event.ActionGroupEvent, ags []event.ActionGroup, as *ApplyStats, ps *PruneStats, ds *DeleteStats, c Collector) error
FormatActionGroupEvent(
age event.ActionGroupEvent,
ags []event.ActionGroup,
as *ApplyStats,
ps *PruneStats,
ds *DeleteStats,
ws *WaitStats,
c Collector,
) error
}
type FormatterFactory func(previewStrategy common.DryRunStrategy) Formatter
@ -94,6 +103,24 @@ func (d *DeleteStats) incFailed() {
d.Failed++
}
type WaitStats struct {
Reconciled int
Timeout int
Skipped int
}
func (w *WaitStats) incReconciled() {
w.Reconciled++
}
func (w *WaitStats) incTimeout() {
w.Timeout++
}
func (w *WaitStats) incSkipped() {
w.Skipped++
}
type Collector interface {
LatestStatus() map[object.ObjMetadata]event.StatusEvent
}
@ -120,6 +147,7 @@ func (b *BaseListPrinter) Print(ch <-chan event.Event, previewStrategy common.Dr
applyStats := &ApplyStats{}
pruneStats := &PruneStats{}
deleteStats := &DeleteStats{}
waitStats := &WaitStats{}
statusCollector := &StatusCollector{
latestStatus: make(map[object.ObjMetadata]event.StatusEvent),
}
@ -172,18 +200,45 @@ func (b *BaseListPrinter) Print(ch <-chan event.Event, previewStrategy common.Dr
if err := formatter.FormatDeleteEvent(e.DeleteEvent); err != nil {
return err
}
case event.WaitType:
switch e.WaitEvent.Operation {
case event.Reconciled:
waitStats.incReconciled()
case event.ReconcileSkipped:
waitStats.incSkipped()
case event.ReconcileTimeout:
waitStats.incTimeout()
}
if err := formatter.FormatWaitEvent(e.WaitEvent); err != nil {
return err
}
case event.ActionGroupType:
if err := formatter.FormatActionGroupEvent(e.ActionGroupEvent, actionGroups, applyStats,
pruneStats, deleteStats, statusCollector); err != nil {
if err := formatter.FormatActionGroupEvent(
e.ActionGroupEvent,
actionGroups,
applyStats,
pruneStats,
deleteStats,
waitStats,
statusCollector,
); err != nil {
return err
}
}
}
failedSum := applyStats.Failed + pruneStats.Failed + deleteStats.Failed
if failedSum > 0 {
switch {
case failedSum > 0 && waitStats.Timeout > 0:
return fmt.Errorf("%d resources failed, %d resources failed to reconcile before timeout",
failedSum, waitStats.Timeout)
case failedSum > 0:
return fmt.Errorf("%d resources failed", failedSum)
case waitStats.Timeout > 0:
return fmt.Errorf("%d resources failed to reconcile before timeout",
waitStats.Timeout)
default:
return nil
}
return nil
}
func ActionGroupByName(name string, ags []event.ActionGroup) (event.ActionGroup, bool) {

View File

@ -82,12 +82,36 @@ func (ef *formatter) FormatDeleteEvent(de event.DeleteEvent) error {
return nil
}
func (ef *formatter) FormatWaitEvent(we event.WaitEvent) error {
gk := we.Identifier.GroupKind
name := we.Identifier.Name
switch we.Operation {
case event.ReconcilePending:
ef.print("%s reconcile pending", resourceIDToString(gk, name))
case event.Reconciled:
ef.print("%s reconciled", resourceIDToString(gk, name))
case event.ReconcileSkipped:
ef.print("%s reconcile skipped", resourceIDToString(gk, name))
case event.ReconcileTimeout:
ef.print("%s reconcile timeout", resourceIDToString(gk, name))
}
return nil
}
func (ef *formatter) FormatErrorEvent(_ event.ErrorEvent) error {
return nil
}
func (ef *formatter) FormatActionGroupEvent(age event.ActionGroupEvent, ags []event.ActionGroup,
as *list.ApplyStats, ps *list.PruneStats, ds *list.DeleteStats, c list.Collector) error {
func (ef *formatter) FormatActionGroupEvent(
age event.ActionGroupEvent,
ags []event.ActionGroup,
as *list.ApplyStats,
ps *list.PruneStats,
ds *list.DeleteStats,
ws *list.WaitStats,
c list.Collector,
) error {
if age.Action == event.ApplyAction &&
age.Type == event.Finished &&
list.IsLastActionGroup(age, ags) {
@ -126,6 +150,12 @@ func (ef *formatter) FormatActionGroupEvent(age event.ActionGroupEvent, ags []ev
}
}
}
if age.Action == event.WaitAction &&
age.Type == event.Finished &&
list.IsLastActionGroup(age, ags) {
ef.print("%d resource(s) reconciled, %d skipped", ws.Reconciled, ds.Skipped)
}
return nil
}

View File

@ -215,6 +215,109 @@ func TestFormatter_FormatDeleteEvent(t *testing.T) {
}
}
func TestFormatter_FormatWaitEvent(t *testing.T) {
testCases := map[string]struct {
previewStrategy common.DryRunStrategy
event event.WaitEvent
waitStats *list.WaitStats
statusCollector list.Collector
expected string
}{
"resource reconciled": {
previewStrategy: common.DryRunNone,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconciled",
},
"resource reconciled (client-side dry-run)": {
previewStrategy: common.DryRunClient,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconciled (preview)",
},
"resource reconciled (server-side dry-run)": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconciled (preview-server)",
},
"resource reconcile timeout": {
previewStrategy: common.DryRunNone,
event: event.WaitEvent{
GroupName: "wait-1",
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
Operation: event.ReconcileTimeout,
},
expected: "deployment.apps/my-dep reconcile timeout",
},
"resource reconcile timeout (client-side dry-run)": {
previewStrategy: common.DryRunClient,
event: event.WaitEvent{
GroupName: "wait-1",
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
Operation: event.ReconcileTimeout,
},
expected: "deployment.apps/my-dep reconcile timeout (preview)",
},
"resource reconcile timeout (server-side dry-run)": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
Operation: event.ReconcileTimeout,
},
expected: "deployment.apps/my-dep reconcile timeout (preview-server)",
},
"resource reconcile skipped": {
previewStrategy: common.DryRunNone,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcileSkipped,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconcile skipped",
},
"resource reconcile skipped (client-side dry-run)": {
previewStrategy: common.DryRunClient,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcileSkipped,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconcile skipped (preview)",
},
"resource reconcile skipped (server-side dry-run)": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcileSkipped,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: "deployment.apps/my-dep reconcile skipped (preview-server)",
},
}
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.FormatWaitEvent(tc.event)
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{}{

View File

@ -69,14 +69,27 @@ func (jf *formatter) FormatDeleteEvent(de event.DeleteEvent) error {
return jf.printEvent("delete", "resourceDeleted", eventInfo)
}
func (jf *formatter) FormatWaitEvent(we event.WaitEvent) error {
eventInfo := jf.baseResourceEvent(we.Identifier)
eventInfo["operation"] = we.Operation.String()
return jf.printEvent("wait", "resourceReconciled", eventInfo)
}
func (jf *formatter) FormatErrorEvent(ee event.ErrorEvent) error {
return jf.printEvent("error", "error", map[string]interface{}{
"error": ee.Err.Error(),
})
}
func (jf *formatter) FormatActionGroupEvent(age event.ActionGroupEvent, ags []event.ActionGroup,
as *list.ApplyStats, ps *list.PruneStats, ds *list.DeleteStats, c list.Collector) error {
func (jf *formatter) FormatActionGroupEvent(
age event.ActionGroupEvent,
ags []event.ActionGroup,
as *list.ApplyStats,
ps *list.PruneStats,
ds *list.DeleteStats,
ws *list.WaitStats,
c list.Collector,
) error {
if age.Action == event.ApplyAction && age.Type == event.Finished {
if err := jf.printEvent("apply", "completed", map[string]interface{}{
"count": as.Sum(),
@ -119,6 +132,15 @@ func (jf *formatter) FormatActionGroupEvent(age event.ActionGroupEvent, ags []ev
}
}
}
if age.Action == event.WaitAction && age.Type == event.Finished {
return jf.printEvent("wait", "completed", map[string]interface{}{
"reconciled": ws.Reconciled,
"skipped": ws.Skipped,
"timeout": ws.Timeout,
})
}
return nil
}

View File

@ -5,6 +5,7 @@ package json
import (
"encoding/json"
"errors"
"strings"
"testing"
@ -82,6 +83,25 @@ func TestFormatter_FormatApplyEvent(t *testing.T) {
},
},
},
"resource apply error": {
previewStrategy: common.DryRunNone,
event: event.ApplyEvent{
Identifier: createIdentifier("apps", "Deployment", "", "my-dep"),
Error: errors.New("example error"),
},
expected: []map[string]interface{}{
{
"eventType": "resourceFailed",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "",
"timestamp": "",
"type": "apply",
"error": "example error",
},
},
},
}
for tn, tc := range testCases {
@ -200,6 +220,23 @@ func TestFormatter_FormatPruneEvent(t *testing.T) {
"type": "prune",
},
},
"resource prune error": {
previewStrategy: common.DryRunNone,
event: event.PruneEvent{
Identifier: createIdentifier("apps", "Deployment", "", "my-dep"),
Error: errors.New("example error"),
},
expected: map[string]interface{}{
"eventType": "resourceFailed",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "",
"timestamp": "",
"type": "prune",
"error": "example error",
},
},
}
for tn, tc := range testCases {
@ -256,6 +293,23 @@ func TestFormatter_FormatDeleteEvent(t *testing.T) {
"type": "delete",
},
},
"resource delete error": {
previewStrategy: common.DryRunNone,
event: event.DeleteEvent{
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
Error: errors.New("example error"),
},
expected: map[string]interface{}{
"eventType": "resourceFailed",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"timestamp": "",
"type": "delete",
"error": "example error",
},
},
}
for tn, tc := range testCases {
@ -270,6 +324,135 @@ func TestFormatter_FormatDeleteEvent(t *testing.T) {
}
}
func TestFormatter_FormatWaitEvent(t *testing.T) {
testCases := map[string]struct {
previewStrategy common.DryRunStrategy
event event.WaitEvent
statusCollector list.Collector
expected map[string]interface{}
}{
"resource reconciled": {
previewStrategy: common.DryRunNone,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Reconciled",
"timestamp": "",
"type": "wait",
},
},
"resource reconciled (client-side dry-run)": {
previewStrategy: common.DryRunClient,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Reconciled",
"timestamp": "",
"type": "wait",
},
},
"resource reconciled (server-side dry-run)": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Reconciled",
"timestamp": "",
"type": "wait",
},
},
"resource reconcile pending": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Pending",
"timestamp": "",
"type": "wait",
},
},
"resource reconcile skipped": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcileSkipped,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Skipped",
"timestamp": "",
"type": "wait",
},
},
"resource reconcile timeout": {
previewStrategy: common.DryRunServer,
event: event.WaitEvent{
GroupName: "wait-1",
Operation: event.ReconcileTimeout,
Identifier: createIdentifier("apps", "Deployment", "default", "my-dep"),
},
expected: map[string]interface{}{
"eventType": "resourceReconciled",
"group": "apps",
"kind": "Deployment",
"name": "my-dep",
"namespace": "default",
"operation": "Timeout",
"timestamp": "",
"type": "wait",
},
},
}
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.FormatWaitEvent(tc.event)
assert.NoError(t, err)
assertOutput(t, tc.expected, out.String())
})
}
}
// nolint:unparam
func assertOutput(t *testing.T, expectedMap map[string]interface{}, actual string) bool {
var m map[string]interface{}

View File

@ -79,6 +79,10 @@ type ResourceInfo struct {
// DeleteOpResult contains the result after
// a delete operation on a resource
DeleteOpResult event.DeleteEventOperation
// WaitOpResult contains the result after
// a wait operation on a resource
WaitOpResult event.WaitEventOperation
}
// Identifier returns the identifier for the given resource.
@ -173,6 +177,8 @@ func (r *ResourceStateCollector) processEvent(ev event.Event) error {
r.processApplyEvent(ev.ApplyEvent)
case event.PruneType:
r.processPruneEvent(ev.PruneEvent)
case event.WaitType:
r.processWaitEvent(ev.WaitEvent)
case event.ErrorType:
return ev.ErrorEvent.Err
}
@ -215,6 +221,18 @@ func (r *ResourceStateCollector) processPruneEvent(e event.PruneEvent) {
previous.PruneOpResult = e.Operation
}
// processPruneEvent handles event related to prune operations.
func (r *ResourceStateCollector) processWaitEvent(e event.WaitEvent) {
identifier := e.Identifier
klog.V(7).Infof("processing wait event for %s", identifier)
previous, found := r.resourceInfos[identifier]
if !found {
klog.V(4).Infof("%s wait event not found in ResourceInfos; no processing", identifier)
return
}
previous.WaitOpResult = e.Operation
}
// ResourceState contains the latest state for all the resources.
type ResourceState struct {
resourceInfos ResourceInfos
@ -251,6 +269,7 @@ func (r *ResourceStateCollector) LatestState() *ResourceState {
ApplyOpResult: ri.ApplyOpResult,
PruneOpResult: ri.PruneOpResult,
DeleteOpResult: ri.DeleteOpResult,
WaitOpResult: ri.WaitOpResult,
})
}
sort.Sort(resourceInfos)

View File

@ -106,11 +106,43 @@ var (
},
}
reconciledColumnDef = table.ColumnDef{
// Column containing the reconciliation status.
ColumnName: "reconciled",
ColumnHeader: "RECONCILED",
ColumnWidth: 10,
PrintResourceFunc: func(w io.Writer, width int, r table.Resource) (
int,
error,
) {
var resInfo *ResourceInfo
switch res := r.(type) {
case *ResourceInfo:
resInfo = res
default:
return 0, nil
}
var text string
switch resInfo.ResourceAction {
case event.WaitAction:
text = resInfo.WaitOpResult.String()
}
if len(text) > width {
text = text[:width]
}
_, err := fmt.Fprint(w, text)
return len(text), err
},
}
columns = []table.ColumnDefinition{
table.MustColumn("namespace"),
table.MustColumn("resource"),
actionColumnDef,
table.MustColumn("status"),
reconciledColumnDef,
table.MustColumn("conditions"),
table.MustColumn("age"),
table.MustColumn("message"),

View File

@ -69,8 +69,9 @@ type ExpDeleteEvent struct {
}
type ExpWaitEvent struct {
GroupName string
Error error
GroupName string
Operation event.WaitEventOperation
Identifier object.ObjMetadata
}
func VerifyEvents(expEvents []ExpEvent, events []event.Event) error {
@ -116,6 +117,8 @@ func isMatch(ee ExpEvent, e event.Event) bool {
return false
}
}
return true
case event.ActionGroupType:
agee := ee.ActionGroupEvent
@ -136,6 +139,8 @@ func isMatch(ee ExpEvent, e event.Event) bool {
if agee.Type != age.Type {
return false
}
return true
case event.ApplyType:
aee := ee.ApplyEvent
// If no more information is specified, we consider it a match.
@ -248,18 +253,26 @@ func isMatch(ee ExpEvent, e event.Event) bool {
}
we := e.WaitEvent
if wee.Identifier != object.NilObjMetadata {
if wee.Identifier != we.Identifier {
return false
}
}
if wee.GroupName != "" {
if wee.GroupName != we.GroupName {
return false
}
}
if wee.Error != nil {
return cmp.Equal(wee.Error, we.Error, cmpopts.EquateErrors())
if wee.Operation != we.Operation {
return false
}
return we.Error == nil
return true
default:
return true
}
return true
}
func EventsToExpEvents(events []event.Event) []ExpEvent {
@ -346,8 +359,9 @@ func EventToExpEvent(e event.Event) ExpEvent {
return ExpEvent{
EventType: event.WaitType,
WaitEvent: &ExpWaitEvent{
GroupName: e.WaitEvent.GroupName,
Error: e.WaitEvent.Error,
GroupName: e.WaitEvent.GroupName,
Identifier: e.WaitEvent.Identifier,
Operation: e.WaitEvent.Operation,
},
}
}
@ -402,6 +416,12 @@ func (ape GroupedEventsByID) Less(i, j int) bool {
return false
}
return ape[i].DeleteEvent.Identifier.String() < ape[j].DeleteEvent.Identifier.String()
case event.WaitType:
if ape[i].WaitEvent.GroupName != ape[j].WaitEvent.GroupName {
// don't change order if not the same task group
return false
}
return ape[i].WaitEvent.Identifier.String() < ape[j].WaitEvent.Identifier.String()
default:
// don't change order if not ApplyType, PruneType, or DeleteType
return false

View File

@ -27,8 +27,9 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
inventoryInfo := createInventoryInfo(invConfig, inventoryName, namespaceName, inventoryID)
deployment1Obj := withNamespace(manifestToUnstructured(deployment1), namespaceName)
resources := []*unstructured.Unstructured{
withNamespace(manifestToUnstructured(deployment1), namespaceName),
deployment1Obj,
}
applierEvents := runCollect(applier.Run(context.TODO(), inventoryInfo, resources, apply.Options{
@ -75,7 +76,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Error: nil,
},
},
@ -97,6 +98,24 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
Type: event.Started,
},
},
{
// Deployment reconcile Pending .
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// Deployment confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -131,7 +150,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
expected := testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.NotFoundStatus,
Error: nil,
},
@ -143,7 +162,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
expected = testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.InProgressStatus,
Error: nil,
},
@ -154,7 +173,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
expected = testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.CurrentStatus,
Error: nil,
},
@ -165,7 +184,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
Expect(received).To(testutil.Equal(expEvents))
By("Verify deployment created")
assertUnstructuredExists(c, withNamespace(manifestToUnstructured(deployment1), namespaceName))
assertUnstructuredExists(c, deployment1Obj)
By("Verify inventory")
invConfig.InvSizeVerifyFunc(c, inventoryName, namespaceName, inventoryID, 1)
@ -197,7 +216,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-0",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Error: nil,
},
},
@ -252,7 +271,7 @@ func applyAndDestroyTest(c client.Client, invConfig InventoryConfig, inventoryNa
Expect(testutil.EventsToExpEvents(destroyerEvents)).To(testutil.Equal(expEvents))
By("Verify deployment deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(deployment1), namespaceName))
assertUnstructuredDoesNotExist(c, deployment1Obj)
}
func createInventoryInfo(invConfig InventoryConfig, inventoryName, namespaceName, inventoryID string) inventory.InventoryInfo {

View File

@ -19,15 +19,19 @@ import (
"sigs.k8s.io/controller-runtime/pkg/client"
)
//nolint:dupl // expEvents similar to mutation tests
func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespaceName string) {
By("apply a set of resources that includes both a crd and a cr")
applier := invConfig.ApplierFactoryFunc()
inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(inventoryName, namespaceName, "test"))
crdObj := manifestToUnstructured(crd)
crObj := manifestToUnstructured(cr)
resources := []*unstructured.Unstructured{
manifestToUnstructured(cr),
manifestToUnstructured(crd),
crObj,
crdObj,
}
applierEvents := runCollect(applier.Run(context.TODO(), inv, resources, apply.Options{
@ -74,7 +78,7 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(manifestToUnstructured(crd)),
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
Error: nil,
},
},
@ -96,6 +100,24 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
Type: event.Started,
},
},
{
// CRD reconcile Pending .
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
},
},
{
// CRD confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -120,7 +142,7 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-1",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(manifestToUnstructured(cr)),
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
Error: nil,
},
},
@ -142,6 +164,24 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
Type: event.Started,
},
},
{
// CR reconcile Pending .
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
},
},
{
// CR confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -198,7 +238,7 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-0",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(manifestToUnstructured(cr)),
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
Error: nil,
},
},
@ -220,6 +260,24 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
Type: event.Started,
},
},
{
// CR reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
},
},
{
// CR confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(crObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -244,7 +302,7 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-1",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(manifestToUnstructured(crd)),
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
Error: nil,
},
},
@ -266,6 +324,24 @@ func crdTest(_ client.Client, invConfig InventoryConfig, inventoryName, namespac
Type: event.Started,
},
},
{
// CRD reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
},
},
{
// CRD confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(crdObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,

View File

@ -24,12 +24,16 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(inventoryName, namespaceName, "test"))
pod1Obj := withDependsOn(withNamespace(manifestToUnstructured(pod1), namespaceName), fmt.Sprintf("/namespaces/%s/Pod/pod3", namespaceName))
pod2Obj := withNamespace(manifestToUnstructured(pod2), namespaceName)
pod3Obj := withDependsOn(withNamespace(manifestToUnstructured(pod3), namespaceName), fmt.Sprintf("/namespaces/%s/Pod/pod2", namespaceName))
// Dependency order: pod1 -> pod3 -> pod2
// Apply order: pod2, pod3, pod1
resources := []*unstructured.Unstructured{
withDependsOn(withNamespace(manifestToUnstructured(pod1), namespaceName), fmt.Sprintf("/namespaces/%s/Pod/pod3", namespaceName)),
withNamespace(manifestToUnstructured(pod2), namespaceName),
withDependsOn(withNamespace(manifestToUnstructured(pod3), namespaceName), fmt.Sprintf("/namespaces/%s/Pod/pod2", namespaceName)),
pod1Obj,
pod2Obj,
pod3Obj,
}
applierEvents := runCollect(applier.Run(context.TODO(), inv, resources, apply.Options{
@ -75,7 +79,7 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod2), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
Error: nil,
},
},
@ -97,6 +101,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod2 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
},
},
{
// Pod2 confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -116,12 +138,12 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
},
},
{
// Apply pod3 second
// Apply Pod3 second
EventType: event.ApplyType,
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-1",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod3), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
Error: nil,
},
},
@ -143,6 +165,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod3 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
},
},
{
// Pod3 confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -162,12 +202,12 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
},
},
{
// Apply pod1 third
// Apply Pod1 third
EventType: event.ApplyType,
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-2",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
Error: nil,
},
},
@ -189,6 +229,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod1 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-2",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
},
},
{
// Pod1 confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-2",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -220,21 +278,21 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Expect(testutil.EventsToExpEvents(applierEvents)).To(testutil.Equal(expEvents))
By("verify pod1 created and ready")
result := assertUnstructuredExists(c, withNamespace(manifestToUnstructured(pod1), namespaceName))
result := assertUnstructuredExists(c, pod1Obj)
podIP, found, err := testutil.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 pod2 created and ready")
result = assertUnstructuredExists(c, withNamespace(manifestToUnstructured(pod2), namespaceName))
result = assertUnstructuredExists(c, pod2Obj)
podIP, found, err = testutil.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 pod3 created and ready")
result = assertUnstructuredExists(c, withNamespace(manifestToUnstructured(pod3), namespaceName))
result = assertUnstructuredExists(c, pod3Obj)
podIP, found, err = testutil.NestedField(result.Object, "status", "podIP")
Expect(err).NotTo(HaveOccurred())
Expect(found).To(BeTrue())
@ -266,7 +324,7 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-0",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
Error: nil,
},
},
@ -288,6 +346,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod1 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
},
},
{
// Pod1 confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -312,7 +388,7 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-1",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod3), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
Error: nil,
},
},
@ -334,6 +410,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod3 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
},
},
{
// Pod3 confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod3Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -358,7 +452,7 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-2",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(pod2), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
Error: nil,
},
},
@ -380,6 +474,24 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Type: event.Started,
},
},
{
// Pod2 reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-2",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
},
},
{
// Pod2 confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-2",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(pod2Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -411,11 +523,11 @@ func dependsOnTest(c client.Client, invConfig InventoryConfig, inventoryName, na
Expect(testutil.EventsToExpEvents(destroyerEvents)).To(testutil.Equal(expEvents))
By("verify pod1 deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(pod1), namespaceName))
assertUnstructuredDoesNotExist(c, pod1Obj)
By("verify pod2 deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(pod2), namespaceName))
assertUnstructuredDoesNotExist(c, pod2Obj)
By("verify pod3 deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(pod3), namespaceName))
assertUnstructuredDoesNotExist(c, pod3Obj)
}

View File

@ -26,8 +26,9 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
firstInvName := randomString("first-inv-")
firstInv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(firstInvName, namespaceName, firstInvName))
deployment1Obj := withNamespace(manifestToUnstructured(deployment1), namespaceName)
firstResources := []*unstructured.Unstructured{
withNamespace(manifestToUnstructured(deployment1), namespaceName),
deployment1Obj,
}
runWithNoErr(applier.Run(context.TODO(), firstInv, firstResources, apply.Options{
@ -38,8 +39,9 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
By("Apply second set of resources")
secondInvName := randomString("second-inv-")
secondInv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(secondInvName, namespaceName, secondInvName))
deployment1Obj = withNamespace(manifestToUnstructured(deployment1), namespaceName)
secondResources := []*unstructured.Unstructured{
withReplicas(withNamespace(manifestToUnstructured(deployment1), namespaceName), 6),
withReplicas(deployment1Obj, 6),
}
applierEvents := runCollect(applier.Run(context.TODO(), secondInv, secondResources, apply.Options{
@ -87,7 +89,7 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
EventType: event.ApplyType,
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Error: testutil.EqualErrorType(
inventory.NewInventoryOverlapError(errors.New("test")),
),
@ -111,6 +113,15 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
Type: event.Started,
},
},
{
// Wait skipped because apply failed
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcileSkipped,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -145,7 +156,7 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
expected := testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.InProgressStatus,
Error: nil,
},
@ -156,7 +167,7 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
expected = testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.CurrentStatus,
Error: nil,
},
@ -167,7 +178,7 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
Expect(received).To(testutil.Equal(expEvents))
By("Verify resource wasn't updated")
result := assertUnstructuredExists(c, withNamespace(manifestToUnstructured(deployment1), namespaceName))
result := assertUnstructuredExists(c, deployment1Obj)
replicas, found, err := testutil.NestedField(result.Object, "spec", "replicas")
Expect(err).NotTo(HaveOccurred())
Expect(found).To(BeTrue())
@ -178,7 +189,8 @@ func inventoryPolicyMustMatchTest(c client.Client, invConfig InventoryConfig, na
func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryConfig, namespaceName string) {
By("Create unmanaged resource")
err := c.Create(context.TODO(), withNamespace(manifestToUnstructured(deployment1), namespaceName))
deployment1Obj := withNamespace(manifestToUnstructured(deployment1), namespaceName)
err := c.Create(context.TODO(), deployment1Obj)
Expect(err).NotTo(HaveOccurred())
By("Apply resources")
@ -186,8 +198,9 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
invName := randomString("test-inv-")
inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(invName, namespaceName, invName))
deployment1Obj = withNamespace(manifestToUnstructured(deployment1), namespaceName)
resources := []*unstructured.Unstructured{
withReplicas(withNamespace(manifestToUnstructured(deployment1), namespaceName), 6),
withReplicas(deployment1Obj, 6),
}
applierEvents := runCollect(applier.Run(context.TODO(), inv, resources, apply.Options{
@ -236,7 +249,7 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Configured,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Error: nil,
},
},
@ -258,6 +271,24 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
Type: event.Started,
},
},
{
// Deployment reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// Deployment confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -292,7 +323,7 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
expected := testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.InProgressStatus,
Error: nil,
},
@ -303,7 +334,7 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
expected = testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.CurrentStatus,
Error: nil,
},
@ -314,7 +345,7 @@ func inventoryPolicyAdoptIfNoInventoryTest(c client.Client, invConfig InventoryC
Expect(received).To(testutil.Equal(expEvents))
By("Verify resource was updated and added to inventory")
result := assertUnstructuredExists(c, withNamespace(manifestToUnstructured(deployment1), namespaceName))
result := assertUnstructuredExists(c, deployment1Obj)
replicas, found, err := testutil.NestedField(result.Object, "spec", "replicas")
Expect(err).NotTo(HaveOccurred())
@ -336,8 +367,9 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
firstInvName := randomString("first-inv-")
firstInv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(firstInvName, namespaceName, firstInvName))
deployment1Obj := withNamespace(manifestToUnstructured(deployment1), namespaceName)
firstResources := []*unstructured.Unstructured{
withNamespace(manifestToUnstructured(deployment1), namespaceName),
deployment1Obj,
}
runWithNoErr(applier.Run(context.TODO(), firstInv, firstResources, apply.Options{
@ -348,8 +380,9 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
By("Apply resources")
secondInvName := randomString("test-inv-")
secondInv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(secondInvName, namespaceName, secondInvName))
deployment1Obj = withNamespace(manifestToUnstructured(deployment1), namespaceName)
secondResources := []*unstructured.Unstructured{
withReplicas(withNamespace(manifestToUnstructured(deployment1), namespaceName), 6),
withReplicas(deployment1Obj, 6),
}
applierEvents := runCollect(applier.Run(context.TODO(), secondInv, secondResources, apply.Options{
@ -398,7 +431,7 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Configured,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Error: nil,
},
},
@ -420,6 +453,24 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
Type: event.Started,
},
},
{
// Deployment reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// Deployment confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -454,7 +505,7 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
expected := testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.InProgressStatus,
Error: nil,
},
@ -465,7 +516,7 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
expected = testutil.ExpEvent{
EventType: event.StatusType,
StatusEvent: &testutil.ExpStatusEvent{
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(deployment1), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(deployment1Obj),
Status: status.CurrentStatus,
Error: nil,
},
@ -476,7 +527,7 @@ func inventoryPolicyAdoptAllTest(c client.Client, invConfig InventoryConfig, nam
Expect(received).To(testutil.Equal(expEvents))
By("Verify resource was updated and added to inventory")
result := assertUnstructuredExists(c, withNamespace(manifestToUnstructured(deployment1), namespaceName))
result := assertUnstructuredExists(c, deployment1Obj)
replicas, found, err := testutil.NestedField(result.Object, "spec", "replicas")
Expect(err).NotTo(HaveOccurred())

View File

@ -31,17 +31,21 @@ import (
// we test a toy example with a pod-a depending on pod-b, injecting the ip and
// port from pod-b into an environment variable of pod-a.
//nolint:dupl // expEvents similar to CRD tests
func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, namespaceName string) {
By("apply resources in order with substitutions based on apply-time-mutation annotation")
applier := invConfig.ApplierFactoryFunc()
inv := invConfig.InvWrapperFunc(invConfig.InventoryFactoryFunc(inventoryName, namespaceName, "test"))
podAObj := withNamespace(manifestToUnstructured(podA), namespaceName)
podBObj := withNamespace(manifestToUnstructured(podB), namespaceName)
// Dependency order: podA -> podB
// Apply order: podB, podA
resources := []*unstructured.Unstructured{
withNamespace(manifestToUnstructured(podA), namespaceName),
withNamespace(manifestToUnstructured(podB), namespaceName),
podAObj,
podBObj,
}
applierEvents := runCollect(applier.Run(context.TODO(), inv, resources, apply.Options{
@ -87,7 +91,7 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-0",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(podB), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
Error: nil,
},
},
@ -109,6 +113,24 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Type: event.Started,
},
},
{
// PodB reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
},
},
{
// PodB confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -128,12 +150,12 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
},
},
{
// Apply pod3 second
// Apply PodA second
EventType: event.ApplyType,
ApplyEvent: &testutil.ExpApplyEvent{
GroupName: "apply-1",
Operation: event.Created,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(podA), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
Error: nil,
},
},
@ -155,6 +177,24 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Type: event.Started,
},
},
{
// PodA reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
},
},
{
// PodA confirmed Current.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -186,7 +226,7 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Expect(testutil.EventsToExpEvents(applierEvents)).To(testutil.Equal(expEvents))
By("verify podB is created and ready")
result := assertUnstructuredExists(c, withNamespace(manifestToUnstructured(podB), namespaceName))
result := assertUnstructuredExists(c, podBObj)
podIP, found, err := testutil.NestedField(result.Object, "status", "podIP")
Expect(err).NotTo(HaveOccurred())
@ -201,7 +241,7 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
host := fmt.Sprintf("%s:%d", podIP, containerPort)
By("verify podA is mutated, created, and ready")
result = assertUnstructuredExists(c, withNamespace(manifestToUnstructured(podA), namespaceName))
result = assertUnstructuredExists(c, podAObj)
podIP, found, err = testutil.NestedField(result.Object, "status", "podIP")
Expect(err).NotTo(HaveOccurred())
@ -234,12 +274,12 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
},
},
{
// Delete podA first
// Delete PodA first
EventType: event.DeleteType,
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-0",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(podA), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
Error: nil,
},
},
@ -261,6 +301,24 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Type: event.Started,
},
},
{
// PodA reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
},
},
{
// PodA confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-0",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(podAObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -280,12 +338,12 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
},
},
{
// Delete pod3 second
// Delete PodB second
EventType: event.DeleteType,
DeleteEvent: &testutil.ExpDeleteEvent{
GroupName: "prune-1",
Operation: event.Deleted,
Identifier: object.UnstructuredToObjMetaOrDie(withNamespace(manifestToUnstructured(podB), namespaceName)),
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
Error: nil,
},
},
@ -307,6 +365,24 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Type: event.Started,
},
},
{
// PodB reconcile Pending.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.ReconcilePending,
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
},
},
{
// PodB confirmed NotFound.
EventType: event.WaitType,
WaitEvent: &testutil.ExpWaitEvent{
GroupName: "wait-1",
Operation: event.Reconciled,
Identifier: object.UnstructuredToObjMetaOrDie(podBObj),
},
},
{
// WaitTask finished
EventType: event.ActionGroupType,
@ -339,8 +415,8 @@ func mutationTest(c client.Client, invConfig InventoryConfig, inventoryName, nam
Expect(testutil.EventsToExpEvents(destroyerEvents)).To(testutil.Equal(expEvents))
By("verify podB deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(podB), namespaceName))
assertUnstructuredDoesNotExist(c, podBObj)
By("verify podA deleted")
assertUnstructuredDoesNotExist(c, withNamespace(manifestToUnstructured(podA), namespaceName))
assertUnstructuredDoesNotExist(c, podAObj)
}