Add (back) component outputs

This change adds back component output properties.  Doing so
requires splitting the RPC interface for creating resources in
half, with an initial RegisterResource which contains all of the
input properties, and a final CompleteResource which optionally
contains any output properties synthesized by the component.
This commit is contained in:
joeduffy 2017-11-20 17:38:09 -08:00
parent 86267b86b9
commit 7e48e8726b
36 changed files with 1436 additions and 650 deletions

View file

@ -20,6 +20,5 @@
"vet",
"vetshadow"
],
"LineLength": 140
"LineLength": 120
}

View file

@ -27,7 +27,7 @@ type localPulumiBackend struct {
func (b *localPulumiBackend) CreateStack(stackName tokens.QName, opts StackCreationOptions) error {
contract.Requiref(opts.Cloud == "", "cloud", "local backend does not support clouds, cloud must be empty")
if _, _, _, err := getStack(stackName); err == nil {
if _, _, _, _, err := getStack(stackName); err == nil {
return errors.Errorf("stack '%v' already exists", stackName)
}
@ -49,7 +49,7 @@ func (b *localPulumiBackend) GetStacks() ([]stackSummary, error) {
}
// Ignore errors, just leave display settings as "n/a".
_, _, snapshot, err := getStack(stack)
_, _, snapshot, _, err := getStack(stack)
if err == nil && snapshot != nil {
summary.LastDeploy = snapshot.Time.String()
summary.ResourceCount = strconv.Itoa(len(snapshot.Resources))
@ -62,7 +62,7 @@ func (b *localPulumiBackend) GetStacks() ([]stackSummary, error) {
}
func (b *localPulumiBackend) RemoveStack(stackName tokens.QName, force bool) error {
name, _, snapshot, err := getStack(stackName)
name, _, snapshot, _, err := getStack(stackName)
if err != nil {
return err
}
@ -228,7 +228,7 @@ func getLocalStacks() ([]tokens.QName, error) {
// Read in this stack's information.
name := tokens.QName(stackfn[:len(stackfn)-len(ext)])
_, _, _, err := getStack(name)
_, _, _, _, err := getStack(name)
if err != nil {
continue // failure reading the stack information.
}

View file

@ -47,9 +47,7 @@ func (p localStackProvider) GetTarget(name tokens.QName) (*deploy.Target, error)
func (p localStackProvider) GetSnapshot(name tokens.QName) (*deploy.Snapshot, error) {
contract.Require(name != "", "name")
_, _, snapshot, err := getStack(name)
_, _, snapshot, _, err := getStack(name)
return snapshot, err
}
@ -64,7 +62,7 @@ func (p localStackProvider) BeginMutation(name tokens.QName) (engine.SnapshotMut
func (m localStackMutation) End(snapshot *deploy.Snapshot) error {
contract.Assert(m.name == snapshot.Namespace)
name, config, _, err := getStack(snapshot.Namespace)
name, config, _, _, err := getStack(snapshot.Namespace)
if err != nil && !os.IsNotExist(err) {
return err
}
@ -72,10 +70,10 @@ func (m localStackMutation) End(snapshot *deploy.Snapshot) error {
return saveStack(name, config, snapshot)
}
func getStack(name tokens.QName) (tokens.QName, map[tokens.ModuleMember]config.Value, *deploy.Snapshot, error) {
func getStack(name tokens.QName) (tokens.QName, map[tokens.ModuleMember]config.Value, *deploy.Snapshot, string, error) {
workspace, err := newWorkspace()
if err != nil {
return "", nil, nil, err
return "", nil, nil, "", err
}
contract.Require(name != "", "name")
@ -84,30 +82,30 @@ func getStack(name tokens.QName) (tokens.QName, map[tokens.ModuleMember]config.V
// Detect the encoding of the file so we can do our initial unmarshaling.
m, ext := encoding.Detect(file)
if m == nil {
return "", nil, nil, errors.Errorf("resource deserialization failed; illegal markup extension: '%v'", ext)
return "", nil, nil, file, errors.Errorf("resource deserialization failed; illegal markup extension: '%v'", ext)
}
// Now read the whole file into a byte blob.
b, err := ioutil.ReadFile(file)
if err != nil {
if os.IsNotExist(err) {
return "", nil, nil, err
return "", nil, nil, file, err
}
return "", nil, nil, err
return "", nil, nil, file, err
}
// Unmarshal the contents into a checkpoint structure.
var checkpoint stack.Checkpoint
if err = m.Unmarshal(b, &checkpoint); err != nil {
return "", nil, nil, err
return "", nil, nil, file, err
}
_, config, snapshot, err := stack.DeserializeCheckpoint(&checkpoint)
if err != nil {
return "", nil, nil, err
return "", nil, nil, file, err
}
return name, config, snapshot, nil
return name, config, snapshot, file, nil
}
func saveStack(name tokens.QName, config map[tokens.ModuleMember]config.Value, snap *deploy.Snapshot) error {

View file

@ -29,29 +29,26 @@ func newStackCmd() *cobra.Command {
return err
}
_, config, snapshot, err := getStack(stackName)
_, config, snapshot, stackFile, err := getStack(stackName)
if err != nil {
return err
}
fmt.Printf("Current stack is %v\n", stackName)
fmt.Printf(" (use `pulumi stack select` to change stack; `pulumi stack ls` lists known ones)\n")
if err != nil {
return err
}
if snapshot != nil {
fmt.Printf("Last update at %v\n", snapshot.Time)
fmt.Printf("\tLast updated at %v\n", snapshot.Time)
}
if len(config) > 0 {
fmt.Printf("%v configuration variables set (see `pulumi config` for details)\n", len(config))
fmt.Printf("\t%v configuration variables set (see `pulumi config` for details)\n", len(config))
}
fmt.Printf("\tCheckpoint file is %s\n", stackFile)
fmt.Printf("\t(Use `pulumi stack select` to change stack; `pulumi stack ls` lists known ones)\n")
var stackResource *resource.State
if snapshot == nil || len(snapshot.Resources) == 0 {
fmt.Printf("No resources currently in this stack\n")
} else {
fmt.Printf("%v resources currently in this stack:\n", len(snapshot.Resources))
fmt.Printf("\n")
fmt.Printf("%-48s %s\n", "TYPE", "NAME")
for _, res := range snapshot.Resources {
if res.Type == stack.RootPulumiStackTypeName {

View file

@ -19,7 +19,8 @@ func Message(msg string) *Diag {
return &Diag{Message: msg}
}
// RawMessage returns an anonymous diagnostic message without any source or ID information that will not be rendered with Sprintf.
// RawMessage returns an anonymous diagnostic message without any source or ID information that will not be rendered
// with Sprintf.
func RawMessage(msg string) *Diag {
return &Diag{Message: msg, Raw: true}
}

View file

@ -153,7 +153,7 @@ func newDeployActions(opts deployOptions, target *deploy.Target, engine *Engine)
}
}
func (acts *deployActions) Run(step deploy.Step) (resource.Status, error) {
func (acts *deployActions) Run(iter *deploy.PlanIterator, step deploy.Step) (resource.Status, error) {
// Report the beginning of the step if appropriate.
if shouldShow(acts.Seen, step, acts.Opts) {
var b bytes.Buffer
@ -167,8 +167,8 @@ func (acts *deployActions) Run(step deploy.Step) (resource.Status, error) {
return resource.StatusOK, err
}
// Apply the step's changes.
status, err := step.Apply()
// Apply the step's changes and save its results in the iterator/checkpoint.
status, err := iter.Apply(step, false)
// Report the result of the step.
stepop := step.Op()
@ -209,8 +209,8 @@ func (acts *deployActions) Run(step deploy.Step) (resource.Status, error) {
}
}
// If necessary, write out the current snapshot. Note that even if a failure has occurred, we should still have a safe checkpoint.
// Note that any error that occurs when writing the checkpoint trumps the error reported above.
// If necessary, write out the current snapshot. Note that even if a failure has occurred, we should still have a
// safe checkpoint. Note that any error that occurs when writing the checkpoint trumps the error reported above.
if mutation != nil {
if endErr := mutation.End(step.Iterator().Snap()); endErr != nil {
return status, endErr

View file

@ -19,8 +19,8 @@ type TargetProvider interface {
// SnapshotMutation abstracts away managing changes to snapshots
type SnapshotMutation interface {
// End indicates that the current mutation has completed and that its results (given by snapshot) should be persisted. See the comments
// on SnapshotProvider.BeginMutation for more details.
// End indicates that the current mutation has completed and that its results (given by snapshot) should be
// persisted. See the comments on SnapshotProvider.BeginMutation for more details.
End(snapshot *deploy.Snapshot) error
}
@ -28,16 +28,16 @@ type SnapshotMutation interface {
type SnapshotProvider interface {
GetSnapshot(name tokens.QName) (*deploy.Snapshot, error)
// BeginMutation and SnapshotMutation.End allow a snapshot provider to be robust in the face of failures that occur between the points
// at which they are called. The semantics are as follows:
// 1. The engine calls `SnapshotProvider.Begin` to indicate that it is about to mutate the state of the resources tracked by the
// snapshot.
// BeginMutation and SnapshotMutation.End allow a snapshot provider to be robust in the face of failures that occur
// between the points at which they are called. The semantics are as follows:
// 1. The engine calls `SnapshotProvider.Begin` to indicate that it is about to mutate the state of the
// resources tracked by the snapshot.
// 2. The engine mutates the state of the resoures tracked by the snapshot.
// 3. The engine calls `SnapshotMutation.End` with the new snapshot to indicate that the mutation(s) it was performing has/have
// finished.
// During (1), the snapshot provider should record the fact that any currently persisted snapshot is being mutated and cannot be
// assumed to represent the actual state of the system. This ensures that if the engine crashes during (2), then the current snapshot
// is known to be unreliable. During (3), the snapshot provider should persist the provided snapshot and record that it is known to be
// reliable.
// 3. The engine calls `SnapshotMutation.End` with the new snapshot to indicate that the mutation(s) it was
// performing has/have finished.
// During (1), the snapshot provider should record the fact that any currently persisted snapshot is being mutated
// and cannot be assumed to represent the actual state of the system. This ensures that if the engine crashes during
// (2), then the current snapshot is known to be unreliable. During (3), the snapshot provider should persist the
// provided snapshot and record that it is known to be reliable.
BeginMutation(name tokens.QName) (SnapshotMutation, error)
}

View file

@ -85,7 +85,7 @@ type planResult struct {
// StepActions is used to process a plan's steps.
type StepActions interface {
// Run is invoked to perform whatever action the implementer uses to process the step.
Run(step deploy.Step) (resource.Status, error)
Run(iter *deploy.PlanIterator, step deploy.Step) (resource.Status, error)
}
// Walk enumerates all steps in the plan, calling out to the provided action at each step. It returns four things: the
@ -110,7 +110,7 @@ func (res *planResult) Walk(actions StepActions) (deploy.PlanSummary, deploy.Ste
for step != nil {
// Perform any per-step actions.
rst, err := actions.Run(step)
rst, err := actions.Run(iter, step)
// If an error occurred, exit early.
if err != nil {
@ -151,14 +151,14 @@ func newPreviewActions(opts deployOptions) *previewActions {
}
}
func (acts *previewActions) Run(step deploy.Step) (resource.Status, error) {
func (acts *previewActions) Run(iter *deploy.PlanIterator, step deploy.Step) (resource.Status, error) {
// Print this step information (resource and all its properties).
if shouldShow(acts.Seen, step, acts.Opts) {
printStep(&acts.Summary, acts.Seen, acts.Shown, step, acts.Opts.Summary, acts.Opts.Detailed, true, "")
}
// Be sure to skip the step so that in-memory state updates are performed.
err := step.Skip()
_, err := iter.Apply(step, true)
// We let `printPlan` handle error reporting for now.
if err == nil {

View file

@ -24,6 +24,7 @@ import (
// in the serialized object match the order they are defined in this struct.
//
// TODO[pulumi/pulumi#423]: use DOM based marshalling so we can roundtrip the seralized structure perfectly.
// nolint: lll
type Package struct {
Name tokens.PackageName `json:"name" yaml:"name"` // a required fully qualified name.
Runtime string `json:"runtime" yaml:"runtime"` // a required runtime that executes code.

View file

@ -217,7 +217,8 @@ func (ops *componentOpsProvider) ListMetrics() []component.MetricName {
}
}
func (ops *componentOpsProvider) GetMetricStatistics(metric component.MetricRequest) ([]component.MetricDataPoint, error) {
func (ops *componentOpsProvider) GetMetricStatistics(metric component.MetricRequest) (
[]component.MetricDataPoint, error) {
var dimensions []*cloudwatch.Dimension
var namespace string
@ -316,7 +317,8 @@ func (ops *componentsOpsProvider) ListMetrics() []component.MetricName {
return []component.MetricName{}
}
func (ops *componentsOpsProvider) GetMetricStatistics(metric component.MetricRequest) ([]component.MetricDataPoint, error) {
func (ops *componentsOpsProvider) GetMetricStatistics(metric component.MetricRequest) (
[]component.MetricDataPoint, error) {
return nil, fmt.Errorf("not yet implemented")
}

View file

@ -28,7 +28,7 @@ import (
// Asset is a serialized asset reference. It is a union: thus, only one of its fields will be non-nil. Several helper
// routines exist as members in order to easily interact with the assets referenced by an instance of this type.
//nolint: lll
// nolint: lll
type Asset struct {
Sig string `json:"4dabf18193072939515e22adb298388d" yaml:"4dabf18193072939515e22adb298388d"` // the unique asset signature (see properties.go).
Hash string `json:"hash,omitempty" yaml:"hash,omitempty"` // the SHA256 hash of the asset contents.
@ -104,9 +104,10 @@ func (a *Asset) GetURIURL() (*url.URL, bool, error) {
return nil, false, nil
}
// Equals returns true if a is value-equal to other. In this case, value equality is determined only by the hash: even if the contents of
// two assets come from different sources, they are treated as equal if their hashes match. Similarly, if the contents of two assets
// come from the same source but the assets have different hashes, the assets are not equal.
// Equals returns true if a is value-equal to other. In this case, value equality is determined only by the hash: even
// if the contents of two assets come from different sources, they are treated as equal if their hashes match.
// Similarly, if the contents of two assets come from the same source but the assets have different hashes, the assets
// are not equal.
func (a *Asset) Equals(other *Asset) bool {
if a == nil {
return other == nil
@ -415,9 +416,10 @@ func (a *Archive) GetURIURL() (*url.URL, bool, error) {
return nil, false, nil
}
// Equals returns true if a is value-equal to other. In this case, value equality is determined only by the hash: even if the contents of
// two archives come from different sources, they are treated as equal if their hashes match. Similarly, if the contents of two archives
// come from the same source but the archives have different hashes, the archives are not equal.
// Equals returns true if a is value-equal to other. In this case, value equality is determined only by the hash: even
// if the contents of two archives come from different sources, they are treated as equal if their hashes match.
// Similarly, if the contents of two archives come from the same source but the archives have different hashes, the
// archives are not equal.
func (a *Archive) Equals(other *Archive) bool {
if a == nil {
return other == nil
@ -528,8 +530,9 @@ func (a *Archive) HasContents() bool {
// ArchiveReader presents the contents of an archive as a stream of named blobs.
type ArchiveReader interface {
// Next returns the name and contents of the next member of the archive. If there are no more members in the archive, this function
// returns ("", nil, io.EOF). The blob returned by a call to Next() must be read in full before the next call to Next().
// Next returns the name and contents of the next member of the archive. If there are no more members in the
// archive, this function returns ("", nil, io.EOF). The blob returned by a call to Next() must be read in full
// before the next call to Next().
Next() (string, *Blob, error)
// Close terminates the stream.
@ -558,7 +561,8 @@ type assetsArchiveReader struct {
func (r *assetsArchiveReader) Next() (string, *Blob, error) {
for {
// If we're currently flattening out a subarchive, first check to see if it has any more members. If it does, return the next member.
// If we're currently flattening out a subarchive, first check to see if it has any more members. If it does,
// return the next member.
if r.archive != nil {
name, blob, err := r.archive.Next()
switch {
@ -800,7 +804,8 @@ func (a *Archive) Archive(format ArchiveFormat, w io.Writer) error {
}
}
// addNextFileToTar adds the next file in the given archive to the given tar file. Returns io.EOF if the archive contains no more files.
// addNextFileToTar adds the next file in the given archive to the given tar file. Returns io.EOF if the archive
// contains no more files.
func addNextFileToTar(r ArchiveReader, tw *tar.Writer) error {
file, data, err := r.Next()
if err != nil {
@ -848,7 +853,8 @@ func (a *Archive) archiveTarGZIP(w io.Writer) error {
return a.archiveTar(z)
}
// addNextFileToZIP adds the next file in the given archive to the given ZIP file. Returns io.EOF if the archive contains no more files.
// addNextFileToZIP adds the next file in the given archive to the given ZIP file. Returns io.EOF if the archive
// contains no more files.
func addNextFileToZIP(r ArchiveReader, zw *zip.Writer) error {
file, data, err := r.Next()
if err != nil {

View file

@ -66,7 +66,8 @@ func roundtripJSON(v Value) (Value, error) {
return roundtrip(v, json.Marshal, json.Unmarshal)
}
func roundtrip(v Value, marshal func(v interface{}) ([]byte, error), unmarshal func([]byte, interface{}) error) (Value, error) {
func roundtrip(v Value, marshal func(v interface{}) ([]byte, error),
unmarshal func([]byte, interface{}) error) (Value, error) {
b, err := marshal(v)
if err != nil {
return Value{}, err

View file

@ -3,6 +3,7 @@
package deploy
import (
"reflect"
"sort"
"time"
@ -44,6 +45,7 @@ func (p *Plan) Start(opts Options) (*PlanIterator, error) {
replaces: make(map[resource.URN]bool),
deletes: make(map[resource.URN]bool),
sames: make(map[resource.URN]bool),
regs: make(map[resource.URN]*FinalState),
dones: make(map[*resource.State]bool),
}, nil
}
@ -107,10 +109,11 @@ type PlanIterator struct {
deletes map[resource.URN]bool // URNs discovered to be deleted.
sames map[resource.URN]bool // URNs discovered to be the same.
stepqueue []Step // a queue of steps to drain.
delqueue []*resource.State // a queue of deletes left to perform.
resources []*resource.State // the resulting ordered resource states.
dones map[*resource.State]bool // true for each old state we're done with.
stepqueue []Step // a queue of steps to drain.
delqueue []*resource.State // a queue of deletes left to perform.
resources []*resource.State // the resulting ordered resource states.
regs map[resource.URN]*FinalState // a map of pending registrations not yet complete.
dones map[*resource.State]bool // true for each old state we're done with.
srcdone bool // true if the source interpreter has been run to completion.
done bool // true if the planning and associated iteration has finished.
@ -129,6 +132,26 @@ func (iter *PlanIterator) Resources() []*resource.State { return iter.resourc
func (iter *PlanIterator) Dones() map[*resource.State]bool { return iter.dones }
func (iter *PlanIterator) Done() bool { return iter.done }
// Apply performs a plan's step and records its result in the iterator's state.
func (iter *PlanIterator) Apply(step Step, skip bool) (resource.Status, error) {
// Ensure we don't have a pending registration for this resource already.
urn := step.URN()
if _, has := iter.regs[urn]; has {
}
// Apply the step.
status, state, err := step.Apply(skip)
if err != nil {
return status, err
}
// Now stash away its resulting state for safe-keeping; we will spill it to the checkpoint later on.
if state != nil {
iter.regs[urn] = state
}
return resource.StatusOK, nil
}
// Close terminates the iteration of this plan.
func (iter *PlanIterator) Close() error {
return iter.src.Close()
@ -138,31 +161,46 @@ func (iter *PlanIterator) Close() error {
// evaluation of the program as much as necessary to determine the next step. If there is no further action to be
// taken, Next will return a nil step pointer.
func (iter *PlanIterator) Next() (Step, error) {
outer:
for !iter.done {
if len(iter.stepqueue) > 0 {
step := iter.stepqueue[0]
iter.stepqueue = iter.stepqueue[1:]
return step, nil
} else if !iter.srcdone {
goal, err := iter.src.Next()
intent, err := iter.src.Next()
if err != nil {
return nil, err
} else if goal != nil {
steps, err := iter.nextResourceSteps(goal)
if err != nil {
return nil, err
} else if intent != nil {
// If we have an intent, drive the behavior based on which kind it is.
switch it := intent.(type) {
case RegisterIntent:
// If the intent is to register a resource, compute the plan steps necessary to do so.
steps, steperr := iter.computeResourceSteps(it)
if err != nil {
return nil, steperr
}
contract.Assert(len(steps) > 0)
if len(steps) > 1 {
iter.stepqueue = steps[1:]
}
return steps[0], nil
case CompleteIntent:
// If the intent is to complete a prior resource registration, do so. We do this by just
// processing the request from the existing state, and do not expose our callers to it.
if err = iter.completeResource(it); err != nil {
return nil, err
}
continue outer
default:
contract.Failf("Unrecognized intent from source iterator: %v", reflect.TypeOf(intent))
}
contract.Assert(len(steps) > 0)
if len(steps) > 1 {
iter.stepqueue = steps[1:]
}
return steps[0], nil
}
// If all returns are nil, the source is done, note it, and don't go back for more. Add any deletions to be
// performed, and then keep going 'round the next iteration of the loop so we can wrap up the planning.
iter.srcdone = true
iter.delqueue = iter.calculateDeletes()
iter.delqueue = iter.computeDeletes()
} else {
// The interpreter has finished, so we need to now drain any deletions that piled up.
if step := iter.nextDeleteStep(); step != nil {
@ -177,14 +215,14 @@ func (iter *PlanIterator) Next() (Step, error) {
return nil, nil
}
// nextResourceSteps produces one or more steps required to achieve the desired resource goal state, or nil if there
// computeResourceSteps produces one or more steps required to achieve the desired resource goal state, or nil if there
// aren't any steps to perform (in other words, the actual known state is equivalent to the goal state). It is
// possible to return multiple steps if the current resource state necessitates it (e.g., replacements).
func (iter *PlanIterator) nextResourceSteps(goal SourceGoal) ([]Step, error) {
func (iter *PlanIterator) computeResourceSteps(reg RegisterIntent) ([]Step, error) {
var invalid bool // will be set to true if this object fails validation.
// Use the resource goal state name to produce a globally unique URN.
res := goal.Resource()
res := reg.Goal()
urn := resource.NewURN(iter.p.Target().Name, iter.p.source.Pkg(), res.Type, res.Name)
if iter.urns[urn] {
invalid = true
@ -311,7 +349,7 @@ func (iter *PlanIterator) nextResourceSteps(goal SourceGoal) ([]Step, error) {
}
return []Step{
NewCreateReplacementStep(iter, goal, old, new, diff.ReplaceKeys),
NewCreateReplacementStep(iter, reg, old, new, diff.ReplaceKeys),
NewReplaceStep(iter, old, new, diff.ReplaceKeys),
}, nil
}
@ -322,7 +360,7 @@ func (iter *PlanIterator) nextResourceSteps(goal SourceGoal) ([]Step, error) {
glog.V(7).Infof("Planner decided to update '%v' (oldprops=%v inputs=%v",
urn, oldinputs, new.AllInputs())
}
return []Step{NewUpdateStep(iter, goal, old, new, diff.StableKeys)}, nil
return []Step{NewUpdateStep(iter, reg, old, new, diff.StableKeys)}, nil
}
// No need to update anything, the properties didn't change.
@ -330,13 +368,13 @@ func (iter *PlanIterator) nextResourceSteps(goal SourceGoal) ([]Step, error) {
if glog.V(7) {
glog.V(7).Infof("Planner decided not to update '%v' (same) (inputs=%v)", urn, new.AllInputs())
}
return []Step{NewSameStep(iter, goal, old, new)}, nil
return []Step{NewSameStep(iter, reg, old, new)}, nil
}
// Otherwise, the resource isn't in the old map, so it must be a resource creation.
iter.creates[urn] = true
glog.V(7).Infof("Planner decided to create '%v' (inputs=%v)", urn, new.AllInputs())
return []Step{NewCreateStep(iter, goal, new)}, nil
return []Step{NewCreateStep(iter, reg, new)}, nil
}
// issueCheckErrors prints any check errors to the diagnostics sink.
@ -357,26 +395,35 @@ func (iter *PlanIterator) issueCheckErrors(new *resource.State, urn resource.URN
return true
}
// nextDeleteStep produces a new step that deletes a resource if necessary.
func (iter *PlanIterator) nextDeleteStep() Step {
if len(iter.delqueue) > 0 {
del := iter.delqueue[0]
iter.delqueue = iter.delqueue[1:]
urn := del.URN
iter.deletes[urn] = true
if iter.replaces[urn] {
glog.V(7).Infof("Planner decided to delete '%v' due to replacement", urn)
} else {
glog.V(7).Infof("Planner decided to delete '%v'", urn)
}
return NewDeleteStep(iter, del, iter.replaces[urn])
func (iter *PlanIterator) completeResource(c CompleteIntent) error {
// Look up the final state in the pending registration list.
urn := c.URN()
final, has := iter.regs[urn]
if !has {
return goerr.Errorf("cannot complete a resource '%v' whose registration isn't pending", urn)
}
contract.Assertf(final != nil, "expected a non-nil final resource state ('%v')", urn)
delete(iter.regs, urn)
// Complete the step and get back our final state.
// If there are any extra properties to add to the outputs, do it now.
if extras := c.Extras(); extras != nil {
final.State.AddExtras(extras)
}
// Add the final state snapshot for this resource so that it gets serialized appropriately.
iter.CompleteStateSnapshot(urn, final.State)
// Now communicate the results back to the language provider, who is waiting on the intent.
c.Done(final)
return nil
}
// calculateDeletes creates a list of deletes to perform. This will include any resources in the snapshot that were
// computeDeletes creates a list of deletes to perform. This will include any resources in the snapshot that were
// not encountered in the input, along with any resources that were replaced.
func (iter *PlanIterator) calculateDeletes() []*resource.State {
func (iter *PlanIterator) computeDeletes() []*resource.State {
// To compute the deletion list, we must walk the list of old resources *backwards*. This is because the list is
// stored in dependency order, and earlier elements are possibly leaf nodes for later elements. We must not delete
// dependencies prior to their dependent nodes.
@ -394,11 +441,27 @@ func (iter *PlanIterator) calculateDeletes() []*resource.State {
return dels
}
// nextDeleteStep produces a new step that deletes a resource if necessary.
func (iter *PlanIterator) nextDeleteStep() Step {
if len(iter.delqueue) > 0 {
del := iter.delqueue[0]
iter.delqueue = iter.delqueue[1:]
urn := del.URN
iter.deletes[urn] = true
if iter.replaces[urn] {
glog.V(7).Infof("Planner decided to delete '%v' due to replacement", urn)
} else {
glog.V(7).Infof("Planner decided to delete '%v'", urn)
}
return NewDeleteStep(iter, del, iter.replaces[urn])
}
return nil
}
// Snap returns a fresh snapshot that takes into account everything that has happened up till this point. Namely, if a
// failure happens partway through, the untouched snapshot elements will be retained, while any updates will be
// preserved. If no failure happens, the snapshot naturally reflects the final state of all resources.
func (iter *PlanIterator) Snap() *Snapshot {
// At this point we have two resource DAGs. One of these is the base DAG for this plan; the other is the current DAG
// for this plan. Any resource r may be present in both DAGs. In order to produce a snapshot, we need to merge these
// DAGs such that all resource dependencies are correctly preserved. Conceptually, the merge proceeds as follows:
@ -406,8 +469,8 @@ func (iter *PlanIterator) Snap() *Snapshot {
// - Begin with an empty merged DAG.
// - For each resource r in the current DAG, insert r and its outgoing edges into the merged DAG.
// - For each resource r in the base DAG:
// - If r is in the merged DAG, we are done: if the resource is in the merged DAG, it must have been in the current
// DAG, which accurately captures its current dependencies.
// - If r is in the merged DAG, we are done: if the resource is in the merged DAG, it must have been in the
// current DAG, which accurately captures its current dependencies.
// - If r is not in the merged DAG, insert it and its outgoing edges into the merged DAG.
//
// Physically, however, each DAG is represented as list of resources without explicit dependency edges. In place of
@ -417,22 +480,23 @@ func (iter *PlanIterator) Snap() *Snapshot {
// topological sort of the merged DAG:
//
// - Begin with an empty merged list.
// - For each resource r in the current list, append r to the merged list. r must be in a correct location in the merged
// list, as its position relative to its assumed dependencies has not changed.
// - For each resource r in the current list, append r to the merged list. r must be in a correct location in the
// merged list, as its position relative to its assumed dependencies has not changed.
// - For each resource r in the base list:
// - If r is in the merged list, we are done by the logic given in the original algorithm.
// - If r is not in the merged list, append r to the merged list. r must be in a correct location in the merged list:
// - If r is not in the merged list, append r to the merged list. r must be in a correct location in the merged
// list:
// - If any of r's dependencies were in the current list, they must already be in the merged list and their
// relative order w.r.t. r has not changed.
// - If any of r's dependencies were not in the current list, they must already be in the merged list, as they
// would have been appended to the list before r.
// - If any of r's dependencies were not in the current list, they must already be in the merged list, as
// they would have been appended to the list before r.
// Start with a copy of the resources produced during the evaluation of the current plan.
resources := make([]*resource.State, len(iter.resources))
copy(resources, iter.resources)
// If the plan has not finished executing, append any resources from the base plan that were not produced by the current
// plan.
// If the plan has not finished executing, append any resources from the base plan that were not produced by the
// current plan.
if !iter.done {
if prev := iter.p.prev; prev != nil {
for _, res := range prev.Resources {
@ -446,15 +510,16 @@ func (iter *PlanIterator) Snap() *Snapshot {
return NewSnapshot(iter.p.Target().Name, time.Now(), resources)
}
// MarkStateSnapshot marks an old state snapshot as being processed. This is done to recover from failures partway
// through the application of a deployment plan. Any old state that has not yet been recovered needs to be kept.
func (iter *PlanIterator) MarkStateSnapshot(state *resource.State) {
iter.dones[state] = true
}
// AppendStateSnapshot appends a resource's state to the current snapshot.
func (iter *PlanIterator) AppendStateSnapshot(state *resource.State) {
iter.resources = append(iter.resources, state)
// CompleteStateSnapshot marks an old state snapshot as being processed, while adding a new version if relevant. This
// is done to recover from failures partway through the application of a deployment plan. Any old state that has not
// yet been recovered needs to be kept.
func (iter *PlanIterator) CompleteStateSnapshot(urn resource.URN, new *resource.State) {
if old := iter.p.olds[urn]; old != nil {
iter.dones[old] = true
}
if new != nil {
iter.resources = append(iter.resources, new)
}
}
// Provider fetches the provider for a given resource type, possibly lazily allocating the plugins for it. If a

View file

@ -107,7 +107,7 @@ func (iter *errorSourceIterator) Close() error {
return nil // nothing to do.
}
func (iter *errorSourceIterator) Next() (SourceGoal, error) {
func (iter *errorSourceIterator) Next() (SourceIntent, error) {
return nil, iter.src.err
}
@ -196,24 +196,24 @@ func TestBasicCRUDPlan(t *testing.T) {
"af1": resource.NewStringProperty("a-value"),
"af2": resource.NewNumberProperty(42),
}, "")
newStateA := &testSourceGoal{Goal: newResA}
newStateA := &testRegInt{goal: newResA}
// - B is updated:
newResB := resource.NewGoal(typB, namB, true, resource.PropertyMap{
"bf1": resource.NewStringProperty("b-value"),
// delete the bf2 field, and add bf3.
"bf3": resource.NewBoolProperty(true),
}, "")
newStateB := &testSourceGoal{Goal: newResB}
newStateB := &testRegInt{goal: newResB}
// - C has no changes:
newResC := resource.NewGoal(typC, namC, true, resource.PropertyMap{
"cf1": resource.NewStringProperty("c-value"),
"cf2": resource.NewNumberProperty(83),
}, "")
newStateC := &testSourceGoal{Goal: newResC}
newStateC := &testRegInt{goal: newResC}
// - No D; it is deleted.
// Use a fixed source that just returns the above predefined objects during planning.
source := NewFixedSource(pkgname, []SourceGoal{newStateA, newStateB, newStateC})
source := NewFixedSource(pkgname, []SourceIntent{newStateA, newStateB, newStateC})
// Next up, create a plan from the new and old, and validate its shape.
plan := NewPlan(ctx, targ, oldsnap, source, nil)
@ -234,7 +234,6 @@ func TestBasicCRUDPlan(t *testing.T) {
var urn resource.URN
var realID bool
var expectOuts resource.PropertyMap
var state *testSourceGoal
switch s := step.(type) {
case *CreateStep: // A is created
old := s.Old()
@ -244,7 +243,7 @@ func TestBasicCRUDPlan(t *testing.T) {
assert.Equal(t, urnA, new.URN)
assert.Equal(t, newResA.Properties, new.Inputs)
assert.Equal(t, newResA.Properties, new.AllInputs())
state, urn, realID = newStateA, urnA, false
urn, realID = urnA, false
case *UpdateStep: // B is updated
old := s.Old()
assert.NotNil(t, old)
@ -255,7 +254,7 @@ func TestBasicCRUDPlan(t *testing.T) {
assert.Equal(t, urnB, new.URN)
assert.Equal(t, newResB.Properties, new.Inputs)
assert.Equal(t, newResB.Properties, new.AllInputs())
state, urn, realID = newStateB, urnB, true
urn, realID = urnB, true
case *SameStep: // C is the same
old := s.Old()
assert.NotNil(t, old)
@ -266,7 +265,7 @@ func TestBasicCRUDPlan(t *testing.T) {
assert.Equal(t, urnC, new.URN)
assert.Equal(t, newResC.Properties, new.Inputs)
assert.Equal(t, newResC.Properties, new.AllInputs())
state, urn, realID, expectOuts = newStateC, urnC, true, oldResC.Outputs
urn, realID, expectOuts = urnC, true, oldResC.Outputs
case *DeleteStep: // D is deleted
old := s.Old()
assert.NotNil(t, old)
@ -278,20 +277,19 @@ func TestBasicCRUDPlan(t *testing.T) {
t.FailNow() // unexpected step kind.
}
if state != nil {
// Ensure the state is empty until until we step.
assert.Nil(t, state.State)
}
err = step.Skip()
var state *FinalState
_, state, err = step.Apply(true)
assert.Nil(t, err)
op := step.Op()
if state != nil {
// The state should be non-nil by now.
// The state should be non-nil by now and it should have a URN.
assert.NotNil(t, state.State)
// Ensure the ID, URN, and properties are populated correctly.
// Ensure the URN, ID, and properties are populated correctly.
assert.Equal(t, urn, state.State.URN,
"Expected op %v to populate a URN equal to %v", op, urn)
if realID {
assert.NotEqual(t, resource.ID(""), state.State.ID,
"Expected op %v to populate a real ID (%v)", op, urn)
@ -299,8 +297,6 @@ func TestBasicCRUDPlan(t *testing.T) {
assert.Equal(t, resource.ID(""), state.State.ID,
"Expected op %v to leave ID blank (%v); got %v", op, urn, state.State.ID)
}
assert.Equal(t, urn, state.State.URN,
"Expected op %v to populate a URN equal to %v", op, urn)
if expectOuts != nil {
props := state.State.All()
@ -329,18 +325,22 @@ func TestBasicCRUDPlan(t *testing.T) {
assert.True(t, iter.Deletes()[urnD])
}
type testSourceGoal struct {
Goal *resource.Goal
State *resource.State
type testRegInt struct {
goal *resource.Goal
urn resource.URN
}
func (g *testSourceGoal) Resource() *resource.Goal {
return g.Goal
var _ RegisterIntent = (*testRegInt)(nil)
func (g *testRegInt) intent() {}
func (g *testRegInt) Goal() *resource.Goal {
return g.goal
}
func (g *testSourceGoal) Done(state *resource.State, stable bool, stables []resource.PropertyKey) {
contract.Assertf(g.State == nil, "Attempt to invoke testSourceGoal.Done more than once")
g.State = state
func (g *testRegInt) Done(urn resource.URN) {
contract.Assertf(g.urn == "", "Attempt to invoke testRegInt.Done more than once")
g.urn = urn
}
type testProviderHost struct {

View file

@ -20,22 +20,42 @@ type Source interface {
Iterate(opts Options) (SourceIterator, error)
}
// A SourceIterator enumerates the list of resources that a source has to offer.
// A SourceIterator enumerates the list of resources that a source has to offer and tracks associated state.
type SourceIterator interface {
io.Closer
// Next returns the next resource from the source. This object contains information produced by the iterator
// about a resource's state; it may be used to communicate the result of the ensuing planning or deployment
// operation. Indeed, its Done function *must* be callled when done. If it is nil, then the iterator has
// completed its job and no subsequent calls to Next should be made.
Next() (SourceGoal, error)
// Next returns the next intent from the source.
Next() (SourceIntent, error)
}
// SourceGoal is an item returned from a source iterator which can be used to inspect input state, and
// communicate back the final results after a plan or deployment operation has been performed.
type SourceGoal interface {
// Resource reflects the goal state for the resource object that was allocated by the program.
Resource() *resource.Goal
// Done indicates that we are done with this resource, and provides the full state (ID, URN, and output properties)
// that resulted from the operation. This *must* be called when the resource element is done with.
Done(state *resource.State, stable bool, stables []resource.PropertyKey)
// SourceIntent is an intent associated with the enumeration of a plan. It is an intent expressed by the source
// program, and it is the responsibility of the engine to make it so.
type SourceIntent interface {
intent()
}
// RegisterIntent is a step that asks the engine to provision a resource.
type RegisterIntent interface {
SourceIntent
// Goal returns the goal state for the resource object that was allocated by the program.
Goal() *resource.Goal
// Done indicates that we are done with this step. It must be called to perform cleanup associated with the step.
Done(urn resource.URN)
}
// CompleteIntent is an intent that asks the engine to complete the provisioning of a resource.
type CompleteIntent interface {
SourceIntent
// URN is the resource URN that this completion applies to.
URN() resource.URN
// Extras returns an optional "extra" property map of output properties to add to a resource before completing.
Extras() resource.PropertyMap
// Done indicates that we are done with this step. It must be called to perform cleanup associated with the step.
Done(res *FinalState)
}
// FinalState is the final source completion information.
type FinalState struct {
State *resource.State // the resource state.
Stable bool // if true, the resource state is stable and may be trusted.
Stables []resource.PropertyKey // an optional list of specific resource properties that are stable.
}

View file

@ -65,18 +65,20 @@ func (src *evalSource) Info() interface{} {
// Iterate will spawn an evaluator coroutine and prepare to interact with it on subsequent calls to Next.
func (src *evalSource) Iterate(opts Options) (SourceIterator, error) {
// First, fire up a resource monitor that will watch for and record resource creation.
reschan := make(chan *evalSourceGoal)
mon, err := newResourceMonitor(src, reschan)
regChan := make(chan *evalRegStep)
compChan := make(chan *evalCompStep)
mon, err := newResourceMonitor(src, regChan, compChan)
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,
finchan: make(chan error),
reschan: reschan,
mon: mon,
src: src,
regChan: regChan,
compChan: compChan,
finChan: make(chan error),
}
// Now invoke Run in a goroutine. All subsequent resource creation events will come in over the gRPC channel,
@ -88,11 +90,12 @@ 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.
finchan chan error // the channel that communicates completion.
reschan chan *evalSourceGoal // the channel that contains resource elements.
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 *evalRegStep // the channel that contains resource registrations.
compChan chan *evalCompStep // the channel that contains resource completions.
finChan chan error // the channel that communicates completion.
done bool // set to true when the evaluation is done.
}
func (iter *evalSourceIterator) Close() error {
@ -100,7 +103,7 @@ func (iter *evalSourceIterator) Close() error {
return iter.mon.Cancel()
}
func (iter *evalSourceIterator) Next() (SourceGoal, error) {
func (iter *evalSourceIterator) Next() (SourceIntent, error) {
// If we are done, quit.
if iter.done {
return nil, nil
@ -113,7 +116,18 @@ func (iter *evalSourceIterator) Next() (SourceGoal, error) {
// Await the program to compute some more state and then inspect what it has to say.
select {
case err := <-iter.finchan:
case reg := <-iter.regChan:
contract.Assert(reg != nil)
goal := reg.Goal()
glog.V(5).Infof("EvalSourceIterator produced a registration: t=%v,name=%v,#props=%v",
goal.Type, goal.Name, len(goal.Properties))
return reg, nil
case comp := <-iter.compChan:
contract.Assert(comp != nil)
glog.V(5).Infof("EvalSourceIterator produced a completion: urn=%v,#extras=%v",
comp.URN(), len(comp.Extras()))
return comp, 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.
iter.done = true
@ -121,12 +135,6 @@ func (iter *evalSourceIterator) Next() (SourceGoal, error) {
glog.V(5).Infof("EvalSourceIterator ended with an error: %v", err)
}
return nil, err
case res := <-iter.reschan:
contract.Assert(res != nil)
goal := res.Resource()
glog.V(5).Infof("EvalSourceIterator produced a new object: t=%v,name=%v,#props=%v",
goal.Type, goal.Name, len(goal.Properties))
return res, nil
}
}
@ -170,7 +178,7 @@ func (iter *evalSourceIterator) forkRun(opts Options) {
}
// Communicate the error, if it exists, or nil if the program exited cleanly.
iter.finchan <- err
iter.finChan <- err
}()
}
}
@ -178,20 +186,22 @@ func (iter *evalSourceIterator) forkRun(opts Options) {
// resmon implements the lumirpc.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.
reschan chan *evalSourceGoal // the channel to send resources 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.
resChan chan *evalRegStep // the channel to send resource registrations to.
compChan chan *evalCompStep // the channel to send resource completions 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, reschan chan *evalSourceGoal) (*resmon, error) {
func newResourceMonitor(src *evalSource, resChan chan *evalRegStep, compChan chan *evalCompStep) (*resmon, error) {
// New up an engine RPC server.
resmon := &resmon{
src: src,
reschan: reschan,
cancel: make(chan bool),
src: src,
resChan: resChan,
compChan: compChan,
cancel: make(chan bool),
}
// Fire up a gRPC server and start listening for incomings.
@ -260,9 +270,9 @@ func (rm *resmon) Invoke(ctx context.Context, req *lumirpc.InvokeRequest) (*lumi
return &lumirpc.InvokeResponse{Return: mret, Failures: chkfails}, nil
}
// NewResource is invoked by a language process when a new resource has been allocated.
func (rm *resmon) NewResource(ctx context.Context,
req *lumirpc.NewResourceRequest) (*lumirpc.NewResourceResponse, error) {
// RegisterResource is invoked by a language process when a new resource has been allocated.
func (rm *resmon) RegisterResource(ctx context.Context,
req *lumirpc.RegisterResourceRequest) (*lumirpc.RegisterResourceResponse, error) {
// Communicate the type, name, and object information to the iterator that is awaiting us.
props, err := plugin.UnmarshalProperties(
@ -271,33 +281,66 @@ func (rm *resmon) NewResource(ctx context.Context,
return nil, err
}
goal := &evalSourceGoal{
goal: resource.NewGoal(
tokens.Type(req.GetType()),
tokens.QName(req.GetName()),
req.GetCustom(),
props,
resource.URN(req.GetParent()),
),
done: make(chan *evalState),
t := tokens.Type(req.GetType())
name := tokens.QName(req.GetName())
custom := req.GetCustom()
parent := resource.URN(req.GetParent())
glog.V(5).Infof("ResourceMonitor.RegisterResource received: t=%v, name=%v, custom=%v, #props=%v, parent=%v",
t, name, custom, len(props), parent)
// Send the goal state to the engine.
step := &evalRegStep{
goal: resource.NewGoal(t, name, custom, props, parent),
done: make(chan resource.URN),
}
glog.V(5).Infof("ResourceMonitor.NewResource received: t=%v, name=%v, custom=%v, #props=%v, parent=%v",
goal.goal.Type, goal.goal.Name, goal.goal.Custom, len(goal.goal.Properties), goal.goal.Parent)
rm.reschan <- goal
rm.resChan <- step
// Now block waiting for the operation to finish.
// FIXME: we probably need some way to cancel this in case of catastrophe.
done := <-goal.done
state := done.State
// IDEA: we probably need some way to cancel this in case of catastrophe.
urn := string(<-step.done)
glog.V(5).Infof(
"ResourceMonitor.RegisterResource operation finished: t=%v, name=%v, urn=%v", t, name, urn)
return &lumirpc.RegisterResourceResponse{Urn: urn}, nil
}
// CompleteResource records some new output properties for a resource that have arrived after its initial
// provisioning. These will make their way into the eventual checkpoint state file for that resource.
func (rm *resmon) CompleteResource(ctx context.Context,
req *lumirpc.CompleteResourceRequest) (*lumirpc.CompleteResourceResponse, error) {
// Obtain and validate the message's inputs (a URN plus the output property map).
urn := resource.URN(req.GetUrn())
if urn == "" {
return nil, errors.New("missing required URN")
}
extras, err := plugin.UnmarshalProperties(
req.GetExtras(), plugin.MarshalOptions{KeepUnknowns: true, ComputeAssetHashes: true})
if err != nil {
return nil, errors.Wrapf(err, "cannot unmarshal output properties")
}
glog.V(5).Infof("ResourceMonitor.CompleteResource received: urn=%v, #extras=%v", urn, len(extras))
// Now send the step over to the engine to perform.
step := &evalCompStep{
urn: urn,
extras: extras,
done: make(chan *FinalState),
}
rm.compChan <- step
// Now block waiting for the operation to finish.
// IDEA: we probably need some way to cancel this in case of catastrophe.
result := <-step.done
state := result.State
outprops := state.Synthesized()
stable := done.Stable
stable := result.Stable
var stables []string
for _, sta := range done.Stables {
for _, sta := range result.Stables {
stables = append(stables, string(sta))
}
glog.V(5).Infof(
"ResourceMonitor.NewResource operation finished: t=%v, urn=%v (name=%v), stable=%v, #stables=%v #outs=%v",
state.Type, state.URN, goal.goal.Name, stable, len(stables), len(outprops))
"ResourceMonitor.CompleteResource operation finished: t=%v, urn=%v, name=%v, stable=%v, #stables=%v #outs=%v",
state.Type, state.URN, stable, len(stables), len(outprops))
// Finally, unpack the response into properties that we can return to the language runtime. This mostly includes
// an ID, URN, and defaults and output properties that will all be blitted back onto the runtime object.
@ -305,35 +348,51 @@ func (rm *resmon) NewResource(ctx context.Context,
if err != nil {
return nil, err
}
return &lumirpc.NewResourceResponse{
return &lumirpc.CompleteResourceResponse{
Id: string(state.ID),
Urn: string(state.URN),
Object: outs,
Stable: stable,
Stables: stables,
}, nil
}
type evalState struct {
State *resource.State // the resource state.
Stable bool // if true, the resource state is stable and may be trusted.
Stables []resource.PropertyKey // an optional list of specific resource properties that are stable.
type evalRegStep struct {
goal *resource.Goal // the resource goal state produced by the iterator.
done chan resource.URN // the channel to communicate with after the resource state is available.
}
type evalSourceGoal struct {
goal *resource.Goal // the resource goal state produced by the iterator.
done chan *evalState // the channel to communicate with after the resource state is available.
}
var _ RegisterIntent = (*evalRegStep)(nil)
func (g *evalSourceGoal) Resource() *resource.Goal {
func (g *evalRegStep) intent() {}
func (g *evalRegStep) Goal() *resource.Goal {
return g.goal
}
func (g *evalSourceGoal) Done(state *resource.State, stable bool, stables []resource.PropertyKey) {
func (g *evalRegStep) Done(urn resource.URN) {
// Communicate the resulting state back to the RPC thread, which is parked awaiting our reply.
g.done <- &evalState{
State: state,
Stable: stable,
Stables: stables,
}
g.done <- urn
}
type evalCompStep struct {
urn resource.URN // the URN to which this completion applies.
extras resource.PropertyMap // an optional property bag for "extra" output properties.
done chan *FinalState // the channel to communicate with after the resource state is available.
}
var _ CompleteIntent = (*evalCompStep)(nil)
func (g *evalCompStep) intent() {}
func (g *evalCompStep) URN() resource.URN {
return g.urn
}
func (g *evalCompStep) Extras() resource.PropertyMap {
return g.extras
}
func (g *evalCompStep) Done(res *FinalState) {
// Communicate the resulting state back to the RPC thread, which is parked awaiting our reply.
g.done <- res
}

View file

@ -6,15 +6,15 @@ import (
"github.com/pulumi/pulumi/pkg/tokens"
)
// NewFixedSource returns a valid planning source that is comprised of a list of pre-computed resource objects.
func NewFixedSource(ctx tokens.PackageName, resources []SourceGoal) Source {
return &fixedSource{ctx: ctx, resources: resources}
// NewFixedSource returns a valid planning source that is comprised of a list of pre-computed steps.
func NewFixedSource(ctx tokens.PackageName, steps []SourceIntent) Source {
return &fixedSource{ctx: ctx, steps: steps}
}
// A fixedSource just returns from a fixed set of resource states.
type fixedSource struct {
ctx tokens.PackageName
resources []SourceGoal
ctx tokens.PackageName
steps []SourceIntent
}
func (src *fixedSource) Close() error {
@ -46,10 +46,10 @@ func (iter *fixedSourceIterator) Close() error {
return nil // nothing to do.
}
func (iter *fixedSourceIterator) Next() (SourceGoal, error) {
func (iter *fixedSourceIterator) Next() (SourceIntent, error) {
iter.current++
if iter.current >= len(iter.src.resources) {
if iter.current >= len(iter.src.steps) {
return nil, nil
}
return iter.src.resources[iter.current], nil
return iter.src.steps[iter.current], nil
}

View file

@ -38,6 +38,6 @@ func (iter *nullSourceIterator) Close() error {
return nil // nothing to do.
}
func (iter *nullSourceIterator) Next() (SourceGoal, error) {
func (iter *nullSourceIterator) Next() (SourceIntent, error) {
return nil, nil // means "done"
}

View file

@ -12,8 +12,7 @@ import (
// Step is a specification for a deployment operation.
type Step interface {
Apply() (resource.Status, error) // applies the action that this step represents.
Skip() error // skips past this step (required when iterating a plan).
Apply(skip bool) (resource.Status, *FinalState, error) // applies or skips the action that this step represents.
Op() StepOp // the operation performed by this step.
URN() resource.URN // the resource URN (for before and after).
@ -25,21 +24,19 @@ type Step interface {
Plan() *Plan // the owning plan.
Iterator() *PlanIterator // the current plan iterator.
}
// SameStep is a mutating step that does nothing.
type SameStep struct {
iter *PlanIterator // the current plan iteration.
goal SourceGoal // the goal state for the resource.
reg RegisterIntent // the registration intent to convey a URN back to.
old *resource.State // the state of the resource before this step.
new *resource.State // the state of the resource after this step.
}
var _ Step = (*SameStep)(nil)
func NewSameStep(iter *PlanIterator, goal SourceGoal, old *resource.State, new *resource.State) Step {
contract.Assert(goal != nil)
func NewSameStep(iter *PlanIterator, reg RegisterIntent, old *resource.State, new *resource.State) Step {
contract.Assert(old != nil)
contract.Assert(old.URN != "")
contract.Assert(old.ID != "" || !old.Custom)
@ -50,7 +47,7 @@ func NewSameStep(iter *PlanIterator, goal SourceGoal, old *resource.State, new *
contract.Assert(!new.Delete)
return &SameStep{
iter: iter,
goal: goal,
reg: reg,
old: old,
new: new,
}
@ -66,24 +63,15 @@ func (s *SameStep) New() *resource.State { return s.new }
func (s *SameStep) Res() *resource.State { return s.new }
func (s *SameStep) Logical() bool { return true }
func (s *SameStep) Apply() (resource.Status, error) {
// Just propagate the ID and output state to the live object and append to the snapshot.
s.goal.Done(s.old, true, nil)
s.iter.MarkStateSnapshot(s.old)
s.iter.AppendStateSnapshot(s.old)
return resource.StatusOK, nil
}
func (s *SameStep) Skip() error {
// In the case of a same, both ID and outputs are identical.
s.goal.Done(s.old, true, nil)
return nil
func (s *SameStep) Apply(skip bool) (resource.Status, *FinalState, error) {
s.reg.Done(s.URN())
return resource.StatusOK, &FinalState{State: s.old, Stable: true}, nil
}
// CreateStep is a mutating step that creates an entirely new resource.
type CreateStep struct {
iter *PlanIterator // the current plan iteration.
goal SourceGoal // the goal state for the resource.
reg RegisterIntent // the registration intent to convey a URN back to.
old *resource.State // the state of the existing resource (only for replacements).
new *resource.State // the state of the resource after this step.
keys []resource.PropertyKey // the keys causing replacement (only for replacements).
@ -92,22 +80,22 @@ type CreateStep struct {
var _ Step = (*CreateStep)(nil)
func NewCreateStep(iter *PlanIterator, goal SourceGoal, new *resource.State) Step {
contract.Assert(goal != nil)
func NewCreateStep(iter *PlanIterator, reg RegisterIntent, new *resource.State) Step {
contract.Assert(reg != nil)
contract.Assert(new != nil)
contract.Assert(new.URN != "")
contract.Assert(new.ID == "")
contract.Assert(!new.Delete)
return &CreateStep{
iter: iter,
goal: goal,
reg: reg,
new: new,
}
}
func NewCreateReplacementStep(iter *PlanIterator, goal SourceGoal,
func NewCreateReplacementStep(iter *PlanIterator, reg RegisterIntent,
old *resource.State, new *resource.State, keys []resource.PropertyKey) Step {
contract.Assert(goal != nil)
contract.Assert(reg != nil)
contract.Assert(old != nil)
contract.Assert(old.URN != "")
contract.Assert(old.ID != "" || !old.Custom)
@ -119,7 +107,7 @@ func NewCreateReplacementStep(iter *PlanIterator, goal SourceGoal,
contract.Assert(old.Type == new.Type)
return &CreateStep{
iter: iter,
goal: goal,
reg: reg,
old: old,
new: new,
keys: keys,
@ -143,16 +131,16 @@ func (s *CreateStep) Res() *resource.State { return s.new }
func (s *CreateStep) Keys() []resource.PropertyKey { return s.keys }
func (s *CreateStep) Logical() bool { return !s.replacing }
func (s *CreateStep) Apply() (resource.Status, error) {
if s.new.Custom {
func (s *CreateStep) Apply(skip bool) (resource.Status, *FinalState, error) {
if !skip && s.new.Custom {
// Invoke the Create RPC function for this provider:
prov, err := getProvider(s)
if err != nil {
return resource.StatusOK, err
return resource.StatusOK, nil, err
}
id, outs, rst, err := prov.Create(s.URN(), s.new.AllInputs())
if err != nil {
return rst, err
return rst, nil, err
}
contract.Assert(id != "")
@ -166,21 +154,8 @@ func (s *CreateStep) Apply() (resource.Status, error) {
s.old.Delete = true
}
// And finish the overall operation.
s.goal.Done(s.new, false, nil)
s.iter.AppendStateSnapshot(s.new)
return resource.StatusOK, nil
}
func (s *CreateStep) Skip() error {
// Mark the old resource as pending deletion if necessary.
if s.replacing {
s.old.Delete = true
}
// In the case of a create, we don't know the ID or output properties. But we do know the defaults and URN.
s.goal.Done(s.new, false, nil)
return nil
s.reg.Done(s.URN())
return resource.StatusOK, &FinalState{State: s.new}, nil
}
// DeleteStep is a mutating step that deletes an existing resource.
@ -219,30 +194,24 @@ func (s *DeleteStep) New() *resource.State { return nil }
func (s *DeleteStep) Res() *resource.State { return s.old }
func (s *DeleteStep) Logical() bool { return !s.replacing }
func (s *DeleteStep) Apply() (resource.Status, error) {
if s.old.Custom {
func (s *DeleteStep) Apply(skip bool) (resource.Status, *FinalState, error) {
if !skip && s.old.Custom {
// Invoke the Delete RPC function for this provider:
prov, err := getProvider(s)
if err != nil {
return resource.StatusOK, err
return resource.StatusOK, nil, err
}
if rst, err := prov.Delete(s.URN(), s.old.ID, s.old.All()); err != nil {
return rst, err
return rst, nil, err
}
}
s.iter.MarkStateSnapshot(s.old)
return resource.StatusOK, nil
}
func (s *DeleteStep) Skip() error {
// In the case of a deletion, there is no state to propagate: the new object doesn't even exist.
return nil
return resource.StatusOK, nil, nil
}
// UpdateStep is a mutating step that updates an existing resource's state.
type UpdateStep struct {
iter *PlanIterator // the current plan iteration.
goal SourceGoal // the goal state for the resource.
reg RegisterIntent // the registration intent to convey a URN back to.
old *resource.State // the state of the existing resource.
new *resource.State // the newly computed state of the resource after updating.
stables []resource.PropertyKey // an optional list of properties that won't change during this update.
@ -250,7 +219,7 @@ type UpdateStep struct {
var _ Step = (*UpdateStep)(nil)
func NewUpdateStep(iter *PlanIterator, goal SourceGoal, old *resource.State,
func NewUpdateStep(iter *PlanIterator, reg RegisterIntent, old *resource.State,
new *resource.State, stables []resource.PropertyKey) Step {
contract.Assert(old != nil)
contract.Assert(old.URN != "")
@ -263,7 +232,7 @@ func NewUpdateStep(iter *PlanIterator, goal SourceGoal, old *resource.State,
contract.Assert(old.Type == new.Type)
return &UpdateStep{
iter: iter,
goal: goal,
reg: reg,
old: old,
new: new,
stables: stables,
@ -280,18 +249,21 @@ func (s *UpdateStep) New() *resource.State { return s.new }
func (s *UpdateStep) Res() *resource.State { return s.new }
func (s *UpdateStep) Logical() bool { return true }
func (s *UpdateStep) Apply() (resource.Status, error) {
if s.new.Custom {
func (s *UpdateStep) Apply(skip bool) (resource.Status, *FinalState, error) {
if skip {
// In the case of an update, the URN, defaults, and ID are the same, however, the outputs remain unknown.
s.new.ID = s.old.ID
} else if s.new.Custom {
// Invoke the Update RPC function for this provider:
prov, err := getProvider(s)
if err != nil {
return resource.StatusOK, err
return resource.StatusOK, nil, err
}
// Update to the combination of the old "all" state (including outputs), but overwritten with new inputs.
news := s.old.All().Merge(s.new.Inputs)
outs, rst, upderr := prov.Update(s.URN(), s.old.ID, s.old.All(), news)
if upderr != nil {
return rst, upderr
return rst, nil, upderr
}
// Now copy any output state back in case the update triggered cascading updates to other properties.
@ -300,17 +272,8 @@ func (s *UpdateStep) Apply() (resource.Status, error) {
}
// Finally, mark this operation as complete.
s.goal.Done(s.new, false, s.stables)
s.iter.MarkStateSnapshot(s.old)
s.iter.AppendStateSnapshot(s.new)
return resource.StatusOK, nil
}
func (s *UpdateStep) Skip() error {
// In the case of an update, the URN, defaults, and ID are the same, however, the outputs remain unknown.
s.new.ID = s.old.ID
s.goal.Done(s.new, false, s.stables)
return nil
s.reg.Done(s.URN())
return resource.StatusOK, &FinalState{State: s.new, Stables: s.stables}, nil
}
// ReplaceStep is a logical step indicating a resource will be replaced. This is comprised of three physical steps:
@ -353,19 +316,14 @@ func (s *ReplaceStep) Res() *resource.State { return s.new }
func (s *ReplaceStep) Keys() []resource.PropertyKey { return s.keys }
func (s *ReplaceStep) Logical() bool { return true }
func (s *ReplaceStep) Apply() (resource.Status, error) {
func (s *ReplaceStep) Apply(skip bool) (resource.Status, *FinalState, error) {
// We should have marked the old resource for deletion in the CreateReplacement step.
contract.Assert(s.old.Delete)
return resource.StatusOK, nil
return resource.StatusOK, nil, nil
}
func (s *ReplaceStep) Skip() error {
return nil
}
// getProvider fetches the provider for the given step.
func getProvider(s Step) (plugin.Provider, error) {
return s.Plan().Provider(s.Type().Package())
func (s *ReplaceStep) Snap() (*FinalState, error) {
return nil, nil
}
// StepOp represents the kind of operation performed by a step. It evaluates to its string label.
@ -457,3 +415,8 @@ func (op StepOp) Suffix() string {
}
return ""
}
// getProvider fetches the provider for the given step.
func getProvider(s Step) (plugin.Provider, error) {
return s.Plan().Provider(s.Type().Package())
}

View file

@ -223,9 +223,9 @@ func (p *plugin) Close() error {
var result error
// On windows, plugins are not loaded directly, instead a cmd script launches each plugin as a child process, so instead
// we need to kill all the children of the PID we have recorded, as well. Otherwise we will block waiting for the child
// processes to close.
// On windows, plugins are not loaded directly, instead a cmd script launches each plugin as a child process, so
// instead we need to kill all the children of the PID we have recorded, as well. Otherwise we will block waiting
// for the child processes to close.
if runtime.GOOS == "windows" {
if err := cmdutil.KillChildren(p.Proc.Pid); err != nil {
result = multierror.Append(result, err)

View file

@ -337,8 +337,8 @@ func MarshalStruct(obj *structpb.Struct, opts MarshalOptions) *structpb.Value {
// MarshalAsset marshals an asset into its wire form for resource provider plugins.
func MarshalAsset(v *resource.Asset, opts MarshalOptions) (*structpb.Value, error) {
// If we are not providing access to an asset's contents, we simply need to record the fact that this asset existed. Serialize the
// asset with only its hash (if present).
// If we are not providing access to an asset's contents, we simply need to record the fact that this asset existed.
// Serialize the asset with only its hash (if present).
if opts.ElideAssetContents {
v = &resource.Asset{Hash: v.Hash}
} else {
@ -358,8 +358,8 @@ func MarshalAsset(v *resource.Asset, opts MarshalOptions) (*structpb.Value, erro
// MarshalArchive marshals an archive into its wire form for resource provider plugins.
func MarshalArchive(v *resource.Archive, opts MarshalOptions) (*structpb.Value, error) {
// If we are not providing access to an asset's contents, we simply need to record the fact that this asset existed. Serialize the
// asset with only its hash (if present).
// If we are not providing access to an asset's contents, we simply need to record the fact that this asset existed.
// Serialize the asset with only its hash (if present).
if opts.ElideAssetContents {
v = &resource.Archive{Hash: v.Hash}
} else {

View file

@ -52,6 +52,11 @@ func (s *State) AllInputs() PropertyMap {
return s.Defaults.Merge(s.Inputs)
}
// AddExtras adds an optional set of extra output properties to the current map.
func (s *State) AddExtras(extras PropertyMap) {
s.Outputs = s.Outputs.Merge(extras)
}
// Synthesized returns all of the resource's "synthesized" state; this includes all properties that appeared in the
// default and output set, which may or may not override some or all of those that appeared in the input set.
func (s *State) Synthesized() PropertyMap {

View file

@ -17,6 +17,7 @@ import (
)
// Checkpoint is a serialized deployment target plus a record of the latest deployment.
// nolint: lll
type Checkpoint struct {
Target tokens.QName `json:"target" yaml:"target"` // the target stack name.
Config map[tokens.ModuleMember]config.Value `json:"config,omitempty" yaml:"config,omitempty"` // optional configuration key/values.

View file

@ -7,8 +7,8 @@ import (
"github.com/pulumi/pulumi/pkg/util/contract"
)
// KillChildren calls os.Process.Kill() on every child process of `pid`'s, stoping after the first error (if any). It also only kills
// direct child process, not any children they may have.
// KillChildren calls os.Process.Kill() on every child process of `pid`'s, stoping after the first error (if any). It
// also only kills direct child process, not any children they may have.
func KillChildren(pid int) error {
contract.Failf("KillChidren only implemented on windows")
return nil

View file

@ -10,8 +10,8 @@ import (
ps "github.com/mitchellh/go-ps"
)
// KillChildren calls os.Process.Kill() on every child process of `pid`'s, stoping after the first error (if any). It also only kills
// direct child process, not any children they may have. This function is only implemented on Windows.
// KillChildren calls os.Process.Kill() on every child process of `pid`'s, stoping after the first error (if any). It
// also only kills direct child process, not any children they may have. This function is only implemented on Windows.
func KillChildren(pid int) error {
procs, err := ps.Processes()
if err != nil {

View file

@ -13,8 +13,8 @@ import (
)
const ProjectFile = "Pulumi" // the base name of a project file.
const GitDir = ".git" // the name of the folder git uses to store information
const BookkeepingDir = ".pulumi" // the name of our bookeeping folder, we store all state information here (like .git for git)
const GitDir = ".git" // the name of the folder git uses to store information.
const BookkeepingDir = ".pulumi" // the name of our bookeeping folder, we store state here (like .git for git).
const StackDir = "stacks" // the name of the directory that holds stack information for projects.
const WorkspaceDir = "workspaces" // the name of the directory that holds workspace information for projects.
const RepoFile = "settings.json" // the name of the file that holds information specific to the entire repository.
@ -24,7 +24,9 @@ const WorkspaceFile = "workspace.json" // the name of the file that holds worksp
// DetectPackage locates the closest package from the given path, searching "upwards" in the directory hierarchy. If no
// Project is found, an empty path is returned. If problems are detected, they are logged to the diag.Sink.
func DetectPackage(path string) (string, error) {
return fsutil.WalkUp(path, isProject, func(s string) bool { return !isRepositoryFolder(filepath.Join(s, BookkeepingDir)) })
return fsutil.WalkUp(path, isProject, func(s string) bool {
return !isRepositoryFolder(filepath.Join(s, BookkeepingDir))
})
}
func isGitFolder(path string) bool {

View file

@ -8,11 +8,11 @@ import * as runtime from "./runtime";
* getProject returns the current project name, or the empty string if there is none.
*/
export function getProject(): string {
return runtime.options.project || "";
return runtime.options.project || "project";
}
/**
* getStack returns the current stack name, or the empty string if there is none.
*/
export function getStack(): string {
return runtime.options.stack || "";
return runtime.options.stack || "stack";
}

View file

@ -37,7 +37,7 @@ export abstract class Resource {
// If there wasn't an explicit parent, and a root stack exists, parent to that.
if (!parent) {
parent = runtime.rootPulumiStack;
parent = runtime.getRootPulumiStack();
}
// Now kick off the resource registration. If we are actually performing a deployment, this resource's
@ -74,6 +74,10 @@ export abstract class CustomResource extends Resource {
*/
constructor(t: string, name: string, props?: ComputedValues, parent?: Resource, dependsOn?: Resource[]) {
super(t, name, true, props, parent, dependsOn);
// Unlike components, a custom resource is done as soon as its registration has happened; automatically
// finish registering custom resource state so that subclasses don't need to do so.
runtime.completeResource(this);
}
}
@ -98,16 +102,10 @@ export class ComponentResource extends Resource {
super(t, name, false, props, parent, dependsOn);
}
// recordOutput sets a property named key to the value val in this component's output properties.
protected recordOutput(key: string, val: any): void {
// TODO[pulumi/pulumi#340]: communicate outputs back to the engine via RPC so that it can record them
// inside of the resulting checkpoint file.
}
// recordOutputs sets all object keys and values from obj as properties in this component's output properties.
protected recordOutputs(obj: any): void {
// TODO[pulumi/pulumi#340]: communicate outputs back to the engine via RPC so that it can record them
// inside of the resulting checkpoint file.
// complete finishes the initialization of this resource, with an optional bag of extra output state. All
// component subclasses *must* call this when done, otherwise they will not be present in the checkpoint file.
protected complete(extras?: ComputedValues): void {
runtime.completeResource(this, extras);
}
}

View file

@ -17,6 +17,17 @@ const resproto = require("../proto/resource_pb.js");
let resourceChain: Promise<void> = Promise.resolve();
let resourceChainLabel: string | undefined = undefined;
const registrations = new Set<Resource>();
const pendingRegistrations = new Map<Resource, PendingRegistration>();
interface PendingRegistration {
t: string;
name: string;
props: ComputedValues | undefined;
resolveID: ((v: ID | undefined) => void) | undefined;
resolveProps: Promise<PropertyTransfer>;
}
/**
* registerResource registers a new resource object with a given type t and name. It returns the auto-generated URN
* and the ID that will resolve after the deployment has completed. All properties will be initialized to property
@ -29,8 +40,14 @@ export function registerResource(res: Resource, t: string, name: string, custom:
log.debug(`Registering resource: t=${t}, name=${name}, custom=${custom}` +
(excessiveDebugOutput ? `, props=${JSON.stringify(props)}` : ``));
if (registrations.has(res)) {
throw new Error("Resource has already been registered");
} else if (pendingRegistrations.has(res)) {
throw new Error("Resource is already in the process of being registered");
}
// Pre-allocate an error so we have a clean stack to print even if an asynchronous operation occurs.
const createError: Error = new Error(`Resouce '${name}' [${t}] could not be created`);
const createError: Error = new Error(`Resource '${name}' [${t}] could not be registered`);
// Simply initialize the URN property and get prepared to resolve it later on.
let resolveURN: ((urn: URN | undefined) => void) | undefined;
@ -48,40 +65,34 @@ export function registerResource(res: Resource, t: string, name: string, custom:
);
}
// Ensure we depend on any children plus any explicit dependsOns.
const allDependsOn: Resource[] = [];
if (parent) {
allDependsOn.push(parent);
}
if (dependsOn) {
allDependsOn.push(...dependsOn);
}
// Now "transfer" all input properties; this simply awaits any promises and resolves when they all do.
const transfer: Promise<PropertyTransfer> = debuggablePromise(
transferProperties(res, label, props, allDependsOn), `transferProperties(${label})`);
transferProperties(res, label, props, dependsOn), `transferProperties(${label})`);
// Mark this resource has pending registration.
pendingRegistrations.set(res, {
t: t,
name: name,
props: props,
resolveID: resolveID,
resolveProps: transfer,
});
// Serialize the invocation if necessary.
const resourceOp: Promise<void> = debuggablePromise(resourceChain.then(async () => {
if (serialize()) {
resourceChainLabel = `${name} [${t}]`;
log.debug(`Resource serialization requested: ${resourceChainLabel} is current`);
log.debug(`RegisterResource serialization requested: ${resourceChainLabel} is current`);
}
// Make sure to propagate these no matter what.
let urn: URN | undefined = undefined;
let id: ID | undefined = undefined;
let propsStruct: any | undefined = undefined;
let stable: boolean = false;
let stables: Set<string> | undefined = undefined;
// During a real deployment, the transfer operation may take some time to settle (we may need to wait on
// other in-flight operations. As a result, we can't launch the RPC request until they are done. At the same
// time, we want to give the illusion of non-blocking code, so we return immediately.
let urn: URN | undefined = undefined;
const result: PropertyTransfer = await transfer;
try {
const obj: any = result.obj;
log.debug(`Resource RPC prepared: t=${t}, name=${name}` +
log.debug(`RegisterResource RPC prepared: t=${t}, name=${name}` +
(excessiveDebugOutput ? `, obj=${JSON.stringify(obj)}` : ``));
// Fetch the monitor and make an RPC request.
@ -92,7 +103,7 @@ export function registerResource(res: Resource, t: string, name: string, custom:
parentURN = await parent.urn;
}
const req = new resproto.NewResourceRequest();
const req = new resproto.RegisterResourceRequest();
req.setType(t);
req.setName(name);
req.setParent(parentURN);
@ -100,30 +111,20 @@ export function registerResource(res: Resource, t: string, name: string, custom:
req.setObject(obj);
const resp: any = await debuggablePromise(new Promise((resolve, reject) => {
monitor.newResource(req, (err: Error, innerResponse: any) => {
log.debug(`Resource RPC finished: t=${t}, name=${name}; err: ${err}, resp: ${innerResponse}`);
monitor.registerResource(req, (err: Error, innerResponse: any) => {
log.debug(`RegisterResource RPC finished: t=${t}, name=${name}; ` +
`err: ${err}, resp: ${innerResponse}`);
if (err) {
log.error(`Failed to register new resource '${name}' [${t}]: ${err}`);
log.error(`Failed to register new resource '${name}' [${t}]: ${err.stack}`);
reject(err);
}
else {
resolve(innerResponse);
}
});
}), `monitor.newResource(${label})`);
}), `monitor.registerResource(${label})`);
urn = resp.getUrn();
id = resp.getId();
propsStruct = resp.getObject();
stable = resp.getStable();
const stablesList: string[] | undefined = resp.getStablesList();
if (stablesList) {
stables = new Set<string>();
for (const sta of stablesList) {
stables.add(sta);
}
}
}
else {
// If the monitor doesn't exist, still make sure to resolve all properties to undefined.
@ -131,17 +132,7 @@ export function registerResource(res: Resource, t: string, name: string, custom:
}
}
finally {
// If an ID is present, then it's safe to say it's final, because the resource planner wouldn't hand
// it back to us otherwise (e.g., if the resource was being replaced, it would be missing). If it isn't
// available, ensure the ID gets resolved, just resolve it to undefined (indicating it isn't known).
if (resolveID) {
resolveID(id || undefined);
}
// Propagate any other properties that were given to us as outputs.
resolveProperties(res, result, t, name, props, propsStruct, stable, stables);
// Finally, the resolution will always have a valid URN, even during planning; set it.
// Always make sure to resolve the URN property, even if it is undefined due to a missing monitor.
resolveURN!(urn);
}
}));
@ -167,3 +158,119 @@ export function registerResource(res: Resource, t: string, name: string, custom:
}
}
}
export function completeResource(res: Resource, extras?: ComputedValues) {
const pending: PendingRegistration | undefined = pendingRegistrations.get(res);
if (!pending) {
throw new Error("Resource is not in the process of being registered");
}
pendingRegistrations.delete(res);
if (registrations.has(res)) {
throw new Error("Resource has already completed registration");
}
registrations.add(res);
const t: string = pending.t;
const name: string = pending.name;
const props: ComputedValues | undefined = pending.props;
const label = `resource:${name}[${t}]`;
log.debug(`Completing resource: t=${t}, name=${name}` +
(excessiveDebugOutput ? `, extras=${JSON.stringify(extras)}` : ``));
// Pre-allocate an error so we have a clean stack to print even if an asynchronous operation occurs.
const completeError: Error = new Error(`Resource '${name}' [${t}] could not be completed`);
// Produce the "extra" values, if any, that we'll use in the RPC call.
const transfer: Promise<PropertyTransfer> = debuggablePromise(
transferProperties(undefined, `completeResource`, extras, undefined));
// Serialize the invocation if necessary.
const resourceOp: Promise<void> = debuggablePromise(resourceChain.then(async () => {
// Make sure to propagate these no matter what.
let id: ID | undefined = undefined;
let propsStruct: any | undefined = undefined;
let stable: boolean = false;
let stables: Set<string> | undefined = undefined;
// The registration could very well still be taking place, so we will need to wait for its URN and extras.
const urn: URN = await res.urn;
const extrasResult: PropertyTransfer = await transfer;
const resolveProps: PropertyTransfer = await pending!.resolveProps;
try {
const extrasObj: any = extrasResult.obj;
log.debug(`CompleteResource RPC prepared: urn=${urn}` +
(excessiveDebugOutput ? `, extras=${JSON.stringify(extrasObj)}` : ``));
// Fetch the monitor and make an RPC request.
const monitor: any = getMonitor();
if (monitor) {
const req = new resproto.CompleteResourceRequest();
req.setUrn(urn);
req.setExtras(extrasObj);
const resp: any = await debuggablePromise(new Promise((resolve, reject) => {
monitor.completeResource(req, (err: Error, innerResponse: any) => {
log.debug(`CompleteResource RPC finished: t=${t}, name=${name}; `+
`err: ${err}, resp: ${innerResponse}`);
if (err) {
log.error(`Failed to complete new resource '${name}' [${t}]: ${err.stack}`);
reject(err);
}
else {
resolve(innerResponse);
}
});
}), `monitor.completeResource(${label})`);
id = resp.getId();
propsStruct = resp.getObject();
stable = resp.getStable();
const stablesList: string[] | undefined = resp.getStablesList();
if (stablesList) {
stables = new Set<string>();
for (const sta of stablesList) {
stables.add(sta);
}
}
}
else {
// If the monitor doesn't exist, still make sure to resolve all properties to undefined.
log.warn(`Not sending RPC to monitor -- it doesn't exist: t=${t}, name=${name}`);
}
}
finally {
// If an ID is present, then it's safe to say it's final, because the resource planner wouldn't hand
// it back to us otherwise (e.g., if the resource was being replaced, it would be missing). If it isn't
// available, ensure the ID gets resolved, just resolve it to undefined (indicating it isn't known).
if (pending!.resolveID) {
pending!.resolveID!(id || undefined);
}
// Propagate any other properties that were given to us as outputs.
resolveProperties(res, resolveProps, t, name, props, propsStruct, stable, stables);
}
}));
// If any errors make it this far, ensure we log them.
const finalOp: Promise<void> = debuggablePromise(resourceOp.catch((err: Error) => {
// At this point, we've gone fully asynchronous, and the stack is missing. To make it easier
// to debug which resource this came from, we will emit the original stack trace too.
log.error(errorString(completeError));
log.error(`Failed to complete resource '${name}' [${t}]: ${errorString(err)}`);
}));
// Ensure the process won't exit until this registerResource call finishes and resolve it when appropriate.
const done: () => void = rpcKeepAlive();
finalOp.then(() => { done(); }, () => { done(); });
// If serialization is requested, wait for the prior resource operation to finish before we proceed, serializing
// them, and make this the current resource operation so that everybody piles up on it.
if (serialize()) {
resourceChain = finalOp;
if (resourceChainLabel) {
log.debug(`Resource serialization requested: ${name} [${t}] is behind ${resourceChainLabel}`);
}
}
}

View file

@ -7,7 +7,10 @@ import { ComponentResource, Resource } from "../resource";
* rootPulumiStack is a root stack that will be used automatically as resource parents. This ensures that all
* resources without explicit parents are parented to a common stack type.
*/
export let rootPulumiStack: Resource | undefined;
let rootPulumiStack: Resource | undefined;
export function getRootPulumiStack(): Resource | undefined {
return rootPulumiStack;
}
/**
* rootPulumiStackTypeName is the type name that should be used to construct the root component in the tree of Pulumi
@ -32,9 +35,9 @@ class Stack extends ComponentResource {
throw new Error("Only one root Pulumi Stack may be active at once");
}
try {
rootPulumiStack = this; // install ourselves as the current root.
const outputs = init(); // run the init code.
super.recordOutputs(outputs); // save the outputs for this component to whatever the init returned.
rootPulumiStack = this; // install ourselves as the current root.
const outputs: any = init(); // run the init code.
super.complete(outputs); // save the outputs for this component to whatever the init returned.
}
finally {
rootPulumiStack = undefined;

View file

@ -9,8 +9,10 @@ import { asyncTest } from "../../util";
const gstruct = require("google-protobuf/google/protobuf/struct_pb.js");
const grpc = require("grpc");
const langrpc = require("../../../proto/languages_grpc_pb.js");
const langproto = require("../../../proto/languages_pb.js");
const langrpc = require("../../../proto/language_grpc_pb.js");
const langproto = require("../../../proto/language_pb.js");
const resrpc = require("../../../proto/resource_grpc_pb.js");
const resproto = require("../../../proto/resource_pb.js");
interface RunCase {
project?: string;
@ -22,8 +24,13 @@ interface RunCase {
expectError?: string;
expectResourceCount?: number;
invoke?: (ctx: any, tok: string, args: any) => { failures: any, ret: any };
createResource?: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
id?: ID, urn?: URN, props?: any };
registerResource?: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => URN | undefined;
completeResource?: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
id: ID | undefined, props: any | undefined };
}
function makeUrn(t: string, name: string): URN {
return `${t}::${name}`;
}
describe("rpc", () => {
@ -50,17 +57,23 @@ describe("rpc", () => {
"one_resource": {
program: path.join(base, "001.one_resource"),
expectResourceCount: 1,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(name, "testResource1");
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
completeResource: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(name, "testResource1");
assert.strictEqual(urn, makeUrn(t, name));
return { id: undefined, props: undefined };
},
},
// A program that allocates ten simple resources.
"ten_resources": {
program: path.join(base, "002.ten_resources"),
expectResourceCount: 10,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
if (ctx.seen) {
assert(!ctx.seen[name],
@ -76,14 +89,26 @@ describe("rpc", () => {
assert(!isNaN(seqnum),
`Expected ${name} to be of the form ${prefix}N; missing N seqnum`);
ctx.seen[name] = true;
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
completeResource: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(urn, makeUrn(t, name));
const prefix = "testResource";
assert.strictEqual(name.substring(0, prefix.length), prefix,
`Expected ${name} to be of the form ${prefix}N; missing prefix`);
const seqnum = parseInt(name.substring(prefix.length), 10);
assert(!isNaN(seqnum),
`Expected ${name} to be of the form ${prefix}N; missing N seqnum`);
ctx.seen[name] = true;
return { id: undefined, props: undefined };
},
},
// A program that allocates a complex resource with lots of input and output properties.
"one_complex_resource": {
program: path.join(base, "003.one_complex_resource"),
expectResourceCount: 1,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(name, "testResource1");
assert.deepEqual(res, {
@ -101,9 +126,29 @@ describe("rpc", () => {
o: { z: "x" },
},
});
return makeUrn(t, name);
},
completeResource: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(name, "testResource1");
assert.strictEqual(urn, makeUrn(t, name));
assert.deepEqual(res, {
inpropB1: false,
inpropB2: true,
inpropN: 42,
inpropS: "a string",
inpropA: [ true, 99, "what a great property" ],
inpropO: {
b1: false,
b2: true,
n: 42,
s: "another string",
a: [ 66, false, "strings galore" ],
o: { z: "x" },
},
});
return {
id: name,
urn: t + "::" + name,
props: {
outprop1: "output properties ftw",
outprop2: 998.6,
@ -115,7 +160,7 @@ describe("rpc", () => {
"ten_complex_resources": {
program: path.join(base, "004.ten_complex_resources"),
expectResourceCount: 10,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
if (ctx.seen) {
assert(!ctx.seen[name],
@ -147,9 +192,36 @@ describe("rpc", () => {
o: { z: "x" },
},
});
return makeUrn(t, name);
},
completeResource: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(urn, makeUrn(t, name));
const prefix = "testResource";
assert.strictEqual(name.substring(0, prefix.length), prefix,
`Expected ${name} to be of the form ${prefix}N; missing prefix`);
const seqnum = parseInt(name.substring(prefix.length), 10);
assert(!isNaN(seqnum),
`Expected ${name} to be of the form ${prefix}N; missing N seqnum`);
ctx.seen[name] = true;
assert.deepEqual(res, {
inseq: seqnum,
inpropB1: false,
inpropB2: true,
inpropN: 42,
inpropS: "a string",
inpropA: [ true, 99, "what a great property" ],
inpropO: {
b1: false,
b2: true,
n: 42,
s: "another string",
a: [ 66, false, "strings galore" ],
o: { z: "x" },
},
});
return {
id: name,
urn: t + "::" + name,
props: {
outseq: seqnum,
outprop1: "output properties ftw",
@ -162,17 +234,12 @@ describe("rpc", () => {
"resource_thens": {
program: path.join(base, "005.resource_thens"),
expectResourceCount: 2,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
switch (t) {
case "test:index:ResourceA": {
assert.strictEqual(name, "resourceA");
assert.deepEqual(res, { inprop: 777 });
const result: any = { urn: t + "::" + name };
if (!dryrun) {
result.id = name;
result.props = { outprop: "output yeah" };
}
return result;
break;
}
case "test:index:ResourceB": {
assert.strictEqual(name, "resourceB");
@ -190,7 +257,45 @@ describe("rpc", () => {
otherOut: "output yeah",
});
}
const result: any = { urn: t + "::" + name };
break;
}
default:
assert.fail(`Unrecognized resource type ${t}`);
throw new Error();
}
return makeUrn(t, name);
},
completeResource: (ctx: any, dryrun: boolean, urn: string, t: string, name: string, res: any, ex: any) => {
switch (t) {
case "test:index:ResourceA": {
assert.strictEqual(name, "resourceA");
assert.strictEqual(urn, makeUrn(t, name));
assert.deepEqual(res, { inprop: 777 });
const result: any = {};
if (!dryrun) {
result.id = name;
result.props = { outprop: "output yeah" };
}
return result;
}
case "test:index:ResourceB": {
assert.strictEqual(name, "resourceB");
assert.strictEqual(urn, makeUrn(t, name));
if (dryrun) {
// If this is a dry-run, we won't have the real values:
assert.deepEqual(res, {
otherIn: runtime.unknownComputedValue,
otherOut: runtime.unknownComputedValue,
});
}
else {
// Otherwise, we will:
assert.deepEqual(res, {
otherIn: 777,
otherOut: "output yeah",
});
}
const result: any = {};
if (!dryrun) {
result.id = name;
}
@ -205,7 +310,7 @@ describe("rpc", () => {
"input_output": {
pwd: path.join(base, "006.asset"),
expectResourceCount: 1,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:FileResource");
assert.strictEqual(name, "file1");
assert.deepEqual(res, {
@ -214,26 +319,26 @@ describe("rpc", () => {
path: "./testdata.txt",
},
});
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
},
"promises_io": {
pwd: path.join(base, "007.promises_io"),
expectResourceCount: 1,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:FileResource");
assert.strictEqual(name, "file1");
assert.deepEqual(res, {
data: "The test worked!\n\nIf you can see some data!\n\n",
});
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
},
// A program that allocates ten simple resources that use dependsOn to depend on one another.
"ten_depends_on_resources": {
program: path.join(base, "008.ten_depends_on_resources"),
expectResourceCount: 10,
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
if (ctx.seen) {
assert(!ctx.seen[name],
@ -249,7 +354,7 @@ describe("rpc", () => {
assert(!isNaN(seqnum),
`Expected ${name} to be of the form ${prefix}N; missing N seqnum`);
ctx.seen[name] = true;
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
},
// A simple test of the invocation RPC pathways.
@ -265,10 +370,10 @@ describe("rpc", () => {
});
return { failures: undefined, ret: args };
},
createResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
assert.strictEqual(t, "test:index:MyResource");
assert.strictEqual(name, "testResource1");
return { id: undefined, urn: undefined, props: undefined };
return makeUrn(t, name);
},
},
// Simply test that certain runtime properties are available.
@ -298,12 +403,14 @@ describe("rpc", () => {
console.log(dryrun ? "PREVIEW:" : "UPDATE:");
// First we need to mock the resource monitor.
const ctx = {};
let rescnt = 0;
const ctx: any = {};
const regs: any = {};
let regCnt = 0;
let compCnt = 0;
const monitor = createMockResourceMonitor(
// Invoke callback
(call: any, callback: any) => {
const resp = new langproto.InvokeResponse();
const resp = new resproto.InvokeResponse();
if (opts.invoke) {
const req: any = call.request;
const args: any = req.getArgs().toJavaScript();
@ -314,21 +421,43 @@ describe("rpc", () => {
}
callback(undefined, resp);
},
// NewResources callback
// RegisterResource callback
(call: any, callback: any) => {
const resp = new langproto.NewResourceResponse();
const resp = new resproto.RegisterResourceResponse();
const req: any = call.request;
// Skip the automatically generated root component resource.
if (req.getType() !== runtime.rootPulumiStackTypeName) {
if (opts.createResource) {
const res: any = req.getObject().toJavaScript();
const { id, urn, props } =
opts.createResource(ctx, dryrun, req.getType(), req.getName(), res);
resp.setId(id);
if (opts.registerResource) {
const t = req.getType();
const name = req.getName();
const props: any = req.getObject().toJavaScript();
const urn = opts.registerResource(ctx, dryrun, t, name, props);
resp.setUrn(urn);
if (urn) {
regs[urn] = { t: t, name: name, props: props };
}
}
regCnt++;
}
callback(undefined, resp);
},
(call: any, callback: any) => {
const resp = new resproto.CompleteResourceResponse();
const req: any = call.request;
const urn = req.getUrn();
const res = regs[urn];
if (res) {
if (opts.completeResource) {
let extras: any;
if (req.getExtras()) {
extras = req.getExtras().toJavaScript();
}
const { id, props } =
opts.completeResource(ctx, dryrun, urn, res.t, res.name, res.props, extras);
resp.setId(id);
resp.setObject(gstruct.Struct.fromJavaScript(props));
}
rescnt++;
compCnt++;
}
callback(undefined, resp);
},
@ -358,8 +487,10 @@ describe("rpc", () => {
if (expectResourceCount === undefined) {
expectResourceCount = 0;
}
assert.strictEqual(rescnt, expectResourceCount,
`Expected exactly ${expectResourceCount} resources; got ${rescnt}`);
assert.strictEqual(regCnt, expectResourceCount,
`Expected exactly ${expectResourceCount} resource registrations; got ${regCnt}`);
assert.strictEqual(compCnt, expectResourceCount,
`Expected exactly ${expectResourceCount} resource completions; got ${compCnt}`);
// Finally, tear down everything so each test case starts anew.
await new Promise<void>((resolve, reject) => {
@ -420,12 +551,14 @@ function mockRun(langHostClient: any, opts: RunCase, dryrun: boolean): Promise<s
function createMockResourceMonitor(
invokeCallback: (call: any, request: any) => any,
newResourceCallback: (call: any, request: any) => any): { server: any, addr: string } {
registerResourceCallback: (call: any, request: any) => any,
completeResourceCallback: (call: any, request: any) => any): { server: any, addr: string } {
// The resource monitor is hosted in the current process so it can record state, etc.
const server = new grpc.Server();
server.addService(langrpc.ResourceMonitorService, {
server.addService(resrpc.ResourceMonitorService, {
invoke: invokeCallback,
newResource: newResourceCallback,
registerResource: registerResourceCallback,
completeResource: completeResourceCallback,
});
const port = server.bind("0.0.0.0:0", grpc.ServerCredentials.createInsecure());
server.start();

View file

@ -32,8 +32,10 @@ It has these top-level messages:
UpdateRequest
UpdateResponse
DeleteRequest
NewResourceRequest
NewResourceResponse
RegisterResourceRequest
RegisterResourceResponse
CompleteResourceRequest
CompleteResourceResponse
*/
package pulumirpc

View file

@ -19,8 +19,8 @@ var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
// NewResourceRequest contains information about a resource object that was newly allocated.
type NewResourceRequest struct {
// RegisterResourceRequest contains information about a resource object that was newly allocated.
type RegisterResourceRequest struct {
Type string `protobuf:"bytes,1,opt,name=type" json:"type,omitempty"`
Name string `protobuf:"bytes,2,opt,name=name" json:"name,omitempty"`
Parent string `protobuf:"bytes,3,opt,name=parent" json:"parent,omitempty"`
@ -28,89 +28,125 @@ type NewResourceRequest struct {
Object *google_protobuf.Struct `protobuf:"bytes,5,opt,name=object" json:"object,omitempty"`
}
func (m *NewResourceRequest) Reset() { *m = NewResourceRequest{} }
func (m *NewResourceRequest) String() string { return proto.CompactTextString(m) }
func (*NewResourceRequest) ProtoMessage() {}
func (*NewResourceRequest) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{0} }
func (m *RegisterResourceRequest) Reset() { *m = RegisterResourceRequest{} }
func (m *RegisterResourceRequest) String() string { return proto.CompactTextString(m) }
func (*RegisterResourceRequest) ProtoMessage() {}
func (*RegisterResourceRequest) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{0} }
func (m *NewResourceRequest) GetType() string {
func (m *RegisterResourceRequest) GetType() string {
if m != nil {
return m.Type
}
return ""
}
func (m *NewResourceRequest) GetName() string {
func (m *RegisterResourceRequest) GetName() string {
if m != nil {
return m.Name
}
return ""
}
func (m *NewResourceRequest) GetParent() string {
func (m *RegisterResourceRequest) GetParent() string {
if m != nil {
return m.Parent
}
return ""
}
func (m *NewResourceRequest) GetCustom() bool {
func (m *RegisterResourceRequest) GetCustom() bool {
if m != nil {
return m.Custom
}
return false
}
func (m *NewResourceRequest) GetObject() *google_protobuf.Struct {
func (m *RegisterResourceRequest) GetObject() *google_protobuf.Struct {
if m != nil {
return m.Object
}
return nil
}
// NewResourceResponse reflects back the properties initialized during creation, if applicable.
type NewResourceResponse struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
Urn string `protobuf:"bytes,2,opt,name=urn" json:"urn,omitempty"`
Object *google_protobuf.Struct `protobuf:"bytes,3,opt,name=object" json:"object,omitempty"`
Stable bool `protobuf:"varint,4,opt,name=stable" json:"stable,omitempty"`
Stables []string `protobuf:"bytes,5,rep,name=stables" json:"stables,omitempty"`
// RegisterResourceResponse reflects back the properties initialized during creation, if applicable.
type RegisterResourceResponse struct {
Urn string `protobuf:"bytes,1,opt,name=urn" json:"urn,omitempty"`
}
func (m *NewResourceResponse) Reset() { *m = NewResourceResponse{} }
func (m *NewResourceResponse) String() string { return proto.CompactTextString(m) }
func (*NewResourceResponse) ProtoMessage() {}
func (*NewResourceResponse) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{1} }
func (m *RegisterResourceResponse) Reset() { *m = RegisterResourceResponse{} }
func (m *RegisterResourceResponse) String() string { return proto.CompactTextString(m) }
func (*RegisterResourceResponse) ProtoMessage() {}
func (*RegisterResourceResponse) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{1} }
func (m *NewResourceResponse) GetId() string {
if m != nil {
return m.Id
}
return ""
}
func (m *NewResourceResponse) GetUrn() string {
func (m *RegisterResourceResponse) GetUrn() string {
if m != nil {
return m.Urn
}
return ""
}
func (m *NewResourceResponse) GetObject() *google_protobuf.Struct {
// CompleteResourceRequest completes the registration of a resource, and optionally adds extra derived output
// properties to an existing resource that is in flight. It must be called once per registration.
type CompleteResourceRequest struct {
Urn string `protobuf:"bytes,1,opt,name=urn" json:"urn,omitempty"`
Extras *google_protobuf.Struct `protobuf:"bytes,2,opt,name=extras" json:"extras,omitempty"`
}
func (m *CompleteResourceRequest) Reset() { *m = CompleteResourceRequest{} }
func (m *CompleteResourceRequest) String() string { return proto.CompactTextString(m) }
func (*CompleteResourceRequest) ProtoMessage() {}
func (*CompleteResourceRequest) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{2} }
func (m *CompleteResourceRequest) GetUrn() string {
if m != nil {
return m.Urn
}
return ""
}
func (m *CompleteResourceRequest) GetExtras() *google_protobuf.Struct {
if m != nil {
return m.Extras
}
return nil
}
// CompleteResourceResponse is returned by the engine after a resource is completed. It includes any state
// that was populated by the resource provider so that the language engine can blit it into the resource objects.
type CompleteResourceResponse struct {
Id string `protobuf:"bytes,1,opt,name=id" json:"id,omitempty"`
Object *google_protobuf.Struct `protobuf:"bytes,2,opt,name=object" json:"object,omitempty"`
Stable bool `protobuf:"varint,3,opt,name=stable" json:"stable,omitempty"`
Stables []string `protobuf:"bytes,4,rep,name=stables" json:"stables,omitempty"`
}
func (m *CompleteResourceResponse) Reset() { *m = CompleteResourceResponse{} }
func (m *CompleteResourceResponse) String() string { return proto.CompactTextString(m) }
func (*CompleteResourceResponse) ProtoMessage() {}
func (*CompleteResourceResponse) Descriptor() ([]byte, []int) { return fileDescriptor4, []int{3} }
func (m *CompleteResourceResponse) GetId() string {
if m != nil {
return m.Id
}
return ""
}
func (m *CompleteResourceResponse) GetObject() *google_protobuf.Struct {
if m != nil {
return m.Object
}
return nil
}
func (m *NewResourceResponse) GetStable() bool {
func (m *CompleteResourceResponse) GetStable() bool {
if m != nil {
return m.Stable
}
return false
}
func (m *NewResourceResponse) GetStables() []string {
func (m *CompleteResourceResponse) GetStables() []string {
if m != nil {
return m.Stables
}
@ -118,8 +154,10 @@ func (m *NewResourceResponse) GetStables() []string {
}
func init() {
proto.RegisterType((*NewResourceRequest)(nil), "pulumirpc.NewResourceRequest")
proto.RegisterType((*NewResourceResponse)(nil), "pulumirpc.NewResourceResponse")
proto.RegisterType((*RegisterResourceRequest)(nil), "pulumirpc.RegisterResourceRequest")
proto.RegisterType((*RegisterResourceResponse)(nil), "pulumirpc.RegisterResourceResponse")
proto.RegisterType((*CompleteResourceRequest)(nil), "pulumirpc.CompleteResourceRequest")
proto.RegisterType((*CompleteResourceResponse)(nil), "pulumirpc.CompleteResourceResponse")
}
// Reference imports to suppress errors if they are not otherwise used.
@ -134,7 +172,8 @@ const _ = grpc.SupportPackageIsVersion4
type ResourceMonitorClient interface {
Invoke(ctx context.Context, in *InvokeRequest, opts ...grpc.CallOption) (*InvokeResponse, error)
NewResource(ctx context.Context, in *NewResourceRequest, opts ...grpc.CallOption) (*NewResourceResponse, error)
RegisterResource(ctx context.Context, in *RegisterResourceRequest, opts ...grpc.CallOption) (*RegisterResourceResponse, error)
CompleteResource(ctx context.Context, in *CompleteResourceRequest, opts ...grpc.CallOption) (*CompleteResourceResponse, error)
}
type resourceMonitorClient struct {
@ -154,9 +193,18 @@ func (c *resourceMonitorClient) Invoke(ctx context.Context, in *InvokeRequest, o
return out, nil
}
func (c *resourceMonitorClient) NewResource(ctx context.Context, in *NewResourceRequest, opts ...grpc.CallOption) (*NewResourceResponse, error) {
out := new(NewResourceResponse)
err := grpc.Invoke(ctx, "/pulumirpc.ResourceMonitor/NewResource", in, out, c.cc, opts...)
func (c *resourceMonitorClient) RegisterResource(ctx context.Context, in *RegisterResourceRequest, opts ...grpc.CallOption) (*RegisterResourceResponse, error) {
out := new(RegisterResourceResponse)
err := grpc.Invoke(ctx, "/pulumirpc.ResourceMonitor/RegisterResource", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *resourceMonitorClient) CompleteResource(ctx context.Context, in *CompleteResourceRequest, opts ...grpc.CallOption) (*CompleteResourceResponse, error) {
out := new(CompleteResourceResponse)
err := grpc.Invoke(ctx, "/pulumirpc.ResourceMonitor/CompleteResource", in, out, c.cc, opts...)
if err != nil {
return nil, err
}
@ -167,7 +215,8 @@ func (c *resourceMonitorClient) NewResource(ctx context.Context, in *NewResource
type ResourceMonitorServer interface {
Invoke(context.Context, *InvokeRequest) (*InvokeResponse, error)
NewResource(context.Context, *NewResourceRequest) (*NewResourceResponse, error)
RegisterResource(context.Context, *RegisterResourceRequest) (*RegisterResourceResponse, error)
CompleteResource(context.Context, *CompleteResourceRequest) (*CompleteResourceResponse, error)
}
func RegisterResourceMonitorServer(s *grpc.Server, srv ResourceMonitorServer) {
@ -192,20 +241,38 @@ func _ResourceMonitor_Invoke_Handler(srv interface{}, ctx context.Context, dec f
return interceptor(ctx, in, info, handler)
}
func _ResourceMonitor_NewResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(NewResourceRequest)
func _ResourceMonitor_RegisterResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(RegisterResourceRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ResourceMonitorServer).NewResource(ctx, in)
return srv.(ResourceMonitorServer).RegisterResource(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/pulumirpc.ResourceMonitor/NewResource",
FullMethod: "/pulumirpc.ResourceMonitor/RegisterResource",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ResourceMonitorServer).NewResource(ctx, req.(*NewResourceRequest))
return srv.(ResourceMonitorServer).RegisterResource(ctx, req.(*RegisterResourceRequest))
}
return interceptor(ctx, in, info, handler)
}
func _ResourceMonitor_CompleteResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(CompleteResourceRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(ResourceMonitorServer).CompleteResource(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/pulumirpc.ResourceMonitor/CompleteResource",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(ResourceMonitorServer).CompleteResource(ctx, req.(*CompleteResourceRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -219,8 +286,12 @@ var _ResourceMonitor_serviceDesc = grpc.ServiceDesc{
Handler: _ResourceMonitor_Invoke_Handler,
},
{
MethodName: "NewResource",
Handler: _ResourceMonitor_NewResource_Handler,
MethodName: "RegisterResource",
Handler: _ResourceMonitor_RegisterResource_Handler,
},
{
MethodName: "CompleteResource",
Handler: _ResourceMonitor_CompleteResource_Handler,
},
},
Streams: []grpc.StreamDesc{},
@ -230,25 +301,28 @@ var _ResourceMonitor_serviceDesc = grpc.ServiceDesc{
func init() { proto.RegisterFile("resource.proto", fileDescriptor4) }
var fileDescriptor4 = []byte{
// 314 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x8c, 0x91, 0x4f, 0x4e, 0xf3, 0x30,
0x10, 0xc5, 0x3f, 0x37, 0x6d, 0x3e, 0xea, 0x4a, 0x05, 0x0d, 0x12, 0x98, 0x08, 0x50, 0xd4, 0x55,
0x56, 0xa9, 0x54, 0x0e, 0xc0, 0x9a, 0x05, 0x5d, 0x84, 0x13, 0x34, 0xe9, 0x50, 0x05, 0x5a, 0x8f,
0xf1, 0x9f, 0x22, 0xae, 0x82, 0xc4, 0x86, 0x93, 0xa2, 0xc4, 0x0e, 0x0a, 0x02, 0x24, 0x76, 0xef,
0xbd, 0x19, 0x79, 0x7e, 0xe3, 0xe1, 0x53, 0x8d, 0x86, 0x9c, 0xae, 0x30, 0x57, 0x9a, 0x2c, 0xc1,
0x58, 0xb9, 0xad, 0xdb, 0xd5, 0x5a, 0x55, 0xc9, 0xf9, 0x86, 0x68, 0xb3, 0xc5, 0x79, 0x5b, 0x28,
0xdd, 0xfd, 0xdc, 0x58, 0xed, 0x2a, 0xeb, 0x1b, 0x93, 0xa9, 0xd2, 0xb4, 0xaf, 0xd7, 0xa8, 0xbd,
0x9f, 0xbd, 0x31, 0x0e, 0x4b, 0x7c, 0x2e, 0xc2, 0x73, 0x05, 0x3e, 0x39, 0x34, 0x16, 0x80, 0x0f,
0xed, 0x8b, 0x42, 0xc1, 0x52, 0x96, 0x8d, 0x8b, 0x56, 0x37, 0x99, 0x5c, 0xed, 0x50, 0x0c, 0x7c,
0xd6, 0x68, 0x38, 0xe1, 0xb1, 0x5a, 0x69, 0x94, 0x56, 0x44, 0x6d, 0x1a, 0x5c, 0x93, 0x57, 0xce,
0x58, 0xda, 0x89, 0x61, 0xca, 0xb2, 0x83, 0x22, 0x38, 0x98, 0xf3, 0x98, 0xca, 0x07, 0xac, 0xac,
0x18, 0xa5, 0x2c, 0x9b, 0x2c, 0x4e, 0x73, 0x4f, 0x9b, 0x77, 0xb4, 0xf9, 0x5d, 0x4b, 0x5b, 0x84,
0xb6, 0xd9, 0x2b, 0xe3, 0xc7, 0x5f, 0xf8, 0x8c, 0x22, 0x69, 0x10, 0xa6, 0x7c, 0x50, 0xaf, 0x03,
0xde, 0xa0, 0x5e, 0xc3, 0x11, 0x8f, 0x9c, 0x96, 0x81, 0xad, 0x91, 0xbd, 0x51, 0xd1, 0x9f, 0x46,
0x35, 0xcc, 0xc6, 0xae, 0xca, 0x2d, 0x76, 0xcc, 0xde, 0x81, 0xe0, 0xff, 0xbd, 0x32, 0x62, 0x94,
0x46, 0xd9, 0xb8, 0xe8, 0xec, 0xe2, 0x9d, 0xf1, 0xc3, 0x8e, 0xec, 0x96, 0x64, 0x6d, 0x49, 0xc3,
0x35, 0x8f, 0x6f, 0xe4, 0x9e, 0x1e, 0x11, 0x44, 0xfe, 0x79, 0x94, 0xdc, 0x47, 0xe1, 0x77, 0x93,
0xb3, 0x1f, 0x2a, 0x7e, 0xaf, 0xd9, 0x3f, 0x58, 0xf2, 0x49, 0x6f, 0x61, 0xb8, 0xe8, 0xf5, 0x7e,
0x3f, 0x54, 0x72, 0xf9, 0x5b, 0xb9, 0x7b, 0xaf, 0x8c, 0xdb, 0x7d, 0xaf, 0x3e, 0x02, 0x00, 0x00,
0xff, 0xff, 0x82, 0xe1, 0x98, 0x15, 0x33, 0x02, 0x00, 0x00,
// 368 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x8c, 0x52, 0xcb, 0x4e, 0xeb, 0x30,
0x14, 0x6c, 0xd2, 0xde, 0xdc, 0xf6, 0x5c, 0xa9, 0xb7, 0xf2, 0x82, 0x9a, 0x88, 0x45, 0x14, 0x36,
0x59, 0xa0, 0x54, 0x2a, 0x1f, 0xc0, 0x82, 0x15, 0x0b, 0x36, 0x66, 0x0b, 0x8b, 0x24, 0x3d, 0x54,
0x81, 0x24, 0x36, 0x7e, 0x54, 0xf0, 0x11, 0x48, 0x7c, 0x03, 0x5f, 0x8a, 0x12, 0xbb, 0xa8, 0x0f,
0x5a, 0xb1, 0x3b, 0x67, 0x3c, 0x1e, 0xcd, 0x8c, 0x0d, 0x63, 0x89, 0x8a, 0x1b, 0x59, 0x60, 0x2a,
0x24, 0xd7, 0x9c, 0x8c, 0x84, 0xa9, 0x4c, 0x5d, 0x4a, 0x51, 0x84, 0x67, 0x4b, 0xce, 0x97, 0x15,
0xce, 0xba, 0x83, 0xdc, 0x3c, 0xce, 0x94, 0x96, 0xa6, 0xd0, 0x96, 0x18, 0x8e, 0x85, 0xe4, 0xab,
0x72, 0x81, 0xd2, 0xee, 0xf1, 0xa7, 0x07, 0x53, 0x86, 0xcb, 0x52, 0x69, 0x94, 0xcc, 0x69, 0x32,
0x7c, 0x31, 0xa8, 0x34, 0x21, 0x30, 0xd0, 0x6f, 0x02, 0xa9, 0x17, 0x79, 0xc9, 0x88, 0x75, 0x73,
0x8b, 0x35, 0x59, 0x8d, 0xd4, 0xb7, 0x58, 0x3b, 0x93, 0x13, 0x08, 0x44, 0x26, 0xb1, 0xd1, 0xb4,
0xdf, 0xa1, 0x6e, 0x6b, 0xf1, 0xc2, 0x28, 0xcd, 0x6b, 0x3a, 0x88, 0xbc, 0x64, 0xc8, 0xdc, 0x46,
0x66, 0x10, 0xf0, 0xfc, 0x09, 0x0b, 0x4d, 0xff, 0x44, 0x5e, 0xf2, 0x6f, 0x3e, 0x4d, 0xad, 0xe5,
0x74, 0x6d, 0x39, 0xbd, 0xeb, 0x2c, 0x33, 0x47, 0x8b, 0x2f, 0x80, 0xee, 0x7b, 0x54, 0x82, 0x37,
0x0a, 0xc9, 0x04, 0xfa, 0x46, 0x36, 0xce, 0x63, 0x3b, 0xc6, 0xf7, 0x30, 0xbd, 0xe6, 0xb5, 0xa8,
0x50, 0xe3, 0x6e, 0xa2, 0x3d, 0x72, 0xeb, 0x05, 0x5f, 0xb5, 0xcc, 0x54, 0x97, 0xe8, 0x98, 0x17,
0x4b, 0x8b, 0xdf, 0x3d, 0xa0, 0xfb, 0xf2, 0xce, 0xcc, 0x18, 0xfc, 0x72, 0xe1, 0xe4, 0xfd, 0x72,
0xb1, 0x91, 0xd4, 0xff, 0x55, 0xd2, 0xb6, 0x32, 0xa5, 0xb3, 0xbc, 0xc2, 0xae, 0xca, 0x21, 0x73,
0x1b, 0xa1, 0xf0, 0xd7, 0x4e, 0x8a, 0x0e, 0xa2, 0x7e, 0x32, 0x62, 0xeb, 0x75, 0xfe, 0xe1, 0xc3,
0xff, 0xb5, 0x8f, 0x5b, 0xde, 0x94, 0x9a, 0x4b, 0x72, 0x05, 0xc1, 0x4d, 0xb3, 0xe2, 0xcf, 0x48,
0x68, 0xfa, 0xfd, 0x31, 0x52, 0x0b, 0xb9, 0x2a, 0xc2, 0xd3, 0x1f, 0x4e, 0x6c, 0x8a, 0xb8, 0x47,
0x1e, 0x60, 0xb2, 0x5b, 0x38, 0x89, 0x37, 0x2e, 0x1c, 0xf8, 0x31, 0xe1, 0xf9, 0x51, 0xce, 0xa6,
0xfc, 0x6e, 0x85, 0x5b, 0xf2, 0x07, 0x9e, 0x6f, 0x4b, 0xfe, 0xd0, 0x1b, 0xc4, 0xbd, 0x3c, 0xe8,
0xda, 0xbd, 0xfc, 0x0a, 0x00, 0x00, 0xff, 0xff, 0x8d, 0xb6, 0x99, 0xaa, 0x25, 0x03, 0x00, 0x00,
}

View file

@ -9,6 +9,28 @@ var resource_pb = require('./resource_pb.js');
var google_protobuf_struct_pb = require('google-protobuf/google/protobuf/struct_pb.js');
var provider_pb = require('./provider_pb.js');
function serialize_pulumirpc_CompleteResourceRequest(arg) {
if (!(arg instanceof resource_pb.CompleteResourceRequest)) {
throw new Error('Expected argument of type pulumirpc.CompleteResourceRequest');
}
return new Buffer(arg.serializeBinary());
}
function deserialize_pulumirpc_CompleteResourceRequest(buffer_arg) {
return resource_pb.CompleteResourceRequest.deserializeBinary(new Uint8Array(buffer_arg));
}
function serialize_pulumirpc_CompleteResourceResponse(arg) {
if (!(arg instanceof resource_pb.CompleteResourceResponse)) {
throw new Error('Expected argument of type pulumirpc.CompleteResourceResponse');
}
return new Buffer(arg.serializeBinary());
}
function deserialize_pulumirpc_CompleteResourceResponse(buffer_arg) {
return resource_pb.CompleteResourceResponse.deserializeBinary(new Uint8Array(buffer_arg));
}
function serialize_pulumirpc_InvokeRequest(arg) {
if (!(arg instanceof provider_pb.InvokeRequest)) {
throw new Error('Expected argument of type pulumirpc.InvokeRequest');
@ -31,26 +53,26 @@ function deserialize_pulumirpc_InvokeResponse(buffer_arg) {
return provider_pb.InvokeResponse.deserializeBinary(new Uint8Array(buffer_arg));
}
function serialize_pulumirpc_NewResourceRequest(arg) {
if (!(arg instanceof resource_pb.NewResourceRequest)) {
throw new Error('Expected argument of type pulumirpc.NewResourceRequest');
function serialize_pulumirpc_RegisterResourceRequest(arg) {
if (!(arg instanceof resource_pb.RegisterResourceRequest)) {
throw new Error('Expected argument of type pulumirpc.RegisterResourceRequest');
}
return new Buffer(arg.serializeBinary());
}
function deserialize_pulumirpc_NewResourceRequest(buffer_arg) {
return resource_pb.NewResourceRequest.deserializeBinary(new Uint8Array(buffer_arg));
function deserialize_pulumirpc_RegisterResourceRequest(buffer_arg) {
return resource_pb.RegisterResourceRequest.deserializeBinary(new Uint8Array(buffer_arg));
}
function serialize_pulumirpc_NewResourceResponse(arg) {
if (!(arg instanceof resource_pb.NewResourceResponse)) {
throw new Error('Expected argument of type pulumirpc.NewResourceResponse');
function serialize_pulumirpc_RegisterResourceResponse(arg) {
if (!(arg instanceof resource_pb.RegisterResourceResponse)) {
throw new Error('Expected argument of type pulumirpc.RegisterResourceResponse');
}
return new Buffer(arg.serializeBinary());
}
function deserialize_pulumirpc_NewResourceResponse(buffer_arg) {
return resource_pb.NewResourceResponse.deserializeBinary(new Uint8Array(buffer_arg));
function deserialize_pulumirpc_RegisterResourceResponse(buffer_arg) {
return resource_pb.RegisterResourceResponse.deserializeBinary(new Uint8Array(buffer_arg));
}
@ -67,16 +89,27 @@ var ResourceMonitorService = exports.ResourceMonitorService = {
responseSerialize: serialize_pulumirpc_InvokeResponse,
responseDeserialize: deserialize_pulumirpc_InvokeResponse,
},
newResource: {
path: '/pulumirpc.ResourceMonitor/NewResource',
registerResource: {
path: '/pulumirpc.ResourceMonitor/RegisterResource',
requestStream: false,
responseStream: false,
requestType: resource_pb.NewResourceRequest,
responseType: resource_pb.NewResourceResponse,
requestSerialize: serialize_pulumirpc_NewResourceRequest,
requestDeserialize: deserialize_pulumirpc_NewResourceRequest,
responseSerialize: serialize_pulumirpc_NewResourceResponse,
responseDeserialize: deserialize_pulumirpc_NewResourceResponse,
requestType: resource_pb.RegisterResourceRequest,
responseType: resource_pb.RegisterResourceResponse,
requestSerialize: serialize_pulumirpc_RegisterResourceRequest,
requestDeserialize: deserialize_pulumirpc_RegisterResourceRequest,
responseSerialize: serialize_pulumirpc_RegisterResourceResponse,
responseDeserialize: deserialize_pulumirpc_RegisterResourceResponse,
},
completeResource: {
path: '/pulumirpc.ResourceMonitor/CompleteResource',
requestStream: false,
responseStream: false,
requestType: resource_pb.CompleteResourceRequest,
responseType: resource_pb.CompleteResourceResponse,
requestSerialize: serialize_pulumirpc_CompleteResourceRequest,
requestDeserialize: deserialize_pulumirpc_CompleteResourceRequest,
responseSerialize: serialize_pulumirpc_CompleteResourceResponse,
responseDeserialize: deserialize_pulumirpc_CompleteResourceResponse,
},
};

View file

@ -13,8 +13,10 @@ var global = Function('return this')();
var google_protobuf_struct_pb = require('google-protobuf/google/protobuf/struct_pb.js');
var provider_pb = require('./provider_pb.js');
goog.exportSymbol('proto.pulumirpc.NewResourceRequest', null, global);
goog.exportSymbol('proto.pulumirpc.NewResourceResponse', null, global);
goog.exportSymbol('proto.pulumirpc.CompleteResourceRequest', null, global);
goog.exportSymbol('proto.pulumirpc.CompleteResourceResponse', null, global);
goog.exportSymbol('proto.pulumirpc.RegisterResourceRequest', null, global);
goog.exportSymbol('proto.pulumirpc.RegisterResourceResponse', null, global);
/**
* Generated by JsPbCodeGenerator.
@ -26,12 +28,12 @@ goog.exportSymbol('proto.pulumirpc.NewResourceResponse', null, global);
* @extends {jspb.Message}
* @constructor
*/
proto.pulumirpc.NewResourceRequest = function(opt_data) {
proto.pulumirpc.RegisterResourceRequest = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, null, null);
};
goog.inherits(proto.pulumirpc.NewResourceRequest, jspb.Message);
goog.inherits(proto.pulumirpc.RegisterResourceRequest, jspb.Message);
if (goog.DEBUG && !COMPILED) {
proto.pulumirpc.NewResourceRequest.displayName = 'proto.pulumirpc.NewResourceRequest';
proto.pulumirpc.RegisterResourceRequest.displayName = 'proto.pulumirpc.RegisterResourceRequest';
}
@ -46,8 +48,8 @@ if (jspb.Message.GENERATE_TO_OBJECT) {
* for transitional soy proto support: http://goto/soy-param-migration
* @return {!Object}
*/
proto.pulumirpc.NewResourceRequest.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.NewResourceRequest.toObject(opt_includeInstance, this);
proto.pulumirpc.RegisterResourceRequest.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.RegisterResourceRequest.toObject(opt_includeInstance, this);
};
@ -56,11 +58,11 @@ proto.pulumirpc.NewResourceRequest.prototype.toObject = function(opt_includeInst
* @param {boolean|undefined} includeInstance Whether to include the JSPB
* instance for transitional soy proto support:
* http://goto/soy-param-migration
* @param {!proto.pulumirpc.NewResourceRequest} msg The msg instance to transform.
* @param {!proto.pulumirpc.RegisterResourceRequest} msg The msg instance to transform.
* @return {!Object}
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.NewResourceRequest.toObject = function(includeInstance, msg) {
proto.pulumirpc.RegisterResourceRequest.toObject = function(includeInstance, msg) {
var f, obj = {
type: jspb.Message.getFieldWithDefault(msg, 1, ""),
name: jspb.Message.getFieldWithDefault(msg, 2, ""),
@ -80,23 +82,23 @@ proto.pulumirpc.NewResourceRequest.toObject = function(includeInstance, msg) {
/**
* Deserializes binary data (in protobuf wire format).
* @param {jspb.ByteSource} bytes The bytes to deserialize.
* @return {!proto.pulumirpc.NewResourceRequest}
* @return {!proto.pulumirpc.RegisterResourceRequest}
*/
proto.pulumirpc.NewResourceRequest.deserializeBinary = function(bytes) {
proto.pulumirpc.RegisterResourceRequest.deserializeBinary = function(bytes) {
var reader = new jspb.BinaryReader(bytes);
var msg = new proto.pulumirpc.NewResourceRequest;
return proto.pulumirpc.NewResourceRequest.deserializeBinaryFromReader(msg, reader);
var msg = new proto.pulumirpc.RegisterResourceRequest;
return proto.pulumirpc.RegisterResourceRequest.deserializeBinaryFromReader(msg, reader);
};
/**
* Deserializes binary data (in protobuf wire format) from the
* given reader into the given message object.
* @param {!proto.pulumirpc.NewResourceRequest} msg The message object to deserialize into.
* @param {!proto.pulumirpc.RegisterResourceRequest} msg The message object to deserialize into.
* @param {!jspb.BinaryReader} reader The BinaryReader to use.
* @return {!proto.pulumirpc.NewResourceRequest}
* @return {!proto.pulumirpc.RegisterResourceRequest}
*/
proto.pulumirpc.NewResourceRequest.deserializeBinaryFromReader = function(msg, reader) {
proto.pulumirpc.RegisterResourceRequest.deserializeBinaryFromReader = function(msg, reader) {
while (reader.nextField()) {
if (reader.isEndGroup()) {
break;
@ -137,9 +139,9 @@ proto.pulumirpc.NewResourceRequest.deserializeBinaryFromReader = function(msg, r
* Serializes the message to binary data (in protobuf wire format).
* @return {!Uint8Array}
*/
proto.pulumirpc.NewResourceRequest.prototype.serializeBinary = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.serializeBinary = function() {
var writer = new jspb.BinaryWriter();
proto.pulumirpc.NewResourceRequest.serializeBinaryToWriter(this, writer);
proto.pulumirpc.RegisterResourceRequest.serializeBinaryToWriter(this, writer);
return writer.getResultBuffer();
};
@ -147,11 +149,11 @@ proto.pulumirpc.NewResourceRequest.prototype.serializeBinary = function() {
/**
* Serializes the given message to binary data (in protobuf wire
* format), writing to the given BinaryWriter.
* @param {!proto.pulumirpc.NewResourceRequest} message
* @param {!proto.pulumirpc.RegisterResourceRequest} message
* @param {!jspb.BinaryWriter} writer
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.NewResourceRequest.serializeBinaryToWriter = function(message, writer) {
proto.pulumirpc.RegisterResourceRequest.serializeBinaryToWriter = function(message, writer) {
var f = undefined;
f = message.getType();
if (f.length > 0) {
@ -196,13 +198,13 @@ proto.pulumirpc.NewResourceRequest.serializeBinaryToWriter = function(message, w
* optional string type = 1;
* @return {string}
*/
proto.pulumirpc.NewResourceRequest.prototype.getType = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.getType = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 1, ""));
};
/** @param {string} value */
proto.pulumirpc.NewResourceRequest.prototype.setType = function(value) {
proto.pulumirpc.RegisterResourceRequest.prototype.setType = function(value) {
jspb.Message.setField(this, 1, value);
};
@ -211,13 +213,13 @@ proto.pulumirpc.NewResourceRequest.prototype.setType = function(value) {
* optional string name = 2;
* @return {string}
*/
proto.pulumirpc.NewResourceRequest.prototype.getName = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.getName = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 2, ""));
};
/** @param {string} value */
proto.pulumirpc.NewResourceRequest.prototype.setName = function(value) {
proto.pulumirpc.RegisterResourceRequest.prototype.setName = function(value) {
jspb.Message.setField(this, 2, value);
};
@ -226,13 +228,13 @@ proto.pulumirpc.NewResourceRequest.prototype.setName = function(value) {
* optional string parent = 3;
* @return {string}
*/
proto.pulumirpc.NewResourceRequest.prototype.getParent = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.getParent = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 3, ""));
};
/** @param {string} value */
proto.pulumirpc.NewResourceRequest.prototype.setParent = function(value) {
proto.pulumirpc.RegisterResourceRequest.prototype.setParent = function(value) {
jspb.Message.setField(this, 3, value);
};
@ -243,13 +245,13 @@ proto.pulumirpc.NewResourceRequest.prototype.setParent = function(value) {
* You should avoid comparisons like {@code val === true/false} in those cases.
* @return {boolean}
*/
proto.pulumirpc.NewResourceRequest.prototype.getCustom = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.getCustom = function() {
return /** @type {boolean} */ (jspb.Message.getFieldWithDefault(this, 4, false));
};
/** @param {boolean} value */
proto.pulumirpc.NewResourceRequest.prototype.setCustom = function(value) {
proto.pulumirpc.RegisterResourceRequest.prototype.setCustom = function(value) {
jspb.Message.setField(this, 4, value);
};
@ -258,19 +260,19 @@ proto.pulumirpc.NewResourceRequest.prototype.setCustom = function(value) {
* optional google.protobuf.Struct object = 5;
* @return {?proto.google.protobuf.Struct}
*/
proto.pulumirpc.NewResourceRequest.prototype.getObject = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.getObject = function() {
return /** @type{?proto.google.protobuf.Struct} */ (
jspb.Message.getWrapperField(this, google_protobuf_struct_pb.Struct, 5));
};
/** @param {?proto.google.protobuf.Struct|undefined} value */
proto.pulumirpc.NewResourceRequest.prototype.setObject = function(value) {
proto.pulumirpc.RegisterResourceRequest.prototype.setObject = function(value) {
jspb.Message.setWrapperField(this, 5, value);
};
proto.pulumirpc.NewResourceRequest.prototype.clearObject = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.clearObject = function() {
this.setObject(undefined);
};
@ -279,7 +281,7 @@ proto.pulumirpc.NewResourceRequest.prototype.clearObject = function() {
* Returns whether this field is set.
* @return {!boolean}
*/
proto.pulumirpc.NewResourceRequest.prototype.hasObject = function() {
proto.pulumirpc.RegisterResourceRequest.prototype.hasObject = function() {
return jspb.Message.getField(this, 5) != null;
};
@ -295,19 +297,347 @@ proto.pulumirpc.NewResourceRequest.prototype.hasObject = function() {
* @extends {jspb.Message}
* @constructor
*/
proto.pulumirpc.NewResourceResponse = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, proto.pulumirpc.NewResourceResponse.repeatedFields_, null);
proto.pulumirpc.RegisterResourceResponse = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, null, null);
};
goog.inherits(proto.pulumirpc.NewResourceResponse, jspb.Message);
goog.inherits(proto.pulumirpc.RegisterResourceResponse, jspb.Message);
if (goog.DEBUG && !COMPILED) {
proto.pulumirpc.NewResourceResponse.displayName = 'proto.pulumirpc.NewResourceResponse';
proto.pulumirpc.RegisterResourceResponse.displayName = 'proto.pulumirpc.RegisterResourceResponse';
}
if (jspb.Message.GENERATE_TO_OBJECT) {
/**
* Creates an object representation of this proto suitable for use in Soy templates.
* Field names that are reserved in JavaScript and will be renamed to pb_name.
* To access a reserved field use, foo.pb_<name>, eg, foo.pb_default.
* For the list of reserved names please see:
* com.google.apps.jspb.JsClassTemplate.JS_RESERVED_WORDS.
* @param {boolean=} opt_includeInstance Whether to include the JSPB instance
* for transitional soy proto support: http://goto/soy-param-migration
* @return {!Object}
*/
proto.pulumirpc.RegisterResourceResponse.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.RegisterResourceResponse.toObject(opt_includeInstance, this);
};
/**
* Static version of the {@see toObject} method.
* @param {boolean|undefined} includeInstance Whether to include the JSPB
* instance for transitional soy proto support:
* http://goto/soy-param-migration
* @param {!proto.pulumirpc.RegisterResourceResponse} msg The msg instance to transform.
* @return {!Object}
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.RegisterResourceResponse.toObject = function(includeInstance, msg) {
var f, obj = {
urn: jspb.Message.getFieldWithDefault(msg, 1, "")
};
if (includeInstance) {
obj.$jspbMessageInstance = msg;
}
return obj;
};
}
/**
* Deserializes binary data (in protobuf wire format).
* @param {jspb.ByteSource} bytes The bytes to deserialize.
* @return {!proto.pulumirpc.RegisterResourceResponse}
*/
proto.pulumirpc.RegisterResourceResponse.deserializeBinary = function(bytes) {
var reader = new jspb.BinaryReader(bytes);
var msg = new proto.pulumirpc.RegisterResourceResponse;
return proto.pulumirpc.RegisterResourceResponse.deserializeBinaryFromReader(msg, reader);
};
/**
* Deserializes binary data (in protobuf wire format) from the
* given reader into the given message object.
* @param {!proto.pulumirpc.RegisterResourceResponse} msg The message object to deserialize into.
* @param {!jspb.BinaryReader} reader The BinaryReader to use.
* @return {!proto.pulumirpc.RegisterResourceResponse}
*/
proto.pulumirpc.RegisterResourceResponse.deserializeBinaryFromReader = function(msg, reader) {
while (reader.nextField()) {
if (reader.isEndGroup()) {
break;
}
var field = reader.getFieldNumber();
switch (field) {
case 1:
var value = /** @type {string} */ (reader.readString());
msg.setUrn(value);
break;
default:
reader.skipField();
break;
}
}
return msg;
};
/**
* Serializes the message to binary data (in protobuf wire format).
* @return {!Uint8Array}
*/
proto.pulumirpc.RegisterResourceResponse.prototype.serializeBinary = function() {
var writer = new jspb.BinaryWriter();
proto.pulumirpc.RegisterResourceResponse.serializeBinaryToWriter(this, writer);
return writer.getResultBuffer();
};
/**
* Serializes the given message to binary data (in protobuf wire
* format), writing to the given BinaryWriter.
* @param {!proto.pulumirpc.RegisterResourceResponse} message
* @param {!jspb.BinaryWriter} writer
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.RegisterResourceResponse.serializeBinaryToWriter = function(message, writer) {
var f = undefined;
f = message.getUrn();
if (f.length > 0) {
writer.writeString(
1,
f
);
}
};
/**
* optional string urn = 1;
* @return {string}
*/
proto.pulumirpc.RegisterResourceResponse.prototype.getUrn = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 1, ""));
};
/** @param {string} value */
proto.pulumirpc.RegisterResourceResponse.prototype.setUrn = function(value) {
jspb.Message.setField(this, 1, value);
};
/**
* Generated by JsPbCodeGenerator.
* @param {Array=} opt_data Optional initial data array, typically from a
* server response, or constructed directly in Javascript. The array is used
* in place and becomes part of the constructed object. It is not cloned.
* If no data is provided, the constructed object will be empty, but still
* valid.
* @extends {jspb.Message}
* @constructor
*/
proto.pulumirpc.CompleteResourceRequest = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, null, null);
};
goog.inherits(proto.pulumirpc.CompleteResourceRequest, jspb.Message);
if (goog.DEBUG && !COMPILED) {
proto.pulumirpc.CompleteResourceRequest.displayName = 'proto.pulumirpc.CompleteResourceRequest';
}
if (jspb.Message.GENERATE_TO_OBJECT) {
/**
* Creates an object representation of this proto suitable for use in Soy templates.
* Field names that are reserved in JavaScript and will be renamed to pb_name.
* To access a reserved field use, foo.pb_<name>, eg, foo.pb_default.
* For the list of reserved names please see:
* com.google.apps.jspb.JsClassTemplate.JS_RESERVED_WORDS.
* @param {boolean=} opt_includeInstance Whether to include the JSPB instance
* for transitional soy proto support: http://goto/soy-param-migration
* @return {!Object}
*/
proto.pulumirpc.CompleteResourceRequest.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.CompleteResourceRequest.toObject(opt_includeInstance, this);
};
/**
* Static version of the {@see toObject} method.
* @param {boolean|undefined} includeInstance Whether to include the JSPB
* instance for transitional soy proto support:
* http://goto/soy-param-migration
* @param {!proto.pulumirpc.CompleteResourceRequest} msg The msg instance to transform.
* @return {!Object}
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.CompleteResourceRequest.toObject = function(includeInstance, msg) {
var f, obj = {
urn: jspb.Message.getFieldWithDefault(msg, 1, ""),
extras: (f = msg.getExtras()) && google_protobuf_struct_pb.Struct.toObject(includeInstance, f)
};
if (includeInstance) {
obj.$jspbMessageInstance = msg;
}
return obj;
};
}
/**
* Deserializes binary data (in protobuf wire format).
* @param {jspb.ByteSource} bytes The bytes to deserialize.
* @return {!proto.pulumirpc.CompleteResourceRequest}
*/
proto.pulumirpc.CompleteResourceRequest.deserializeBinary = function(bytes) {
var reader = new jspb.BinaryReader(bytes);
var msg = new proto.pulumirpc.CompleteResourceRequest;
return proto.pulumirpc.CompleteResourceRequest.deserializeBinaryFromReader(msg, reader);
};
/**
* Deserializes binary data (in protobuf wire format) from the
* given reader into the given message object.
* @param {!proto.pulumirpc.CompleteResourceRequest} msg The message object to deserialize into.
* @param {!jspb.BinaryReader} reader The BinaryReader to use.
* @return {!proto.pulumirpc.CompleteResourceRequest}
*/
proto.pulumirpc.CompleteResourceRequest.deserializeBinaryFromReader = function(msg, reader) {
while (reader.nextField()) {
if (reader.isEndGroup()) {
break;
}
var field = reader.getFieldNumber();
switch (field) {
case 1:
var value = /** @type {string} */ (reader.readString());
msg.setUrn(value);
break;
case 2:
var value = new google_protobuf_struct_pb.Struct;
reader.readMessage(value,google_protobuf_struct_pb.Struct.deserializeBinaryFromReader);
msg.setExtras(value);
break;
default:
reader.skipField();
break;
}
}
return msg;
};
/**
* Serializes the message to binary data (in protobuf wire format).
* @return {!Uint8Array}
*/
proto.pulumirpc.CompleteResourceRequest.prototype.serializeBinary = function() {
var writer = new jspb.BinaryWriter();
proto.pulumirpc.CompleteResourceRequest.serializeBinaryToWriter(this, writer);
return writer.getResultBuffer();
};
/**
* Serializes the given message to binary data (in protobuf wire
* format), writing to the given BinaryWriter.
* @param {!proto.pulumirpc.CompleteResourceRequest} message
* @param {!jspb.BinaryWriter} writer
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.CompleteResourceRequest.serializeBinaryToWriter = function(message, writer) {
var f = undefined;
f = message.getUrn();
if (f.length > 0) {
writer.writeString(
1,
f
);
}
f = message.getExtras();
if (f != null) {
writer.writeMessage(
2,
f,
google_protobuf_struct_pb.Struct.serializeBinaryToWriter
);
}
};
/**
* optional string urn = 1;
* @return {string}
*/
proto.pulumirpc.CompleteResourceRequest.prototype.getUrn = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 1, ""));
};
/** @param {string} value */
proto.pulumirpc.CompleteResourceRequest.prototype.setUrn = function(value) {
jspb.Message.setField(this, 1, value);
};
/**
* optional google.protobuf.Struct extras = 2;
* @return {?proto.google.protobuf.Struct}
*/
proto.pulumirpc.CompleteResourceRequest.prototype.getExtras = function() {
return /** @type{?proto.google.protobuf.Struct} */ (
jspb.Message.getWrapperField(this, google_protobuf_struct_pb.Struct, 2));
};
/** @param {?proto.google.protobuf.Struct|undefined} value */
proto.pulumirpc.CompleteResourceRequest.prototype.setExtras = function(value) {
jspb.Message.setWrapperField(this, 2, value);
};
proto.pulumirpc.CompleteResourceRequest.prototype.clearExtras = function() {
this.setExtras(undefined);
};
/**
* Returns whether this field is set.
* @return {!boolean}
*/
proto.pulumirpc.CompleteResourceRequest.prototype.hasExtras = function() {
return jspb.Message.getField(this, 2) != null;
};
/**
* Generated by JsPbCodeGenerator.
* @param {Array=} opt_data Optional initial data array, typically from a
* server response, or constructed directly in Javascript. The array is used
* in place and becomes part of the constructed object. It is not cloned.
* If no data is provided, the constructed object will be empty, but still
* valid.
* @extends {jspb.Message}
* @constructor
*/
proto.pulumirpc.CompleteResourceResponse = function(opt_data) {
jspb.Message.initialize(this, opt_data, 0, -1, proto.pulumirpc.CompleteResourceResponse.repeatedFields_, null);
};
goog.inherits(proto.pulumirpc.CompleteResourceResponse, jspb.Message);
if (goog.DEBUG && !COMPILED) {
proto.pulumirpc.CompleteResourceResponse.displayName = 'proto.pulumirpc.CompleteResourceResponse';
}
/**
* List of repeated fields within this message type.
* @private {!Array<number>}
* @const
*/
proto.pulumirpc.NewResourceResponse.repeatedFields_ = [5];
proto.pulumirpc.CompleteResourceResponse.repeatedFields_ = [4];
@ -322,8 +652,8 @@ if (jspb.Message.GENERATE_TO_OBJECT) {
* for transitional soy proto support: http://goto/soy-param-migration
* @return {!Object}
*/
proto.pulumirpc.NewResourceResponse.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.NewResourceResponse.toObject(opt_includeInstance, this);
proto.pulumirpc.CompleteResourceResponse.prototype.toObject = function(opt_includeInstance) {
return proto.pulumirpc.CompleteResourceResponse.toObject(opt_includeInstance, this);
};
@ -332,17 +662,16 @@ proto.pulumirpc.NewResourceResponse.prototype.toObject = function(opt_includeIns
* @param {boolean|undefined} includeInstance Whether to include the JSPB
* instance for transitional soy proto support:
* http://goto/soy-param-migration
* @param {!proto.pulumirpc.NewResourceResponse} msg The msg instance to transform.
* @param {!proto.pulumirpc.CompleteResourceResponse} msg The msg instance to transform.
* @return {!Object}
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.NewResourceResponse.toObject = function(includeInstance, msg) {
proto.pulumirpc.CompleteResourceResponse.toObject = function(includeInstance, msg) {
var f, obj = {
id: jspb.Message.getFieldWithDefault(msg, 1, ""),
urn: jspb.Message.getFieldWithDefault(msg, 2, ""),
object: (f = msg.getObject()) && google_protobuf_struct_pb.Struct.toObject(includeInstance, f),
stable: jspb.Message.getFieldWithDefault(msg, 4, false),
stablesList: jspb.Message.getRepeatedField(msg, 5)
stable: jspb.Message.getFieldWithDefault(msg, 3, false),
stablesList: jspb.Message.getRepeatedField(msg, 4)
};
if (includeInstance) {
@ -356,23 +685,23 @@ proto.pulumirpc.NewResourceResponse.toObject = function(includeInstance, msg) {
/**
* Deserializes binary data (in protobuf wire format).
* @param {jspb.ByteSource} bytes The bytes to deserialize.
* @return {!proto.pulumirpc.NewResourceResponse}
* @return {!proto.pulumirpc.CompleteResourceResponse}
*/
proto.pulumirpc.NewResourceResponse.deserializeBinary = function(bytes) {
proto.pulumirpc.CompleteResourceResponse.deserializeBinary = function(bytes) {
var reader = new jspb.BinaryReader(bytes);
var msg = new proto.pulumirpc.NewResourceResponse;
return proto.pulumirpc.NewResourceResponse.deserializeBinaryFromReader(msg, reader);
var msg = new proto.pulumirpc.CompleteResourceResponse;
return proto.pulumirpc.CompleteResourceResponse.deserializeBinaryFromReader(msg, reader);
};
/**
* Deserializes binary data (in protobuf wire format) from the
* given reader into the given message object.
* @param {!proto.pulumirpc.NewResourceResponse} msg The message object to deserialize into.
* @param {!proto.pulumirpc.CompleteResourceResponse} msg The message object to deserialize into.
* @param {!jspb.BinaryReader} reader The BinaryReader to use.
* @return {!proto.pulumirpc.NewResourceResponse}
* @return {!proto.pulumirpc.CompleteResourceResponse}
*/
proto.pulumirpc.NewResourceResponse.deserializeBinaryFromReader = function(msg, reader) {
proto.pulumirpc.CompleteResourceResponse.deserializeBinaryFromReader = function(msg, reader) {
while (reader.nextField()) {
if (reader.isEndGroup()) {
break;
@ -384,19 +713,15 @@ proto.pulumirpc.NewResourceResponse.deserializeBinaryFromReader = function(msg,
msg.setId(value);
break;
case 2:
var value = /** @type {string} */ (reader.readString());
msg.setUrn(value);
break;
case 3:
var value = new google_protobuf_struct_pb.Struct;
reader.readMessage(value,google_protobuf_struct_pb.Struct.deserializeBinaryFromReader);
msg.setObject(value);
break;
case 4:
case 3:
var value = /** @type {boolean} */ (reader.readBool());
msg.setStable(value);
break;
case 5:
case 4:
var value = /** @type {string} */ (reader.readString());
msg.addStables(value);
break;
@ -413,9 +738,9 @@ proto.pulumirpc.NewResourceResponse.deserializeBinaryFromReader = function(msg,
* Serializes the message to binary data (in protobuf wire format).
* @return {!Uint8Array}
*/
proto.pulumirpc.NewResourceResponse.prototype.serializeBinary = function() {
proto.pulumirpc.CompleteResourceResponse.prototype.serializeBinary = function() {
var writer = new jspb.BinaryWriter();
proto.pulumirpc.NewResourceResponse.serializeBinaryToWriter(this, writer);
proto.pulumirpc.CompleteResourceResponse.serializeBinaryToWriter(this, writer);
return writer.getResultBuffer();
};
@ -423,11 +748,11 @@ proto.pulumirpc.NewResourceResponse.prototype.serializeBinary = function() {
/**
* Serializes the given message to binary data (in protobuf wire
* format), writing to the given BinaryWriter.
* @param {!proto.pulumirpc.NewResourceResponse} message
* @param {!proto.pulumirpc.CompleteResourceResponse} message
* @param {!jspb.BinaryWriter} writer
* @suppress {unusedLocalVariables} f is only used for nested messages
*/
proto.pulumirpc.NewResourceResponse.serializeBinaryToWriter = function(message, writer) {
proto.pulumirpc.CompleteResourceResponse.serializeBinaryToWriter = function(message, writer) {
var f = undefined;
f = message.getId();
if (f.length > 0) {
@ -436,17 +761,10 @@ proto.pulumirpc.NewResourceResponse.serializeBinaryToWriter = function(message,
f
);
}
f = message.getUrn();
if (f.length > 0) {
writer.writeString(
2,
f
);
}
f = message.getObject();
if (f != null) {
writer.writeMessage(
3,
2,
f,
google_protobuf_struct_pb.Struct.serializeBinaryToWriter
);
@ -454,14 +772,14 @@ proto.pulumirpc.NewResourceResponse.serializeBinaryToWriter = function(message,
f = message.getStable();
if (f) {
writer.writeBool(
4,
3,
f
);
}
f = message.getStablesList();
if (f.length > 0) {
writer.writeRepeatedString(
5,
4,
f
);
}
@ -472,49 +790,34 @@ proto.pulumirpc.NewResourceResponse.serializeBinaryToWriter = function(message,
* optional string id = 1;
* @return {string}
*/
proto.pulumirpc.NewResourceResponse.prototype.getId = function() {
proto.pulumirpc.CompleteResourceResponse.prototype.getId = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 1, ""));
};
/** @param {string} value */
proto.pulumirpc.NewResourceResponse.prototype.setId = function(value) {
proto.pulumirpc.CompleteResourceResponse.prototype.setId = function(value) {
jspb.Message.setField(this, 1, value);
};
/**
* optional string urn = 2;
* @return {string}
*/
proto.pulumirpc.NewResourceResponse.prototype.getUrn = function() {
return /** @type {string} */ (jspb.Message.getFieldWithDefault(this, 2, ""));
};
/** @param {string} value */
proto.pulumirpc.NewResourceResponse.prototype.setUrn = function(value) {
jspb.Message.setField(this, 2, value);
};
/**
* optional google.protobuf.Struct object = 3;
* optional google.protobuf.Struct object = 2;
* @return {?proto.google.protobuf.Struct}
*/
proto.pulumirpc.NewResourceResponse.prototype.getObject = function() {
proto.pulumirpc.CompleteResourceResponse.prototype.getObject = function() {
return /** @type{?proto.google.protobuf.Struct} */ (
jspb.Message.getWrapperField(this, google_protobuf_struct_pb.Struct, 3));
jspb.Message.getWrapperField(this, google_protobuf_struct_pb.Struct, 2));
};
/** @param {?proto.google.protobuf.Struct|undefined} value */
proto.pulumirpc.NewResourceResponse.prototype.setObject = function(value) {
jspb.Message.setWrapperField(this, 3, value);
proto.pulumirpc.CompleteResourceResponse.prototype.setObject = function(value) {
jspb.Message.setWrapperField(this, 2, value);
};
proto.pulumirpc.NewResourceResponse.prototype.clearObject = function() {
proto.pulumirpc.CompleteResourceResponse.prototype.clearObject = function() {
this.setObject(undefined);
};
@ -523,40 +826,40 @@ proto.pulumirpc.NewResourceResponse.prototype.clearObject = function() {
* Returns whether this field is set.
* @return {!boolean}
*/
proto.pulumirpc.NewResourceResponse.prototype.hasObject = function() {
return jspb.Message.getField(this, 3) != null;
proto.pulumirpc.CompleteResourceResponse.prototype.hasObject = function() {
return jspb.Message.getField(this, 2) != null;
};
/**
* optional bool stable = 4;
* optional bool stable = 3;
* Note that Boolean fields may be set to 0/1 when serialized from a Java server.
* You should avoid comparisons like {@code val === true/false} in those cases.
* @return {boolean}
*/
proto.pulumirpc.NewResourceResponse.prototype.getStable = function() {
return /** @type {boolean} */ (jspb.Message.getFieldWithDefault(this, 4, false));
proto.pulumirpc.CompleteResourceResponse.prototype.getStable = function() {
return /** @type {boolean} */ (jspb.Message.getFieldWithDefault(this, 3, false));
};
/** @param {boolean} value */
proto.pulumirpc.NewResourceResponse.prototype.setStable = function(value) {
jspb.Message.setField(this, 4, value);
proto.pulumirpc.CompleteResourceResponse.prototype.setStable = function(value) {
jspb.Message.setField(this, 3, value);
};
/**
* repeated string stables = 5;
* repeated string stables = 4;
* @return {!Array.<string>}
*/
proto.pulumirpc.NewResourceResponse.prototype.getStablesList = function() {
return /** @type {!Array.<string>} */ (jspb.Message.getRepeatedField(this, 5));
proto.pulumirpc.CompleteResourceResponse.prototype.getStablesList = function() {
return /** @type {!Array.<string>} */ (jspb.Message.getRepeatedField(this, 4));
};
/** @param {!Array.<string>} value */
proto.pulumirpc.NewResourceResponse.prototype.setStablesList = function(value) {
jspb.Message.setField(this, 5, value || []);
proto.pulumirpc.CompleteResourceResponse.prototype.setStablesList = function(value) {
jspb.Message.setField(this, 4, value || []);
};
@ -564,12 +867,12 @@ proto.pulumirpc.NewResourceResponse.prototype.setStablesList = function(value) {
* @param {!string} value
* @param {number=} opt_index
*/
proto.pulumirpc.NewResourceResponse.prototype.addStables = function(value, opt_index) {
jspb.Message.addToRepeatedField(this, 5, value, opt_index);
proto.pulumirpc.CompleteResourceResponse.prototype.addStables = function(value, opt_index) {
jspb.Message.addToRepeatedField(this, 4, value, opt_index);
};
proto.pulumirpc.NewResourceResponse.prototype.clearStablesList = function() {
proto.pulumirpc.CompleteResourceResponse.prototype.clearStablesList = function() {
this.setStablesList([]);
};

View file

@ -10,11 +10,12 @@ package pulumirpc;
// ResourceMonitor is the interface a source uses to talk back to the planning monitor orchestrating the execution.
service ResourceMonitor {
rpc Invoke(InvokeRequest) returns (InvokeResponse) {}
rpc NewResource(NewResourceRequest) returns (NewResourceResponse) {}
rpc RegisterResource(RegisterResourceRequest) returns (RegisterResourceResponse) {}
rpc CompleteResource(CompleteResourceRequest) returns (CompleteResourceResponse) {}
}
// NewResourceRequest contains information about a resource object that was newly allocated.
message NewResourceRequest {
// RegisterResourceRequest contains information about a resource object that was newly allocated.
message RegisterResourceRequest {
string type = 1; // the type of the object allocated.
string name = 2; // the name, for URN purposes, of the object.
string parent = 3; // an optional parent URN that this child resource belongs to.
@ -22,11 +23,23 @@ message NewResourceRequest {
google.protobuf.Struct object = 5; // an object produced by the interpreter/source.
}
// NewResourceResponse reflects back the properties initialized during creation, if applicable.
message NewResourceResponse {
string id = 1; // the unique ID assigned by the provider.
string urn = 2; // the URN assigned by the fabric.
google.protobuf.Struct object = 3; // the resulting object properties, including provider defaults.
bool stable = 4; // if true, the object's state is stable and may be trusted not to change.
repeated string stables = 5; // an optional list of guaranteed-stable properties.
// RegisterResourceResponse reflects back the properties initialized during creation, if applicable.
message RegisterResourceResponse {
string urn = 1; // the URN assigned by the fabric.
}
// CompleteResourceRequest completes the registration of a resource, and optionally adds extra derived output
// properties to an existing resource that is in flight. It must be called once per registration.
message CompleteResourceRequest {
string urn = 1; // the URN for the resource to attach output properties to.
google.protobuf.Struct extras = 2; // optional additional output properties to add to the existing resource.
}
// CompleteResourceResponse is returned by the engine after a resource is completed. It includes any state
// that was populated by the resource provider so that the language engine can blit it into the resource objects.
message CompleteResourceResponse {
string id = 1; // the unique ID assigned by the provider.
google.protobuf.Struct object = 2; // the resulting object properties, including provider defaults.
bool stable = 3; // if true, the object's state is stable and may be trusted not to change.
repeated string stables = 4; // an optional list of guaranteed-stable properties.
}