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"
|
2018-08-06 23:46:17 +00:00
|
|
|
"fmt"
|
|
|
|
"sync"
|
|
|
|
"sync/atomic"
|
|
|
|
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/diag"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/contract"
|
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/logging"
|
2018-08-06 23:46:17 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
|
|
|
// Dummy workerID for synchronous operations.
|
|
|
|
synchronousWorkerID = -1
|
2018-10-17 22:33:26 +00:00
|
|
|
infiniteWorkerID = -2
|
2018-08-06 23:46:17 +00:00
|
|
|
|
|
|
|
// Utility constant for easy debugging.
|
|
|
|
stepExecutorLogLevel = 4
|
|
|
|
)
|
|
|
|
|
2023-10-31 17:37:41 +00:00
|
|
|
// StepApplyFailed is a sentinel error for errors that arise when step application fails.
|
2023-03-03 16:36:39 +00:00
|
|
|
// We (the step executor) are not responsible for reporting those errors so this sentinel ensures
|
|
|
|
// that we don't do so.
|
2023-10-31 17:37:41 +00:00
|
|
|
type StepApplyFailed struct {
|
|
|
|
Err error
|
|
|
|
}
|
|
|
|
|
|
|
|
func (saf StepApplyFailed) Error() string {
|
|
|
|
return fmt.Sprintf("step application failed: %s", saf.Err)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (saf StepApplyFailed) Unwrap() error {
|
|
|
|
return saf.Err
|
|
|
|
}
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2018-09-27 22:49:08 +00:00
|
|
|
// The step executor operates in terms of "chains" and "antichains". A chain is set of steps that are totally ordered
|
|
|
|
// when ordered by dependency; each step in a chain depends directly on the step that comes before it. An antichain
|
|
|
|
// is a set of steps that is completely incomparable when ordered by dependency. The step executor is aware that chains
|
|
|
|
// must be executed serially and antichains can be executed concurrently.
|
|
|
|
//
|
|
|
|
// See https://en.wikipedia.org/wiki/Antichain for more complete definitions. The below type aliases are useful for
|
|
|
|
// documentation purposes.
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
// A Chain is a sequence of Steps that must be executed in the given order.
|
2018-09-27 22:49:08 +00:00
|
|
|
type chain = []Step
|
|
|
|
|
|
|
|
// An Antichain is a set of Steps that can be executed in parallel.
|
|
|
|
type antichain = []Step
|
|
|
|
|
|
|
|
// A CompletionToken is a token returned by the step executor that is completed when the chain has completed execution.
|
|
|
|
// Callers can use it to optionally wait synchronously on the completion of a chain.
|
|
|
|
type completionToken struct {
|
|
|
|
channel chan bool
|
|
|
|
}
|
|
|
|
|
|
|
|
// Wait blocks until the completion token is signalled or until the given context completes, whatever occurs first.
|
|
|
|
func (c completionToken) Wait(ctx context.Context) {
|
|
|
|
select {
|
|
|
|
case <-c.channel:
|
|
|
|
case <-ctx.Done():
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// incomingChain represents a request to the step executor to execute a chain.
|
|
|
|
type incomingChain struct {
|
|
|
|
Chain chain // The chain we intend to execute
|
|
|
|
CompletionChan chan bool // A completion channel to be closed when the chain has completed execution
|
|
|
|
}
|
2018-08-06 23:46:17 +00:00
|
|
|
|
|
|
|
// stepExecutor is the component of the engine responsible for taking steps and executing
|
|
|
|
// them, possibly in parallel if requested. The step generator operates on the granularity
|
|
|
|
// of "chains", which are sequences of steps that must be executed exactly in the given order.
|
|
|
|
// Chains are a simplification of the full dependency graph DAG within Pulumi programs. Since
|
|
|
|
// Pulumi language hosts can only invoke the resource monitor once all of their dependencies have
|
|
|
|
// resolved, we (the engine) can assume that any chain given to us by the step generator is already
|
|
|
|
// ready to execute.
|
|
|
|
type stepExecutor struct {
|
2020-11-18 17:47:52 +00:00
|
|
|
deployment *Deployment // The deployment currently being executed.
|
|
|
|
opts Options // The options for this current deployment.
|
|
|
|
preview bool // Whether or not we are doing a preview.
|
|
|
|
pendingNews sync.Map // Resources that have been created but are pending a RegisterResourceOutputs.
|
|
|
|
continueOnError bool // True if we want to continue the deployment after a step error.
|
2018-08-06 23:46:17 +00:00
|
|
|
|
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
|
|
|
// Lock protecting the running of workers. This can be used to synchronize with step executor.
|
|
|
|
workerLock sync.RWMutex
|
|
|
|
|
2018-09-27 22:49:08 +00:00
|
|
|
workers sync.WaitGroup // WaitGroup tracking the worker goroutines that are owned by this step executor.
|
|
|
|
incomingChains chan incomingChain // Incoming chains that we are to execute
|
2018-08-06 23:46:17 +00:00
|
|
|
|
2023-10-17 06:47:32 +00:00
|
|
|
ctx context.Context // cancellation context for the current deployment.
|
|
|
|
cancel context.CancelFunc // CancelFunc that cancels the above context.
|
|
|
|
|
|
|
|
// atomic error indicating an error seen by the step executor, if multiple errors are seen this will only hold one.
|
|
|
|
sawError atomic.Value
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
//
|
|
|
|
// The stepExecutor communicates with a stepGenerator by listening to a channel. As the step generator
|
|
|
|
// generates new chains that need to be executed, the step executor will listen to this channel to execute
|
|
|
|
// those steps.
|
|
|
|
//
|
|
|
|
|
|
|
|
// Execute submits a Chain for asynchronous execution. The execution of the chain will begin as soon as there
|
|
|
|
// is a worker available to execute it.
|
2018-09-27 22:49:08 +00:00
|
|
|
func (se *stepExecutor) ExecuteSerial(chain chain) completionToken {
|
2018-08-06 23:46:17 +00:00
|
|
|
// The select here is to avoid blocking on a send to se.incomingChains if a cancellation is pending.
|
|
|
|
// If one is pending, we should exit early - we will shortly be tearing down the engine and exiting.
|
2018-09-27 22:49:08 +00:00
|
|
|
|
|
|
|
completion := make(chan bool)
|
2018-08-06 23:46:17 +00:00
|
|
|
select {
|
2018-09-27 22:49:08 +00:00
|
|
|
case se.incomingChains <- incomingChain{Chain: chain, CompletionChan: completion}:
|
2018-08-06 23:46:17 +00:00
|
|
|
case <-se.ctx.Done():
|
2018-09-27 22:49:08 +00:00
|
|
|
close(completion)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
2018-09-27 22:49:08 +00:00
|
|
|
|
|
|
|
return completionToken{channel: completion}
|
|
|
|
}
|
|
|
|
|
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
|
|
|
// Locks the step executor from executing any more steps. This is used to synchronize with the step executor.
|
|
|
|
func (se *stepExecutor) Lock() {
|
|
|
|
se.workerLock.Lock()
|
|
|
|
}
|
|
|
|
|
|
|
|
// Unlocks the step executor to allow it to execute more steps. This is used to synchronize with the step executor.
|
|
|
|
func (se *stepExecutor) Unlock() {
|
|
|
|
se.workerLock.Unlock()
|
|
|
|
}
|
|
|
|
|
2018-09-27 22:49:08 +00:00
|
|
|
// ExecuteParallel submits an antichain for parallel execution. All of the steps within the antichain are submitted for
|
|
|
|
// concurrent execution.
|
|
|
|
func (se *stepExecutor) ExecuteParallel(antichain antichain) completionToken {
|
|
|
|
var wg sync.WaitGroup
|
|
|
|
|
|
|
|
// ExecuteParallel is implemented in terms of ExecuteSerial - it executes each step individually and waits for all
|
|
|
|
// of the steps to complete.
|
|
|
|
wg.Add(len(antichain))
|
|
|
|
for _, step := range antichain {
|
|
|
|
tok := se.ExecuteSerial(chain{step})
|
|
|
|
go func() {
|
|
|
|
defer wg.Done()
|
|
|
|
tok.Wait(se.ctx)
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
|
|
|
done := make(chan bool)
|
|
|
|
go func() {
|
|
|
|
wg.Wait()
|
|
|
|
close(done)
|
|
|
|
}()
|
|
|
|
|
|
|
|
return completionToken{channel: done}
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// ExecuteRegisterResourceOutputs services a RegisterResourceOutputsEvent synchronously on the calling goroutine.
|
2023-09-11 12:35:48 +00:00
|
|
|
func (se *stepExecutor) ExecuteRegisterResourceOutputs(e RegisterResourceOutputsEvent) error {
|
2018-08-06 23:46:17 +00:00
|
|
|
// Look up the final state in the pending registration list.
|
|
|
|
urn := e.URN()
|
|
|
|
value, has := se.pendingNews.Load(urn)
|
2023-09-29 19:42:18 +00:00
|
|
|
if !has {
|
|
|
|
return fmt.Errorf("cannot complete a resource '%v' whose registration isn't pending", urn)
|
|
|
|
}
|
2018-08-06 23:46:17 +00:00
|
|
|
reg := value.(Step)
|
|
|
|
contract.Assertf(reg != nil, "expected a non-nil resource step ('%v')", urn)
|
|
|
|
se.pendingNews.Delete(urn)
|
|
|
|
// Unconditionally set the resource's outputs to what was provided. This intentionally overwrites whatever
|
|
|
|
// might already be there, since otherwise "deleting" outputs would have no affect.
|
|
|
|
outs := e.Outputs()
|
|
|
|
se.log(synchronousWorkerID,
|
|
|
|
"registered resource outputs %s: old=#%d, new=#%d", urn, len(reg.New().Outputs), len(outs))
|
2022-01-31 10:31:51 +00:00
|
|
|
reg.New().Outputs = outs
|
|
|
|
|
|
|
|
old := se.deployment.Olds()[urn]
|
|
|
|
var oldOuts resource.PropertyMap
|
|
|
|
if old != nil {
|
|
|
|
oldOuts = old.Outputs
|
|
|
|
}
|
|
|
|
|
|
|
|
// If a plan is present check that these outputs match what we recorded before
|
|
|
|
if se.deployment.plan != nil {
|
|
|
|
resourcePlan, ok := se.deployment.plan.ResourcePlans[urn]
|
|
|
|
if !ok {
|
2023-09-11 12:35:48 +00:00
|
|
|
return fmt.Errorf("no plan for resource %v", urn)
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if err := resourcePlan.checkOutputs(oldOuts, outs); err != nil {
|
2023-09-11 12:35:48 +00:00
|
|
|
return fmt.Errorf("resource violates plan: %w", err)
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-10 08:14:26 +00:00
|
|
|
// If we're generating plans save these new outputs to the plan
|
|
|
|
if se.opts.GeneratePlan {
|
2022-01-31 10:31:51 +00:00
|
|
|
if resourcePlan, ok := se.deployment.newPlans.get(urn); ok {
|
|
|
|
resourcePlan.Goal.OutputDiff = NewPlanDiff(oldOuts.Diff(outs))
|
|
|
|
resourcePlan.Outputs = outs
|
|
|
|
} else {
|
2023-09-11 12:35:48 +00:00
|
|
|
return fmt.Errorf(
|
|
|
|
"resource should already have a plan from when we called register resources [urn=%v]", urn)
|
2022-01-31 10:31:51 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
// If there is an event subscription for finishing the resource, execute them.
|
|
|
|
if e := se.opts.Events; e != nil {
|
|
|
|
if eventerr := e.OnResourceOutputs(reg); eventerr != nil {
|
|
|
|
se.log(synchronousWorkerID, "register resource outputs failed: %s", eventerr.Error())
|
|
|
|
|
|
|
|
// This is a bit of a kludge, but ExecuteRegisterResourceOutputs is an odd duck
|
|
|
|
// in that it doesn't execute on worker goroutines. Arguably, it should, but today it's
|
|
|
|
// not possible to express RegisterResourceOutputs as a step. We could 1) more generally allow
|
|
|
|
// clients of stepExecutor to do work on worker threads by e.g. scheduling arbitrary callbacks
|
|
|
|
// or 2) promote RRE to be step-like so that it can be scheduled as if it were a step. Neither
|
|
|
|
// of these are particularly appealing right now.
|
2021-11-13 02:37:17 +00:00
|
|
|
outErr := fmt.Errorf("resource complete event returned an error: %w", eventerr)
|
2020-07-09 14:19:12 +00:00
|
|
|
diagMsg := diag.RawMessage(reg.URN(), outErr.Error())
|
2020-11-18 17:47:52 +00:00
|
|
|
se.deployment.Diag().Errorf(diagMsg)
|
2023-10-17 06:47:32 +00:00
|
|
|
se.cancelDueToError(eventerr)
|
2022-01-31 10:31:51 +00:00
|
|
|
return nil
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
e.Done()
|
2022-01-31 10:31:51 +00:00
|
|
|
return nil
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2019-11-19 04:28:25 +00:00
|
|
|
// Errored returns whether or not this step executor saw a step whose execution ended in failure.
|
2023-10-17 06:47:32 +00:00
|
|
|
func (se *stepExecutor) Errored() error {
|
|
|
|
err := se.sawError.Load()
|
|
|
|
if err == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return err.(error)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// SignalCompletion signals to the stepExecutor that there are no more chains left to execute. All worker
|
|
|
|
// threads will terminate as soon as they retire all of the work they are currently executing.
|
|
|
|
func (se *stepExecutor) SignalCompletion() {
|
|
|
|
close(se.incomingChains)
|
|
|
|
}
|
|
|
|
|
|
|
|
// WaitForCompletion blocks the calling goroutine until the step executor completes execution of all in-flight
|
|
|
|
// chains.
|
|
|
|
func (se *stepExecutor) WaitForCompletion() {
|
|
|
|
se.log(synchronousWorkerID, "StepExecutor.waitForCompletion(): waiting for worker threads to exit")
|
|
|
|
se.workers.Wait()
|
|
|
|
se.log(synchronousWorkerID, "StepExecutor.waitForCompletion(): worker threads all exited")
|
|
|
|
}
|
|
|
|
|
|
|
|
//
|
|
|
|
// As calls to `Execute` submit chains for execution, some number of worker goroutines will continuously
|
|
|
|
// read from `incomingChains` and execute any chains that are received. The core execution logic is in
|
|
|
|
// the next few functions.
|
|
|
|
//
|
|
|
|
|
|
|
|
// executeChain executes a chain, one step at a time. If any step in the chain fails to execute, or if the
|
|
|
|
// context is canceled, the chain stops execution.
|
2018-09-27 22:49:08 +00:00
|
|
|
func (se *stepExecutor) executeChain(workerID int, chain chain) {
|
2018-08-06 23:46:17 +00:00
|
|
|
for _, step := range chain {
|
|
|
|
select {
|
|
|
|
case <-se.ctx.Done():
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v canceled", step.Op(), step.URN())
|
2018-08-06 23:46:17 +00:00
|
|
|
return
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
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
|
|
|
// Take the work lock before executing the step, this uses the "read" side of the lock because we're ok with as
|
|
|
|
// many workers as possible executing steps in parallel.
|
|
|
|
se.workerLock.RLock()
|
|
|
|
err := se.executeStep(workerID, step)
|
|
|
|
// Regardless of error we need to release the lock here.
|
|
|
|
se.workerLock.RUnlock()
|
|
|
|
|
|
|
|
if err != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v failed, signalling cancellation", step.Op(), step.URN())
|
2023-10-17 06:47:32 +00:00
|
|
|
se.cancelDueToError(err)
|
2023-10-31 17:37:41 +00:00
|
|
|
|
|
|
|
var saf StepApplyFailed
|
|
|
|
if !errors.As(err, &saf) {
|
2018-08-06 23:46:17 +00:00
|
|
|
// Step application errors are recorded by the OnResourceStepPost callback. This is confusing,
|
|
|
|
// but it means that at this level we shouldn't be logging any errors that came from there.
|
|
|
|
//
|
2023-10-31 17:37:41 +00:00
|
|
|
// The StepApplyFailed sentinel signals that the error that failed this chain was a step apply
|
2018-08-06 23:46:17 +00:00
|
|
|
// error and that we shouldn't log it. Everything else should be logged to the diag system as usual.
|
2020-07-09 14:19:12 +00:00
|
|
|
diagMsg := diag.RawMessage(step.URN(), err.Error())
|
2020-11-18 17:47:52 +00:00
|
|
|
se.deployment.Diag().Errorf(diagMsg)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-10-17 06:47:32 +00:00
|
|
|
func (se *stepExecutor) cancelDueToError(err error) {
|
|
|
|
se.sawError.Store(err)
|
2018-09-05 21:00:28 +00:00
|
|
|
if !se.continueOnError {
|
|
|
|
se.cancel()
|
|
|
|
}
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
//
|
|
|
|
// The next few functions are responsible for executing individual steps. The basic flow of step
|
|
|
|
// execution is
|
|
|
|
// 1. The pre-step event is raised, if there are any attached callbacks to the engine
|
|
|
|
// 2. If successful, the step is executed (if not a preview)
|
|
|
|
// 3. The post-step event is raised, if there are any attached callbacks to the engine
|
|
|
|
//
|
|
|
|
// The pre-step event returns an interface{}, which is some arbitrary context that must be passed
|
|
|
|
// verbatim to the post-step event.
|
|
|
|
//
|
|
|
|
|
|
|
|
// executeStep executes a single step, returning true if the step execution was successful and
|
|
|
|
// false if it was not.
|
|
|
|
func (se *stepExecutor) executeStep(workerID int, step Step) error {
|
|
|
|
var payload interface{}
|
|
|
|
events := se.opts.Events
|
|
|
|
if events != nil {
|
|
|
|
var err error
|
|
|
|
payload, err = events.OnResourceStepPre(step)
|
|
|
|
if err != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v failed pre-resource step: %v", step.Op(), step.URN(), err)
|
2021-11-13 02:37:17 +00:00
|
|
|
return fmt.Errorf("pre-step event returned an error: %w", err)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "applying step %v on %v (preview %v)", step.Op(), step.URN(), se.preview)
|
2018-08-08 20:45:48 +00:00
|
|
|
status, stepComplete, err := step.Apply(se.preview)
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
if err == nil {
|
|
|
|
// If we have a state object, and this is a create or update, remember it, as we may need to update it later.
|
|
|
|
if step.Logical() && step.New() != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
if prior, has := se.pendingNews.Load(step.URN()); has {
|
2021-11-13 02:37:17 +00:00
|
|
|
return fmt.Errorf("resource '%s' registered twice (%s and %s)", step.URN(), prior.(Step).Op(), step.Op())
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2020-07-09 14:19:12 +00:00
|
|
|
se.pendingNews.Store(step.URN(), step)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-12-09 03:21:58 +00:00
|
|
|
// Ensure that any secrets properties in the output are marked as such and that the resource is tracked in the set
|
|
|
|
// of registered resources.
|
2019-04-23 00:45:26 +00:00
|
|
|
if step.New() != nil {
|
|
|
|
newState := step.New()
|
2019-05-09 21:27:34 +00:00
|
|
|
for _, k := range newState.AdditionalSecretOutputs {
|
2022-06-06 16:34:04 +00:00
|
|
|
if k == "id" {
|
|
|
|
se.deployment.Diag().Warningf(&diag.Diag{
|
|
|
|
URN: step.URN(),
|
|
|
|
Message: "The 'id' property cannot be made secret. See pulumi/pulumi#2717 for more details.",
|
|
|
|
})
|
|
|
|
} else {
|
|
|
|
if v, has := newState.Outputs[k]; has && !v.IsSecret() {
|
|
|
|
newState.Outputs[k] = resource.MakeSecret(v)
|
golangci-lint: Enable staticcheck
Remove staticcheck from the list of disabled linters.
It's enabled by default in golangci-lint.
This also fixes minor remaining staticcheck issues
that don't merit their own pull requests,
or opts out of those that cannot be fixed yet.
Notably, we're opting out of:
- Resource.Name is deprecated (#9469)
- github.com/golang/protobuf is deprecated (#11869)
- strings.Title has been deprecated (#11870)
Besides that, other issues addressed in this change are:
```
// all issues are in pkg
codegen/schema/docs_parser.go:103:4: SA4006: this value of `text` is never used (staticcheck)
codegen/schema/loader.go:253:3: SA9003: empty branch (staticcheck)
resource/deploy/step_executor.go:328:12: SA9003: empty branch (staticcheck)
resource/deploy/step_generator.go:141:10: SA9003: empty branch (staticcheck)
codegen/pcl/invoke.go:97:10: SA9003: empty branch (staticcheck)
codegen/hcl2/model/type_const.go:57:2: SA9003: empty branch (staticcheck)
codegen/hcl2/model/type_enum.go:99:9: SA4001: &*x will be simplified to x. It will not copy x. (staticcheck)
codegen/go/gen_test.go:399:19: SA4017: HasPrefix is a pure function but its return value is ignored (staticcheck)
```
Depends on #11857, #11858, #11859, #11860, #11862, #11865, #11866, #11867, #11868
Resolves #11808
2023-01-11 19:53:41 +00:00
|
|
|
} else if !has { //nolint:staticcheck // https://github.com/pulumi/pulumi/issues/9926
|
2022-06-22 12:52:42 +00:00
|
|
|
// TODO (https://github.com/pulumi/pulumi/issues/9926): We want to re-enable this warning
|
|
|
|
// but it requires that providers always return back _every_ output even in preview. We
|
|
|
|
// might need to add a new "unset" PropertyValue to do this as there might be optional
|
|
|
|
// secret outputs and the engine needs to be able to tell the difference between "this
|
|
|
|
// isn't a valid output of the resource" and "this value just hasn't been set in this
|
|
|
|
// instance". Arguably for user side additionalSecretOutputs that distinction probably
|
|
|
|
// doesn't matter (if you ask for an optional output to be made secret but then the
|
|
|
|
// provider doesn't return it maybe you want the warning that nothing is actually being
|
|
|
|
// affected?). But for SDK generated we always send the same list and the user doesn't
|
|
|
|
// control it so we need to make sure that if there is an optional output that this
|
|
|
|
// warning doesn't get triggered.
|
|
|
|
|
2022-06-06 16:34:04 +00:00
|
|
|
// User asked us to make k a secret, but we don't have a property k. This is probably a
|
|
|
|
// mistake (mostly likely due to casing, eg my_prop vs myProp) but warn the user so they know
|
|
|
|
// the key didn't do anything.
|
2023-03-03 16:36:39 +00:00
|
|
|
// msg := fmt.Sprintf("Could not find property '%s' listed in additional secret outputs.", k)
|
|
|
|
// se.deployment.Diag().Warningf(diag.RawMessage(step.URN(), msg))
|
2022-06-06 16:34:04 +00:00
|
|
|
}
|
2019-04-23 00:45:26 +00:00
|
|
|
}
|
|
|
|
}
|
2020-12-09 03:21:58 +00:00
|
|
|
|
2023-06-23 16:17:01 +00:00
|
|
|
// If an input secret is potentially leaked as an output, preemptively mark it as secret.
|
|
|
|
for k, out := range newState.Outputs {
|
|
|
|
if !out.IsSecret() {
|
|
|
|
in, has := newState.Inputs[k]
|
|
|
|
if !has {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if in.IsSecret() {
|
|
|
|
newState.Outputs[k] = resource.MakeSecret(out)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-12-09 03:21:58 +00:00
|
|
|
// If this is not a resource that is managed by Pulumi, then we can ignore it.
|
2020-12-22 19:04:49 +00:00
|
|
|
if _, hasGoal := se.deployment.goals.get(newState.URN); hasGoal {
|
2020-12-09 03:21:58 +00:00
|
|
|
se.deployment.news.set(newState.URN, newState)
|
|
|
|
}
|
2022-01-31 10:31:51 +00:00
|
|
|
|
2022-10-10 08:14:26 +00:00
|
|
|
// If we're generating plans update the resource's outputs in the generated plan.
|
|
|
|
if se.opts.GeneratePlan {
|
2022-01-31 10:31:51 +00:00
|
|
|
if resourcePlan, ok := se.deployment.newPlans.get(newState.URN); ok {
|
|
|
|
resourcePlan.Outputs = newState.Outputs
|
|
|
|
}
|
|
|
|
}
|
2019-04-23 00:45:26 +00:00
|
|
|
}
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
if events != nil {
|
|
|
|
if postErr := events.OnResourceStepPost(payload, step, status, err); postErr != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v failed post-resource step: %v", step.Op(), step.URN(), postErr)
|
2021-11-13 02:37:17 +00:00
|
|
|
return fmt.Errorf("post-step event returned an error: %w", postErr)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-08 20:45:48 +00:00
|
|
|
// Calling stepComplete allows steps that depend on this step to continue. OnResourceStepPost saved the results
|
|
|
|
// of the step in the snapshot, so we are ready to go.
|
|
|
|
if stepComplete != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v retired", step.Op(), step.URN())
|
2018-08-08 20:45:48 +00:00
|
|
|
stepComplete()
|
|
|
|
}
|
|
|
|
|
2018-08-06 23:46:17 +00:00
|
|
|
if err != nil {
|
2020-07-09 14:19:12 +00:00
|
|
|
se.log(workerID, "step %v on %v failed with an error: %v", step.Op(), step.URN(), err)
|
2023-10-31 17:37:41 +00:00
|
|
|
return StepApplyFailed{err}
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// log is a simple logging helper for the step executor.
|
|
|
|
func (se *stepExecutor) log(workerID int, msg string, args ...interface{}) {
|
|
|
|
if logging.V(stepExecutorLogLevel) {
|
|
|
|
message := fmt.Sprintf(msg, args...)
|
|
|
|
logging.V(stepExecutorLogLevel).Infof("StepExecutor worker(%d): %s", workerID, message)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
//
|
|
|
|
// The step executor owns a number of goroutines that it considers to be "workers", responsible for
|
|
|
|
// executing steps. By default, as we ease into the waters of parallelism, there is at most one worker
|
|
|
|
// active.
|
|
|
|
//
|
|
|
|
// Workers continuously pull from se.incomingChains, executing chains as they are provided to the executor.
|
|
|
|
// There are two reasons why a worker would exit:
|
|
|
|
//
|
|
|
|
// 1. A worker exits if se.ctx is canceled. There are two ways that se.ctx gets canceled: first, if there is
|
2020-11-18 17:47:52 +00:00
|
|
|
// a step error in another worker, it will cancel the context. Second, if the deployment executor experiences an
|
2018-08-06 23:46:17 +00:00
|
|
|
// error when generating steps or doing pre or post-step events, it will cancel the context.
|
|
|
|
// 2. A worker exits if it experiences an error when running a step.
|
|
|
|
//
|
|
|
|
|
|
|
|
// worker is the base function for all step executor worker goroutines. It continuously polls for new chains
|
2018-10-17 22:33:26 +00:00
|
|
|
// and executes any that it gets from the channel. If `launchAsync` is true, worker launches a new goroutine
|
|
|
|
// that will execute the chain so that the execution continues asynchronously and this worker can proceed to
|
|
|
|
// the next chain.
|
|
|
|
func (se *stepExecutor) worker(workerID int, launchAsync bool) {
|
2018-08-06 23:46:17 +00:00
|
|
|
se.log(workerID, "worker coming online")
|
|
|
|
defer se.workers.Done()
|
|
|
|
|
2018-10-17 22:33:26 +00:00
|
|
|
oneshotWorkerID := 0
|
2018-08-06 23:46:17 +00:00
|
|
|
for {
|
|
|
|
se.log(workerID, "worker waiting for incoming chains")
|
|
|
|
select {
|
2018-09-27 22:49:08 +00:00
|
|
|
case request := <-se.incomingChains:
|
|
|
|
if request.Chain == nil {
|
2018-08-06 23:46:17 +00:00
|
|
|
se.log(workerID, "worker received nil chain, exiting")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
se.log(workerID, "worker received chain for execution")
|
2018-10-17 22:33:26 +00:00
|
|
|
if !launchAsync {
|
|
|
|
se.executeChain(workerID, request.Chain)
|
|
|
|
close(request.CompletionChan)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
// If we're launching asynchronously, make up a new worker ID for this new oneshot worker and record its
|
|
|
|
// launch with our worker wait group.
|
|
|
|
se.workers.Add(1)
|
|
|
|
newWorkerID := oneshotWorkerID
|
|
|
|
go func() {
|
|
|
|
defer se.workers.Done()
|
|
|
|
se.log(newWorkerID, "launching oneshot worker")
|
|
|
|
se.executeChain(newWorkerID, request.Chain)
|
|
|
|
close(request.CompletionChan)
|
|
|
|
}()
|
|
|
|
|
|
|
|
oneshotWorkerID++
|
2018-08-06 23:46:17 +00:00
|
|
|
case <-se.ctx.Done():
|
|
|
|
se.log(workerID, "worker exiting due to cancellation")
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-18 17:47:52 +00:00
|
|
|
func newStepExecutor(ctx context.Context, cancel context.CancelFunc, deployment *Deployment, opts Options,
|
2023-03-03 16:36:39 +00:00
|
|
|
preview, continueOnError bool,
|
|
|
|
) *stepExecutor {
|
2018-08-06 23:46:17 +00:00
|
|
|
exec := &stepExecutor{
|
2020-11-18 17:47:52 +00:00
|
|
|
deployment: deployment,
|
2018-09-05 21:00:28 +00:00
|
|
|
opts: opts,
|
|
|
|
preview: preview,
|
|
|
|
continueOnError: continueOnError,
|
2018-09-27 22:49:08 +00:00
|
|
|
incomingChains: make(chan incomingChain),
|
2018-09-05 21:00:28 +00:00
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
2018-10-17 22:33:26 +00:00
|
|
|
// If we're being asked to run as parallel as possible, spawn a single worker that launches chain executions
|
|
|
|
// asynchronously.
|
|
|
|
if opts.InfiniteParallelism() {
|
|
|
|
exec.workers.Add(1)
|
|
|
|
go exec.worker(infiniteWorkerID, true /*launchAsync*/)
|
|
|
|
return exec
|
|
|
|
}
|
|
|
|
|
|
|
|
// Otherwise, launch a worker goroutine for each degree of parallelism.
|
2018-08-06 23:46:17 +00:00
|
|
|
fanout := opts.DegreeOfParallelism()
|
|
|
|
for i := 0; i < fanout; i++ {
|
|
|
|
exec.workers.Add(1)
|
2018-10-17 22:33:26 +00:00
|
|
|
go exec.worker(i, false /*launchAsync*/)
|
2018-08-06 23:46:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return exec
|
|
|
|
}
|