2018-08-06 23:46:17 +00:00
|
|
|
// Copyright 2016-2018, Pulumi Corporation.
|
|
|
|
//
|
|
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
|
|
// you may not use this file except in compliance with the License.
|
|
|
|
// You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
|
|
|
|
|
|
|
package deploy
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2021-11-13 02:37:17 +00:00
|
|
|
"errors"
|
2020-11-24 17:14:09 +00:00
|
|
|
"fmt"
|
2019-10-01 06:41:56 +00:00
|
|
|
"strings"
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2024-03-22 09:22:40 +00:00
|
|
|
mapset "github.com/deckarep/golang-set/v2"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/pkg/v3/resource/deploy/providers"
|
|
|
|
"github.com/pulumi/pulumi/pkg/v3/resource/graph"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/diag"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource"
|
2024-04-22 11:12:45 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource/urn"
|
2023-06-28 16:02:04 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/slice"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/contract"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/logging"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/result"
|
2018-08-06 23:46:17 +00:00
|
|
|
)
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// deploymentExecutor is responsible for taking a deployment and driving it to completion.
|
2018-08-06 23:46:17 +00:00
|
|
|
// Its primary responsibility is to own a `stepGenerator` and `stepExecutor`, serving
|
|
|
|
// as the glue that links the two subsystems together.
|
2020-11-18 17:47:52 +00:00
|
|
|
type deploymentExecutor struct {
|
|
|
|
deployment *Deployment // The deployment that we are executing
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
stepGen *stepGenerator // step generator owned by this deployment
|
|
|
|
stepExec *stepExecutor // step executor owned by this deployment
|
2024-04-22 11:12:45 +00:00
|
|
|
|
|
|
|
skipped mapset.Set[urn.URN] // The set of resources that have failed
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2019-10-01 06:41:56 +00:00
|
|
|
// checkTargets validates that all the targets passed in refer to existing resources. Diagnostics
|
|
|
|
// are generated for any target that cannot be found. The target must either have existed in the stack
|
|
|
|
// prior to running the operation, or it must be the urn for a resource that was created.
|
2023-10-09 10:44:05 +00:00
|
|
|
func (ex *deploymentExecutor) checkTargets(targets UrnTargets) error {
|
2022-10-25 19:05:35 +00:00
|
|
|
if !targets.IsConstrained() {
|
2019-10-01 06:41:56 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
olds := ex.deployment.olds
|
2019-10-01 06:41:56 +00:00
|
|
|
var news map[resource.URN]bool
|
2020-11-18 17:47:52 +00:00
|
|
|
if ex.stepGen != nil {
|
|
|
|
news = ex.stepGen.urns
|
2019-10-01 06:41:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
hasUnknownTarget := false
|
2022-10-25 19:05:35 +00:00
|
|
|
for _, target := range targets.Literals() {
|
|
|
|
hasOld := olds != nil && olds[target] != nil
|
2019-10-01 06:41:56 +00:00
|
|
|
hasNew := news != nil && news[target]
|
|
|
|
if !hasOld && !hasNew {
|
|
|
|
hasUnknownTarget = true
|
|
|
|
|
2023-05-23 20:17:59 +00:00
|
|
|
logging.V(7).Infof("Targeted resource could not be found in the stack [urn=%v]", target)
|
2019-10-01 06:41:56 +00:00
|
|
|
if strings.Contains(string(target), "$") {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.deployment.Diag().Errorf(diag.GetTargetCouldNotBeFoundError(), target)
|
2019-10-01 06:41:56 +00:00
|
|
|
} else {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.deployment.Diag().Errorf(diag.GetTargetCouldNotBeFoundDidYouForgetError(), target)
|
2019-10-01 06:41:56 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if hasUnknownTarget {
|
2023-10-09 10:44:05 +00:00
|
|
|
return result.BailErrorf("one or more targets could not be found in the stack")
|
2019-10-01 06:41:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-04-05 20:44:11 +00:00
|
|
|
func (ex *deploymentExecutor) printPendingOperationsWarning() {
|
|
|
|
pendingOperations := ""
|
|
|
|
for _, op := range ex.deployment.prev.PendingOperations {
|
|
|
|
pendingOperations = pendingOperations + fmt.Sprintf(" * %s, interrupted while %s\n", op.Resource.URN, op.Type)
|
|
|
|
}
|
|
|
|
|
2022-08-18 14:31:22 +00:00
|
|
|
resolutionMessage := "" +
|
|
|
|
"These resources are in an unknown state because the Pulumi CLI was interrupted while " +
|
|
|
|
"waiting for changes to these resources to complete. You should confirm whether or not the " +
|
|
|
|
"operations listed completed successfully by checking the state of the appropriate provider. " +
|
|
|
|
"For example, if you are using AWS, you can confirm using the AWS Console.\n" +
|
|
|
|
"\n" +
|
|
|
|
"Once you have confirmed the status of the interrupted operations, you can repair your stack " +
|
|
|
|
"using `pulumi refresh` which will refresh the state from the provider you are using and " +
|
|
|
|
"clear the pending operations if there are any.\n" +
|
|
|
|
"\n" +
|
2022-08-24 11:40:01 +00:00
|
|
|
"Note that `pulumi refresh` will need to be run interactively to clear pending CREATE operations."
|
2022-04-05 20:44:11 +00:00
|
|
|
|
|
|
|
warning := "Attempting to deploy or update resources " +
|
|
|
|
fmt.Sprintf("with %d pending operations from previous deployment.\n", len(ex.deployment.prev.PendingOperations)) +
|
|
|
|
pendingOperations +
|
|
|
|
resolutionMessage
|
|
|
|
|
|
|
|
ex.deployment.Diag().Warningf(diag.RawMessage("" /*urn*/, warning))
|
|
|
|
}
|
|
|
|
|
2019-03-21 20:23:46 +00:00
|
|
|
// reportExecResult issues an appropriate diagnostic depending on went wrong.
|
2020-11-18 17:47:52 +00:00
|
|
|
func (ex *deploymentExecutor) reportExecResult(message string, preview bool) {
|
2018-08-21 21:05:00 +00:00
|
|
|
kind := "update"
|
|
|
|
if preview {
|
|
|
|
kind = "preview"
|
|
|
|
}
|
2019-03-20 21:56:12 +00:00
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportError("", errors.New(kind+" "+message))
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// reportError reports a single error to the executor's diag stream with the indicated URN for context.
|
2020-11-18 17:47:52 +00:00
|
|
|
func (ex *deploymentExecutor) reportError(urn resource.URN, err error) {
|
|
|
|
ex.deployment.Diag().Errorf(diag.RawMessage(urn, err.Error()))
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// Execute executes a deployment to completion, using the given cancellation context and running a preview
|
2018-08-06 23:46:17 +00:00
|
|
|
// or update.
|
2023-10-09 10:44:05 +00:00
|
|
|
func (ex *deploymentExecutor) Execute(callerCtx context.Context, opts Options, preview bool) (*Plan, error) {
|
2020-11-18 17:47:52 +00:00
|
|
|
// Set up a goroutine that will signal cancellation to the deployment's plugins if the caller context is cancelled.
|
|
|
|
// We do not hang this off of the context we create below because we do not want the failure of a single step to
|
|
|
|
// cause other steps to fail.
|
2024-04-22 11:12:45 +00:00
|
|
|
ex.skipped = mapset.NewSet[urn.URN]()
|
2018-08-21 21:05:00 +00:00
|
|
|
done := make(chan bool)
|
2018-09-10 22:18:25 +00:00
|
|
|
defer close(done)
|
2018-08-06 23:46:17 +00:00
|
|
|
go func() {
|
|
|
|
select {
|
2018-08-23 00:52:46 +00:00
|
|
|
case <-callerCtx.Done():
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): signalling cancellation to providers...")
|
|
|
|
cancelErr := ex.deployment.ctx.Host.SignalCancellation()
|
2018-08-21 21:05:00 +00:00
|
|
|
if cancelErr != nil {
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): failed to signal cancellation to providers: %v", cancelErr)
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
|
|
|
case <-done:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): exiting provider canceller")
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// If this deployment is an import, run the imports and exit.
|
|
|
|
if ex.deployment.isImport {
|
|
|
|
return ex.importResources(callerCtx, opts, preview)
|
2020-10-14 11:51:53 +00:00
|
|
|
}
|
|
|
|
|
2018-08-23 00:52:46 +00:00
|
|
|
// Before doing anything else, optionally refresh each resource in the base checkpoint.
|
|
|
|
if opts.Refresh {
|
2023-10-09 10:44:05 +00:00
|
|
|
if err := ex.refresh(callerCtx, opts, preview); err != nil {
|
|
|
|
return nil, err
|
2018-08-23 00:52:46 +00:00
|
|
|
}
|
|
|
|
if opts.RefreshOnly {
|
2022-01-31 10:31:51 +00:00
|
|
|
return nil, nil
|
2018-08-23 00:52:46 +00:00
|
|
|
}
|
2022-04-05 20:44:11 +00:00
|
|
|
} else if ex.deployment.prev != nil && len(ex.deployment.prev.PendingOperations) > 0 && !preview {
|
|
|
|
// Print a warning for users that there are pending operations.
|
|
|
|
// Explain that these operations can be cleared using pulumi refresh (except for CREATE operations)
|
|
|
|
// since these require user intevention:
|
|
|
|
ex.printPendingOperationsWarning()
|
2018-08-23 00:52:46 +00:00
|
|
|
}
|
|
|
|
|
2023-10-09 10:44:05 +00:00
|
|
|
if err := ex.checkTargets(opts.ReplaceTargets); err != nil {
|
|
|
|
return nil, err
|
2019-10-31 00:16:55 +00:00
|
|
|
}
|
2019-10-01 06:41:56 +00:00
|
|
|
|
2018-08-23 00:52:46 +00:00
|
|
|
// Begin iterating the source.
|
2023-09-20 14:34:24 +00:00
|
|
|
src, err := ex.deployment.source.Iterate(callerCtx, opts, ex.deployment)
|
|
|
|
if err != nil {
|
2023-10-09 10:44:05 +00:00
|
|
|
return nil, err
|
2018-08-23 00:52:46 +00:00
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// Set up a step generator for this deployment.
|
2023-05-23 20:17:59 +00:00
|
|
|
ex.stepGen = newStepGenerator(ex.deployment, opts, opts.Targets, opts.ReplaceTargets)
|
2018-09-10 23:48:14 +00:00
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// Derive a cancellable context for this deployment. We will only cancel this context if some piece of the
|
|
|
|
// deployment's execution fails.
|
2018-08-23 00:52:46 +00:00
|
|
|
ctx, cancel := context.WithCancel(callerCtx)
|
2018-08-21 21:05:00 +00:00
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// Set up a step generator and executor for this deployment.
|
|
|
|
ex.stepExec = newStepExecutor(ctx, cancel, ex.deployment, opts, preview, false)
|
2018-08-21 21:05:00 +00:00
|
|
|
|
|
|
|
// We iterate the source in its own goroutine because iteration is blocking and we want the main loop to be able to
|
|
|
|
// respond to cancellation requests promptly.
|
2018-08-06 23:46:17 +00:00
|
|
|
type nextEvent struct {
|
2023-09-12 15:01:16 +00:00
|
|
|
Event SourceEvent
|
|
|
|
Error error
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
incomingEvents := make(chan nextEvent)
|
|
|
|
go func() {
|
|
|
|
for {
|
2023-09-12 15:01:16 +00:00
|
|
|
event, err := src.Next()
|
2018-08-06 23:46:17 +00:00
|
|
|
select {
|
2023-09-12 15:01:16 +00:00
|
|
|
case incomingEvents <- nextEvent{event, err}:
|
2018-08-21 21:05:00 +00:00
|
|
|
if event == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
case <-done:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): incoming events goroutine exiting")
|
2018-08-06 23:46:17 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
// The main loop. We'll continuously select for incoming events and the cancellation signal. There are
|
|
|
|
// a three ways we can exit this loop:
|
|
|
|
// 1. The SourceIterator sends us a `nil` event. This means that we're done processing source events and
|
|
|
|
// we should begin processing deletes.
|
|
|
|
// 2. The SourceIterator sends us an error. This means some error occurred in the source program and we
|
|
|
|
// should bail.
|
|
|
|
// 3. The stepExecCancel cancel context gets canceled. This means some error occurred in the step executor
|
|
|
|
// and we need to bail. This can also happen if the user hits Ctrl-C.
|
2023-10-09 10:44:05 +00:00
|
|
|
canceled, err := func() (bool, error) {
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): waiting for incoming events")
|
2018-08-21 21:05:00 +00:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case event := <-incomingEvents:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): incoming event (nil? %v, %v)", event.Event == nil,
|
2023-09-12 15:01:16 +00:00
|
|
|
event.Error)
|
2018-08-21 21:05:00 +00:00
|
|
|
|
2023-09-12 15:01:16 +00:00
|
|
|
if event.Error != nil {
|
|
|
|
if !result.IsBail(event.Error) {
|
|
|
|
ex.reportError("", event.Error)
|
2019-03-19 23:21:50 +00:00
|
|
|
}
|
2018-08-21 21:05:00 +00:00
|
|
|
cancel()
|
2019-03-19 23:21:50 +00:00
|
|
|
|
|
|
|
// We reported any errors above. So we can just bail now.
|
2023-10-09 10:44:05 +00:00
|
|
|
return false, result.BailError(event.Error)
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if event.Event == nil {
|
2023-05-23 20:17:59 +00:00
|
|
|
// Check targets before performDeletes mutates the initial Snapshot.
|
|
|
|
targetErr := ex.checkTargets(opts.Targets)
|
|
|
|
|
2023-10-09 10:44:05 +00:00
|
|
|
err := ex.performDeletes(ctx, opts.Targets)
|
|
|
|
if err != nil {
|
|
|
|
if !result.IsBail(err) {
|
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): error performing deletes: %v", err)
|
|
|
|
ex.reportError("", err)
|
|
|
|
return false, result.BailError(err)
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
}
|
2023-05-23 20:17:59 +00:00
|
|
|
|
|
|
|
if targetErr != nil {
|
|
|
|
// Propagate the target error as it hasn't been reported yet.
|
|
|
|
return false, targetErr
|
|
|
|
}
|
|
|
|
return false, nil
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
|
|
|
|
2023-09-25 12:25:23 +00:00
|
|
|
if err := ex.handleSingleEvent(event.Event); err != nil {
|
|
|
|
if !result.IsBail(err) {
|
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): error handling event: %v", err)
|
|
|
|
ex.reportError(ex.deployment.generateEventURN(event.Event), err)
|
2018-09-05 22:08:09 +00:00
|
|
|
}
|
2018-08-21 21:05:00 +00:00
|
|
|
cancel()
|
2023-10-09 10:44:05 +00:00
|
|
|
return false, result.BailError(err)
|
2018-08-21 21:05:00 +00:00
|
|
|
}
|
|
|
|
case <-ctx.Done():
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): context finished: %v", ctx.Err())
|
2018-08-21 21:05:00 +00:00
|
|
|
|
2018-08-23 00:52:46 +00:00
|
|
|
// NOTE: we use the presence of an error in the caller context in order to distinguish caller-initiated
|
2018-08-21 21:05:00 +00:00
|
|
|
// cancellation from internally-initiated cancellation.
|
2018-08-23 00:52:46 +00:00
|
|
|
return callerCtx.Err() != nil, nil
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}
|
2018-08-21 21:05:00 +00:00
|
|
|
}()
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.stepExec.WaitForCompletion()
|
2024-03-25 22:37:46 +00:00
|
|
|
|
|
|
|
stepExecutorError := ex.stepExec.Errored()
|
|
|
|
|
|
|
|
// Finalize the stack outputs.
|
|
|
|
if e := ex.stepExec.stackOutputsEvent; e != nil {
|
|
|
|
errored := err != nil || stepExecutorError != nil || ex.stepGen.Errored()
|
|
|
|
finalizingStackOutputs := true
|
|
|
|
if err := ex.stepExec.executeRegisterResourceOutputs(e, errored, finalizingStackOutputs); err != nil {
|
|
|
|
return nil, result.BailError(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): step executor has completed")
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2022-01-31 10:31:51 +00:00
|
|
|
// Check that we did operations for everything expected in the plan. We mutate ResourcePlan.Ops as we run
|
|
|
|
// so by the time we get here everything in the map should have an empty ops list (except for unneeded
|
|
|
|
// deletes). We skip this check if we already have an error, chances are if the deployment failed lots of
|
|
|
|
// operations wouldn't have got a chance to run so we'll spam errors about all of those failed operations
|
|
|
|
// making it less clear to the user what the root cause error was.
|
2023-10-09 10:44:05 +00:00
|
|
|
if err == nil && ex.deployment.plan != nil {
|
2022-01-31 10:31:51 +00:00
|
|
|
for urn, resourcePlan := range ex.deployment.plan.ResourcePlans {
|
|
|
|
if len(resourcePlan.Ops) != 0 {
|
|
|
|
if len(resourcePlan.Ops) == 1 && resourcePlan.Ops[0] == OpDelete {
|
|
|
|
// We haven't done a delete for this resource check if it was in the snapshot,
|
|
|
|
// if it's already gone this wasn't done because it wasn't needed
|
|
|
|
found := false
|
|
|
|
for i := range ex.deployment.prev.Resources {
|
|
|
|
if ex.deployment.prev.Resources[i].URN == urn {
|
|
|
|
found = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Didn't find the resource in the old snapshot so this was just an unneeded delete
|
|
|
|
if !found {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-10-09 10:44:05 +00:00
|
|
|
rErr := fmt.Errorf("expected resource operations for %v but none were seen", urn)
|
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): error handling event: %v", rErr)
|
|
|
|
ex.reportError(urn, rErr)
|
|
|
|
err = errors.Join(err, rErr)
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
}
|
2023-10-09 10:44:05 +00:00
|
|
|
// If we made any errors above wrap it in a bail
|
|
|
|
if err != nil {
|
|
|
|
err = result.BailError(err)
|
|
|
|
}
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
|
2023-10-09 10:44:05 +00:00
|
|
|
if err != nil && result.IsBail(err) {
|
|
|
|
return nil, err
|
2019-03-19 23:21:50 +00:00
|
|
|
}
|
|
|
|
|
2019-10-25 15:29:02 +00:00
|
|
|
// If the step generator and step executor were both successful, then we send all the resources
|
|
|
|
// observed to be analyzed. Otherwise, this step is skipped.
|
2023-10-17 06:47:32 +00:00
|
|
|
if err == nil && stepExecutorError == nil {
|
2023-10-06 15:57:27 +00:00
|
|
|
err := ex.stepGen.AnalyzeResources()
|
|
|
|
if err != nil {
|
|
|
|
if !result.IsBail(err) {
|
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): error analyzing resources: %v", err)
|
|
|
|
ex.reportError("", err)
|
2019-12-12 22:26:27 +00:00
|
|
|
}
|
2023-10-09 10:44:05 +00:00
|
|
|
return nil, result.BailErrorf("failed to analyze resources: %v", err)
|
2019-10-25 15:29:02 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-21 21:05:00 +00:00
|
|
|
// Figure out if execution failed and why. Step generation and execution errors trump cancellation.
|
2023-10-17 06:47:32 +00:00
|
|
|
if err != nil || stepExecutorError != nil || ex.stepGen.Errored() {
|
2019-03-20 21:56:12 +00:00
|
|
|
// TODO(cyrusn): We seem to be losing any information about the original 'res's errors. Should
|
|
|
|
// we be doing a merge here?
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportExecResult("failed", preview)
|
2023-10-09 10:44:05 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, result.BailError(err)
|
|
|
|
}
|
2023-10-17 06:47:32 +00:00
|
|
|
if stepExecutorError != nil {
|
|
|
|
return nil, result.BailErrorf("step executor errored: %w", stepExecutorError)
|
2023-10-09 10:44:05 +00:00
|
|
|
}
|
|
|
|
return nil, result.BailErrorf("step generator errored")
|
2018-08-21 21:05:00 +00:00
|
|
|
} else if canceled {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportExecResult("canceled", preview)
|
2023-10-09 10:44:05 +00:00
|
|
|
return nil, result.BailErrorf("canceled")
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
2019-03-20 21:56:12 +00:00
|
|
|
|
2023-10-09 10:44:05 +00:00
|
|
|
return ex.deployment.newPlans.plan(), err
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
func (ex *deploymentExecutor) performDeletes(
|
2023-05-23 20:17:59 +00:00
|
|
|
ctx context.Context, targetsOpt UrnTargets,
|
2023-10-09 10:44:05 +00:00
|
|
|
) error {
|
2019-09-20 02:28:14 +00:00
|
|
|
defer func() {
|
|
|
|
// We're done here - signal completion so that the step executor knows to terminate.
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.stepExec.SignalCompletion()
|
2019-09-20 02:28:14 +00:00
|
|
|
}()
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
prev := ex.deployment.prev
|
2019-09-20 02:28:14 +00:00
|
|
|
if prev == nil || len(prev.Resources) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
logging.V(7).Infof("performDeletes(...): beginning")
|
|
|
|
|
Deepcopy event payloads on construction _not_ on access (#14049)
<!---
Thanks so much for your contribution! If this is your first time
contributing, please ensure that you have read the
[CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md)
documentation.
-->
# Description
<!--- Please include a summary of the change and which issue is fixed.
Please also include relevant motivation and context. -->
Fixes https://github.com/pulumi/pulumi/issues/14036.
Not sure this is perfect (it's pretty hard to reason about mutation of
the whole system given what Go gives us to work with), but it seems
better.
We copy event payloads on construction, which should be synchronous with
step execution, rather than on access which could be in parallel with
the step executor or deployment executor mutating resource state.
Just copying on construction flagged up that the deployment executor and
step executor both race to modify resource state when the deployment
executor starts scheduling deletes. We add a synchronisation point
between the two parts in this change so that the deployment executor
waits for the step executor to go quite, and keeps it quite while it
computes the deletes, once it starts scheduling them the step executor
can run again.
## Checklist
- [x] I have run `make tidy` to update any new dependencies
- [x] I have run `make lint` to verify my code passes the lint check
- [ ] I have formatted my code using `gofumpt`
<!--- Please provide details if the checkbox below is to be left
unchecked. -->
- [ ] I have added tests that prove my fix is effective or that my
feature works
<!---
User-facing changes require a CHANGELOG entry.
-->
- [x] I have run `make changelog` and committed the
`changelog/pending/<file>` documenting my change
<!--
If the change(s) in this PR is a modification of an existing call to the
Pulumi Cloud,
then the service should honor older versions of the CLI where this
change would not exist.
You must then bump the API version in
/pkg/backend/httpstate/client/api.go, as well as add
it to the service.
-->
- [ ] Yes, there are changes in this PR that warrants bumping the Pulumi
Cloud API version
<!-- @Pulumi employees: If yes, you must submit corresponding changes in
the service repo. -->
2023-10-02 14:15:10 +00:00
|
|
|
// GenerateDeletes mutates state we need to lock the step executor while we do this.
|
|
|
|
ex.stepExec.Lock()
|
|
|
|
|
2019-10-01 06:41:56 +00:00
|
|
|
// At this point we have generated the set of resources above that we would normally want to
|
|
|
|
// delete. However, if the user provided -target's we will only actually delete the specific
|
|
|
|
// resources that are in the set explicitly asked for.
|
2023-09-25 12:25:23 +00:00
|
|
|
deleteSteps, err := ex.stepGen.GenerateDeletes(targetsOpt)
|
Deepcopy event payloads on construction _not_ on access (#14049)
<!---
Thanks so much for your contribution! If this is your first time
contributing, please ensure that you have read the
[CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md)
documentation.
-->
# Description
<!--- Please include a summary of the change and which issue is fixed.
Please also include relevant motivation and context. -->
Fixes https://github.com/pulumi/pulumi/issues/14036.
Not sure this is perfect (it's pretty hard to reason about mutation of
the whole system given what Go gives us to work with), but it seems
better.
We copy event payloads on construction, which should be synchronous with
step execution, rather than on access which could be in parallel with
the step executor or deployment executor mutating resource state.
Just copying on construction flagged up that the deployment executor and
step executor both race to modify resource state when the deployment
executor starts scheduling deletes. We add a synchronisation point
between the two parts in this change so that the deployment executor
waits for the step executor to go quite, and keeps it quite while it
computes the deletes, once it starts scheduling them the step executor
can run again.
## Checklist
- [x] I have run `make tidy` to update any new dependencies
- [x] I have run `make lint` to verify my code passes the lint check
- [ ] I have formatted my code using `gofumpt`
<!--- Please provide details if the checkbox below is to be left
unchecked. -->
- [ ] I have added tests that prove my fix is effective or that my
feature works
<!---
User-facing changes require a CHANGELOG entry.
-->
- [x] I have run `make changelog` and committed the
`changelog/pending/<file>` documenting my change
<!--
If the change(s) in this PR is a modification of an existing call to the
Pulumi Cloud,
then the service should honor older versions of the CLI where this
change would not exist.
You must then bump the API version in
/pkg/backend/httpstate/client/api.go, as well as add
it to the service.
-->
- [ ] Yes, there are changes in this PR that warrants bumping the Pulumi
Cloud API version
<!-- @Pulumi employees: If yes, you must submit corresponding changes in
the service repo. -->
2023-10-02 14:15:10 +00:00
|
|
|
// Regardless of if this error'd or not the step executor needs unlocking
|
|
|
|
ex.stepExec.Unlock()
|
2023-09-25 12:25:23 +00:00
|
|
|
if err != nil {
|
2019-09-20 02:28:14 +00:00
|
|
|
logging.V(7).Infof("performDeletes(...): generating deletes produced error result")
|
2023-10-09 10:44:05 +00:00
|
|
|
return err
|
2019-09-20 02:28:14 +00:00
|
|
|
}
|
|
|
|
|
2024-03-22 09:22:40 +00:00
|
|
|
deleteChains := ex.stepGen.ScheduleDeletes(deleteSteps)
|
2019-09-20 02:28:14 +00:00
|
|
|
|
|
|
|
// ScheduleDeletes gives us a list of lists of steps. Each list of steps can safely be executed
|
|
|
|
// in parallel, but each list must execute completes before the next list can safely begin
|
|
|
|
// executing.
|
|
|
|
//
|
|
|
|
// This is not "true" delete parallelism, since there may be resources that could safely begin
|
|
|
|
// deleting but we won't until the previous set of deletes fully completes. This approximation
|
|
|
|
// is conservative, but correct.
|
2024-03-22 09:22:40 +00:00
|
|
|
erroredDeps := mapset.NewSet[*resource.State]()
|
|
|
|
seenErrors := mapset.NewSet[Step]()
|
|
|
|
for _, antichain := range deleteChains {
|
|
|
|
if ex.stepExec.opts.ContinueOnError {
|
|
|
|
erroredSteps := ex.stepExec.GetErroredSteps()
|
|
|
|
for _, step := range erroredSteps {
|
|
|
|
if seenErrors.Contains(step) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
deps := ex.deployment.depGraph.TransitiveDependenciesOf(step.Res())
|
|
|
|
erroredDeps = erroredDeps.Union(deps)
|
|
|
|
}
|
|
|
|
seenErrors.Append(erroredSteps...)
|
|
|
|
newChain := make([]Step, 0, len(antichain))
|
|
|
|
for _, step := range antichain {
|
|
|
|
if !erroredDeps.Contains(step.Res()) {
|
|
|
|
newChain = append(newChain, step)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
antichain = newChain
|
|
|
|
}
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): beginning delete antichain")
|
|
|
|
tok := ex.stepExec.ExecuteParallel(antichain)
|
2019-09-20 02:28:14 +00:00
|
|
|
tok.Wait(ctx)
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.Execute(...): antichain complete")
|
2019-09-20 02:28:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
// handleSingleEvent handles a single source event. For all incoming events, it produces a chain that needs
|
|
|
|
// to be executed and schedules the chain for execution.
|
2023-09-25 12:25:23 +00:00
|
|
|
func (ex *deploymentExecutor) handleSingleEvent(event SourceEvent) error {
|
2023-02-17 20:05:48 +00:00
|
|
|
contract.Requiref(event != nil, "event", "must not be nil")
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2018-08-21 21:05:00 +00:00
|
|
|
var steps []Step
|
2023-09-25 12:25:23 +00:00
|
|
|
var err error
|
2018-08-06 23:46:17 +00:00
|
|
|
switch e := event.(type) {
|
|
|
|
case RegisterResourceEvent:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.handleSingleEvent(...): received RegisterResourceEvent")
|
2023-09-25 12:25:23 +00:00
|
|
|
steps, err = ex.stepGen.GenerateSteps(e)
|
2018-08-06 23:46:17 +00:00
|
|
|
case ReadResourceEvent:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.handleSingleEvent(...): received ReadResourceEvent")
|
2023-09-25 12:25:23 +00:00
|
|
|
steps, err = ex.stepGen.GenerateReadSteps(e)
|
2018-08-06 23:46:17 +00:00
|
|
|
case RegisterResourceOutputsEvent:
|
2020-11-18 17:47:52 +00:00
|
|
|
logging.V(4).Infof("deploymentExecutor.handleSingleEvent(...): received register resource outputs")
|
2023-09-25 12:25:23 +00:00
|
|
|
return ex.stepExec.ExecuteRegisterResourceOutputs(e)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2023-09-25 12:25:23 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
2024-04-22 11:12:45 +00:00
|
|
|
// Exclude the steps that depend on errored steps if ContinueOnError is set.
|
|
|
|
var newSteps []Step
|
|
|
|
skipped := false
|
|
|
|
if ex.stepExec.opts.ContinueOnError {
|
|
|
|
for _, errored := range ex.stepExec.GetErroredSteps() {
|
|
|
|
ex.skipped.Add(errored.Res().URN)
|
|
|
|
}
|
|
|
|
outer:
|
|
|
|
for _, step := range steps {
|
|
|
|
for _, dep := range step.Res().Dependencies {
|
|
|
|
if ex.skipped.Contains(dep) {
|
|
|
|
step.Skip()
|
|
|
|
ex.skipped.Add(step.Res().URN)
|
|
|
|
skipped = true
|
|
|
|
continue outer
|
|
|
|
}
|
|
|
|
}
|
|
|
|
newSteps = append(newSteps, step)
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
newSteps = steps
|
|
|
|
}
|
|
|
|
|
|
|
|
// If we pass an empty chain to the step executors the workers will shut down. However we don't want that
|
|
|
|
// if we just skipped a step because its dependencies errored out. Return early in that case.
|
|
|
|
if skipped && len(newSteps) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
2018-09-05 22:08:09 +00:00
|
|
|
|
2024-04-22 11:12:45 +00:00
|
|
|
ex.stepExec.ExecuteSerial(newSteps)
|
2018-08-21 21:05:00 +00:00
|
|
|
return nil
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
2018-08-23 00:52:46 +00:00
|
|
|
|
2020-10-14 11:51:53 +00:00
|
|
|
// import imports a list of resources into a stack.
|
2022-01-31 10:31:51 +00:00
|
|
|
func (ex *deploymentExecutor) importResources(
|
|
|
|
callerCtx context.Context,
|
|
|
|
opts Options,
|
2023-03-03 16:36:39 +00:00
|
|
|
preview bool,
|
2023-10-09 10:44:05 +00:00
|
|
|
) (*Plan, error) {
|
2020-11-18 17:47:52 +00:00
|
|
|
if len(ex.deployment.imports) == 0 {
|
2022-01-31 10:31:51 +00:00
|
|
|
return nil, nil
|
2020-10-14 11:51:53 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Create an executor for this import.
|
|
|
|
ctx, cancel := context.WithCancel(callerCtx)
|
2020-11-18 17:47:52 +00:00
|
|
|
stepExec := newStepExecutor(ctx, cancel, ex.deployment, opts, preview, true)
|
2020-10-14 11:51:53 +00:00
|
|
|
|
|
|
|
importer := &importer{
|
2020-11-18 17:47:52 +00:00
|
|
|
deployment: ex.deployment,
|
|
|
|
executor: stepExec,
|
|
|
|
preview: preview,
|
2020-10-14 11:51:53 +00:00
|
|
|
}
|
2023-10-01 14:21:32 +00:00
|
|
|
err := importer.importResources(ctx)
|
2020-10-14 11:51:53 +00:00
|
|
|
stepExec.SignalCompletion()
|
|
|
|
stepExec.WaitForCompletion()
|
|
|
|
|
|
|
|
// NOTE: we use the presence of an error in the caller context in order to distinguish caller-initiated
|
|
|
|
// cancellation from internally-initiated cancellation.
|
|
|
|
canceled := callerCtx.Err() != nil
|
|
|
|
|
2023-10-17 06:47:32 +00:00
|
|
|
stepExecutorError := stepExec.Errored()
|
|
|
|
if err != nil || stepExecutorError != nil {
|
2023-10-01 14:21:32 +00:00
|
|
|
if err != nil && !result.IsBail(err) {
|
|
|
|
ex.reportExecResult(fmt.Sprintf("failed: %s", err), preview)
|
2020-11-24 17:14:09 +00:00
|
|
|
} else {
|
|
|
|
ex.reportExecResult("failed", preview)
|
|
|
|
}
|
2023-10-09 10:44:05 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, result.BailError(err)
|
|
|
|
}
|
2023-10-17 06:47:32 +00:00
|
|
|
return nil, result.BailErrorf("step executor errored: %w", stepExecutorError)
|
2020-10-14 11:51:53 +00:00
|
|
|
} else if canceled {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportExecResult("canceled", preview)
|
2023-10-09 10:44:05 +00:00
|
|
|
return nil, result.BailErrorf("canceled")
|
2020-10-14 11:51:53 +00:00
|
|
|
}
|
2022-01-31 10:31:51 +00:00
|
|
|
return ex.deployment.newPlans.plan(), nil
|
2020-10-14 11:51:53 +00:00
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
// refresh refreshes the state of the base checkpoint file for the current deployment in memory.
|
2023-10-09 10:44:05 +00:00
|
|
|
func (ex *deploymentExecutor) refresh(callerCtx context.Context, opts Options, preview bool) error {
|
2020-11-18 17:47:52 +00:00
|
|
|
prev := ex.deployment.prev
|
2018-08-23 00:52:46 +00:00
|
|
|
if prev == nil || len(prev.Resources) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-09-21 00:50:44 +00:00
|
|
|
// Make sure if there were any targets specified, that they all refer to existing resources.
|
2023-10-09 10:44:05 +00:00
|
|
|
if err := ex.checkTargets(opts.Targets); err != nil {
|
|
|
|
return err
|
2019-09-18 21:28:42 +00:00
|
|
|
}
|
|
|
|
|
2019-09-18 01:14:10 +00:00
|
|
|
// If the user did not provide any --target's, create a refresh step for each resource in the
|
|
|
|
// old snapshot. If they did provider --target's then only create refresh steps for those
|
|
|
|
// specific targets.
|
|
|
|
steps := []Step{}
|
|
|
|
resourceToStep := map[*resource.State]Step{}
|
|
|
|
for _, res := range prev.Resources {
|
2023-05-23 20:17:59 +00:00
|
|
|
if opts.Targets.Contains(res.URN) {
|
2023-04-12 09:35:20 +00:00
|
|
|
// For each resource we're going to refresh we need to ensure we have a provider for it
|
|
|
|
err := ex.deployment.EnsureProvider(res.Provider)
|
|
|
|
if err != nil {
|
2023-10-09 10:44:05 +00:00
|
|
|
return fmt.Errorf("could not load provider for resource %v: %w", res.URN, err)
|
2023-04-12 09:35:20 +00:00
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
step := NewRefreshStep(ex.deployment, res, nil)
|
2019-09-18 01:14:10 +00:00
|
|
|
steps = append(steps, step)
|
|
|
|
resourceToStep[res] = step
|
|
|
|
}
|
2018-08-30 04:00:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Fire up a worker pool and issue each refresh in turn.
|
|
|
|
ctx, cancel := context.WithCancel(callerCtx)
|
2020-11-18 17:47:52 +00:00
|
|
|
stepExec := newStepExecutor(ctx, cancel, ex.deployment, opts, preview, true)
|
2018-09-27 22:49:08 +00:00
|
|
|
stepExec.ExecuteParallel(steps)
|
2018-08-23 00:52:46 +00:00
|
|
|
stepExec.SignalCompletion()
|
|
|
|
stepExec.WaitForCompletion()
|
|
|
|
|
2023-12-05 17:19:10 +00:00
|
|
|
ex.rebuildBaseState(resourceToStep)
|
2019-09-20 02:28:14 +00:00
|
|
|
|
|
|
|
// NOTE: we use the presence of an error in the caller context in order to distinguish caller-initiated
|
|
|
|
// cancellation from internally-initiated cancellation.
|
|
|
|
canceled := callerCtx.Err() != nil
|
|
|
|
|
2023-10-17 06:47:32 +00:00
|
|
|
stepExecutorError := stepExec.Errored()
|
|
|
|
if stepExecutorError != nil {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportExecResult("failed", preview)
|
2023-10-17 06:47:32 +00:00
|
|
|
return result.BailErrorf("step executor errored: %w", stepExecutorError)
|
2019-09-20 02:28:14 +00:00
|
|
|
} else if canceled {
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.reportExecResult("canceled", preview)
|
2023-10-09 10:44:05 +00:00
|
|
|
return result.BailErrorf("canceled")
|
2019-09-20 02:28:14 +00:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-12-05 17:19:10 +00:00
|
|
|
func (ex *deploymentExecutor) rebuildBaseState(resourceToStep map[*resource.State]Step) {
|
2020-11-18 17:47:52 +00:00
|
|
|
// Rebuild this deployment's map of old resources and dependency graph, stripping out any deleted
|
2019-09-18 01:14:10 +00:00
|
|
|
// resources and repairing dependency lists as necessary. Note that this updates the base
|
|
|
|
// snapshot _in memory_, so it is critical that any components that use the snapshot refer to
|
|
|
|
// the same instance and avoid reading it concurrently with this rebuild.
|
2018-08-23 00:52:46 +00:00
|
|
|
//
|
2019-09-18 01:14:10 +00:00
|
|
|
// The process of repairing dependency lists is a bit subtle. Because multiple physical
|
|
|
|
// resources may share a URN, the ability of a particular URN to be referenced in a dependency
|
|
|
|
// list can change based on the dependent resource's position in the resource list. For example,
|
|
|
|
// consider the following list of resources, where each resource is a (URN, ID, Dependencies)
|
|
|
|
// tuple:
|
2018-08-23 00:52:46 +00:00
|
|
|
//
|
|
|
|
// [ (A, 0, []), (B, 0, [A]), (A, 1, []), (A, 2, []), (C, 0, [A]) ]
|
|
|
|
//
|
2019-09-18 01:14:10 +00:00
|
|
|
// Let `(A, 0, [])` and `(A, 2, [])` be deleted by the refresh. This produces the following
|
|
|
|
// intermediate list before dependency lists are repaired:
|
2018-08-23 00:52:46 +00:00
|
|
|
//
|
|
|
|
// [ (B, 0, [A]), (A, 1, []), (C, 0, [A]) ]
|
|
|
|
//
|
2019-09-18 01:14:10 +00:00
|
|
|
// In order to repair the dependency lists, we iterate over the intermediate resource list,
|
|
|
|
// keeping track of which URNs refer to at least one physical resource at each point in the
|
|
|
|
// list, and remove any dependencies that refer to URNs that do not refer to any physical
|
|
|
|
// resources. This process produces the following final list:
|
2018-08-23 00:52:46 +00:00
|
|
|
//
|
|
|
|
// [ (B, 0, []), (A, 1, []), (C, 0, [A]) ]
|
|
|
|
//
|
2019-09-18 01:14:10 +00:00
|
|
|
// Note that the correctness of this process depends on the fact that the list of resources is a
|
|
|
|
// topological sort of its corresponding dependency graph, so a resource always appears in the
|
|
|
|
// list after any resources on which it may depend.
|
|
|
|
resources := []*resource.State{}
|
2018-08-23 00:52:46 +00:00
|
|
|
referenceable := make(map[resource.URN]bool)
|
|
|
|
olds := make(map[resource.URN]*resource.State)
|
2020-11-18 17:47:52 +00:00
|
|
|
for _, s := range ex.deployment.prev.Resources {
|
2019-09-18 01:14:10 +00:00
|
|
|
var old, new *resource.State
|
|
|
|
if step, has := resourceToStep[s]; has {
|
2020-10-14 11:51:53 +00:00
|
|
|
// We produced a refresh step for this specific resource. Use the new information about
|
2019-09-18 01:14:10 +00:00
|
|
|
// its dependencies during the update.
|
|
|
|
old = step.Old()
|
|
|
|
new = step.New()
|
|
|
|
} else {
|
|
|
|
// We didn't do anything with this resource. However, we still may want to update its
|
|
|
|
// dependencies. So use this resource itself as the 'new' one to update.
|
|
|
|
old = s
|
|
|
|
new = s
|
|
|
|
}
|
|
|
|
|
2018-08-23 00:52:46 +00:00
|
|
|
if new == nil {
|
2023-12-05 17:19:10 +00:00
|
|
|
contract.Assertf(old.Custom, "expected custom resource")
|
|
|
|
contract.Assertf(!providers.IsProviderType(old.Type), "expected non-provider resource")
|
2018-08-23 00:52:46 +00:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// Remove any deleted resources from this resource's dependency list.
|
|
|
|
if len(new.Dependencies) != 0 {
|
2023-06-28 16:02:04 +00:00
|
|
|
deps := slice.Prealloc[resource.URN](len(new.Dependencies))
|
2018-08-23 00:52:46 +00:00
|
|
|
for _, d := range new.Dependencies {
|
|
|
|
if referenceable[d] {
|
|
|
|
deps = append(deps, d)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
new.Dependencies = deps
|
|
|
|
}
|
|
|
|
|
Better handle property dependencies and `deletedWith` (#16088)
A resource `A` depends on a resource `B` if:
1. `B` is `A`'s `Provider`.
2. `B` is `A`'s `Parent`.
3. `B` appears in `A`'s `Dependencies`.
4. `B` appears in one or more of `A`'s `PropertyDependencies`.
5. `B` is referenced by `A`'s `DeletedWith` field.
While cases 1, 2, and 3 (providers, parents, and dependencies) are
handled fairly consistently, there have been a number of cases where the
newer features of `PropertyDependencies` (case 4) and `DeletedWith`
(case 5) have been neglected. This commit addresses some of these
omissions. Specifically:
* When refreshing state, it's important that we remove URNs that point
to resources that we've identified as deleted. Presently we check
pointers to parents and dependencies, but not property dependencies or
`deletedWith`. This commit fixes these gaps where dangling URNs could
lurk.
* Linked to the above, this commit extends snapshot integrity checks to
include property dependencies and `deletedWith`. Some tests that
previously used now invalid states have to be repaired or removed as a
result of this.
* Fixing snapshot integrity checking reveals that dependency graph
checks also fail to consider property dependencies and `deletedWith`.
This probably hasn't bitten us since property dependencies are currently
rolled up into dependencies (for temporary backwards compatibility) and
`deletedWith` is typically an optimisation (moreover, one that only
matters during deletes), so operations being parallelised due a
perceived lack of dependency still succeed. However, tests that
previously passed now fail as we can spot these races with our better
integrity checks. This commit thus fixes up dependency graph
construction and bulks out its test suite to cover the new cases.
These bugs were discovered as part of the investigation into #16052,
though they may not be directly responsible for it (though the issues
with dependency graphs are certainly a candidate).
2024-05-03 17:08:06 +00:00
|
|
|
// Remove any deleted resources from this resource's property dependencies
|
|
|
|
// lists. If we end up emptying a property dependency list, we'll remove the
|
|
|
|
// property from the map altogether.
|
|
|
|
for prop, deps := range new.PropertyDependencies {
|
|
|
|
if len(deps) != 0 {
|
|
|
|
newDeps := slice.Prealloc[resource.URN](len(deps))
|
|
|
|
for _, d := range deps {
|
|
|
|
if referenceable[d] {
|
|
|
|
newDeps = append(newDeps, d)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(newDeps) > 0 {
|
|
|
|
new.PropertyDependencies[prop] = newDeps
|
|
|
|
} else {
|
|
|
|
delete(new.PropertyDependencies, prop)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Remove any deleted resources from DeletedWith properties.
|
|
|
|
if new.DeletedWith != "" && !referenceable[new.DeletedWith] {
|
|
|
|
new.DeletedWith = ""
|
|
|
|
}
|
|
|
|
|
2018-08-23 00:52:46 +00:00
|
|
|
// Add this resource to the resource list and mark it as referenceable.
|
|
|
|
resources = append(resources, new)
|
|
|
|
referenceable[new.URN] = true
|
|
|
|
|
|
|
|
// Do not record resources that are pending deletion in the "olds" lookup table.
|
|
|
|
if !new.Delete {
|
|
|
|
olds[new.URN] = new
|
|
|
|
}
|
|
|
|
}
|
2019-09-18 01:14:10 +00:00
|
|
|
|
2022-07-08 21:23:58 +00:00
|
|
|
undangleParentResources(olds, resources)
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
ex.deployment.prev.Resources = resources
|
|
|
|
ex.deployment.olds, ex.deployment.depGraph = olds, graph.NewDependencyGraph(resources)
|
2018-08-23 00:52:46 +00:00
|
|
|
}
|
2022-07-08 21:23:58 +00:00
|
|
|
|
|
|
|
func undangleParentResources(undeleted map[resource.URN]*resource.State, resources []*resource.State) {
|
|
|
|
// Since a refresh may delete arbitrary resources, we need to handle the case where
|
|
|
|
// the parent of a still existing resource is deleted.
|
|
|
|
//
|
|
|
|
// Invalid parents need to be fixed since otherwise they leave the state invalid, and
|
|
|
|
// the user sees an error:
|
|
|
|
// ```
|
|
|
|
// snapshot integrity failure; refusing to use it: child resource ${validURN} refers to missing parent ${deletedURN}
|
|
|
|
// ```
|
|
|
|
// To solve the problem we traverse the topologically sorted list of resources in
|
|
|
|
// order, setting newly invalidated parent URNS to the URN of the parent's parent.
|
|
|
|
//
|
|
|
|
// This can be illustrated by an example. Consider the graph of resource parents:
|
|
|
|
//
|
|
|
|
// A xBx
|
|
|
|
// / \ |
|
|
|
|
// xCx D xEx
|
|
|
|
// | / \ |
|
|
|
|
// F G xHx I
|
|
|
|
//
|
|
|
|
// When a capital letter is marked for deletion, it is bracketed by `x`s.
|
|
|
|
// We can obtain a topological sort by reading left to right, top to bottom.
|
|
|
|
//
|
|
|
|
// A..D -> valid parents, so we do nothing
|
|
|
|
// E -> The parent of E is marked for deletion, so set E.Parent to E.Parent.Parent.
|
|
|
|
// Since B (E's parent) has no parent, we set E.Parent to "".
|
|
|
|
// F -> The parent of F is marked for deletion, so set F.Parent to F.Parent.Parent.
|
|
|
|
// We set F.Parent to "A"
|
|
|
|
// G, H -> valid parents, do nothing
|
|
|
|
// I -> The parent of I is marked for deletion, so set I.Parent to I.Parent.Parent.
|
|
|
|
// The parent of I has parent "", (since we addressed the parent of E
|
|
|
|
// previously), so we set I.Parent = "".
|
|
|
|
//
|
|
|
|
// The new graph looks like this:
|
|
|
|
//
|
|
|
|
// A xBx xEx I
|
|
|
|
// / | \
|
|
|
|
// xCx F D
|
|
|
|
// / \
|
|
|
|
// G xHx
|
|
|
|
// We observe that it is perfectly valid for deleted nodes to be leaf nodes, but they
|
|
|
|
// cannot be intermediary nodes.
|
|
|
|
_, hasEmptyValue := undeleted[""]
|
|
|
|
contract.Assertf(!hasEmptyValue, "the zero value for an URN is not a valid URN")
|
|
|
|
availableParents := map[resource.URN]resource.URN{}
|
|
|
|
for _, r := range resources {
|
|
|
|
if _, ok := undeleted[r.Parent]; !ok {
|
|
|
|
// Since existing must obey a topological sort, we have already addressed
|
|
|
|
// p.Parent. Since we know that it doesn't dangle, and that r.Parent no longer
|
|
|
|
// exists, we set r.Parent as r.Parent.Parent.
|
|
|
|
r.Parent = availableParents[r.Parent]
|
|
|
|
}
|
|
|
|
availableParents[r.URN] = r.Parent
|
|
|
|
}
|
|
|
|
}
|