2021-09-21 17:00:44 +00:00
|
|
|
// Copyright 2016-2021, Pulumi Corporation.
|
2020-11-18 19:16:30 +00:00
|
|
|
//
|
|
|
|
// 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 engine
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2021-11-13 02:37:17 +00:00
|
|
|
"errors"
|
|
|
|
"fmt"
|
2020-11-18 19:16:30 +00:00
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/opentracing/opentracing-go"
|
2022-11-01 15:15:09 +00:00
|
|
|
"google.golang.org/grpc"
|
2021-11-13 02:37:17 +00:00
|
|
|
|
2023-09-18 11:01:28 +00:00
|
|
|
"github.com/pulumi/pulumi/pkg/v3/display"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/pkg/v3/resource/deploy"
|
|
|
|
"github.com/pulumi/pulumi/pkg/v3/resource/deploy/providers"
|
2022-11-01 15:15:09 +00:00
|
|
|
interceptors "github.com/pulumi/pulumi/pkg/v3/util/rpcdebug"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/diag"
|
2022-12-15 14:46:39 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/env"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource"
|
2023-03-31 10:22:50 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource/config"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/resource/plugin"
|
2021-06-04 12:08:40 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/tokens"
|
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/fsutil"
|
ctrl-c should cause Pulumi to call Cancel operation on providers (#14057)
<!---
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
Fixes #14054
This PR fixes a problem that the engine cannot forward a cancellation
signal to the provider, because the plugin context is already closed. An
[earlier
commit](https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114)
made the plugin context be closed too eagerly, with the intent of
cancelling plugin installation. This PR attempts to decouple the
cancellation of plugin installation from the lifecycle of the plugin
context, so that plugin installation may be cancelled during the
cancelation phase as opposed to the termination phase. Then, it closes
the plugin context in termination phase.
There's an existing test case in the engine lifecycle tests called
`TestProviderCancellation`, but it didn't catch the problem because it
uses a fake plugin host that behaves differently after being closed. The
issue was fixed in https://github.com/pulumi/pulumi/pull/14063 and the
test was temporarily disabled. This PR re-enables the test case.
A new test case `TestSourceFuncCancellation` is added to test
cancellation of the source func (where plugin installation happens, see
[update.go](https://github.com/pulumi/pulumi/pull/14057/files#diff-7d2ca3e83a05073b332435271496050e28466b4f7af8c0c91bbc77947a75a3a2R378)),
as this was the original motivation of
https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114.
## 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. -->
- [x] 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-09-29 22:12:35 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/util/logging"
|
2021-03-17 13:20:05 +00:00
|
|
|
"github.com/pulumi/pulumi/sdk/v3/go/common/workspace"
|
2020-11-18 19:16:30 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
const clientRuntimeName = "client"
|
|
|
|
|
|
|
|
// ProjectInfoContext returns information about the current project, including its pwd, main, and plugin context.
|
2022-07-25 11:34:49 +00:00
|
|
|
func ProjectInfoContext(projinfo *Projinfo, host plugin.Host,
|
2020-11-18 19:16:30 +00:00
|
|
|
diag, statusDiag diag.Sink, disableProviderPreview bool,
|
2023-03-31 10:22:50 +00:00
|
|
|
tracingSpan opentracing.Span, config map[config.Key]string,
|
2023-03-03 16:36:39 +00:00
|
|
|
) (string, string, *plugin.Context, error) {
|
2023-02-15 01:28:14 +00:00
|
|
|
contract.Requiref(projinfo != nil, "projinfo", "must not be nil")
|
2020-11-18 19:16:30 +00:00
|
|
|
|
|
|
|
// If the package contains an override for the main entrypoint, use it.
|
|
|
|
pwd, main, err := projinfo.GetPwdMain()
|
|
|
|
if err != nil {
|
|
|
|
return "", "", nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create a context for plugins.
|
2022-07-25 11:34:49 +00:00
|
|
|
ctx, err := plugin.NewContextWithRoot(diag, statusDiag, host, pwd, projinfo.Root,
|
2023-03-31 10:22:50 +00:00
|
|
|
projinfo.Proj.Runtime.Options(), disableProviderPreview, tracingSpan, projinfo.Proj.Plugins, config)
|
2020-11-18 19:16:30 +00:00
|
|
|
if err != nil {
|
|
|
|
return "", "", nil, err
|
|
|
|
}
|
|
|
|
|
2022-12-15 14:46:39 +00:00
|
|
|
if logFile := env.DebugGRPC.Value(); logFile != "" {
|
2022-11-01 15:15:09 +00:00
|
|
|
di, err := interceptors.NewDebugInterceptor(interceptors.DebugInterceptorOptions{
|
|
|
|
LogFile: logFile,
|
|
|
|
Mutex: ctx.DebugTraceMutex,
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
return "", "", nil, err
|
|
|
|
}
|
|
|
|
ctx.DialOptions = func(metadata interface{}) []grpc.DialOption {
|
|
|
|
return di.DialOptions(interceptors.LogOptions{
|
|
|
|
Metadata: metadata,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-18 19:16:30 +00:00
|
|
|
// If the project wants to connect to an existing language runtime, do so now.
|
|
|
|
if projinfo.Proj.Runtime.Name() == clientRuntimeName {
|
|
|
|
addressValue, ok := projinfo.Proj.Runtime.Options()["address"]
|
|
|
|
if !ok {
|
|
|
|
return "", "", nil, errors.New("missing address of language runtime service")
|
|
|
|
}
|
|
|
|
address, ok := addressValue.(string)
|
|
|
|
if !ok {
|
|
|
|
return "", "", nil, errors.New("address of language runtime service must be a string")
|
|
|
|
}
|
|
|
|
host, err := connectToLanguageRuntime(ctx, address)
|
|
|
|
if err != nil {
|
|
|
|
return "", "", nil, err
|
|
|
|
}
|
|
|
|
ctx.Host = host
|
|
|
|
}
|
|
|
|
|
|
|
|
return pwd, main, ctx, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// newDeploymentContext creates a context for a subsequent deployment. Callers must call Close on the context after the
|
|
|
|
// associated deployment completes.
|
|
|
|
func newDeploymentContext(u UpdateInfo, opName string, parentSpan opentracing.SpanContext) (*deploymentContext, error) {
|
2023-02-15 01:28:14 +00:00
|
|
|
contract.Requiref(u != nil, "u", "must not be nil")
|
2020-11-18 19:16:30 +00:00
|
|
|
|
|
|
|
// Create a root span for the operation
|
|
|
|
opts := []opentracing.StartSpanOption{}
|
|
|
|
if opName != "" {
|
|
|
|
opts = append(opts, opentracing.Tag{Key: "operation", Value: opName})
|
|
|
|
}
|
|
|
|
if parentSpan != nil {
|
|
|
|
opts = append(opts, opentracing.ChildOf(parentSpan))
|
|
|
|
}
|
|
|
|
tracingSpan := opentracing.StartSpan("pulumi-plan", opts...)
|
|
|
|
|
|
|
|
return &deploymentContext{
|
|
|
|
Update: u,
|
|
|
|
TracingSpan: tracingSpan,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type deploymentContext struct {
|
|
|
|
Update UpdateInfo // The update being processed.
|
|
|
|
TracingSpan opentracing.Span // An OpenTracing span to parent deployment operations within.
|
|
|
|
}
|
|
|
|
|
|
|
|
func (ctx *deploymentContext) Close() {
|
|
|
|
ctx.TracingSpan.Finish()
|
|
|
|
}
|
|
|
|
|
|
|
|
// deploymentOptions includes a full suite of options for performing a deployment.
|
|
|
|
type deploymentOptions struct {
|
|
|
|
UpdateOptions
|
|
|
|
|
|
|
|
// SourceFunc is a factory that returns an EvalSource to use during deployment. This is the thing that
|
|
|
|
// creates resources to compare against the current checkpoint state (e.g., by evaluating a program, etc).
|
|
|
|
SourceFunc deploymentSourceFunc
|
|
|
|
|
|
|
|
DOT bool // true if we should print the DOT file for this deployment.
|
|
|
|
Events eventEmitter // the channel to write events from the engine to.
|
|
|
|
Diag diag.Sink // the sink to use for diag'ing.
|
|
|
|
StatusDiag diag.Sink // the sink to use for diag'ing status messages.
|
|
|
|
|
|
|
|
isImport bool // True if this is an import.
|
|
|
|
imports []deploy.Import // Resources to import, if this is an import.
|
|
|
|
|
|
|
|
// true if we're executing a refresh.
|
|
|
|
isRefresh bool
|
|
|
|
}
|
|
|
|
|
|
|
|
// deploymentSourceFunc is a callback that will be used to prepare for, and evaluate, the "new" state for a stack.
|
|
|
|
type deploymentSourceFunc func(
|
ctrl-c should cause Pulumi to call Cancel operation on providers (#14057)
<!---
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
Fixes #14054
This PR fixes a problem that the engine cannot forward a cancellation
signal to the provider, because the plugin context is already closed. An
[earlier
commit](https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114)
made the plugin context be closed too eagerly, with the intent of
cancelling plugin installation. This PR attempts to decouple the
cancellation of plugin installation from the lifecycle of the plugin
context, so that plugin installation may be cancelled during the
cancelation phase as opposed to the termination phase. Then, it closes
the plugin context in termination phase.
There's an existing test case in the engine lifecycle tests called
`TestProviderCancellation`, but it didn't catch the problem because it
uses a fake plugin host that behaves differently after being closed. The
issue was fixed in https://github.com/pulumi/pulumi/pull/14063 and the
test was temporarily disabled. This PR re-enables the test case.
A new test case `TestSourceFuncCancellation` is added to test
cancellation of the source func (where plugin installation happens, see
[update.go](https://github.com/pulumi/pulumi/pull/14057/files#diff-7d2ca3e83a05073b332435271496050e28466b4f7af8c0c91bbc77947a75a3a2R378)),
as this was the original motivation of
https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114.
## 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. -->
- [x] 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-09-29 22:12:35 +00:00
|
|
|
ctx context.Context,
|
2023-10-09 18:31:17 +00:00
|
|
|
client deploy.BackendClient, opts *deploymentOptions, proj *workspace.Project, pwd, main, projectRoot string,
|
2020-11-18 19:16:30 +00:00
|
|
|
target *deploy.Target, plugctx *plugin.Context, dryRun bool) (deploy.Source, error)
|
|
|
|
|
|
|
|
// newDeployment creates a new deployment with the given context and options.
|
2023-10-09 18:31:17 +00:00
|
|
|
func newDeployment(ctx *Context, info *deploymentContext, opts *deploymentOptions,
|
2023-03-03 16:36:39 +00:00
|
|
|
dryRun bool,
|
|
|
|
) (*deployment, error) {
|
2023-02-15 01:28:14 +00:00
|
|
|
contract.Assertf(info != nil, "a deployment context must be provided")
|
|
|
|
contract.Assertf(info.Update != nil, "update info cannot be nil")
|
|
|
|
contract.Assertf(opts.SourceFunc != nil, "a source factory must be provided")
|
2020-11-18 19:16:30 +00:00
|
|
|
|
|
|
|
// First, load the package metadata and the deployment target in preparation for executing the package's program
|
|
|
|
// and creating resources. This includes fetching its pwd and main overrides.
|
|
|
|
proj, target := info.Update.GetProject(), info.Update.GetTarget()
|
2023-02-15 01:28:14 +00:00
|
|
|
contract.Assertf(proj != nil, "update project cannot be nil")
|
|
|
|
contract.Assertf(target != nil, "update target cannot be nil")
|
2020-11-18 19:16:30 +00:00
|
|
|
projinfo := &Projinfo{Proj: proj, Root: info.Update.GetRoot()}
|
2023-03-31 10:22:50 +00:00
|
|
|
|
|
|
|
// Decrypt the configuration.
|
|
|
|
config, err := target.Config.Decrypt(target.Decrypter)
|
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("failed to decrypt config: %w", err)
|
|
|
|
}
|
|
|
|
|
2022-07-25 11:34:49 +00:00
|
|
|
pwd, main, plugctx, err := ProjectInfoContext(projinfo, opts.Host,
|
2023-03-31 10:22:50 +00:00
|
|
|
opts.Diag, opts.StatusDiag, opts.DisableProviderPreview, info.TracingSpan, config)
|
2020-11-18 19:16:30 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
ctrl-c should cause Pulumi to call Cancel operation on providers (#14057)
<!---
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
Fixes #14054
This PR fixes a problem that the engine cannot forward a cancellation
signal to the provider, because the plugin context is already closed. An
[earlier
commit](https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114)
made the plugin context be closed too eagerly, with the intent of
cancelling plugin installation. This PR attempts to decouple the
cancellation of plugin installation from the lifecycle of the plugin
context, so that plugin installation may be cancelled during the
cancelation phase as opposed to the termination phase. Then, it closes
the plugin context in termination phase.
There's an existing test case in the engine lifecycle tests called
`TestProviderCancellation`, but it didn't catch the problem because it
uses a fake plugin host that behaves differently after being closed. The
issue was fixed in https://github.com/pulumi/pulumi/pull/14063 and the
test was temporarily disabled. This PR re-enables the test case.
A new test case `TestSourceFuncCancellation` is added to test
cancellation of the source func (where plugin installation happens, see
[update.go](https://github.com/pulumi/pulumi/pull/14057/files#diff-7d2ca3e83a05073b332435271496050e28466b4f7af8c0c91bbc77947a75a3a2R378)),
as this was the original motivation of
https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114.
## 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. -->
- [x] 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-09-29 22:12:35 +00:00
|
|
|
|
|
|
|
// Keep the plugin context open until the context is terminated, to allow for graceful provider cancellation.
|
|
|
|
plugctx = plugctx.WithCancelChannel(ctx.Cancel.Terminated())
|
|
|
|
|
|
|
|
// Set up a goroutine that will signal cancellation to the source if the caller context
|
|
|
|
// is cancelled.
|
|
|
|
cancelCtx, cancelFunc := context.WithCancel(context.Background())
|
|
|
|
go func() {
|
|
|
|
<-ctx.Cancel.Canceled()
|
|
|
|
logging.V(7).Infof("engine.newDeployment(...): received cancellation signal")
|
|
|
|
cancelFunc()
|
|
|
|
}()
|
2020-11-18 19:16:30 +00:00
|
|
|
|
|
|
|
// Now create the state source. This may issue an error if it can't create the source. This entails,
|
|
|
|
// for example, loading any plugins which will be required to execute a program, among other things.
|
ctrl-c should cause Pulumi to call Cancel operation on providers (#14057)
<!---
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
Fixes #14054
This PR fixes a problem that the engine cannot forward a cancellation
signal to the provider, because the plugin context is already closed. An
[earlier
commit](https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114)
made the plugin context be closed too eagerly, with the intent of
cancelling plugin installation. This PR attempts to decouple the
cancellation of plugin installation from the lifecycle of the plugin
context, so that plugin installation may be cancelled during the
cancelation phase as opposed to the termination phase. Then, it closes
the plugin context in termination phase.
There's an existing test case in the engine lifecycle tests called
`TestProviderCancellation`, but it didn't catch the problem because it
uses a fake plugin host that behaves differently after being closed. The
issue was fixed in https://github.com/pulumi/pulumi/pull/14063 and the
test was temporarily disabled. This PR re-enables the test case.
A new test case `TestSourceFuncCancellation` is added to test
cancellation of the source func (where plugin installation happens, see
[update.go](https://github.com/pulumi/pulumi/pull/14057/files#diff-7d2ca3e83a05073b332435271496050e28466b4f7af8c0c91bbc77947a75a3a2R378)),
as this was the original motivation of
https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114.
## 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. -->
- [x] 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-09-29 22:12:35 +00:00
|
|
|
source, err := opts.SourceFunc(
|
|
|
|
cancelCtx, ctx.BackendClient, opts, proj, pwd, main, projinfo.Root, target, plugctx, dryRun)
|
2020-11-18 19:16:30 +00:00
|
|
|
if err != nil {
|
|
|
|
contract.IgnoreClose(plugctx)
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
localPolicyPackPaths := ConvertLocalPolicyPacksToPaths(opts.LocalPolicyPacks)
|
|
|
|
|
|
|
|
var depl *deploy.Deployment
|
|
|
|
if !opts.isImport {
|
|
|
|
depl, err = deploy.NewDeployment(
|
2023-01-11 16:04:14 +00:00
|
|
|
plugctx, target, target.Snapshot, opts.Plan, source,
|
|
|
|
localPolicyPackPaths, dryRun, ctx.BackendClient)
|
2020-11-18 19:16:30 +00:00
|
|
|
} else {
|
ctrl-c should cause Pulumi to call Cancel operation on providers (#14057)
<!---
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
Fixes #14054
This PR fixes a problem that the engine cannot forward a cancellation
signal to the provider, because the plugin context is already closed. An
[earlier
commit](https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114)
made the plugin context be closed too eagerly, with the intent of
cancelling plugin installation. This PR attempts to decouple the
cancellation of plugin installation from the lifecycle of the plugin
context, so that plugin installation may be cancelled during the
cancelation phase as opposed to the termination phase. Then, it closes
the plugin context in termination phase.
There's an existing test case in the engine lifecycle tests called
`TestProviderCancellation`, but it didn't catch the problem because it
uses a fake plugin host that behaves differently after being closed. The
issue was fixed in https://github.com/pulumi/pulumi/pull/14063 and the
test was temporarily disabled. This PR re-enables the test case.
A new test case `TestSourceFuncCancellation` is added to test
cancellation of the source func (where plugin installation happens, see
[update.go](https://github.com/pulumi/pulumi/pull/14057/files#diff-7d2ca3e83a05073b332435271496050e28466b4f7af8c0c91bbc77947a75a3a2R378)),
as this was the original motivation of
https://github.com/pulumi/pulumi/pull/9793/commits/a9ae602867834efc9821abd866ef388c1b051114.
## 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. -->
- [x] 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-09-29 22:12:35 +00:00
|
|
|
_, defaultProviderInfo, pluginErr := installPlugins(cancelCtx, proj, pwd, main, target, plugctx,
|
2020-11-18 19:16:30 +00:00
|
|
|
false /*returnInstallErrors*/)
|
|
|
|
if pluginErr != nil {
|
|
|
|
return nil, pluginErr
|
|
|
|
}
|
|
|
|
for i := range opts.imports {
|
|
|
|
imp := &opts.imports[i]
|
2023-11-13 17:58:35 +00:00
|
|
|
if imp.Component {
|
|
|
|
if imp.ID != "" {
|
|
|
|
return nil, fmt.Errorf("import %s cannot specify an ID as it's a component", imp.Name)
|
|
|
|
}
|
2021-06-04 12:08:40 +00:00
|
|
|
}
|
2023-11-13 17:58:35 +00:00
|
|
|
|
|
|
|
if !imp.Component || imp.Remote {
|
|
|
|
_, err := tokens.ParseTypeToken(imp.Type.String())
|
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("import type %q is not a valid resource type token. "+
|
|
|
|
"Type tokens must be of the format <package>:<module>:<type> - "+
|
|
|
|
"refer to the import section of the provider resource documentation.", imp.Type.String())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if imp.Provider == "" && (!imp.Component || imp.Remote) {
|
2021-12-17 22:52:01 +00:00
|
|
|
if imp.Version == nil {
|
|
|
|
imp.Version = defaultProviderInfo[imp.Type.Package()].Version
|
|
|
|
}
|
|
|
|
if imp.PluginDownloadURL == "" {
|
|
|
|
imp.PluginDownloadURL = defaultProviderInfo[imp.Type.Package()].PluginDownloadURL
|
|
|
|
}
|
2023-09-11 15:54:07 +00:00
|
|
|
if imp.PluginChecksums == nil {
|
|
|
|
imp.PluginChecksums = defaultProviderInfo[imp.Type.Package()].Checksums
|
|
|
|
}
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
depl, err = deploy.NewImportDeployment(plugctx, target, proj.Name, opts.imports, dryRun)
|
|
|
|
}
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
contract.IgnoreClose(plugctx)
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return &deployment{
|
|
|
|
Ctx: info,
|
|
|
|
Plugctx: plugctx,
|
|
|
|
Deployment: depl,
|
|
|
|
Options: opts,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
type deployment struct {
|
|
|
|
Ctx *deploymentContext // deployment context information.
|
|
|
|
Plugctx *plugin.Context // the context containing plugins and their state.
|
|
|
|
Deployment *deploy.Deployment // the deployment created by this command.
|
2023-10-09 18:31:17 +00:00
|
|
|
Options *deploymentOptions // the options used while deploying.
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
type runActions interface {
|
|
|
|
deploy.Events
|
|
|
|
|
2022-06-27 14:08:06 +00:00
|
|
|
Changes() display.ResourceChanges
|
2020-11-18 19:16:30 +00:00
|
|
|
MaybeCorrupt() bool
|
|
|
|
}
|
|
|
|
|
|
|
|
// run executes the deployment. It is primarily responsible for handling cancellation.
|
2023-10-09 18:31:17 +00:00
|
|
|
func (deployment *deployment) run(cancelCtx *Context, actions runActions,
|
2023-03-03 16:36:39 +00:00
|
|
|
preview bool,
|
2023-10-09 14:48:10 +00:00
|
|
|
) (*deploy.Plan, display.ResourceChanges, error) {
|
2020-11-18 19:16:30 +00:00
|
|
|
// Change into the plugin context's working directory.
|
|
|
|
chdir, err := fsutil.Chdir(deployment.Plugctx.Pwd)
|
|
|
|
if err != nil {
|
2023-10-09 14:48:10 +00:00
|
|
|
return nil, nil, err
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
|
|
|
defer chdir()
|
|
|
|
|
|
|
|
// Create a new context for cancellation and tracing.
|
|
|
|
ctx, cancelFunc := context.WithCancel(context.Background())
|
|
|
|
|
|
|
|
// Inject our opentracing span into the context.
|
|
|
|
if deployment.Ctx.TracingSpan != nil {
|
|
|
|
ctx = opentracing.ContextWithSpan(ctx, deployment.Ctx.TracingSpan)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Emit an appropriate prelude event.
|
|
|
|
deployment.Options.Events.preludeEvent(preview, deployment.Ctx.Update.GetTarget().Config)
|
|
|
|
|
|
|
|
// Execute the deployment.
|
|
|
|
start := time.Now()
|
|
|
|
|
|
|
|
done := make(chan bool)
|
2022-01-31 10:31:51 +00:00
|
|
|
var newPlan *deploy.Plan
|
2023-10-09 14:48:10 +00:00
|
|
|
var walkError error
|
2020-11-18 19:16:30 +00:00
|
|
|
go func() {
|
|
|
|
opts := deploy.Options{
|
2020-12-16 20:38:20 +00:00
|
|
|
Events: actions,
|
|
|
|
Parallel: deployment.Options.Parallel,
|
|
|
|
Refresh: deployment.Options.Refresh,
|
|
|
|
RefreshOnly: deployment.Options.isRefresh,
|
|
|
|
ReplaceTargets: deployment.Options.ReplaceTargets,
|
2023-05-23 20:17:59 +00:00
|
|
|
Targets: deployment.Options.Targets,
|
2020-12-16 20:38:20 +00:00
|
|
|
TargetDependents: deployment.Options.TargetDependents,
|
|
|
|
UseLegacyDiff: deployment.Options.UseLegacyDiff,
|
|
|
|
DisableResourceReferences: deployment.Options.DisableResourceReferences,
|
2021-09-15 21:16:00 +00:00
|
|
|
DisableOutputValues: deployment.Options.DisableOutputValues,
|
2022-10-10 08:14:26 +00:00
|
|
|
GeneratePlan: deployment.Options.UpdateOptions.GeneratePlan,
|
2024-03-22 09:22:40 +00:00
|
|
|
ContinueOnError: deployment.Options.ContinueOnError,
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
2023-10-09 14:48:10 +00:00
|
|
|
newPlan, walkError = deployment.Deployment.Execute(ctx, opts, preview)
|
2020-11-18 19:16:30 +00:00
|
|
|
close(done)
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Asynchronously listen for cancellation, and deliver that signal to the deployment.
|
|
|
|
go func() {
|
|
|
|
select {
|
|
|
|
case <-cancelCtx.Cancel.Canceled():
|
|
|
|
// Cancel the deployment's execution context, so it begins to shut down.
|
|
|
|
cancelFunc()
|
|
|
|
case <-done:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for the deployment to finish executing or for the user to terminate the run.
|
|
|
|
select {
|
|
|
|
case <-cancelCtx.Cancel.Terminated():
|
2023-10-09 14:48:10 +00:00
|
|
|
err = cancelCtx.Cancel.TerminateErr()
|
2020-11-18 19:16:30 +00:00
|
|
|
|
|
|
|
case <-done:
|
2023-10-09 14:48:10 +00:00
|
|
|
err = walkError
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
duration := time.Since(start)
|
|
|
|
changes := actions.Changes()
|
|
|
|
|
2023-10-09 18:31:17 +00:00
|
|
|
// Refresh and Import do not execute Policy Packs.
|
|
|
|
policies := map[string]string{}
|
|
|
|
if !deployment.Options.isRefresh && !deployment.Options.isImport {
|
|
|
|
for _, p := range deployment.Options.RequiredPolicies {
|
|
|
|
policies[p.Name()] = p.Version()
|
|
|
|
}
|
|
|
|
for _, pack := range deployment.Options.LocalPolicyPacks {
|
|
|
|
packName := pack.NameForEvents()
|
|
|
|
policies[packName] = pack.Version
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-18 19:16:30 +00:00
|
|
|
// Emit a summary event.
|
2023-10-09 18:31:17 +00:00
|
|
|
deployment.Options.Events.summaryEvent(preview, actions.MaybeCorrupt(), duration, changes, policies)
|
2020-11-18 19:16:30 +00:00
|
|
|
|
2023-10-09 14:48:10 +00:00
|
|
|
return newPlan, changes, err
|
2020-11-18 19:16:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (deployment *deployment) Close() error {
|
|
|
|
return deployment.Plugctx.Close()
|
|
|
|
}
|
|
|
|
|
|
|
|
func assertSeen(seen map[resource.URN]deploy.Step, step deploy.Step) {
|
|
|
|
_, has := seen[step.URN()]
|
|
|
|
contract.Assertf(has, "URN '%v' had not been marked as seen", step.URN())
|
|
|
|
}
|
|
|
|
|
|
|
|
func isDefaultProviderStep(step deploy.Step) bool {
|
|
|
|
return providers.IsDefaultProvider(step.URN())
|
|
|
|
}
|
2023-05-23 20:17:59 +00:00
|
|
|
|
|
|
|
func checkTargets(targetUrns deploy.UrnTargets, snap *deploy.Snapshot) error {
|
|
|
|
if !targetUrns.IsConstrained() {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
if snap == nil {
|
2023-12-12 12:19:42 +00:00
|
|
|
return errors.New("targets specified, but snapshot was nil")
|
2023-05-23 20:17:59 +00:00
|
|
|
}
|
|
|
|
urns := map[resource.URN]struct{}{}
|
|
|
|
for _, res := range snap.Resources {
|
|
|
|
urns[res.URN] = struct{}{}
|
|
|
|
}
|
|
|
|
for _, target := range targetUrns.Literals() {
|
|
|
|
if _, ok := urns[target]; !ok {
|
|
|
|
return fmt.Errorf("no resource named '%s' found", target)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|