More tests

This commit is contained in:
Fraser Waters 2021-11-03 15:56:42 +00:00
parent 83e2cd7e4a
commit b1e23a5287
3 changed files with 145 additions and 365 deletions

View file

@ -2691,3 +2691,112 @@ func TestPlannedUpdate(t *testing.T) {
})
assert.Equal(t, expected, snap.Resources[1].Outputs)
}
func TestUnplannedCreate(t *testing.T) {
loaders := []*deploytest.ProviderLoader{
deploytest.NewProviderLoader("pkgA", semver.MustParse("1.0.0"), func() (plugin.Provider, error) {
return &deploytest.Provider{
CreateF: func(urn resource.URN, news resource.PropertyMap, timeout float64,
preview bool) (resource.ID, resource.PropertyMap, resource.Status, error) {
return "created-id", news, resource.StatusOK, nil
},
}, nil
}),
}
ins := resource.NewPropertyMapFromMap(map[string]interface{}{
"foo": "bar",
})
createResource := false
program := deploytest.NewLanguageRuntime(func(_ plugin.RunInfo, monitor *deploytest.ResourceMonitor) error {
if createResource {
_, _, _, err := monitor.RegisterResource("pkgA:m:typA", "resA", true, deploytest.ResourceOptions{
Inputs: ins,
})
assert.NoError(t, err)
}
return nil
})
host := deploytest.NewPluginHost(nil, nil, program, loaders...)
p := &TestPlan{
Options: UpdateOptions{Host: host},
}
project := p.GetProject()
// Create a plan to do nothing
plan, res := TestOp(Update).Plan(project, p.GetTarget(nil), p.Options, p.BackendClient, nil)
assert.Nil(t, res)
// Now set the flag for the language runtime to create a resource, and run update with the plan
createResource = true
p.Options.Plan = plan
snap, res := TestOp(Update).Run(project, p.GetTarget(nil), p.Options, false, p.BackendClient, nil)
assert.NotNil(t, res)
// Check nothing was was created
assert.NotNil(t, snap)
if !assert.Len(t, snap.Resources, 0) {
return
}
}
func TestUnplannedDelete(t *testing.T) {
loaders := []*deploytest.ProviderLoader{
deploytest.NewProviderLoader("pkgA", semver.MustParse("1.0.0"), func() (plugin.Provider, error) {
return &deploytest.Provider{
DeleteF: func(urn resource.URN, id resource.ID, olds resource.PropertyMap, timeout float64) (resource.Status, error) {
return resource.StatusOK, nil
},
}, nil
}),
}
ins := resource.NewPropertyMapFromMap(map[string]interface{}{
"foo": "bar",
})
createAllResources := true
program := deploytest.NewLanguageRuntime(func(_ plugin.RunInfo, monitor *deploytest.ResourceMonitor) error {
_, _, _, err := monitor.RegisterResource("pkgA:m:typA", "resA", true, deploytest.ResourceOptions{
Inputs: ins,
})
assert.NoError(t, err)
if createAllResources {
_, _, _, err = monitor.RegisterResource("pkgA:m:typA", "resB", true, deploytest.ResourceOptions{
Inputs: ins,
})
assert.NoError(t, err)
}
return nil
})
host := deploytest.NewPluginHost(nil, nil, program, loaders...)
p := &TestPlan{
Options: UpdateOptions{Host: host},
}
project := p.GetProject()
// Create an initial snapshot that resA and resB exist
snap, res := TestOp(Update).Run(project, p.GetTarget(nil), p.Options, false, p.BackendClient, nil)
assert.Nil(t, res)
// Create a plan that resA and resB won't change
plan, res := TestOp(Update).Plan(project, p.GetTarget(snap), p.Options, p.BackendClient, nil)
assert.Nil(t, res)
// Now set the flag for the language runtime to not create resB and run an update with the no-op plan, this should block the delete
createAllResources = false
p.Options.Plan = plan
snap, res = TestOp(Update).Run(project, p.GetTarget(snap), p.Options, false, p.BackendClient, nil)
assert.NotNil(t, snap)
assert.NotNil(t, res)
// Check both resources and the provider are still listed in the snapshot
if !assert.Len(t, snap.Resources, 3) {
return
}
}

View file

