Save resources obtained from ".get" in the snapshot (#1654)

* Protobuf changes to record dependencies for read resources

* Add a number of tests for read resources, especially around replacement

* Place read resources in the snapshot with "external" bit set

Fixes pulumi/pulumi#1521. This commit introduces two new step ops: Read
and ReadReplacement. The engine generates Read and ReadReplacement steps
when servicing ReadResource RPC calls from the language host.

* Fix an omission of OpReadReplace from the step list

* Rebase against master

* Transition to use V2 Resources by default

* Add a semantic "relinquish" operation to the engine

If the engine observes that a resource is read and also that the
resource exists in the snapshot as a non-external resource, it will not
delete the resource if the IDs of the old and new resources match.

* Typo fix

* CR: add missing comments, DeserializeDeployment -> DeserializeDeploymentV2, ID check
This commit is contained in:
Sean Gillespie 2018-08-03 14:06:00 -07:00 committed by GitHub
parent a9ff410360
commit 48aa5e73f8
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
48 changed files with 1204 additions and 203 deletions

View file

@ -74,7 +74,7 @@ func newStackImportCmd() *cobra.Command {
// We do, however, now want to unmarshal the json.RawMessage into a real, typed deployment. We do this so
// we can check that the deployment doesn't contain resources from a stack other than the selected one. This
// catches errors wherein someone imports the wrong stack's deployment (which can seriously hork things).
snapshot, err := stack.DeserializeDeployment(&deployment)
snapshot, err := stack.DeserializeUntypedDeployment(&deployment)
if err != nil {
switch err {
case stack.ErrDeploymentSchemaVersionTooOld:

View file

@ -39,7 +39,7 @@ import (
const (
// DeploymentSchemaVersionCurrent is the current version of the `Deployment` schema.
// Any deployments newer than this version will be rejected.
DeploymentSchemaVersionCurrent = 1
DeploymentSchemaVersionCurrent = 2
)
// VersionedCheckpoint is a version number plus a json document. The version number describes what

View file

@ -492,7 +492,7 @@ func (pc *Client) InvalidateUpdateCheckpoint(ctx context.Context, update UpdateI
}
// PatchUpdateCheckpoint patches the checkpoint for the indicated update with the given contents.
func (pc *Client) PatchUpdateCheckpoint(ctx context.Context, update UpdateIdentifier, deployment *apitype.DeploymentV1,
func (pc *Client) PatchUpdateCheckpoint(ctx context.Context, update UpdateIdentifier, deployment *apitype.DeploymentV2,
token string) error {
rawDeployment, err := json.Marshal(deployment)
@ -501,7 +501,7 @@ func (pc *Client) PatchUpdateCheckpoint(ctx context.Context, update UpdateIdenti
}
req := apitype.PatchUpdateCheckpointRequest{
Version: 1,
Version: 2,
Deployment: rawDeployment,
}

View file

@ -241,7 +241,7 @@ func (b *cloudBackend) getSnapshot(ctx context.Context, stackRef backend.StackRe
return nil, err
}
snapshot, err := stack.DeserializeDeployment(untypedDeployment)
snapshot, err := stack.DeserializeUntypedDeployment(untypedDeployment)
if err != nil {
return nil, err
}

View file

@ -373,7 +373,7 @@ func (b *localBackend) ExportDeployment(ctx context.Context,
}
return &apitype.UntypedDeployment{
Version: 1,
Version: 2,
Deployment: json.RawMessage(data),
}, nil
}
@ -387,7 +387,7 @@ func (b *localBackend) ImportDeployment(ctx context.Context, stackRef backend.St
return err
}
snap, err := stack.DeserializeDeployment(deployment)
snap, err := stack.DeserializeUntypedDeployment(deployment)
if err != nil {
return err
}

View file

@ -1009,6 +1009,8 @@ func (display *ProgressDisplay) getStepDoneDescription(step engine.StepEventMeta
return "deleting failed"
case deploy.OpReplace:
return "replacing failed"
case deploy.OpRead, deploy.OpReadReplacement:
return "reading failed"
}
} else {
switch op {
@ -1026,6 +1028,10 @@ func (display *ProgressDisplay) getStepDoneDescription(step engine.StepEventMeta
return "created replacement"
case deploy.OpDeleteReplaced:
return "deleted original"
case deploy.OpRead:
return "read"
case deploy.OpReadReplacement:
return "read for replacement"
}
}
@ -1056,6 +1062,10 @@ func (display *ProgressDisplay) getPreviewText(op deploy.StepOp) string {
return "create replacement"
case deploy.OpDeleteReplaced:
return "delete original"
case deploy.OpRead:
return "read"
case deploy.OpReadReplacement:
return "read for replacement"
}
contract.Failf("Unrecognized resource step op: %v", op)
@ -1120,6 +1130,10 @@ func (display *ProgressDisplay) getStepInProgressDescription(step engine.StepEve
return "creating replacement"
case deploy.OpDeleteReplaced:
return "deleting original"
case deploy.OpRead:
return "reading"
case deploy.OpReadReplacement:
return "reading for replacement"
}
contract.Failf("Unrecognized resource step op: %v", op)

View file

@ -136,7 +136,7 @@ func (b *localBackend) getStack(name tokens.QName) (config.Map, *deploy.Snapshot
}
// GetCheckpoint loads a checkpoint file for the given stack in this project, from the current project workspace.
func (b *localBackend) getCheckpoint(stackName tokens.QName) (*apitype.CheckpointV1, error) {
func (b *localBackend) getCheckpoint(stackName tokens.QName) (*apitype.CheckpointV2, error) {
chkpath := b.stackPath(stackName)
bytes, err := ioutil.ReadFile(chkpath)
if err != nil {

View file

@ -156,6 +156,8 @@ func (sm *SnapshotManager) BeginMutation(step deploy.Step) (engine.SnapshotMutat
return &deleteSnapshotMutation{sm}, nil
case deploy.OpReplace:
return &replaceSnapshotMutation{}, nil
case deploy.OpRead, deploy.OpReadReplacement:
return &readSnapshotMutation{sm}, nil
}
contract.Failf("unknown StepOp: %s", step.Op())
@ -243,6 +245,24 @@ type replaceSnapshotMutation struct{}
func (rsm *replaceSnapshotMutation) End(step deploy.Step, successful bool) error { return nil }
type readSnapshotMutation struct {
manager *SnapshotManager
}
func (rsm *readSnapshotMutation) End(step deploy.Step, successful bool) error {
contract.Require(step != nil, "step != nil")
logging.V(9).Infof("SnapshotManager: readSnapshotMutation.End(..., %v)", successful)
return rsm.manager.mutate(func() {
if successful {
if step.Old() != nil {
rsm.manager.markDone(step.Old())
}
rsm.manager.markNew(step.New())
}
})
}
// refresh does a no-op mutation that forces the SnapshotManager to persist the
// snapshot exactly as it is currently to disk. This is useful when a mutation
// has failed and we do not intend to persist the failed mutation.

View file

@ -27,7 +27,7 @@ import (
)
func getPulumiResources(t *testing.T, path string) *Resource {
var checkpoint apitype.CheckpointV1
var checkpoint apitype.CheckpointV2
byts, err := ioutil.ReadFile(path)
assert.NoError(t, err)
err = json.Unmarshal(byts, &checkpoint)

View file

@ -195,6 +195,17 @@ outer:
return nil, err
}
continue outer
case ReadResourceEvent:
steps, steperr := iter.stepGen.GenerateReadSteps(e)
if steperr != nil {
return nil, steperr
}
contract.Assert(len(steps) > 0)
if len(steps) > 1 {
iter.stepqueue = steps[1:]
}
return steps[0], nil
default:
contract.Failf("Unrecognized intent from source iterator: %v", reflect.TypeOf(event))
}

View file

@ -174,6 +174,7 @@ func TestBasicCRUDPlan(t *testing.T) {
nil,
"",
false,
false,
nil,
[]string{},
)
@ -188,6 +189,7 @@ func TestBasicCRUDPlan(t *testing.T) {
},
"",
false,
false,
nil,
[]string{},
)
@ -199,6 +201,7 @@ func TestBasicCRUDPlan(t *testing.T) {
nil,
"",
false,
false,
nil,
[]string{},
)

View file

@ -77,3 +77,27 @@ type RegisterResourceOutputsEvent interface {
// Done indicates that we are done with this step. It must be called to perform cleanup associated with the step.
Done()
}
// ReadResourceEvent is an event that asks the engine to read the state of a resource that already exists.
type ReadResourceEvent interface {
SourceEvent
// ID is the requested ID of this read.
ID() resource.ID
// Name is the requested name of this read.
Name() tokens.QName
// Type is type of the resource being read.
Type() tokens.Type
// Parent is the parent resource of the resource being read.
Parent() resource.URN
// Properties is the property bag that will be passed to Read as search parameters.
Properties() resource.PropertyMap
// Dependencies returns the list of URNs upon which this read depends.
Dependencies() []resource.URN
// Done indicates that we are done with this event.
Done(result *ReadResult)
}
type ReadResult struct {
State *resource.State
}

View file

@ -82,18 +82,20 @@ func (src *evalSource) Iterate(opts Options) (SourceIterator, error) {
// First, fire up a resource monitor that will watch for and record resource creation.
regChan := make(chan *registerResourceEvent)
regOutChan := make(chan *registerResourceOutputsEvent)
mon, err := newResourceMonitor(src, regChan, regOutChan)
regReadChan := make(chan *readResourceEvent)
mon, err := newResourceMonitor(src, regChan, regOutChan, regReadChan)
if err != nil {
return nil, errors.Wrap(err, "failed to start resource monitor")
}
// Create a new iterator with appropriate channels, and gear up to go!
iter := &evalSourceIterator{
mon: mon,
src: src,
regChan: regChan,
regOutChan: regOutChan,
finChan: make(chan error),
mon: mon,
src: src,
regChan: regChan,
regOutChan: regOutChan,
regReadChan: regReadChan,
finChan: make(chan error),
}
// Now invoke Run in a goroutine. All subsequent resource creation events will come in over the gRPC channel,
@ -105,12 +107,13 @@ func (src *evalSource) Iterate(opts Options) (SourceIterator, error) {
}
type evalSourceIterator struct {
mon *resmon // the resource monitor, per iterator.
src *evalSource // the owning eval source object.
regChan chan *registerResourceEvent // the channel that contains resource registrations.
regOutChan chan *registerResourceOutputsEvent // the channel that contains resource completions.
finChan chan error // the channel that communicates completion.
done bool // set to true when the evaluation is done.
mon *resmon // the resource monitor, per iterator.
src *evalSource // the owning eval source object.
regChan chan *registerResourceEvent // the channel that contains resource registrations.
regOutChan chan *registerResourceOutputsEvent // the channel that contains resource completions.
regReadChan chan *readResourceEvent // the channel that contains read resource requests.
finChan chan error // the channel that communicates completion.
done bool // set to true when the evaluation is done.
}
func (iter *evalSourceIterator) Close() error {
@ -137,6 +140,10 @@ func (iter *evalSourceIterator) Next() (SourceEvent, error) {
logging.V(5).Infof("EvalSourceIterator produced a completion: urn=%v,#outs=%v",
regOut.URN(), len(regOut.Outputs()))
return regOut, nil
case read := <-iter.regReadChan:
contract.Assert(read != nil)
logging.V(5).Infoln("EvalSourceIterator produced a read")
return read, nil
case err := <-iter.finChan:
// If we are finished, we can safely exit. The contract with the language provider is that this implies
// that the language runtime has exited and so calling Close on the plugin is fine.
@ -199,23 +206,25 @@ func (iter *evalSourceIterator) forkRun(opts Options) {
// resmon implements the pulumirpc.ResourceMonitor interface and acts as the gateway between a language runtime's
// evaluation of a program and the internal resource planning and deployment logic.
type resmon struct {
src *evalSource // the evaluation source.
regChan chan *registerResourceEvent // the channel to send resource registrations to.
regOutChan chan *registerResourceOutputsEvent // the channel to send resource output registrations to.
addr string // the address the host is listening on.
cancel chan bool // a channel that can cancel the server.
done chan error // a channel that resolves when the server completes.
src *evalSource // the evaluation source.
regChan chan *registerResourceEvent // the channel to send resource registrations to.
regOutChan chan *registerResourceOutputsEvent // the channel to send resource output registrations to.
regReadChan chan *readResourceEvent // the channel to send resource reads to.
addr string // the address the host is listening on.
cancel chan bool // a channel that can cancel the server.
done chan error // a channel that resolves when the server completes.
}
// newResourceMonitor creates a new resource monitor RPC server.
func newResourceMonitor(src *evalSource, regChan chan *registerResourceEvent,
regOutChan chan *registerResourceOutputsEvent) (*resmon, error) {
regOutChan chan *registerResourceOutputsEvent, regReadChan chan *readResourceEvent) (*resmon, error) {
// New up an engine RPC server.
resmon := &resmon{
src: src,
regChan: regChan,
regOutChan: regOutChan,
cancel: make(chan bool),
src: src,
regChan: regChan,
regOutChan: regOutChan,
regReadChan: regReadChan,
cancel: make(chan bool),
}
// Fire up a gRPC server and start listening for incomings.
@ -300,41 +309,59 @@ func (rm *resmon) ReadResource(ctx context.Context,
return nil, errors.Errorf("could not load resource provider for package '%v' from $PATH", t.Package())
}
// Manufacture a URN that is based on the program evaluation context.
var pt tokens.Type
if parent != "" {
pt = parent.Type()
}
urn := resource.NewURN(rm.src.Stack(), rm.src.Project(), pt, t, name)
// Now get the ID. If it is an unknown value -- as might happen during planning when, for example, reading
// the output of another resource's output property -- then we can skip the RPC as it can't possibly do anything.
id := resource.ID(req.GetId())
label := fmt.Sprintf("ResourceMonitor.ReadResource(%s, %s, %s)", id, t, name)
resp := &pulumirpc.ReadResourceResponse{Urn: string(urn)}
if id != plugin.UnknownStringValue {
// Unmarshal any additional state that came with the message.
props, err := plugin.UnmarshalProperties(
req.GetProperties(), plugin.MarshalOptions{Label: label, KeepUnknowns: true})
if err != nil {
return nil, errors.Wrapf(err, "failed to unmarshal read properties for resource %s", id)
}
// Now actually call the plugin to read the state and then return the results.
logging.V(5).Infof("ResourceMonitor.ReadResource received: %s #props=%d", label, len(props))
result, err := prov.Read(urn, id, props)
if err != nil {
return nil, errors.Wrapf(err, "reading resource %s state", urn)
}
marshaled, err := plugin.MarshalProperties(result, plugin.MarshalOptions{Label: label, KeepUnknowns: true})
if err != nil {
return nil, errors.Wrapf(err, "failed to marshal %s return state", urn)
}
resp.Properties = marshaled
var deps []resource.URN
for _, depURN := range req.GetDependencies() {
deps = append(deps, resource.URN(depURN))
}
return resp, nil
props, err := plugin.UnmarshalProperties(req.GetProperties(), plugin.MarshalOptions{
Label: label,
KeepUnknowns: true,
})
if err != nil {
return nil, err
}
event := &readResourceEvent{
id: id,
name: name,
baseType: t,
parent: parent,
props: props,
dependencies: deps,
done: make(chan *ReadResult),
}
select {
case rm.regReadChan <- event:
case <-rm.cancel:
logging.V(5).Infof("ResourceMonitor.ReadResource operation canceled, name=%s", name)
return nil, rpcerror.New(codes.Unavailable, "resource monitor shut down while sending resource registration")
}
// Now block waiting for the operation to finish.
var result *ReadResult
select {
case result = <-event.done:
case <-rm.cancel:
logging.V(5).Infof("ResourceMonitor.ReadResource operation canceled, name=%s", name)
return nil, rpcerror.New(codes.Unavailable, "resource monitor shut down while waiting on step's done channel")
}
contract.Assert(result != nil)
marshaled, err := plugin.MarshalProperties(result.State.Outputs, plugin.MarshalOptions{
Label: label,
KeepUnknowns: true,
})
if err != nil {
return nil, errors.Wrapf(err, "failed to marshal %s return state", result.State.URN)
}
return &pulumirpc.ReadResourceResponse{
Urn: string(result.State.URN),
Properties: marshaled,
}, nil
}
// RegisterResource is invoked by a language process when a new resource has been allocated.
@ -497,3 +524,27 @@ func (g *registerResourceOutputsEvent) Done() {
// Communicate the resulting state back to the RPC thread, which is parked awaiting our reply.
g.done <- true
}
type readResourceEvent struct {
id resource.ID
name tokens.QName
baseType tokens.Type
parent resource.URN
props resource.PropertyMap
dependencies []resource.URN
done chan *ReadResult
}
var _ ReadResourceEvent = (*readResourceEvent)(nil)
func (g *readResourceEvent) event() {}
func (g *readResourceEvent) ID() resource.ID { return g.id }
func (g *readResourceEvent) Name() tokens.QName { return g.name }
func (g *readResourceEvent) Type() tokens.Type { return g.baseType }
func (g *readResourceEvent) Parent() resource.URN { return g.parent }
func (g *readResourceEvent) Properties() resource.PropertyMap { return g.props }
func (g *readResourceEvent) Dependencies() []resource.URN { return g.dependencies }
func (g *readResourceEvent) Done(result *ReadResult) {
g.done <- result
}

View file

@ -101,7 +101,8 @@ func (iter *refreshSourceIterator) newRefreshGoal(s *resource.State) (*resource.
return nil, nil // the resource was deleted.
}
s = resource.NewState(
s.Type, s.URN, s.Custom, s.Delete, s.ID, s.Inputs, refreshed, s.Parent, s.Protect, s.Dependencies, s.InitErrors)
s.Type, s.URN, s.Custom, s.Delete, s.ID, s.Inputs, refreshed,
s.Parent, s.Protect, s.External, s.Dependencies, s.InitErrors)
}
// Now just return the actual state as the goal state.

View file

@ -102,6 +102,7 @@ func NewCreateStep(plan *Plan, reg RegisterResourceEvent, new *resource.State) S
contract.Assert(new.URN != "")
contract.Assert(new.ID == "")
contract.Assert(!new.Delete)
contract.Assert(!new.External)
return &CreateStep{
plan: plan,
reg: reg,
@ -121,6 +122,7 @@ func NewCreateReplacementStep(plan *Plan, reg RegisterResourceEvent,
contract.Assert(new.ID == "")
contract.Assert(!new.Delete)
contract.Assert(old.Type == new.Type)
contract.Assert(!new.External)
return &CreateStep{
plan: plan,
reg: reg,
@ -191,7 +193,8 @@ func (s *CreateStep) Apply(preview bool) (resource.Status, error) {
return resourceStatus, resourceError
}
// DeleteStep is a mutating step that deletes an existing resource.
// DeleteStep is a mutating step that deletes an existing resource. If `old` is marked "External",
// DeleteStep is a no-op.
type DeleteStep struct {
plan *Plan // the current plan.
old *resource.State // the state of the existing resource.
@ -244,7 +247,8 @@ func (s *DeleteStep) Apply(preview bool) (resource.Status, error) {
errors.Errorf("refusing to delete protected resource '%s'", s.old.URN)
}
if !preview {
// Deleting an External resource is a no-op, since Pulumi does not own the lifecycle.
if !preview && !s.old.External {
if s.old.Custom && !s.plan.IsRefresh() {
// Invoke the Delete RPC function for this provider:
prov, err := getProvider(s)
@ -282,6 +286,8 @@ func NewUpdateStep(plan *Plan, reg RegisterResourceEvent, old *resource.State,
contract.Assert(new.ID == "")
contract.Assert(!new.Delete)
contract.Assert(old.Type == new.Type)
contract.Assert(!new.External)
contract.Assert(!old.External)
return &UpdateStep{
plan: plan,
reg: reg,
@ -364,7 +370,7 @@ func NewReplaceStep(plan *Plan, old *resource.State, new *resource.State,
contract.Assert(!old.Delete)
contract.Assert(new != nil)
contract.Assert(new.URN != "")
contract.Assert(new.ID == "")
// contract.Assert(new.ID == "")
contract.Assert(!new.Delete)
return &ReplaceStep{
plan: plan,
@ -391,6 +397,107 @@ func (s *ReplaceStep) Apply(preview bool) (resource.Status, error) {
return resource.StatusOK, nil
}
// ReadStep is a step indicating that an existing resources will be "read" and projected into the Pulumi object
// model. Resources that are read are marked with the "External" bit which indicates to the engine that it does
// not own this resource's lifeycle.
//
// A resource with a given URN can transition freely between an "external" state and a non-external state. If
// a URN that was previously marked "External" (i.e. was the target of a ReadStep in a previous plan) is the
// target of a RegisterResource in the next plan, a CreateReplacement step will be issued to indicate the transition
// from external to owned. If a URN that was previously not marked "External" is the target of a ReadResource in the
// next plan, a ReadReplacement step will be issued to indicate the transition from owned to external.
type ReadStep struct {
plan *Plan // the plan that produced this read
event ReadResourceEvent // the event that should be signaled upon completion
old *resource.State // the old resource state, if one exists for this urn
new *resource.State // the new resource state, to be used to query the provider
replacing bool // whether or not the new resource is replacing the old resource
}
// NewReadStep creates a new Read step.
func NewReadStep(plan *Plan, event ReadResourceEvent, old *resource.State, new *resource.State) Step {
contract.Assert(new != nil)
contract.Assertf(new.External, "target of Read step must be marked External")
contract.Assertf(new.Custom, "target of Read step must be Custom")
// If Old was given, it's either an external resource or its ID is equal to the
// ID that we are preparing to read.
if old != nil {
contract.Assert(old.ID == new.ID || old.External)
}
return &ReadStep{
plan: plan,
event: event,
old: old,
new: new,
replacing: false,
}
}
// NewReadReplacementStep creates a new Read step with the `replacing` flag set. When executed,
// it will pend deletion of the "old" resource, which must not be an external resource.
func NewReadReplacementStep(plan *Plan, event ReadResourceEvent, old *resource.State, new *resource.State) Step {
contract.Assert(new != nil)
contract.Assertf(new.External, "target of ReadReplacement step must be marked External")
contract.Assertf(new.Custom, "target of ReadReplacement step must be Custom")
contract.Assert(old != nil)
contract.Assertf(!old.External, "old target of ReadReplacement step must not be External")
return &ReadStep{
plan: plan,
event: event,
old: old,
new: new,
replacing: true,
}
}
func (s *ReadStep) Op() StepOp {
if s.replacing {
return OpReadReplacement
}
return OpRead
}
func (s *ReadStep) Plan() *Plan { return s.plan }
func (s *ReadStep) Type() tokens.Type { return s.new.Type }
func (s *ReadStep) URN() resource.URN { return s.new.URN }
func (s *ReadStep) Old() *resource.State { return s.old }
func (s *ReadStep) New() *resource.State { return s.new }
func (s *ReadStep) Res() *resource.State { return s.new }
func (s *ReadStep) Logical() bool { return !s.replacing }
func (s *ReadStep) Apply(preview bool) (resource.Status, error) {
urn := s.new.URN
id := s.new.ID
// Unlike most steps, Read steps run during previews. The only time
// we can't run is if the ID we are given is unknown.
if id != "" {
prov, err := getProvider(s)
if err != nil {
return resource.StatusOK, err
}
result, err := prov.Read(urn, id, s.new.Inputs)
if err != nil {
return resource.StatusUnknown, err
}
s.new.Outputs = result
}
// If we were asked to replace an existing, non-External resource, pend the
// deletion here.
if s.replacing {
s.old.Delete = true
}
s.event.Done(&ReadResult{State: s.new})
return resource.StatusOK, nil
}
// StepOp represents the kind of operation performed by a step. It evaluates to its string label.
type StepOp string
@ -402,6 +509,8 @@ const (
OpReplace StepOp = "replace" // replacing a resource with a new one.
OpCreateReplacement StepOp = "create-replacement" // creating a new resource for a replacement.
OpDeleteReplaced StepOp = "delete-replaced" // deleting an existing resource after replacement.
OpRead StepOp = "read" // reading an existing resource.
OpReadReplacement StepOp = "read-replacement" // reading an existing resource for a replacement.
)
// StepOps contains the full set of step operation types.
@ -413,6 +522,8 @@ var StepOps = []StepOp{
OpReplace,
OpCreateReplacement,
OpDeleteReplaced,
OpRead,
OpReadReplacement,
}
// Color returns a suggested color for lines of this op type.
@ -432,6 +543,10 @@ func (op StepOp) Color() string {
return colors.SpecCreateReplacement
case OpDeleteReplaced:
return colors.SpecDeleteReplaced
case OpRead:
return colors.SpecCreate
case OpReadReplacement:
return colors.SpecReplace
default:
contract.Failf("Unrecognized resource step op: '%v'", op)
return ""
@ -460,6 +575,10 @@ func (op StepOp) RawPrefix() string {
return "++"
case OpDeleteReplaced:
return "--"
case OpRead:
return ">-"
case OpReadReplacement:
return ">~"
default:
contract.Failf("Unrecognized resource step op: %v", op)
return ""
@ -468,8 +587,10 @@ func (op StepOp) RawPrefix() string {
func (op StepOp) PastTense() string {
switch op {
case OpSame, OpCreate, OpDelete, OpReplace, OpCreateReplacement, OpDeleteReplaced, OpUpdate:
case OpSame, OpCreate, OpDelete, OpReplace, OpCreateReplacement, OpDeleteReplaced, OpUpdate, OpReadReplacement:
return string(op) + "d"
case OpRead:
return "read"
default:
contract.Failf("Unexpected resource step op: %v", op)
return ""
@ -478,7 +599,7 @@ func (op StepOp) PastTense() string {
// Suffix returns a suggested suffix for lines of this op type.
func (op StepOp) Suffix() string {
if op == OpCreateReplacement || op == OpUpdate || op == OpReplace {
if op == OpCreateReplacement || op == OpUpdate || op == OpReplace || op == OpReadReplacement {
return colors.Reset // updates and replacements colorize individual lines; get has none
}
return ""

View file

@ -34,6 +34,7 @@ type stepGenerator struct {
opts Options // options for this step generator
urns map[resource.URN]bool // set of URNs discovered for this plan
reads map[resource.URN]bool // set of URNs read for this plan
deletes map[resource.URN]bool // set of URNs deleted in this plan
replaces map[resource.URN]bool // set of URNs replaced in this plan
updates map[resource.URN]bool // set of URNs updated in this plan
@ -41,6 +42,57 @@ type stepGenerator struct {
sames map[resource.URN]bool // set of URNs that were not changed in this plan
}
// GenerateReadSteps is responsible for producing one or more steps required to service
// a ReadResourceEvent coming from the language host.
func (sg *stepGenerator) GenerateReadSteps(event ReadResourceEvent) ([]Step, error) {
urn := sg.generateURN(event.Parent(), event.Type(), event.Name())
newState := resource.NewState(event.Type(),
urn,
true, /*custom*/
false, /*delete*/
event.ID(),
event.Properties(),
make(resource.PropertyMap), /* outputs */
event.Parent(),
false, /*protect*/
true, /*external*/
event.Dependencies(),
nil /* initErrors */)
old, hasOld := sg.plan.Olds()[urn]
// If the snapshot has an old resource for this URN and it's not external, we're going
// to have to delete the old resource and conceptually replace it with the resource we
// are about to read.
//
// We accomplish this through the "read-replacement" step, which atomically reads a resource
// and marks the resource it is replacing as pending deletion.
//
// In the event that the new "read" resource's ID matches the existing resource,
// we do not need to delete the resource - we know exactly what resource we are going
// to get from the read.
//
// This operation is tenatively called "relinquish" - it semantically represents the
// release of a resource from the management of Pulumi.
if hasOld && !old.External && old.ID != event.ID() {
logging.V(7).Infof(
"stepGenerator.GenerateReadSteps(...): replacing existing resource %s, ids don't match", urn)
sg.replaces[urn] = true
return []Step{
NewReadReplacementStep(sg.plan, event, old, newState),
NewReplaceStep(sg.plan, old, newState, nil, true),
}, nil
}
if logging.V(7) && old.ID == event.ID() {
logging.V(7).Infof("stepGenerator.GenerateReadSteps(...): recognized relinquish of resource %s", urn)
}
sg.reads[urn] = true
return []Step{
NewReadStep(sg.plan, event, old, newState),
}, nil
}
// GenerateSteps produces one or more steps required to achieve the goal state
// specified by the incoming RegisterResourceEvent.
//
@ -52,7 +104,7 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
goal := event.Goal()
// generate an URN for this new resource.
urn := sg.generateURN(event)
urn := sg.generateURN(goal.Parent, goal.Type, goal.Name)
if sg.urns[urn] {
invalid = true
// TODO[pulumi/pulumi-framework#19]: improve this error message!
@ -91,14 +143,18 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
// We may be re-creating this resource if it got deleted earlier in the execution of this plan.
_, recreating := sg.deletes[urn]
// We may be creating this resource if it previously existed in the snapshot as an External resource
wasExternal := hasOld && old.External
// If this isn't a refresh, ensure the provider is okay with this resource and fetch the inputs to pass to
// subsequent methods. If these are not inputs, we are just going to blindly store the outputs, so skip this.
if prov != nil && !refresh {
var failures []plugin.CheckFailure
// If we are re-creating this resource because it was deleted earlier, the old inputs are now
// invalid (they got deleted) so don't consider them.
if recreating {
// invalid (they got deleted) so don't consider them. Similarly, if the old resource was External,
// don't consider those inputs since Pulumi does not own them.
if recreating || wasExternal {
inputs, failures, err = prov.Check(urn, nil, goal.Properties, allowUnknowns)
} else {
inputs, failures, err = prov.Check(urn, oldInputs, inputs, allowUnknowns)
@ -139,7 +195,7 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
return nil, errors.New("One or more resource validation errors occurred; refusing to proceed")
}
// There are three cases we need to consider when figuring out what to do with this resource.
// There are four cases we need to consider when figuring out what to do with this resource.
//
// Case 1: recreating
// In this case, we have seen a resource with this URN before and we have already issued a
@ -167,7 +223,28 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
}, nil
}
// Case 2: hasOld
// Case 2: wasExternal
// In this case, the resource we are operating upon exists in the old snapshot, but it
// was "external" - Pulumi does not own its lifecycle. Conceptually, this operation is
// akin to "taking ownership" of a resource that we did not previously control.
//
// Since we are not allowed to manipulate the existing resource, we must create a resource
// to take its place. Since this is technically a replacement operation, we pend deletion of
// read until the end of the plan.
if wasExternal {
logging.V(7).Infof("Planner recognized '%s' as old external resource, creating instead", urn)
sg.creates[urn] = true
if err != nil {
return nil, err
}
return []Step{
NewCreateReplacementStep(sg.plan, event, old, new, nil, true),
NewReplaceStep(sg.plan, old, new, nil, true),
}, nil
}
// Case 3: hasOld
// In this case, the resource we are operating upon now exists in the old snapshot.
// It must be an update or a replace. Which operation we do depends on the provider's
// response to `Diff`. We must:
@ -252,8 +329,8 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
logging.V(7).Infof("Planner decided to delete '%v' due to dependence on condemned resource '%v'",
dependentResource.URN, urn)
steps = append(steps, NewDeleteReplacementStep(sg.plan, dependentResource, false))
steps = append(steps, NewDeleteReplacementStep(sg.plan, dependentResource, false))
// Mark the condemned resource as deleted. We won't know until later in the plan whether
// or not we're going to be replacing this resource.
sg.deletes[dependentResource.URN] = true
@ -289,7 +366,7 @@ func (sg *stepGenerator) GenerateSteps(event RegisterResourceEvent) ([]Step, err
return []Step{NewSameStep(sg.plan, event, old, new)}, nil
}
// Case 3: Not Case 1 or Case 2
// Case 4: Not Case 1, 2, or 3
// If a resource isn't being recreated and it's not being updated or replaced,
// it's just being created.
sg.creates[urn] = true
@ -330,7 +407,8 @@ func (sg *stepGenerator) GenerateDeletes() []Step {
}
sg.deletes[res.URN] = true
dels = append(dels, NewDeleteReplacementStep(sg.plan, res, true))
} else if !sg.sames[res.URN] && !sg.updates[res.URN] && !sg.replaces[res.URN] && !sg.deletes[res.URN] {
} else if !sg.sames[res.URN] && !sg.updates[res.URN] && !sg.replaces[res.URN] &&
!sg.deletes[res.URN] && !sg.reads[res.URN] {
// In addition to the above comment, I am fairly certain there is a bug here. If a resource
// is not registered in a plan, but there exists a pending delete copy of that resource in the
// snapshot, we will choose not to delete the live resource and instead be content with deleting
@ -406,21 +484,18 @@ func (sg *stepGenerator) getResourcePropertyStates(urn resource.URN, goal *resou
}
return props, inputs, outputs,
resource.NewState(goal.Type, urn, goal.Custom, false, "",
inputs, outputs, goal.Parent, goal.Protect, goal.Dependencies, []string{})
inputs, outputs, goal.Parent, goal.Protect, false, goal.Dependencies, []string{})
}
func (sg *stepGenerator) generateURN(e RegisterResourceEvent) resource.URN {
func (sg *stepGenerator) generateURN(parent resource.URN, ty tokens.Type, name tokens.QName) resource.URN {
// Use the resource goal state name to produce a globally unique URN.
goal := e.Goal()
parentType := tokens.Type("")
if p := goal.Parent; p != "" && p.Type() != resource.RootStackType {
if parent != "" && parent.Type() != resource.RootStackType {
// Skip empty parents and don't use the root stack type; otherwise, use the full qualified type.
parentType = p.QualifiedType()
parentType = parent.QualifiedType()
}
return resource.NewURN(sg.plan.Target().Name, sg.plan.source.Project(), parentType, goal.Type, goal.Name)
return resource.NewURN(sg.plan.Target().Name, sg.plan.source.Project(), parentType, ty, name)
}
// issueCheckErrors prints any check errors to the diagnostics sink.
@ -467,6 +542,7 @@ func newStepGenerator(plan *Plan, opts Options) *stepGenerator {
plan: plan,
opts: opts,
urns: make(map[resource.URN]bool),
reads: make(map[resource.URN]bool),
creates: make(map[resource.URN]bool),
sames: make(map[resource.URN]bool),
replaces: make(map[resource.URN]bool),

View file

@ -32,14 +32,15 @@ type State struct {
Outputs PropertyMap // the resource's complete output state (as returned by the resource provider).
Parent URN // an optional parent URN that this resource belongs to.
Protect bool // true to "protect" this resource (protected resources cannot be deleted).
External bool // true if this resource is "external" to Pulumi and we don't control the lifecycle
Dependencies []URN // the resource's dependencies
InitErrors []string // the set of errors encountered in the process of initializing resource.
}
// NewState creates a new resource value from existing resource state information.
func NewState(t tokens.Type, urn URN, custom bool, del bool, id ID,
inputs PropertyMap, outputs PropertyMap, parent URN, protect bool, dependencies []URN,
initErrors []string) *State {
inputs PropertyMap, outputs PropertyMap, parent URN, protect bool,
external bool, dependencies []URN, initErrors []string) *State {
contract.Assertf(t != "", "type was empty")
contract.Assertf(custom || id == "", "is custom or had empty ID")
contract.Assertf(inputs != nil, "inputs was non-nil")
@ -53,6 +54,7 @@ func NewState(t tokens.Type, urn URN, custom bool, del bool, id ID,
Outputs: outputs,
Parent: parent,
Protect: protect,
External: external,
Dependencies: dependencies,
InitErrors: initErrors,
}

View file

@ -19,19 +19,18 @@ package stack
import (
"encoding/json"
"github.com/blang/semver"
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/apitype"
"github.com/pulumi/pulumi/pkg/apitype/migrate"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/resource/config"
"github.com/pulumi/pulumi/pkg/resource/deploy"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
"github.com/pulumi/pulumi/pkg/workspace"
)
func UnmarshalVersionedCheckpointToLatestCheckpoint(bytes []byte) (*apitype.CheckpointV1, error) {
func UnmarshalVersionedCheckpointToLatestCheckpoint(bytes []byte) (*apitype.CheckpointV2, error) {
var versionedCheckpoint apitype.VersionedCheckpoint
if err := json.Unmarshal(bytes, &versionedCheckpoint); err != nil {
return nil, err
@ -43,19 +42,28 @@ func UnmarshalVersionedCheckpointToLatestCheckpoint(bytes []byte) (*apitype.Chec
// json package did not support strict marshalling before 1.10, and we use 1.9 in our toolchain today.
// After we upgrade, we could consider rewriting this code to use DisallowUnknownFields() on the decoder
// to have the old checkpoint not even deserialize as an apitype.VersionedCheckpoint.
var checkpoint apitype.CheckpointV1
if err := json.Unmarshal(bytes, &checkpoint); err != nil {
var v1checkpoint apitype.CheckpointV1
if err := json.Unmarshal(bytes, &v1checkpoint); err != nil {
return nil, err
}
checkpoint := migrate.UpToCheckpointV2(v1checkpoint)
return &checkpoint, nil
case 1:
var checkpoint apitype.CheckpointV1
if err := json.Unmarshal(versionedCheckpoint.Checkpoint, &checkpoint); err != nil {
var v1checkpoint apitype.CheckpointV1
if err := json.Unmarshal(versionedCheckpoint.Checkpoint, &v1checkpoint); err != nil {
return nil, err
}
checkpoint := migrate.UpToCheckpointV2(v1checkpoint)
return &checkpoint, nil
case 2:
var v2checkpoint apitype.CheckpointV2
if err := json.Unmarshal(versionedCheckpoint.Checkpoint, &v2checkpoint); err != nil {
return nil, err
}
return &v2checkpoint, nil
default:
return nil, errors.Errorf("unsupported checkpoint version %d", versionedCheckpoint.Version)
}
@ -64,12 +72,12 @@ func UnmarshalVersionedCheckpointToLatestCheckpoint(bytes []byte) (*apitype.Chec
// SerializeCheckpoint turns a snapshot into a data structure suitable for serialization.
func SerializeCheckpoint(stack tokens.QName, config config.Map, snap *deploy.Snapshot) *apitype.VersionedCheckpoint {
// If snap is nil, that's okay, we will just create an empty deployment; otherwise, serialize the whole snapshot.
var latest *apitype.DeploymentV1
var latest *apitype.DeploymentV2
if snap != nil {
latest = SerializeDeployment(snap)
}
b, err := json.Marshal(apitype.CheckpointV1{
b, err := json.Marshal(apitype.CheckpointV2{
Stack: stack,
Config: config,
Latest: latest,
@ -82,48 +90,15 @@ func SerializeCheckpoint(stack tokens.QName, config config.Map, snap *deploy.Sna
}
}
// DeserializeCheckpoint takes a serialized deployment record and returns its associated snapshot.
func DeserializeCheckpoint(chkpoint *apitype.CheckpointV1) (*deploy.Snapshot, error) {
// DeserializeCheckpoint takes a serialized deployment record and returns its associated snapshot. Returns nil
// if there have been no deployments performed on this checkpoint.
func DeserializeCheckpoint(chkpoint *apitype.CheckpointV2) (*deploy.Snapshot, error) {
contract.Require(chkpoint != nil, "chkpoint")
var snap *deploy.Snapshot
if latest := chkpoint.Latest; latest != nil {
// Unpack the versions.
manifest := deploy.Manifest{
Time: latest.Manifest.Time,
Magic: latest.Manifest.Magic,
Version: latest.Manifest.Version,
}
for _, plug := range latest.Manifest.Plugins {
var version *semver.Version
if v := plug.Version; v != "" {
sv, err := semver.ParseTolerant(v)
if err != nil {
return nil, err
}
version = &sv
}
manifest.Plugins = append(manifest.Plugins, workspace.PluginInfo{
Name: plug.Name,
Kind: plug.Type,
Version: version,
})
}
// For every serialized resource vertex, create a ResourceDeployment out of it.
var resources []*resource.State
for _, res := range latest.Resources {
desres, err := DeserializeResource(res)
if err != nil {
return nil, err
}
resources = append(resources, desres)
}
snap = deploy.NewSnapshot(manifest, resources)
if chkpoint.Latest != nil {
return DeserializeDeploymentV2(*chkpoint.Latest)
}
return snap, nil
return nil, nil
}
// GetRootStackResource returns the root stack resource from a given snapshot, or nil if not found. If the stack

View file

@ -19,10 +19,13 @@ import (
"fmt"
"reflect"
"github.com/blang/semver"
"github.com/pulumi/pulumi/pkg/apitype"
"github.com/pulumi/pulumi/pkg/apitype/migrate"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/resource/deploy"
"github.com/pulumi/pulumi/pkg/util/contract"
"github.com/pulumi/pulumi/pkg/workspace"
)
const (
@ -44,7 +47,7 @@ var (
)
// SerializeDeployment serializes an entire snapshot as a deploy record.
func SerializeDeployment(snap *deploy.Snapshot) *apitype.DeploymentV1 {
func SerializeDeployment(snap *deploy.Snapshot) *apitype.DeploymentV2 {
contract.Require(snap != nil, "snap")
// Capture the version information into a manifest.
@ -67,21 +70,21 @@ func SerializeDeployment(snap *deploy.Snapshot) *apitype.DeploymentV1 {
}
// Serialize all vertices and only include a vertex section if non-empty.
var resources []apitype.ResourceV1
var resources []apitype.ResourceV2
for _, res := range snap.Resources {
resources = append(resources, SerializeResource(res))
}
return &apitype.DeploymentV1{
return &apitype.DeploymentV2{
Manifest: manifest,
Resources: resources,
}
}
// DeserializeDeployment deserializes an untyped deployment and produces a `deploy.Snapshot`
// DeserializeUntypedDeployment deserializes an untyped deployment and produces a `deploy.Snapshot`
// from it. DeserializeDeployment will return an error if the untyped deployment's version is
// not within the range `DeploymentSchemaVersionCurrent` and `DeploymentSchemaVersionOldestSupported`.
func DeserializeDeployment(deployment *apitype.UntypedDeployment) (*deploy.Snapshot, error) {
func DeserializeUntypedDeployment(deployment *apitype.UntypedDeployment) (*deploy.Snapshot, error) {
contract.Require(deployment != nil, "deployment")
switch {
case deployment.Version > apitype.DeploymentSchemaVersionCurrent:
@ -90,19 +93,65 @@ func DeserializeDeployment(deployment *apitype.UntypedDeployment) (*deploy.Snaps
return nil, ErrDeploymentSchemaVersionTooOld
}
var checkpoint apitype.CheckpointV1
var v2deployment apitype.DeploymentV2
switch deployment.Version {
case 1:
if err := json.Unmarshal([]byte(deployment.Deployment), &checkpoint.Latest); err != nil {
var v1deployment apitype.DeploymentV1
if err := json.Unmarshal([]byte(deployment.Deployment), &v1deployment); err != nil {
return nil, err
}
v2deployment = migrate.UpToDeploymentV2(v1deployment)
case 2:
if err := json.Unmarshal([]byte(deployment.Deployment), &v2deployment); err != nil {
return nil, err
}
default:
contract.Failf("unrecognized version: %d", deployment.Version)
}
return DeserializeCheckpoint(&checkpoint)
return DeserializeDeploymentV2(v2deployment)
}
// DeserializeDeploymentV2 deserializes a typed DeploymentV2 into a `deploy.Snapshot`.
func DeserializeDeploymentV2(deployment apitype.DeploymentV2) (*deploy.Snapshot, error) {
// Unpack the versions.
manifest := deploy.Manifest{
Time: deployment.Manifest.Time,
Magic: deployment.Manifest.Magic,
Version: deployment.Manifest.Version,
}
for _, plug := range deployment.Manifest.Plugins {
var version *semver.Version
if v := plug.Version; v != "" {
sv, err := semver.ParseTolerant(v)
if err != nil {
return nil, err
}
version = &sv
}
manifest.Plugins = append(manifest.Plugins, workspace.PluginInfo{
Name: plug.Name,
Kind: plug.Type,
Version: version,
})
}
// For every serialized resource vertex, create a ResourceDeployment out of it.
var resources []*resource.State
for _, res := range deployment.Resources {
desres, err := DeserializeResource(res)
if err != nil {
return nil, err
}
resources = append(resources, desres)
}
return deploy.NewSnapshot(manifest, resources), nil
}
// SerializeResource turns a resource into a structure suitable for serialization.
func SerializeResource(res *resource.State) apitype.ResourceV1 {
func SerializeResource(res *resource.State) apitype.ResourceV2 {
contract.Assert(res != nil)
contract.Assertf(string(res.URN) != "", "Unexpected empty resource resource.URN")
@ -116,7 +165,7 @@ func SerializeResource(res *resource.State) apitype.ResourceV1 {
outputs = SerializeProperties(outp)
}
return apitype.ResourceV1{
return apitype.ResourceV2{
URN: res.URN,
Custom: res.Custom,
Delete: res.Delete,
@ -126,6 +175,7 @@ func SerializeResource(res *resource.State) apitype.ResourceV1 {
Inputs: inputs,
Outputs: outputs,
Protect: res.Protect,
External: res.External,
Dependencies: res.Dependencies,
InitErrors: res.InitErrors,
}
@ -177,7 +227,7 @@ func SerializePropertyValue(prop resource.PropertyValue) interface{} {
}
// DeserializeResource turns a serialized resource back into its usual form.
func DeserializeResource(res apitype.ResourceV1) (*resource.State, error) {
func DeserializeResource(res apitype.ResourceV2) (*resource.State, error) {
// Deserialize the resource properties, if they exist.
inputs, err := DeserializeProperties(res.Inputs)
if err != nil {
@ -189,8 +239,8 @@ func DeserializeResource(res apitype.ResourceV1) (*resource.State, error) {
}
return resource.NewState(
res.Type, res.URN, res.Custom, res.Delete, res.ID, inputs, outputs, res.Parent, res.Protect,
res.Dependencies, res.InitErrors), nil
res.Type, res.URN, res.Custom, res.Delete, res.ID,
inputs, outputs, res.Parent, res.Protect, res.External, res.Dependencies, res.InitErrors), nil
}
// DeserializeProperties deserializes an entire map of deploy properties into a resource property map.

View file

@ -69,6 +69,7 @@ func TestDeploymentSerialization(t *testing.T) {
}),
"",
false,
false,
[]resource.URN{
resource.URN("foo:bar:baz"),
resource.URN("foo:bar:boo"),
@ -150,7 +151,7 @@ func TestLoadTooNewDeployment(t *testing.T) {
Version: apitype.DeploymentSchemaVersionCurrent + 1,
}
deployment, err := DeserializeDeployment(untypedDeployment)
deployment, err := DeserializeUntypedDeployment(untypedDeployment)
assert.Nil(t, deployment)
assert.Error(t, err)
assert.Equal(t, ErrDeploymentSchemaVersionTooNew, err)
@ -161,7 +162,7 @@ func TestLoadTooOldDeployment(t *testing.T) {
Version: DeploymentSchemaVersionOldestSupported - 1,
}
deployment, err := DeserializeDeployment(untypedDeployment)
deployment, err := DeserializeUntypedDeployment(untypedDeployment)
assert.Nil(t, deployment)
assert.Error(t, err)
assert.Equal(t, ErrDeploymentSchemaVersionTooOld, err)

View file

@ -31,12 +31,19 @@ goog.exportSymbol('proto.pulumirpc.RegisterResourceResponse', null, global);
* @constructor
*/
proto.pulumirpc.ReadResourceRequest = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, null, null);
jspb.Message.initialize(this, opt_data, 0, -1, proto.pulumirpc.ReadResourceRequest.repeatedFields_, null);
};
goog.inherits(proto.pulumirpc.ReadResourceRequest, jspb.Message);
if (goog.DEBUG && !COMPILED) {
proto.pulumirpc.ReadResourceRequest.displayName = 'proto.pulumirpc.ReadResourceRequest';
}
/**
* List of repeated fields within this message type.
* @private {!Array<number>}
* @const
*/
proto.pulumirpc.ReadResourceRequest.repeatedFields_ = [6];
if (jspb.Message.GENERATE_TO_OBJECT) {
@ -70,7 +77,8 @@ proto.pulumirpc.ReadResourceRequest.toObject = function(includeInstance, msg) {
type: jspb.Message.getFieldWithDefault(msg, 2, ""),
name: jspb.Message.getFieldWithDefault(msg, 3, ""),
parent: jspb.Message.getFieldWithDefault(msg, 4, ""),
properties: (f = msg.getProperties()) && google_protobuf_struct_pb.Struct.toObject(includeInstance, f)
properties: (f = msg.getProperties()) && google_protobuf_struct_pb.Struct.toObject(includeInstance, f),
dependenciesList: jspb.Message.getRepeatedField(msg, 6)
};
if (includeInstance) {
@ -128,6 +136,10 @@ proto.pulumirpc.ReadResourceRequest.deserializeBinaryFromReader = function(msg,
reader.readMessage(value,google_protobuf_struct_pb.Struct.deserializeBinaryFromReader);
msg.setProperties(value);
break;
case 6:
var value = /** @type {string} */ (reader.readString());
msg.addDependencies(value);
break;
default:
reader.skipField();
break;
@ -193,6 +205,13 @@ proto.pulumirpc.ReadResourceRequest.serializeBinaryToWriter = function(message,
google_protobuf_struct_pb.Struct.serializeBinaryToWriter
);
}
f = message.getDependenciesList();
if (f.length > 0) {
writer.writeRepeatedString(
6,
f
);
}
};
@ -286,6 +305,35 @@ proto.pulumirpc.ReadResourceRequest.prototype.hasProperties = function() {
};
/**
* repeated string dependencies = 6;
* @return {!Array.<string>}
*/
proto.pulumirpc.ReadResourceRequest.prototype.getDependenciesList = function() {
return /** @type {!Array.<string>} */ (jspb.Message.getRepeatedField(this, 6));
};
/** @param {!Array.<string>} value */
proto.pulumirpc.ReadResourceRequest.prototype.setDependenciesList = function(value) {
jspb.Message.setField(this, 6, value || []);
};
/**
* @param {!string} value
* @param {number=} opt_index
*/
proto.pulumirpc.ReadResourceRequest.prototype.addDependencies = function(value, opt_index) {
jspb.Message.addToRepeatedField(this, 6, value, opt_index);
};
proto.pulumirpc.ReadResourceRequest.prototype.clearDependenciesList = function() {
this.setDependenciesList([]);
};
/**
* Generated by JsPbCodeGenerator.

View file

@ -73,6 +73,7 @@ export function readResource(res: Resource, t: string, name: string, props: Inpu
req.setId(resolvedID);
req.setParent(resop.parentURN);
req.setProperties(gstruct.Struct.fromJavaScript(resop.serializedProps));
req.setDependenciesList(Array.from(resop.dependencies));
// Now run the operation, serializing the invocation if necessary.
const opLabel = `monitor.readResource(${label})`;

View file

@ -32,6 +32,7 @@ type ReadResourceRequest struct {
Name string `protobuf:"bytes,3,opt,name=name" json:"name,omitempty"`
Parent string `protobuf:"bytes,4,opt,name=parent" json:"parent,omitempty"`
Properties *_struct.Struct `protobuf:"bytes,5,opt,name=properties" json:"properties,omitempty"`
Dependencies []string `protobuf:"bytes,6,rep,name=dependencies" json:"dependencies,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
XXX_sizecache int32 `json:"-"`
@ -41,7 +42,7 @@ func (m *ReadResourceRequest) Reset() { *m = ReadResourceRequest{} }
func (m *ReadResourceRequest) String() string { return proto.CompactTextString(m) }
func (*ReadResourceRequest) ProtoMessage() {}
func (*ReadResourceRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_resource_55fabfa8edee605e, []int{0}
return fileDescriptor_resource_773fb6366ec12ccd, []int{0}
}
func (m *ReadResourceRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReadResourceRequest.Unmarshal(m, b)
@ -96,6 +97,13 @@ func (m *ReadResourceRequest) GetProperties() *_struct.Struct {
return nil
}
func (m *ReadResourceRequest) GetDependencies() []string {
if m != nil {
return m.Dependencies
}
return nil
}
// ReadResourceResponse contains the result of reading a resource's state.
type ReadResourceResponse struct {
Urn string `protobuf:"bytes,1,opt,name=urn" json:"urn,omitempty"`
@ -109,7 +117,7 @@ func (m *ReadResourceResponse) Reset() { *m = ReadResourceResponse{} }
func (m *ReadResourceResponse) String() string { return proto.CompactTextString(m) }
func (*ReadResourceResponse) ProtoMessage() {}
func (*ReadResourceResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_resource_55fabfa8edee605e, []int{1}
return fileDescriptor_resource_773fb6366ec12ccd, []int{1}
}
func (m *ReadResourceResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_ReadResourceResponse.Unmarshal(m, b)
@ -161,7 +169,7 @@ func (m *RegisterResourceRequest) Reset() { *m = RegisterResourceRequest
func (m *RegisterResourceRequest) String() string { return proto.CompactTextString(m) }
func (*RegisterResourceRequest) ProtoMessage() {}
func (*RegisterResourceRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_resource_55fabfa8edee605e, []int{2}
return fileDescriptor_resource_773fb6366ec12ccd, []int{2}
}
func (m *RegisterResourceRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RegisterResourceRequest.Unmarshal(m, b)
@ -247,7 +255,7 @@ func (m *RegisterResourceResponse) Reset() { *m = RegisterResourceRespon
func (m *RegisterResourceResponse) String() string { return proto.CompactTextString(m) }
func (*RegisterResourceResponse) ProtoMessage() {}
func (*RegisterResourceResponse) Descriptor() ([]byte, []int) {
return fileDescriptor_resource_55fabfa8edee605e, []int{3}
return fileDescriptor_resource_773fb6366ec12ccd, []int{3}
}
func (m *RegisterResourceResponse) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RegisterResourceResponse.Unmarshal(m, b)
@ -315,7 +323,7 @@ func (m *RegisterResourceOutputsRequest) Reset() { *m = RegisterResource
func (m *RegisterResourceOutputsRequest) String() string { return proto.CompactTextString(m) }
func (*RegisterResourceOutputsRequest) ProtoMessage() {}
func (*RegisterResourceOutputsRequest) Descriptor() ([]byte, []int) {
return fileDescriptor_resource_55fabfa8edee605e, []int{4}
return fileDescriptor_resource_773fb6366ec12ccd, []int{4}
}
func (m *RegisterResourceOutputsRequest) XXX_Unmarshal(b []byte) error {
return xxx_messageInfo_RegisterResourceOutputsRequest.Unmarshal(m, b)
@ -528,38 +536,38 @@ var _ResourceMonitor_serviceDesc = grpc.ServiceDesc{
Metadata: "resource.proto",
}
func init() { proto.RegisterFile("resource.proto", fileDescriptor_resource_55fabfa8edee605e) }
func init() { proto.RegisterFile("resource.proto", fileDescriptor_resource_773fb6366ec12ccd) }
var fileDescriptor_resource_55fabfa8edee605e = []byte{
// 472 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0x3b, 0x8f, 0xd3, 0x40,
0x10, 0x3e, 0xdb, 0x87, 0x43, 0x86, 0x53, 0x38, 0x0d, 0x28, 0x67, 0x0c, 0x3a, 0x22, 0xd3, 0x84,
0xc6, 0x11, 0x47, 0x41, 0x49, 0x45, 0x41, 0x81, 0x10, 0xa6, 0x06, 0xc9, 0xb1, 0x87, 0xc8, 0x90,
0x78, 0x97, 0x7d, 0x9c, 0x74, 0x7f, 0x06, 0xfe, 0x1a, 0x05, 0x3f, 0x04, 0xad, 0xd7, 0x1b, 0xe2,
0xc7, 0x3d, 0xba, 0x79, 0x79, 0xe6, 0xfb, 0xbe, 0x99, 0x35, 0xcc, 0x04, 0x49, 0xa6, 0x45, 0x41,
0x29, 0x17, 0x4c, 0x31, 0x9c, 0x72, 0xbd, 0xd5, 0xbb, 0x4a, 0xf0, 0x22, 0x7e, 0xba, 0x61, 0x6c,
0xb3, 0xa5, 0x55, 0x93, 0x58, 0xeb, 0x6f, 0x2b, 0xda, 0x71, 0x75, 0x65, 0xeb, 0xe2, 0x67, 0xfd,
0xa4, 0x54, 0x42, 0x17, 0xaa, 0xcd, 0xce, 0xb8, 0x60, 0x97, 0x55, 0x49, 0xc2, 0xfa, 0xc9, 0x2f,
0x0f, 0x1e, 0x65, 0x94, 0x97, 0x59, 0x3b, 0x2c, 0xa3, 0x9f, 0x9a, 0xa4, 0xc2, 0x19, 0xf8, 0x55,
0x19, 0x79, 0x0b, 0x6f, 0x39, 0xcd, 0xfc, 0xaa, 0x44, 0x84, 0x63, 0x75, 0xc5, 0x29, 0xf2, 0x9b,
0x48, 0x63, 0x9b, 0x58, 0x9d, 0xef, 0x28, 0x0a, 0x6c, 0xcc, 0xd8, 0x38, 0x87, 0x90, 0xe7, 0x82,
0x6a, 0x15, 0x1d, 0x37, 0xd1, 0xd6, 0xc3, 0x37, 0x00, 0x5c, 0x30, 0x4e, 0x42, 0x55, 0x24, 0xa3,
0x7b, 0x0b, 0x6f, 0xf9, 0xe0, 0xe2, 0x2c, 0xb5, 0x50, 0x53, 0x07, 0x35, 0xfd, 0xdc, 0x40, 0xcd,
0x0e, 0x4a, 0x93, 0x1c, 0x1e, 0x77, 0xf1, 0x49, 0xce, 0x6a, 0x49, 0x78, 0x0a, 0x81, 0x16, 0x75,
0x8b, 0xd0, 0x98, 0xbd, 0x11, 0xfe, 0xdd, 0x47, 0xfc, 0xf1, 0xe0, 0x2c, 0xa3, 0x4d, 0x25, 0x15,
0x89, 0xbe, 0x0e, 0x8e, 0xb7, 0x37, 0xc2, 0xdb, 0x1f, 0xe5, 0x1d, 0x74, 0x78, 0xcf, 0x21, 0x2c,
0xb4, 0x54, 0x6c, 0xd7, 0xe8, 0x71, 0x3f, 0x6b, 0x3d, 0x5c, 0x41, 0xc8, 0xd6, 0xdf, 0xa9, 0x50,
0xb7, 0x69, 0xd1, 0x96, 0x61, 0x04, 0x13, 0x93, 0x32, 0x5f, 0x84, 0x4d, 0x27, 0xe7, 0x62, 0x02,
0x27, 0x25, 0x71, 0xaa, 0x4b, 0xaa, 0x0b, 0xc3, 0x7c, 0xb2, 0x08, 0x96, 0xd3, 0xac, 0x13, 0x4b,
0x7e, 0x7b, 0x10, 0x0d, 0x29, 0x5e, 0x2b, 0xa5, 0xdd, 0xbe, 0xbf, 0xdf, 0xfe, 0x7f, 0xb4, 0xc1,
0xdd, 0xd0, 0xce, 0x21, 0x94, 0x2a, 0x5f, 0x6f, 0xc9, 0xd1, 0xb6, 0x9e, 0x61, 0x61, 0x2d, 0x73,
0x03, 0x06, 0xa6, 0x73, 0x13, 0x82, 0xf3, 0x3e, 0xc0, 0x8f, 0x5a, 0x71, 0xad, 0xa4, 0x5b, 0xc5,
0x10, 0xe6, 0x2b, 0x98, 0x30, 0x5b, 0x73, 0xdb, 0xba, 0x5d, 0xdd, 0xc5, 0x5f, 0x1f, 0x1e, 0xba,
0xfe, 0x1f, 0x58, 0x5d, 0x29, 0x26, 0xf0, 0x2d, 0x84, 0xef, 0xeb, 0x4b, 0xf6, 0x83, 0x30, 0x4a,
0xf7, 0x8f, 0x2c, 0xb5, 0xa1, 0x76, 0x78, 0xfc, 0x64, 0x24, 0x63, 0xe5, 0x4b, 0x8e, 0xf0, 0x13,
0x9c, 0x1c, 0xde, 0x28, 0x9e, 0x1f, 0x14, 0x8f, 0x3c, 0xae, 0xf8, 0xf9, 0xb5, 0xf9, 0x7d, 0xcb,
0x2f, 0x70, 0xda, 0x97, 0x03, 0x93, 0xce, 0x67, 0xa3, 0xf7, 0x1a, 0xbf, 0xb8, 0xb1, 0x66, 0xdf,
0xfe, 0xeb, 0xf0, 0xe2, 0x5b, 0xb5, 0xf1, 0xe5, 0x0d, 0x1d, 0xba, 0x1b, 0x89, 0xe7, 0x03, 0xb9,
0xdf, 0x99, 0x1f, 0x51, 0x72, 0xb4, 0x0e, 0x9b, 0xc8, 0xeb, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff,
0x20, 0x92, 0x11, 0xa7, 0xc5, 0x04, 0x00, 0x00,
var fileDescriptor_resource_773fb6366ec12ccd = []byte{
// 480 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x94, 0xbd, 0x8e, 0xd3, 0x40,
0x10, 0xc7, 0xcf, 0xf6, 0xe1, 0x90, 0xe1, 0x14, 0x4e, 0x03, 0xca, 0x19, 0x83, 0x8e, 0xc8, 0x34,
0xa1, 0x71, 0xc4, 0x51, 0x50, 0x52, 0x51, 0x50, 0x20, 0x84, 0xa9, 0x41, 0x72, 0xec, 0x21, 0x32,
0x24, 0xde, 0x65, 0x3f, 0x4e, 0xba, 0xa7, 0xe1, 0x5d, 0x78, 0x12, 0x0a, 0x1e, 0x04, 0xad, 0xd7,
0x1b, 0xe2, 0x8f, 0xfb, 0xe8, 0xe6, 0x6b, 0xc7, 0xff, 0xf9, 0xed, 0xac, 0x61, 0x26, 0x48, 0x32,
0x2d, 0x0a, 0x4a, 0xb9, 0x60, 0x8a, 0xe1, 0x94, 0xeb, 0xad, 0xde, 0x55, 0x82, 0x17, 0xf1, 0xd3,
0x0d, 0x63, 0x9b, 0x2d, 0xad, 0x9a, 0xc4, 0x5a, 0x7f, 0x5b, 0xd1, 0x8e, 0xab, 0x2b, 0x5b, 0x17,
0x3f, 0xeb, 0x27, 0xa5, 0x12, 0xba, 0x50, 0x6d, 0x76, 0xc6, 0x05, 0xbb, 0xac, 0x4a, 0x12, 0xd6,
0x4f, 0x7e, 0x7b, 0xf0, 0x28, 0xa3, 0xbc, 0xcc, 0xda, 0x8f, 0x65, 0xf4, 0x53, 0x93, 0x54, 0x38,
0x03, 0xbf, 0x2a, 0x23, 0x6f, 0xe1, 0x2d, 0xa7, 0x99, 0x5f, 0x95, 0x88, 0x70, 0xac, 0xae, 0x38,
0x45, 0x7e, 0x13, 0x69, 0x6c, 0x13, 0xab, 0xf3, 0x1d, 0x45, 0x81, 0x8d, 0x19, 0x1b, 0xe7, 0x10,
0xf2, 0x5c, 0x50, 0xad, 0xa2, 0xe3, 0x26, 0xda, 0x7a, 0xf8, 0x06, 0x80, 0x0b, 0xc6, 0x49, 0xa8,
0x8a, 0x64, 0x74, 0x6f, 0xe1, 0x2d, 0x1f, 0x5c, 0x9c, 0xa5, 0x56, 0x6a, 0xea, 0xa4, 0xa6, 0x9f,
0x1b, 0xa9, 0xd9, 0x41, 0x29, 0x26, 0x70, 0x52, 0x12, 0xa7, 0xba, 0xa4, 0xba, 0x30, 0x47, 0xc3,
0x45, 0xb0, 0x9c, 0x66, 0x9d, 0x58, 0x92, 0xc3, 0xe3, 0xee, 0x0c, 0x92, 0xb3, 0x5a, 0x12, 0x9e,
0x42, 0xa0, 0x45, 0xdd, 0x4e, 0x61, 0xcc, 0x9e, 0x0c, 0xff, 0xce, 0x32, 0x92, 0x3f, 0x1e, 0x9c,
0x65, 0xb4, 0xa9, 0xa4, 0x22, 0xd1, 0x67, 0xe5, 0xd8, 0x78, 0x23, 0x6c, 0xfc, 0x51, 0x36, 0x41,
0x87, 0xcd, 0x1c, 0xc2, 0x42, 0x4b, 0xc5, 0x76, 0x0d, 0xb3, 0xfb, 0x59, 0xeb, 0xe1, 0x0a, 0x42,
0xb6, 0xfe, 0x4e, 0x85, 0xba, 0x8d, 0x57, 0x5b, 0x86, 0x11, 0x4c, 0x4c, 0xca, 0x9c, 0x08, 0x9b,
0x4e, 0xce, 0x1d, 0x50, 0x9c, 0x8c, 0x50, 0xfc, 0xe5, 0x41, 0x34, 0x1c, 0xf1, 0x5a, 0x94, 0x76,
0x43, 0xfc, 0xfd, 0x86, 0xfc, 0x57, 0x1b, 0xdc, 0x4d, 0xed, 0x1c, 0x42, 0xa9, 0xf2, 0xf5, 0x96,
0xdc, 0xd8, 0xd6, 0x33, 0x53, 0x58, 0xcb, 0xec, 0x89, 0x91, 0xe9, 0xdc, 0x84, 0xe0, 0xbc, 0x2f,
0xf0, 0xa3, 0x56, 0x5c, 0x2b, 0xe9, 0xae, 0x62, 0x28, 0xf3, 0x15, 0x4c, 0x98, 0xad, 0xb9, 0xed,
0xba, 0x5d, 0xdd, 0xc5, 0x5f, 0x1f, 0x1e, 0xba, 0xfe, 0x1f, 0x58, 0x5d, 0x29, 0x26, 0xf0, 0x2d,
0x84, 0xef, 0xeb, 0x4b, 0xf6, 0x83, 0x30, 0x4a, 0xf7, 0x0f, 0x31, 0xb5, 0xa1, 0xf6, 0xe3, 0xf1,
0x93, 0x91, 0x8c, 0xc5, 0x97, 0x1c, 0xe1, 0x27, 0x38, 0x39, 0xdc, 0x51, 0x3c, 0x3f, 0x28, 0x1e,
0x79, 0x80, 0xf1, 0xf3, 0x6b, 0xf3, 0xfb, 0x96, 0x5f, 0xe0, 0xb4, 0x8f, 0x03, 0x93, 0xce, 0xb1,
0xd1, 0x7d, 0x8d, 0x5f, 0xdc, 0x58, 0xb3, 0x6f, 0xff, 0x75, 0xb8, 0xf1, 0x2d, 0x6d, 0x7c, 0x79,
0x43, 0x87, 0xee, 0x8d, 0xc4, 0xf3, 0x01, 0xee, 0x77, 0xe6, 0x67, 0x95, 0x1c, 0xad, 0xc3, 0x26,
0xf2, 0xfa, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x5d, 0xe5, 0xc8, 0x38, 0xe9, 0x04, 0x00, 0x00,
}

View file

@ -35,6 +35,7 @@ message ReadResourceRequest {
string name = 3; // the name, for URN purposes, of the object.
string parent = 4; // an optional parent URN that this child resource belongs to.
google.protobuf.Struct properties = 5; // optional state sufficient to uniquely identify the resource.
repeated string dependencies = 6; // a list of URNs that this read depends on, as observed by the language host.
}
// ReadResourceResponse contains the result of reading a resource's state.

View file

@ -22,7 +22,7 @@ DESCRIPTOR = _descriptor.FileDescriptor(
name='resource.proto',
package='pulumirpc',
syntax='proto3',
serialized_pb=_b('\n\x0eresource.proto\x12\tpulumirpc\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1cgoogle/protobuf/struct.proto\x1a\x0eprovider.proto\"z\n\x13ReadResourceRequest\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\x12\x0c\n\x04name\x18\x03 \x01(\t\x12\x0e\n\x06parent\x18\x04 \x01(\t\x12+\n\nproperties\x18\x05 \x01(\x0b\x32\x17.google.protobuf.Struct\"P\n\x14ReadResourceResponse\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12+\n\nproperties\x18\x02 \x01(\x0b\x32\x17.google.protobuf.Struct\"\xa5\x01\n\x17RegisterResourceRequest\x12\x0c\n\x04type\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0e\n\x06parent\x18\x03 \x01(\t\x12\x0e\n\x06\x63ustom\x18\x04 \x01(\x08\x12\'\n\x06object\x18\x05 \x01(\x0b\x32\x17.google.protobuf.Struct\x12\x0f\n\x07protect\x18\x06 \x01(\x08\x12\x14\n\x0c\x64\x65pendencies\x18\x07 \x03(\t\"}\n\x18RegisterResourceResponse\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12\n\n\x02id\x18\x02 \x01(\t\x12\'\n\x06object\x18\x03 \x01(\x0b\x32\x17.google.protobuf.Struct\x12\x0e\n\x06stable\x18\x04 \x01(\x08\x12\x0f\n\x07stables\x18\x05 \x03(\t\"W\n\x1eRegisterResourceOutputsRequest\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12(\n\x07outputs\x18\x02 \x01(\x0b\x32\x17.google.protobuf.Struct2\xe4\x02\n\x0fResourceMonitor\x12?\n\x06Invoke\x12\x18.pulumirpc.InvokeRequest\x1a\x19.pulumirpc.InvokeResponse\"\x00\x12Q\n\x0cReadResource\x12\x1e.pulumirpc.ReadResourceRequest\x1a\x1f.pulumirpc.ReadResourceResponse\"\x00\x12]\n\x10RegisterResource\x12\".pulumirpc.RegisterResourceRequest\x1a#.pulumirpc.RegisterResourceResponse\"\x00\x12^\n\x17RegisterResourceOutputs\x12).pulumirpc.RegisterResourceOutputsRequest\x1a\x16.google.protobuf.Empty\"\x00\x62\x06proto3')
serialized_pb=_b('\n\x0eresource.proto\x12\tpulumirpc\x1a\x1bgoogle/protobuf/empty.proto\x1a\x1cgoogle/protobuf/struct.proto\x1a\x0eprovider.proto\"\x90\x01\n\x13ReadResourceRequest\x12\n\n\x02id\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\x12\x0c\n\x04name\x18\x03 \x01(\t\x12\x0e\n\x06parent\x18\x04 \x01(\t\x12+\n\nproperties\x18\x05 \x01(\x0b\x32\x17.google.protobuf.Struct\x12\x14\n\x0c\x64\x65pendencies\x18\x06 \x03(\t\"P\n\x14ReadResourceResponse\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12+\n\nproperties\x18\x02 \x01(\x0b\x32\x17.google.protobuf.Struct\"\xa5\x01\n\x17RegisterResourceRequest\x12\x0c\n\x04type\x18\x01 \x01(\t\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0e\n\x06parent\x18\x03 \x01(\t\x12\x0e\n\x06\x63ustom\x18\x04 \x01(\x08\x12\'\n\x06object\x18\x05 \x01(\x0b\x32\x17.google.protobuf.Struct\x12\x0f\n\x07protect\x18\x06 \x01(\x08\x12\x14\n\x0c\x64\x65pendencies\x18\x07 \x03(\t\"}\n\x18RegisterResourceResponse\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12\n\n\x02id\x18\x02 \x01(\t\x12\'\n\x06object\x18\x03 \x01(\x0b\x32\x17.google.protobuf.Struct\x12\x0e\n\x06stable\x18\x04 \x01(\x08\x12\x0f\n\x07stables\x18\x05 \x03(\t\"W\n\x1eRegisterResourceOutputsRequest\x12\x0b\n\x03urn\x18\x01 \x01(\t\x12(\n\x07outputs\x18\x02 \x01(\x0b\x32\x17.google.protobuf.Struct2\xe4\x02\n\x0fResourceMonitor\x12?\n\x06Invoke\x12\x18.pulumirpc.InvokeRequest\x1a\x19.pulumirpc.InvokeResponse\"\x00\x12Q\n\x0cReadResource\x12\x1e.pulumirpc.ReadResourceRequest\x1a\x1f.pulumirpc.ReadResourceResponse\"\x00\x12]\n\x10RegisterResource\x12\".pulumirpc.RegisterResourceRequest\x1a#.pulumirpc.RegisterResourceResponse\"\x00\x12^\n\x17RegisterResourceOutputs\x12).pulumirpc.RegisterResourceOutputsRequest\x1a\x16.google.protobuf.Empty\"\x00\x62\x06proto3')
,
dependencies=[google_dot_protobuf_dot_empty__pb2.DESCRIPTOR,google_dot_protobuf_dot_struct__pb2.DESCRIPTOR,provider__pb2.DESCRIPTOR,])
@ -71,6 +71,13 @@ _READRESOURCEREQUEST = _descriptor.Descriptor(
message_type=None, enum_type=None, containing_type=None,
is_extension=False, extension_scope=None,
options=None, file=DESCRIPTOR),
_descriptor.FieldDescriptor(
name='dependencies', full_name='pulumirpc.ReadResourceRequest.dependencies', index=5,
number=6, type=9, cpp_type=9, label=3,
has_default_value=False, default_value=[],
message_type=None, enum_type=None, containing_type=None,
is_extension=False, extension_scope=None,
options=None, file=DESCRIPTOR),
],
extensions=[
],
@ -83,8 +90,8 @@ _READRESOURCEREQUEST = _descriptor.Descriptor(
extension_ranges=[],
oneofs=[
],
serialized_start=104,
serialized_end=226,
serialized_start=105,
serialized_end=249,
)
@ -121,8 +128,8 @@ _READRESOURCERESPONSE = _descriptor.Descriptor(
extension_ranges=[],
oneofs=[
],
serialized_start=228,
serialized_end=308,
serialized_start=251,
serialized_end=331,
)
@ -194,8 +201,8 @@ _REGISTERRESOURCEREQUEST = _descriptor.Descriptor(
extension_ranges=[],
oneofs=[
],
serialized_start=311,
serialized_end=476,
serialized_start=334,
serialized_end=499,
)
@ -253,8 +260,8 @@ _REGISTERRESOURCERESPONSE = _descriptor.Descriptor(
extension_ranges=[],
oneofs=[
],
serialized_start=478,
serialized_end=603,
serialized_start=501,
serialized_end=626,
)
@ -291,8 +298,8 @@ _REGISTERRESOURCEOUTPUTSREQUEST = _descriptor.Descriptor(
extension_ranges=[],
oneofs=[
],
serialized_start=605,
serialized_end=692,
serialized_start=628,
serialized_end=715,
)
_READRESOURCEREQUEST.fields_by_name['properties'].message_type = google_dot_protobuf_dot_struct__pb2._STRUCT
@ -350,8 +357,8 @@ _RESOURCEMONITOR = _descriptor.ServiceDescriptor(
file=DESCRIPTOR,
index=0,
options=None,
serialized_start=695,
serialized_end=1051,
serialized_start=718,
serialized_end=1074,
methods=[
_descriptor.MethodDescriptor(
name='Invoke',

View file

@ -0,0 +1,28 @@
// Copyright 2016-2018, Pulumi Corporation. All rights reserved.
package ints
import (
"testing"
"github.com/pulumi/pulumi/pkg/testing/integration"
)
// Test that the engine tolerates two deletions of the same URN in the same plan.
func TestReadDBR(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dir: "step1",
Dependencies: []string{"@pulumi/pulumi"},
Quick: true,
EditDirs: []integration.EditDir{
{
Dir: "step2",
Additive: true,
},
{
Dir: "step3",
Additive: true,
},
},
})
}

View file

@ -0,0 +1,3 @@
name: read_dbr
description: A program that has a DBR resource depending on a read resource
runtime: nodejs

View file

@ -0,0 +1,22 @@
// 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.
import { Resource } from "./resource";
// Setup: Resource A is external, Resource B is not but it depends on A. Resource C is external and
// has an input dependent on B.
const a = new Resource("a", { state: 42 }, { id: "existing-id"} );
const b = new Resource("b", { state: a.state.apply((b: any) => b + 1)});
const c = new Resource("c", { state: b.state }, { id: "another-existing-id" })

View file

@ -0,0 +1,14 @@
{
"name": "read_dbr",
"main": "bin/index.js",
"typings": "bin/index.d.ts",
"scripts": {
"build": "tsc"
},
"devDependencies": {
"typescript": "^2.9.0"
},
"peerDependencies": {
"@pulumi/pulumi": "latest"
}
}

View file

@ -0,0 +1,71 @@
// 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.
import * as pulumi from "@pulumi/pulumi";
import * as dynamic from "@pulumi/pulumi/dynamic";
export class Provider implements dynamic.ResourceProvider {
public static readonly instance = new Provider();
private id: number = 0;
public async check(olds: any, news: any): Promise<dynamic.CheckResult> {
return {
inputs: news,
}
}
public async diff(id: pulumi.ID, olds: any, news: any): Promise<dynamic.DiffResult> {
if (news.state !== olds.state) {
return {
changes: true,
replaces: ["state"],
deleteBeforeReplace: true,
};
}
return {
changes: false,
}
}
public async create(inputs: any): Promise<dynamic.CreateResult> {
return {
id: (this.id++).toString(),
outs: inputs,
}
}
public async update(id: string, olds: any, news: any): Promise<dynamic.UpdateResult> {
throw Error("this resource is replace-only and can't be updated");
}
public async read(id: pulumi.ID, props: any): Promise<dynamic.ReadResult> {
return {
props: {
state: {
foo: 42
}
}
}
}
}
export class Resource extends pulumi.dynamic.Resource {
public readonly state: pulumi.Output<any>;
constructor(name: string, props: any, opts?: pulumi.ResourceOptions) {
super(Provider.instance, name, props, opts);
}
}

View file

@ -0,0 +1,21 @@
{
"compilerOptions": {
"outDir": "bin",
"target": "es6",
"module": "commonjs",
"moduleResolution": "node",
"declaration": true,
"sourceMap": true,
"stripInternal": true,
"experimentalDecorators": true,
"pretty": true,
"noFallthroughCasesInSwitch": true,
"noImplicitReturns": true,
"forceConsistentCasingInFileNames": true,
"strictNullChecks": true
},
"files": [
"index.ts"
]
}

View file

@ -0,0 +1,31 @@
// 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.
import { Resource } from "./resource";
const a = new Resource("a", { state: 42 }, { id: "existing-id"} );
// B must be replaced, but it is a DBR replacement.
const b = new Resource("b", { state: a.state.apply((b: any) => b + 2)});
// C depends on B, so it gets re-read. Before the read, it is removed from the
// snapshot due to the deletion of B.
const c = new Resource("c", { state: b.state }, { id: "another-existing-id" })
// The engine generates:
// A: Same
// C: DeleteReplacement (read)
// B: DeleteReplacement
// B: Create
// C: Read

View file

@ -0,0 +1,19 @@
// 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.
import { Resource } from "./resource";
const a = new Resource("a", { state: 42 }, { id: "existing-id"} );
const b = new Resource("b", { state: a.state.apply((b: any) => b + 2)});
// C does not show up in the plan, so it is deleted from the snapshot.

View file

@ -0,0 +1,24 @@
// Copyright 2016-2018, Pulumi Corporation. All rights reserved.
package ints
import (
"testing"
"github.com/pulumi/pulumi/pkg/testing/integration"
)
// Test that the engine is capable of relinquishing control of a resource without deleting it.
func TestReadRelinquish(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dir: "step1",
Dependencies: []string{"@pulumi/pulumi"},
Quick: true,
EditDirs: []integration.EditDir{
{
Dir: "step2",
Additive: true,
},
},
})
}

View file

@ -0,0 +1,3 @@
name: read_relinquish
description: A program that relinquishes a resource without deleting it
runtime: nodejs

View file

@ -0,0 +1,18 @@
// 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.
import { Resource } from "./resource";
// Setup: "a" is a protected non-external resource.
const a = new Resource("a", { state: 42 }, { protect: true });

View file

@ -0,0 +1,14 @@
{
"name": "read_relinquish",
"main": "bin/index.js",
"typings": "bin/index.d.ts",
"scripts": {
"build": "tsc"
},
"devDependencies": {
"typescript": "^2.9.0"
},
"peerDependencies": {
"@pulumi/pulumi": "latest"
}
}

View file

@ -0,0 +1,70 @@
// 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.
import * as pulumi from "@pulumi/pulumi";
import * as dynamic from "@pulumi/pulumi/dynamic";
export class Provider implements dynamic.ResourceProvider {
public static readonly instance = new Provider();
private id: number = 0;
public async check(olds: any, news: any): Promise<dynamic.CheckResult> {
return {
inputs: news,
}
}
public async diff(id: pulumi.ID, olds: any, news: any): Promise<dynamic.DiffResult> {
if (news.state !== olds.state) {
return {
changes: true,
replaces: ["state"],
};
}
return {
changes: false,
}
}
public async create(inputs: any): Promise<dynamic.CreateResult> {
return {
id: (this.id++).toString(),
outs: inputs,
}
}
public async update(id: string, olds: any, news: any): Promise<dynamic.UpdateResult> {
throw Error("this resource is replace-only and can't be updated");
}
public async read(id: pulumi.ID, props: any): Promise<dynamic.ReadResult> {
return {
props: {
state: {
foo: 42
}
}
}
}
}
export class Resource extends pulumi.dynamic.Resource {
public readonly state: pulumi.Output<any>;
constructor(name: string, props: any, opts?: pulumi.ResourceOptions) {
super(Provider.instance, name, props, opts);
}
}

View file

@ -0,0 +1,21 @@
{
"compilerOptions": {
"outDir": "bin",
"target": "es6",
"module": "commonjs",
"moduleResolution": "node",
"declaration": true,
"sourceMap": true,
"stripInternal": true,
"experimentalDecorators": true,
"pretty": true,
"noFallthroughCasesInSwitch": true,
"noImplicitReturns": true,
"forceConsistentCasingInFileNames": true,
"strictNullChecks": true
},
"files": [
"index.ts"
]
}

View file

@ -0,0 +1,24 @@
// 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.
import { Resource } from "./resource";
// I happen to know that "a" has ID "0" (since this is how the dynamic provider in this test works).
//
// Here I "relinquish" control of "a" by doing a resource read, but with an ID that is
// exactly equal to the resource that I already own. The resource will recognize this
// and not delete "a".
//
// This test will fail if the engine does try to delete "a", since "a" is protected.
const a = new Resource("a", { state: 99 }, { id: "0" });

View file

@ -0,0 +1,29 @@
// Copyright 2016-2018, Pulumi Corporation. All rights reserved.
package ints
import (
"testing"
"github.com/pulumi/pulumi/pkg/testing/integration"
)
// Test that the engine handles the replacement of an external resource with a
// owned once gracefully.
func TestReadReplace(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dir: "step1",
Dependencies: []string{"@pulumi/pulumi"},
Quick: true,
EditDirs: []integration.EditDir{
{
Dir: "step2",
Additive: true,
},
{
Dir: "step3",
Additive: true,
},
},
})
}

View file

@ -0,0 +1,3 @@
name: read_dbr
description: A program that has a DBR resource depending on a read resource
runtime: nodejs

View file

@ -0,0 +1,20 @@
// 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.
import { Resource } from "./resource";
// Setup: Resource A is external, Resource B is not.
const a = new Resource("a", { state: 42 }, { id: "existing-id"} );
const b = new Resource("b", { state: a.state.apply((b: any) => b + 1)});

View file

@ -0,0 +1,14 @@
{
"name": "read_dbr",
"main": "bin/index.js",
"typings": "bin/index.d.ts",
"scripts": {
"build": "tsc"
},
"devDependencies": {
"typescript": "^2.9.0"
},
"peerDependencies": {
"@pulumi/pulumi": "latest"
}
}

View file

@ -0,0 +1,70 @@
// 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.
import * as pulumi from "@pulumi/pulumi";
import * as dynamic from "@pulumi/pulumi/dynamic";
export class Provider implements dynamic.ResourceProvider {
public static readonly instance = new Provider();
private id: number = 0;
public async check(olds: any, news: any): Promise<dynamic.CheckResult> {
return {
inputs: news,
}
}
public async diff(id: pulumi.ID, olds: any, news: any): Promise<dynamic.DiffResult> {
if (news.state !== olds.state) {
return {
changes: true,
replaces: ["state"],
};
}
return {
changes: false,
}
}
public async create(inputs: any): Promise<dynamic.CreateResult> {
return {
id: (this.id++).toString(),
outs: inputs,
}
}
public async update(id: string, olds: any, news: any): Promise<dynamic.UpdateResult> {
throw Error("this resource is replace-only and can't be updated");
}
public async read(id: pulumi.ID, props: any): Promise<dynamic.ReadResult> {
return {
props: {
state: {
foo: 42
}
}
}
}
}
export class Resource extends pulumi.dynamic.Resource {
public readonly state: pulumi.Output<any>;
constructor(name: string, props: any, opts?: pulumi.ResourceOptions) {
super(Provider.instance, name, props, opts);
}
}

View file

@ -0,0 +1,21 @@
{
"compilerOptions": {
"outDir": "bin",
"target": "es6",
"module": "commonjs",
"moduleResolution": "node",
"declaration": true,
"sourceMap": true,
"stripInternal": true,
"experimentalDecorators": true,
"pretty": true,
"noFallthroughCasesInSwitch": true,
"noImplicitReturns": true,
"forceConsistentCasingInFileNames": true,
"strictNullChecks": true
},
"files": [
"index.ts"
]
}

View file

@ -0,0 +1,27 @@
// 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.
import { Resource } from "./resource";
// Resource A was read in the previous plan, but it's now created.
const a = new Resource("a", { state: 42 });
// B must be replaced.
const b = new Resource("b", { state: a.state.apply((b: any) => b + 1)});
// The engine generates:
// A: CreateReplacement
// B: CreateReplacement
// B: DeleteReplacement
// A: DeleteReplacement

View file

@ -0,0 +1,20 @@
// 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.
import { Resource } from "./resource";
// Now go back the other way and make sure that "A" is external again.
const a = new Resource("a", { state: 42 }, { id: "existing-id"} );
const b = new Resource("b", { state: a.state.apply((b: any) => b + 1)});