@ -1,357 +0,0 @@
//nolint:revive
package lifecycletest
import (
"context"
"reflect"
"testing"
"github.com/mitchellh/copystructure"
"github.com/stretchr/testify/assert"
. "github.com/pulumi/pulumi/pkg/v3/engine"
"github.com/pulumi/pulumi/pkg/v3/resource/deploy"
"github.com/pulumi/pulumi/pkg/v3/resource/deploy/providers"
"github.com/pulumi/pulumi/pkg/v3/util/cancel"
"github.com/pulumi/pulumi/sdk/v3/go/common/resource"
"github.com/pulumi/pulumi/sdk/v3/go/common/resource/config"
"github.com/pulumi/pulumi/sdk/v3/go/common/tokens"
"github.com/pulumi/pulumi/sdk/v3/go/common/util/contract"
"github.com/pulumi/pulumi/sdk/v3/go/common/util/result"
"github.com/pulumi/pulumi/sdk/v3/go/common/workspace"
)
type updateInfo struct {
project workspace.Project
target deploy.Target
}
func (u *updateInfo) GetRoot() string {
return ""
}
func (u *updateInfo) GetProject() *workspace.Project {
return &u.project
}
func (u *updateInfo) GetTarget() *deploy.Target {
return &u.target
}
func ImportOp(imports []deploy.Import) TestOp {
return TestOp(func(info UpdateInfo, ctx *Context, opts UpdateOptions,
dryRun bool) (deploy.Plan, ResourceChanges, result.Result) {
return Import(info, ctx, opts, imports, dryRun)
})
}
type TestOp func(UpdateInfo, *Context, UpdateOptions, bool) (deploy.Plan, ResourceChanges, result.Result)
type ValidateFunc func(project workspace.Project, target deploy.Target, entries JournalEntries,
events []Event, res result.Result) result.Result
func (op TestOp) Plan(project workspace.Project, target deploy.Target, opts UpdateOptions,
backendClient deploy.BackendClient, validate ValidateFunc) (deploy.Plan, result.Result) {
plan, _, res := op.runWithContext(context.Background(), project, target, opts, true, backendClient, validate)
return plan, res
}
func (op TestOp) Run(project workspace.Project, target deploy.Target, opts UpdateOptions,
dryRun bool, backendClient deploy.BackendClient, validate ValidateFunc) (*deploy.Snapshot, result.Result) {
return op.RunWithContext(context.Background(), project, target, opts, dryRun, backendClient, validate)
}
func (op TestOp) RunWithContext(
callerCtx context.Context, project workspace.Project,
target deploy.Target, opts UpdateOptions, dryRun bool,
backendClient deploy.BackendClient, validate ValidateFunc) (*deploy.Snapshot, result.Result) {
_, snap, res := op.runWithContext(callerCtx, project, target, opts, dryRun, backendClient, validate)
return snap, res
}
func (op TestOp) runWithContext(
callerCtx context.Context, project workspace.Project,
target deploy.Target, opts UpdateOptions, dryRun bool,
backendClient deploy.BackendClient, validate ValidateFunc) (deploy.Plan, *deploy.Snapshot, result.Result) {
// Create an appropriate update info and context.
info := &updateInfo{project: project, target: target}
cancelCtx, cancelSrc := cancel.NewContext(context.Background())
done := make(chan bool)
defer close(done)
go func() {
select {
case <-callerCtx.Done():
cancelSrc.Cancel()
case <-done:
}
}()
events := make(chan Event)
journal := NewJournal()
ctx := &Context{
Cancel: cancelCtx,
Events: events,
SnapshotManager: journal,
BackendClient: backendClient,
}
// Begin draining events.
var firedEvents []Event
go func() {
for e := range events {
firedEvents = append(firedEvents, e)
}
}()
// Run the step and its validator.
plan, _, res := op(info, ctx, opts, dryRun)
contract.IgnoreClose(journal)
if dryRun {
return plan, nil, res
}
if validate != nil {
res = validate(project, target, journal.Entries(), firedEvents, res)
}
snap := journal.Snap(target.Snapshot)
if res == nil && snap != nil {
res = result.WrapIfNonNil(snap.VerifyIntegrity())
}
return nil, snap, res
}
type TestStep struct {
Op TestOp
ExpectFailure bool
SkipPreview bool
Validate ValidateFunc
}
type TestPlan struct {
Project string
Stack string
Runtime string
RuntimeOptions map[string]interface{}
Config config.Map
Decrypter config.Decrypter
BackendClient deploy.BackendClient
Options UpdateOptions
Steps []TestStep
}
//nolint: goconst
func (p *TestPlan) getNames() (stack tokens.QName, project tokens.PackageName, runtime string) {
project = tokens.PackageName(p.Project)
if project == "" {
project = "test"
}
runtime = p.Runtime
if runtime == "" {
runtime = "test"
}
stack = tokens.QName(p.Stack)
if stack == "" {
stack = "test"
}
return stack, project, runtime
}
func (p *TestPlan) NewURN(typ tokens.Type, name string, parent resource.URN) resource.URN {
stack, project, _ := p.getNames()
var pt tokens.Type
if parent != "" {
pt = parent.Type()
}
return resource.NewURN(stack, project, pt, typ, tokens.QName(name))
}
func (p *TestPlan) NewProviderURN(pkg tokens.Package, name string, parent resource.URN) resource.URN {
return p.NewURN(providers.MakeProviderType(pkg), name, parent)
}
func (p *TestPlan) GetProject() workspace.Project {
_, projectName, runtime := p.getNames()
return workspace.Project{
Name: projectName,
Runtime: workspace.NewProjectRuntimeInfo(runtime, p.RuntimeOptions),
}
}
func (p *TestPlan) GetTarget(snapshot *deploy.Snapshot) deploy.Target {
stack, _, _ := p.getNames()
cfg := p.Config
if cfg == nil {
cfg = config.Map{}
}
return deploy.Target{
Name: stack,
Config: cfg,
Decrypter: p.Decrypter,
Snapshot: snapshot,
}
}
func assertIsErrorOrBailResult(t *testing.T, res result.Result) {
assert.NotNil(t, res)
}
// CloneSnapshot makes a deep copy of the given snapshot and returns a pointer to the clone.
func CloneSnapshot(t *testing.T, snap *deploy.Snapshot) *deploy.Snapshot {
t.Helper()
if snap != nil {
copiedSnap := copystructure.Must(copystructure.Copy(*snap)).(deploy.Snapshot)
assert.True(t, reflect.DeepEqual(*snap, copiedSnap))
return &copiedSnap
}
return snap
}
func (p *TestPlan) Run(t *testing.T, snapshot *deploy.Snapshot) *deploy.Snapshot {
project := p.GetProject()
snap := snapshot
for _, step := range p.Steps {
// note: it's really important that the preview and update operate on different snapshots. the engine can and
// does mutate the snapshot in-place, even in previews, and sharing a snapshot between preview and update can
// cause state changes from the preview to persist even when doing an update.
if !step.SkipPreview {
previewSnap := CloneSnapshot(t, snap)
previewTarget := p.GetTarget(previewSnap)
_, res := step.Op.Run(project, previewTarget, p.Options, true, p.BackendClient, step.Validate)
if step.ExpectFailure {
assertIsErrorOrBailResult(t, res)
continue
}
assert.Nil(t, res)
}
var res result.Result
target := p.GetTarget(snap)
snap, res = step.Op.Run(project, target, p.Options, false, p.BackendClient, step.Validate)
if step.ExpectFailure {
assertIsErrorOrBailResult(t, res)
continue
}
if res != nil {
if res.IsBail() {
t.Logf("Got unexpected bail result")
t.FailNow()
} else {
t.Logf("Got unexpected error result: %v", res.Error())
t.FailNow()
}
}
assert.Nil(t, res)
}
return snap
}
func MakeBasicLifecycleSteps(t *testing.T, resCount int) []TestStep {
return []TestStep{
// Initial update
{
Op: Update,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
// Should see only creates or reads.
for _, entry := range entries {
op := entry.Step.Op()
assert.True(t, op == deploy.OpCreate || op == deploy.OpRead)
}
assert.Len(t, entries.Snap(target.Snapshot).Resources, resCount)
return res
},
},
// No-op refresh
{
Op: Refresh,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
// Should see only refresh-sames.
for _, entry := range entries {
assert.Equal(t, deploy.OpRefresh, entry.Step.Op())
assert.Equal(t, deploy.OpSame, entry.Step.(*deploy.RefreshStep).ResultOp())
}
assert.Len(t, entries.Snap(target.Snapshot).Resources, resCount)
return res
},
},
// No-op update
{
Op: Update,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
// Should see only sames.
for _, entry := range entries {
op := entry.Step.Op()
assert.True(t, op == deploy.OpSame || op == deploy.OpRead)
}
assert.Len(t, entries.Snap(target.Snapshot).Resources, resCount)
return res
},
},
// No-op refresh
{
Op: Refresh,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
// Should see only refresh-sames.
for _, entry := range entries {
assert.Equal(t, deploy.OpRefresh, entry.Step.Op())
assert.Equal(t, deploy.OpSame, entry.Step.(*deploy.RefreshStep).ResultOp())
}
assert.Len(t, entries.Snap(target.Snapshot).Resources, resCount)
return res
},
},
// Destroy
{
Op: Destroy,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
// Should see only deletes.
for _, entry := range entries {
switch entry.Step.Op() {
case deploy.OpDelete, deploy.OpReadDiscard:
// ok
default:
assert.Fail(t, "expected OpDelete or OpReadDiscard")
}
}
assert.Len(t, entries.Snap(target.Snapshot).Resources, 0)
return res
},
},
// No-op refresh
{
Op: Refresh,
Validate: func(project workspace.Project, target deploy.Target, entries JournalEntries,
_ []Event, res result.Result) result.Result {
assert.Len(t, entries, 0)
assert.Len(t, entries.Snap(target.Snapshot).Resources, 0)
return res
},
},
}
}

View file

@ -158,16 +158,20 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, res
// Check each proposed step against the relevant resource plan, if any, and generate any output resource plans.
for _, s := range steps {
if resourcePlan, ok := sg.deployment.plan[s.URN()]; ok {
if len(resourcePlan.Ops) == 0 {
return nil, result.Errorf("%v is not allowed by the plan: no more steps were expected for this resource", s.Op())
}
if sg.deployment.plan != nil {
if resourcePlan, ok := sg.deployment.plan[s.URN()]; ok {
if len(resourcePlan.Ops) == 0 {
return nil, result.Errorf("%v is not allowed by the plan: no more steps were expected for this resource", s.Op())
}
constraint := resourcePlan.Ops[0]
if !s.Op().ConstrainedTo(constraint) {
return nil, result.Errorf("%v is not allowed by the plan: this resource is constrained to %v", s.Op(), constraint)
constraint := resourcePlan.Ops[0]
if !s.Op().ConstrainedTo(constraint) {
return nil, result.Errorf("%v is not allowed by the plan: this resource is constrained to %v", s.Op(), constraint)
}
resourcePlan.Ops = resourcePlan.Ops[1:]
} else {
return nil, result.Errorf("%v is not allowed by the plan: no steps were expected for this resource", s.Op())
}
resourcePlan.Ops = resourcePlan.Ops[1:]
}
resourcePlan, ok := sg.deployment.newPlans.get(s.URN())
@ -760,6 +764,30 @@ func (sg *stepGenerator) GenerateDeletes(targetsOpt map[resource.URN]bool) ([]St
}
}
// Check each proposed delete against the relevant resource plan
for _, s := range dels {
if resourcePlan, ok := sg.deployment.plan[s.URN()]; ok {
if len(resourcePlan.Ops) == 0 {
return nil, result.Errorf("%v is not allowed by the plan: no more steps were expected for this resource", s.Op())
}
constraint := resourcePlan.Ops[0]
if !s.Op().ConstrainedTo(constraint) {
return nil, result.Errorf("%v is not allowed by the plan: this resource is constrained to %v", s.Op(), constraint)
}
resourcePlan.Ops = resourcePlan.Ops[1:]
}
resourcePlan, ok := sg.deployment.newPlans.get(s.URN())
if !ok {
// TODO(pdg-plan): using the program inputs means that non-determinism could sneak in as part of default
// application. However, it is necessary in the face of computed inputs.
resourcePlan = &ResourcePlan{}
sg.deployment.newPlans.set(s.URN(), resourcePlan)
}
resourcePlan.Ops = append(resourcePlan.Ops, s.Op())
}
// If -target was provided to either `pulumi update` or `pulumi destroy` then only delete
// resources that were specified.
allowedResourcesToDelete, res := sg.determineAllowedResourcesToDeleteFromTargets(targetsOpt)