Rename Monikers to URNs

This change is mostly just a rename of Moniker to URN.  It does also
prefix resource URNs to have a standard URN namespace; in other words,
"urn🥥<name>", where <name> is the same as the prior Moniker.

This is a minor step that helps to prepare us for pulumi/coconut#109.
This commit is contained in:
joeduffy 2017-03-02 17:10:10 -08:00
parent 2ce75cb946
commit 076d689a05
14 changed files with 250 additions and 238 deletions

View file

@ -631,12 +631,12 @@ func printStep(b *bytes.Buffer, step resource.Step, summary bool, indent string)
// First print out the operation's prefix.
b.WriteString(step.Op().Prefix())
// Next print the resource moniker, properties, etc.
// Next print the resource URN, properties, etc.
printResourceHeader(b, step.Old(), step.New(), indent)
b.WriteString(step.Op().Suffix())
var replaces []resource.PropertyKey
if step.Old() != nil {
m := step.Old().Moniker()
m := step.Old().URN()
replaceMap := step.Plan().Replaces()
replaces = replaceMap[m]
}
@ -662,20 +662,20 @@ func printResourceProperties(b *bytes.Buffer, old resource.Resource, new resourc
computed resource.PropertyMap, replaces []resource.PropertyKey, summary bool, indent string) {
indent += detailsIndent
// Print out the moniker and, if present, the ID, as "pseudo-properties".
// Print out the URN and, if present, the ID, as "pseudo-properties".
var id resource.ID
var moniker resource.Moniker
var URN resource.URN
if old == nil {
id = new.ID()
moniker = new.Moniker()
URN = new.URN()
} else {
id = old.ID()
moniker = old.Moniker()
URN = old.URN()
}
if id != "" {
b.WriteString(fmt.Sprintf("%s[id=%s]\n", indent, string(id)))
}
b.WriteString(fmt.Sprintf("%s[mk=%s]\n", indent, string(moniker)))
b.WriteString(fmt.Sprintf("%s[urn=%s]\n", indent, URN.Name()))
if !summary {
// Print all of the properties associated with this resource.

View file

@ -10,7 +10,7 @@ var (
ErrorPlanApplyFailed = newError(2003, "Plan apply failed: %v")
ErrorIllegalMarkupExtension = newError(2004, "Resource serialization failed; illegal markup extension '%v'")
ErrorCantReadDeployment = newError(2005, "Could not read deployment file '%v': %v")
ErrorDuplicateMonikerNames = newError(2006, "Duplicate objects with the same name: %v")
ErrorDuplicateURNNames = newError(2006, "Duplicate objects with the same URN: %v")
ErrorInvalidHuskName = newError(2007, "Invalid husk '%v'; could not be found in the workspace")
ErrorIllegalConfigToken = newError(2008,
"Configs may only target module properties and class static properties; %v is neither")

View file

@ -35,7 +35,7 @@ func Print(g graph.Graph, w io.Writer) error {
}
// For now, we auto-generate IDs.
// TODO[pulumi/coconut#76]: use the object monikers instead, once we have them.
// TODO[pulumi/coconut#76]: use the object URNs instead, once we have them.
c := 0
ids := make(map[graph.Vertex]string)
getID := func(v graph.Vertex) string {

View file

@ -17,24 +17,24 @@ type Context struct {
Diag diag.Sink // the diagnostics sink to use for messages.
Plugins map[tokens.Package]*Plugin // a cache of plugins and their processes.
ObjRes objectResourceMap // the resources held inside of this snapshot.
ObjMks objectMonikerMap // a convenient lookup map for object to moniker.
MksRes monikerResourceMap // a convenient lookup map for moniker to resource.
MksOldIDs monikerIDMap // a convenient lookup map for monikers to old IDs.
ObjURN objectURNMap // a convenient lookup map for object to urn.
URNRes urnResourceMap // a convenient lookup map for urn to resource.
URNOldIDs urnIDMap // a convenient lookup map for urns to old IDs.
}
type objectMonikerMap map[*rt.Object]Moniker
type objectURNMap map[*rt.Object]URN
type objectResourceMap map[*rt.Object]Resource
type monikerResourceMap map[Moniker]Resource
type monikerIDMap map[Moniker]ID
type urnResourceMap map[URN]Resource
type urnIDMap map[URN]ID
func NewContext(d diag.Sink) *Context {
return &Context{
Diag: d,
Plugins: make(map[tokens.Package]*Plugin),
ObjRes: make(objectResourceMap),
ObjMks: make(objectMonikerMap),
MksRes: make(monikerResourceMap),
MksOldIDs: make(monikerIDMap),
ObjURN: make(objectURNMap),
URNRes: make(urnResourceMap),
URNOldIDs: make(urnIDMap),
}
}

View file

@ -21,7 +21,7 @@ type DeploymentRecord struct {
Reftag *string `json:"reftag,omitempty"` // the ref alias, if any (`#ref` by default).
Package tokens.Package `json:"package"` // the nut that this husk belongs to.
Args *core.Args `json:"args,omitempty"` // the blueprint args for graph creation.
Resources *ResourceDeploymentMap `json:"resources,omitempty"` // a map of monikers to resource vertices.
Resources *ResourceDeploymentMap `json:"resources,omitempty"` // a map of URNs to resource vertices.
}
// DefaultDeploymentReftag is the default ref tag for intra-graph edges.
@ -51,9 +51,9 @@ func serializeDeploymentRecord(snap Snapshot, reftag string) *DeploymentRecord {
if snapres := snap.Resources(); len(snapres) > 0 {
resm = NewResourceDeploymentMap()
for _, res := range snap.Resources() {
m := res.Moniker()
contract.Assertf(string(m) != "", "Unexpected empty resource moniker")
contract.Assertf(!resm.Has(m), "Unexpected duplicate resource moniker '%v'", m)
m := res.URN()
contract.Assertf(string(m) != "", "Unexpected empty resource URN")
contract.Assertf(!resm.Has(m), "Unexpected duplicate resource URN '%v'", m)
resm.Add(m, serializeResourceDeployment(res, reftag))
}
}
@ -136,7 +136,7 @@ func serializeProperty(prop PropertyValue, reftag string) (interface{}, bool) {
return serializeProperties(prop.ObjectValue(), reftag)
}
// Morph resources into their equivalent `{ "#ref": "<moniker>" }` form.
// Morph resources into their equivalent `{ "#ref": "<URN>" }` form.
if prop.IsResource() {
return map[string]string{
reftag: string(prop.ResourceValue()),
@ -171,11 +171,11 @@ func deserializeProperty(v interface{}, reftag string) PropertyValue {
}
return NewPropertyArray(arr)
case map[string]interface{}:
// If the map has a single entry and it is the reftag, this is a moniker.
// If the map has a single entry and it is the reftag, this is a URN.
if len(w) == 1 {
if tag, has := w[reftag]; has {
if tagstr, isstring := tag.(string); isstring {
return NewPropertyResource(Moniker(tagstr))
return NewPropertyResource(URN(tagstr))
}
}
}
@ -191,29 +191,29 @@ func deserializeProperty(v interface{}, reftag string) PropertyValue {
return NewPropertyNull()
}
// ResourceDeploymentMap is a map of moniker to resource, that also preserves a stable order of its keys. This ensures
// ResourceDeploymentMap is a map of URN to resource, that also preserves a stable order of its keys. This ensures
// enumerations are ordered deterministically, versus Go's built-in map type whose enumeration is randomized.
// Additionally, because of this stable ordering, marshaling to and from JSON also preserves the order of keys.
type ResourceDeploymentMap struct {
m map[Moniker]*ResourceDeployment
keys []Moniker
m map[URN]*ResourceDeployment
keys []URN
}
func NewResourceDeploymentMap() *ResourceDeploymentMap {
return &ResourceDeploymentMap{m: make(map[Moniker]*ResourceDeployment)}
return &ResourceDeploymentMap{m: make(map[URN]*ResourceDeployment)}
}
func (m *ResourceDeploymentMap) Keys() []Moniker { return m.keys }
func (m *ResourceDeploymentMap) Len() int { return len(m.keys) }
func (m *ResourceDeploymentMap) Keys() []URN { return m.keys }
func (m *ResourceDeploymentMap) Len() int { return len(m.keys) }
func (m *ResourceDeploymentMap) Add(k Moniker, v *ResourceDeployment) {
func (m *ResourceDeploymentMap) Add(k URN, v *ResourceDeployment) {
_, has := m.m[k]
contract.Assertf(!has, "Unexpected duplicate key '%v' added to map")
m.m[k] = v
m.keys = append(m.keys, k)
}
func (m *ResourceDeploymentMap) Delete(k Moniker) {
func (m *ResourceDeploymentMap) Delete(k URN) {
_, has := m.m[k]
contract.Assertf(has, "Unexpected delete of non-existent key key '%v'")
delete(m.m, k)
@ -227,29 +227,29 @@ func (m *ResourceDeploymentMap) Delete(k Moniker) {
}
}
func (m *ResourceDeploymentMap) Get(k Moniker) (*ResourceDeployment, bool) {
func (m *ResourceDeploymentMap) Get(k URN) (*ResourceDeployment, bool) {
v, has := m.m[k]
return v, has
}
func (m *ResourceDeploymentMap) Has(k Moniker) bool {
func (m *ResourceDeploymentMap) Has(k URN) bool {
_, has := m.m[k]
return has
}
func (m *ResourceDeploymentMap) Must(k Moniker) *ResourceDeployment {
func (m *ResourceDeploymentMap) Must(k URN) *ResourceDeployment {
v, has := m.m[k]
contract.Assertf(has, "Expected key '%v' to exist in this map", k)
return v
}
func (m *ResourceDeploymentMap) Set(k Moniker, v *ResourceDeployment) {
func (m *ResourceDeploymentMap) Set(k URN, v *ResourceDeployment) {
_, has := m.m[k]
contract.Assertf(has, "Expected key '%v' to exist in this map for setting an element", k)
m.m[k] = v
}
func (m *ResourceDeploymentMap) SetOrAdd(k Moniker, v *ResourceDeployment) {
func (m *ResourceDeploymentMap) SetOrAdd(k URN, v *ResourceDeployment) {
if _, has := m.m[k]; has {
m.Set(k, v)
} else {
@ -258,7 +258,7 @@ func (m *ResourceDeploymentMap) SetOrAdd(k Moniker, v *ResourceDeployment) {
}
type ResourceDeploymentKeyValue struct {
Key Moniker
Key URN
Value *ResourceDeployment
}
@ -299,7 +299,7 @@ func (m *ResourceDeploymentMap) MarshalJSON() ([]byte, error) {
func (m *ResourceDeploymentMap) UnmarshalJSON(b []byte) error {
contract.Assert(m.m == nil)
m.m = make(map[Moniker]*ResourceDeployment)
m.m = make(map[URN]*ResourceDeployment)
// Do a pass and read keys and values in the right order.
rdr := bytes.NewReader(b)
@ -313,7 +313,7 @@ func (m *ResourceDeploymentMap) UnmarshalJSON(b []byte) error {
}
contract.Assert(opencurly.(json.Delim) == '{')
// Parse out every resource key (Moniker) and element (*ResourceDeployment):
// Parse out every resource key (URN) and element (*ResourceDeployment):
for dec.More() {
// See if we've reached the closing '}'; if yes, chew on it and break.
token, err := dec.Token()
@ -325,7 +325,7 @@ func (m *ResourceDeploymentMap) UnmarshalJSON(b []byte) error {
break
}
k := Moniker(token.(string))
k := URN(token.(string))
contract.Assert(dec.More())
var v *ResourceDeployment
if err := dec.Decode(&v); err != nil {

View file

@ -1,65 +0,0 @@
// Copyright 2016 Pulumi, Inc. All rights reserved.
package resource
import (
"strings"
"github.com/pulumi/coconut/pkg/tokens"
"github.com/pulumi/coconut/pkg/util/contract"
)
// Moniker is a friendly, but unique, name for a resource, most often auto-assigned by Coconut. These monikers
// are used as unique IDs for objects, and help to to perform graph diffing and resolution of resource object changes.
//
// In theory, we could support manually assigned monikers in the future (e.g., think UUIDs). For the time being,
// however, we have opted to simplify developers' lives by mostly automating the process. The one caveat where it isn't
// truly automatic is that a developer -- or resource provider -- must provide a semi-unique name.
//
// Each moniker is of the form:
//
// <Namespace>::<AllocModule>::<Type>::<Name>
//
// wherein each element is the following:
//
// <Namespace> The namespace being deployed into
// <AllocModule> The module token in which the object was allocated
// <Type> The object type's full type token
// <Name> The human-friendly name identifier assigned by the developer or provider
//
// In the future, we may add elements to the moniker; it is more important that it is unique than it is human-typable.
type Moniker string
const MonikerDelimiter = "::" // the delimiter between elements of the moniker.
// NewMoniker creates a unique moniker for the given object.
func NewMoniker(ns tokens.QName, alloc tokens.Module, t tokens.Type, name tokens.QName) Moniker {
m := Moniker(
string(ns) +
MonikerDelimiter + string(alloc) +
MonikerDelimiter + string(t) +
MonikerDelimiter + string(name),
)
contract.Assert(!m.Replacement())
return m
}
// replaceMonikerSuffix is the suffix for monikers referring to resources that are being replaced.
const replaceMonikerSuffix = Moniker("#<new-id(replace)>")
// Replace returns a new, modified replacement moniker (used to tag resources that are meant to be replaced).
func (m Moniker) Replace() Moniker {
contract.Assert(!m.Replacement())
return m + replaceMonikerSuffix
}
// Unreplace returns the underlying replacement's moniker.
func (m Moniker) Unreplace() Moniker {
contract.Assert(m.Replacement())
return m[:len(m)-len(replaceMonikerSuffix)]
}
// Replacement returns true if this moniker refers to a resource that is meant to be replaced.
func (m Moniker) Replacement() bool {
return strings.HasSuffix(string(m), string(replaceMonikerSuffix))
}

View file

@ -22,7 +22,7 @@ import (
type Plan interface {
Empty() bool // true if the plan is empty.
Steps() Step // the first step to perform, linked to the rest.
Replaces() map[Moniker][]PropertyKey // resources being replaced and their properties.
Replaces() map[URN][]PropertyKey // resources being replaced and their properties.
Unchanged() map[Resource]Resource // the resources untouched by this plan.
Apply(prog Progress) (Snapshot, error, Step, ResourceState) // performs the operations specified in this plan.
}
@ -126,20 +126,20 @@ func NewPlan(ctx *Context, old Snapshot, new Snapshot) (Plan, error) {
}
type plan struct {
ctx *Context // this plan's context.
ns tokens.QName // the husk/namespace target being deployed into.
pkg tokens.Package // the package from which this snapshot came.
args core.Args // the arguments used to compile this package.
first *step // the first step to take.
replaces map[Moniker][]PropertyKey // resources being replaced and their properties.
unchanged map[Resource]Resource // the resources that are remaining the same without modification.
ctx *Context // this plan's context.
ns tokens.QName // the husk/namespace target being deployed into.
pkg tokens.Package // the package from which this snapshot came.
args core.Args // the arguments used to compile this package.
first *step // the first step to take.
replaces map[URN][]PropertyKey // resources being replaced and their properties.
unchanged map[Resource]Resource // the resources that are remaining the same without modification.
}
var _ Plan = (*plan)(nil)
func (p *plan) Replaces() map[Moniker][]PropertyKey { return p.replaces }
func (p *plan) Unchanged() map[Resource]Resource { return p.unchanged }
func (p *plan) Empty() bool { return p.Steps() == nil }
func (p *plan) Replaces() map[URN][]PropertyKey { return p.replaces }
func (p *plan) Unchanged() map[Resource]Resource { return p.unchanged }
func (p *plan) Empty() bool { return p.Steps() == nil }
func (p *plan) Steps() Step {
if p.first == nil {
@ -254,7 +254,7 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
//
// Any property changes that require replacement are applied, recursively, in a cascading manner.
for _, old := range pb.OldRes {
m := old.Moniker()
m := old.URN()
pb.Olds[m] = old
contract.Assert(old.HasID())
// Keep track of which dependents exist for all resources.
@ -263,12 +263,12 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
}
}
for _, new := range pb.NewRes {
pb.News[new.Moniker()] = new
pb.News[new.URN()] = new
}
// Find those things in old but not new, and add them to the delete queue.
for _, old := range pb.OldRes {
m := old.Moniker()
m := old.URN()
if _, hasnew := pb.News[m]; !hasnew {
step := newDeleteStep(pb.P, old)
pb.Deletes[m] = newPlanVertex(step)
@ -278,19 +278,19 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
// Find creates and updates: creates are those in new but not old, and updates are those in both.
for _, new := range pb.NewRes {
m := new.Moniker()
m := new.URN()
if old, hasold := pb.Olds[m]; hasold {
// The resource exists in both new and old; it could be an update. This resource is an update if one of
// these two conditions exist: 1) either the old and new properties don't match or 2) the update impact
// is assessed as having to replace the resource, in which case the ID will change. This might have a
// cascading impact on subsequent updates too, since those IDs must trigger recreations, etc.
contract.Assert(old.Type() == new.Type())
computed := new.Properties().ReplaceResources(func(r Moniker) Moniker {
computed := new.Properties().ReplaceResources(func(r URN) URN {
if pb.Replace(r) {
// If the resource is being replaced, simply mangle the moniker so that it's different; this value
// If the resource is being replaced, simply mangle the URN so that it's different; this value
// won't actually be used for anything other than the diffing algorithms below.
r = r.Replace()
glog.V(7).Infof("Patched resource '%v's moniker property: %v", m, r)
glog.V(7).Infof("Patched resource '%v's URN property: %v", m, r)
}
return r
})
@ -359,7 +359,7 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
// Clearly we must prohibit cycles in this overall graph of resource operations (hence the DAG part). To ensure
// this ordering, we will produce a plan graph whose vertices are operations and whose edges encode dependencies.
for _, old := range pb.OldRes {
m := old.Moniker()
m := old.URN()
if delete, isdelete := pb.Deletes[m]; isdelete {
pb.ConnectDelete(m, delete) // connect this delete so it happens before dependencies.
} else if update, isupdate := pb.Updates[m]; isupdate {
@ -367,7 +367,7 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
}
}
for _, new := range pb.NewRes {
m := new.Moniker()
m := new.URN()
if create, iscreate := pb.Creates[m]; iscreate {
pb.ConnectCreate(m, create) // connect this create so it happens after dependencies are created/updated.
}
@ -379,17 +379,17 @@ func newPlan(ctx *Context, old Snapshot, new Snapshot) (*plan, error) {
// planBuilder records a lot of the necessary information during the creation of a plan.
type planBuilder struct {
P *plan // the plan under construction.
Olds map[Moniker]Resource // a map of moniker to old resource.
OldRes []Resource // a flat list of old resources (in topological order).
News map[Moniker]Resource // a map of moniker to new resource.
NewRes []Resource // a flat list of new resources (in topological order).
Depends map[Moniker][]Moniker // a map of moniker to all existing (old) dependencies.
Creates map[Moniker]*planVertex // a map of pending creates to their associated vertex.
Updates map[Moniker]*planVertex // a map of pending updates to their associated vertex.
Deletes map[Moniker]*planVertex // a map of pending deletes to their associated vertex.
Replaces map[Moniker][]PropertyKey // a map of monikers scheduled for replacement to properties being replaced.
Unchanged map[Resource]Resource // a map of unchanged resources to their ID-stamped state.
P *plan // the plan under construction.
Olds map[URN]Resource // a map of URN to old resource.
OldRes []Resource // a flat list of old resources (in topological order).
News map[URN]Resource // a map of URN to new resource.
NewRes []Resource // a flat list of new resources (in topological order).
Depends map[URN][]URN // a map of URN to all existing (old) dependencies.
Creates map[URN]*planVertex // a map of pending creates to their associated vertex.
Updates map[URN]*planVertex // a map of pending updates to their associated vertex.
Deletes map[URN]*planVertex // a map of pending deletes to their associated vertex.
Replaces map[URN][]PropertyKey // a map of URNs scheduled for replacement to properties being replaced.
Unchanged map[Resource]Resource // a map of unchanged resources to their ID-stamped state.
}
// newPlanBuilder initializes a fresh plan state instance, ready to use for planning.
@ -427,32 +427,32 @@ func newPlanBuilder(ctx *Context, old Snapshot, new Snapshot) *planBuilder {
return &planBuilder{
P: p,
Olds: make(map[Moniker]Resource),
Olds: make(map[URN]Resource),
OldRes: oldres,
News: make(map[Moniker]Resource),
News: make(map[URN]Resource),
NewRes: newres,
Depends: make(map[Moniker][]Moniker),
Creates: make(map[Moniker]*planVertex),
Updates: make(map[Moniker]*planVertex),
Deletes: make(map[Moniker]*planVertex),
Replaces: make(map[Moniker][]PropertyKey),
Depends: make(map[URN][]URN),
Creates: make(map[URN]*planVertex),
Updates: make(map[URN]*planVertex),
Deletes: make(map[URN]*planVertex),
Replaces: make(map[URN][]PropertyKey),
Unchanged: make(map[Resource]Resource),
}
}
func (pb *planBuilder) Replace(m Moniker) bool {
func (pb *planBuilder) Replace(m URN) bool {
return len(pb.Replaces[m]) > 0
}
func (pb *planBuilder) ConnectCreate(m Moniker, v *planVertex) {
func (pb *planBuilder) ConnectCreate(m URN, v *planVertex) {
pb.connectCreateUpdate(m, v, false)
}
func (pb *planBuilder) ConnectUpdate(m Moniker, v *planVertex) {
func (pb *planBuilder) ConnectUpdate(m URN, v *planVertex) {
pb.connectCreateUpdate(m, v, true)
}
func (pb *planBuilder) connectCreateUpdate(m Moniker, v *planVertex, update bool) {
func (pb *planBuilder) connectCreateUpdate(m URN, v *planVertex, update bool) {
var label string
if update {
label = "Updating"
@ -482,7 +482,7 @@ func (pb *planBuilder) connectCreateUpdate(m Moniker, v *planVertex, update bool
}
}
func (pb *planBuilder) ConnectDelete(m Moniker, v *planVertex) {
func (pb *planBuilder) ConnectDelete(m URN, v *planVertex) {
// Add edges to:
// - any dependents that used to refer to this (and are necessarily being deleted or updated)
for _, dep := range pb.Depends[m] {
@ -501,7 +501,7 @@ func (pb *planBuilder) ConnectDelete(m Moniker, v *planVertex) {
func (pb *planBuilder) Plan() (*plan, error) {
// For all plan vertices with no ins, make them root nodes.
var roots []*planEdge
for _, vs := range []map[Moniker]*planVertex{pb.Creates, pb.Updates, pb.Deletes} {
for _, vs := range []map[URN]*planVertex{pb.Creates, pb.Updates, pb.Deletes} {
for _, v := range vs {
if len(v.Ins()) == 0 {
roots = append(roots, &planEdge{to: v})

View file

@ -145,8 +145,8 @@ func (p *Plugin) Name(t tokens.Type, props PropertyMap) (tokens.QName, error) {
req := &cocorpc.NameRequest{
Type: string(t),
Properties: MarshalProperties(p.ctx, props, MarshalOptions{
PermitOlds: true, // permit old monikers, since this is pre-update.
RawMonikers: true, // often used during moniker creation; IDs won't be ready.
PermitOlds: true, // permit old URNs, since this is pre-update.
RawURNs: true, // often used during URN creation; IDs won't be ready.
}),
}
@ -215,7 +215,7 @@ func (p *Plugin) Update(id ID, t tokens.Type, olds PropertyMap, news PropertyMap
Id: string(id),
Type: string(t),
Olds: MarshalProperties(p.ctx, olds, MarshalOptions{
PermitOlds: true, // permit old monikers since these are the old values.
PermitOlds: true, // permit old URNs since these are the old values.
}),
News: MarshalProperties(p.ctx, news, MarshalOptions{}),
}
@ -242,10 +242,10 @@ func (p *Plugin) UpdateImpact(id ID, t tokens.Type,
Id: string(id),
Type: string(t),
Olds: MarshalProperties(p.ctx, olds, MarshalOptions{
RawMonikers: true, // often used during moniker creation; IDs won't be ready.
RawURNs: true, // often used during URN creation; IDs won't be ready.
}),
News: MarshalProperties(p.ctx, news, MarshalOptions{
RawMonikers: true, // often used during moniker creation; IDs won't be ready.
RawURNs: true, // often used during URN creation; IDs won't be ready.
}),
}

View file

@ -150,7 +150,7 @@ func (m PropertyMap) ObjectOrErr(k PropertyKey, req bool) (*PropertyMap, error)
}
// ResourceOrErr checks that the given property is a resource, issuing an error if not; req indicates if required.
func (m PropertyMap) ResourceOrErr(k PropertyKey, req bool) (*Moniker, error) {
func (m PropertyMap) ResourceOrErr(k PropertyKey, req bool) (*URN, error) {
if v, has := m[k]; has && !v.IsNull() {
if !v.IsResource() {
return nil, fmt.Errorf("property '%v' is not an object (%v)", k, reflect.TypeOf(v.V))
@ -226,11 +226,11 @@ func (m PropertyMap) ReqObjectOrErr(k PropertyKey) (PropertyMap, error) {
return *o, nil
}
// ReqResourceOrErr checks that the given property exists and has the type moniker.
func (m PropertyMap) ReqResourceOrErr(k PropertyKey) (Moniker, error) {
// ReqResourceOrErr checks that the given property exists and has the type URN.
func (m PropertyMap) ReqResourceOrErr(k PropertyKey) (URN, error) {
r, err := m.ResourceOrErr(k, true)
if err != nil {
return Moniker(""), err
return URN(""), err
}
return *r, nil
}
@ -270,25 +270,25 @@ func (m PropertyMap) OptObjectOrErr(k PropertyKey) (*PropertyMap, error) {
return m.ObjectOrErr(k, false)
}
// OptResourceOrErr checks that the given property has the type moniker, if it exists.
func (m PropertyMap) OptResourceOrErr(k PropertyKey) (*Moniker, error) {
// OptResourceOrErr checks that the given property has the type URN, if it exists.
func (m PropertyMap) OptResourceOrErr(k PropertyKey) (*URN, error) {
return m.ResourceOrErr(k, false)
}
// AllResources finds all resource monikers, transitively throughout the property map, and returns them.
func (props PropertyMap) AllResources() map[Moniker]bool {
monikers := make(map[Moniker]bool)
// AllResources finds all resource URNs, transitively throughout the property map, and returns them.
func (props PropertyMap) AllResources() map[URN]bool {
URNs := make(map[URN]bool)
for _, k := range StablePropertyKeys(props) {
for m, v := range props[k].AllResources() {
monikers[m] = v
URNs[m] = v
}
}
return monikers
return URNs
}
// ReplaceResources finds all resources and lets an updater function update them if necessary. This is often used
// during a "replacement"-style updated, to replace all monikers of a certain value with another.
func (props PropertyMap) ReplaceResources(updater func(Moniker) Moniker) PropertyMap {
// during a "replacement"-style updated, to replace all URNs of a certain value with another.
func (props PropertyMap) ReplaceResources(updater func(URN) URN) PropertyMap {
result := make(PropertyMap)
for _, k := range StablePropertyKeys(props) {
result[k] = props[k].ReplaceResources(updater)
@ -302,14 +302,14 @@ func NewPropertyNumber(v float64) PropertyValue { return PropertyValue{v}
func NewPropertyString(v string) PropertyValue { return PropertyValue{v} }
func NewPropertyArray(v []PropertyValue) PropertyValue { return PropertyValue{v} }
func NewPropertyObject(v PropertyMap) PropertyValue { return PropertyValue{v} }
func NewPropertyResource(v Moniker) PropertyValue { return PropertyValue{v} }
func NewPropertyResource(v URN) PropertyValue { return PropertyValue{v} }
func (v PropertyValue) BoolValue() bool { return v.V.(bool) }
func (v PropertyValue) NumberValue() float64 { return v.V.(float64) }
func (v PropertyValue) StringValue() string { return v.V.(string) }
func (v PropertyValue) ArrayValue() []PropertyValue { return v.V.([]PropertyValue) }
func (v PropertyValue) ObjectValue() PropertyMap { return v.V.(PropertyMap) }
func (v PropertyValue) ResourceValue() Moniker { return v.V.(Moniker) }
func (v PropertyValue) ResourceValue() URN { return v.V.(URN) }
func (b PropertyValue) IsNull() bool {
return b.V == nil
@ -335,32 +335,32 @@ func (b PropertyValue) IsObject() bool {
return is
}
func (b PropertyValue) IsResource() bool {
_, is := b.V.(Moniker)
_, is := b.V.(URN)
return is
}
// AllResources finds all resource monikers, transitively throughout the property value, and returns them.
func (v PropertyValue) AllResources() map[Moniker]bool {
monikers := make(map[Moniker]bool)
// AllResources finds all resource URNs, transitively throughout the property value, and returns them.
func (v PropertyValue) AllResources() map[URN]bool {
URNs := make(map[URN]bool)
if v.IsResource() {
monikers[v.ResourceValue()] = true
URNs[v.ResourceValue()] = true
} else if v.IsArray() {
for _, elem := range v.ArrayValue() {
for m, v := range elem.AllResources() {
monikers[m] = v
URNs[m] = v
}
}
} else if v.IsObject() {
for m, v := range v.ObjectValue().AllResources() {
monikers[m] = v
URNs[m] = v
}
}
return monikers
return URNs
}
// ReplaceResources finds all resources and lets an updater function update them if necessary. This is often used
// during a "replacement"-style updated, to replace all monikers of a certain value with another.
func (v PropertyValue) ReplaceResources(updater func(Moniker) Moniker) PropertyValue {
// during a "replacement"-style updated, to replace all URNs of a certain value with another.
func (v PropertyValue) ReplaceResources(updater func(URN) URN) PropertyValue {
if v.IsResource() {
m := v.ResourceValue()
return NewPropertyResource(updater(m))

View file

@ -24,13 +24,13 @@ type ID string
// Resource is an instance of a resource with an ID, type, and bag of state.
type Resource interface {
ID() ID // the resource's unique ID, assigned by the resource provider (or blank if uncreated).
Moniker() Moniker // the resource's object moniker, a human-friendly, unique name for the resource.
URN() URN // the resource's object urn, a human-friendly, unique name for the resource.
Type() tokens.Type // the resource's type.
Properties() PropertyMap // the resource's property map.
HasID() bool // returns true if the resource has been assigned an ID.
SetID(id ID) // assignes an ID to this resource, for those under creation.
HasMoniker() bool // returns true if the resource has been assigned moniker.
SetMoniker(m Moniker) // assignes a moniker to this resource, for those under creation.
HasURN() bool // returns true if the resource has been assigned urn.
SetURN(m URN) // assignes a urn to this resource, for those under creation.
}
// ResourceState is returned when an error has occurred during a resource provider operation. It indicates whether the
@ -47,13 +47,13 @@ func IsResourceVertex(v *heapstate.ObjectVertex) bool { return IsResourceType(v.
type resource struct {
id ID // the resource's unique ID, assigned by the resource provider (or blank if uncreated).
moniker Moniker // the resource's object moniker, a human-friendly, unique name for the resource.
urn URN // the resource's object urn, a human-friendly, unique name for the resource.
t tokens.Type // the resource's type.
properties PropertyMap // the resource's property map.
}
func (r *resource) ID() ID { return r.id }
func (r *resource) Moniker() Moniker { return r.moniker }
func (r *resource) URN() URN { return r.urn }
func (r *resource) Type() tokens.Type { return r.t }
func (r *resource) Properties() PropertyMap { return r.properties }
@ -63,17 +63,17 @@ func (r *resource) SetID(id ID) {
r.id = id
}
func (r *resource) HasMoniker() bool { return (string(r.moniker) != "") }
func (r *resource) SetMoniker(m Moniker) {
contract.Requiref(!r.HasMoniker(), "moniker", "empty")
r.moniker = m
func (r *resource) HasURN() bool { return (string(r.urn) != "") }
func (r *resource) SetURN(m URN) {
contract.Requiref(!r.HasURN(), "urn", "empty")
r.urn = m
}
// NewResource creates a new resource from the information provided.
func NewResource(id ID, moniker Moniker, t tokens.Type, properties PropertyMap) Resource {
func NewResource(id ID, urn URN, t tokens.Type, properties PropertyMap) Resource {
return &resource{
id: id,
moniker: moniker,
urn: urn,
t: t,
properties: properties,
}
@ -85,9 +85,9 @@ func NewObjectResource(ctx *Context, obj *rt.Object) Resource {
t := obj.Type()
contract.Assert(IsResourceType(t))
// Extract the moniker. This must already exist.
m, hasm := ctx.ObjMks[obj]
contract.Assertf(!hasm, "Object already assigned a moniker '%v'; double allocation detected", m)
// Extract the urn. This must already exist.
urn, hasm := ctx.ObjURN[obj]
contract.Assertf(!hasm, "Object already assigned a urn '%v'; double allocation detected", urn)
// Do a deep copy of the resource properties. This ensures property serializability.
props := cloneObject(ctx, obj)
@ -101,7 +101,7 @@ func NewObjectResource(ctx *Context, obj *rt.Object) Resource {
// cloneObject creates a property map out of a runtime object. The result is fully serializable in the sense that it
// can be stored in a JSON or YAML file, serialized over an RPC interface, etc. In particular, any references to other
// resources are replaced with their moniker equivalents, which the runtime understands.
// resources are replaced with their urn equivalents, which the runtime understands.
func cloneObject(ctx *Context, obj *rt.Object) PropertyMap {
contract.Assert(obj != nil)
src := obj.PropertyValues()
@ -120,10 +120,10 @@ func cloneObject(ctx *Context, obj *rt.Object) PropertyMap {
func cloneObjectValue(ctx *Context, obj *rt.Object) (PropertyValue, bool) {
t := obj.Type()
if IsResourceType(t) {
// For resources, simply look up the moniker from the resource map.
m, hasm := ctx.ObjMks[obj]
// For resources, simply look up the urn from the resource map.
urn, hasm := ctx.ObjURN[obj]
contract.Assertf(hasm, "Missing object reference; possible out of order dependency walk")
return NewPropertyResource(m), true
return NewPropertyResource(urn), true
}
switch t {

View file

@ -14,28 +14,28 @@ type resourceGraph struct {
var _ graph.Graph = (*resourceGraph)(nil)
// newResourceGraph produces a DAG using the resources' properties embedded moniker information.
// newResourceGraph produces a DAG using the resources' properties embedded URN information.
func newResourceGraph(resources []Resource) *resourceGraph {
// First make two maps: one with monikers to resources, the other with resources to vertices.
mks := make(map[Moniker]Resource)
// First make two maps: one with URNs to resources, the other with resources to vertices.
urns := make(map[URN]Resource)
verts := make(map[Resource]*resourceVertex)
for _, res := range resources {
contract.Assert(res != nil)
m := res.Moniker()
contract.Assertf(mks[m] == nil, "Unexpected duplicate entry '%v' in resource list", m)
mks[m] = res
urn := res.URN()
contract.Assertf(urns[urn] == nil, "Unexpected duplicate entry '%v' in resource list", urn)
urns[urn] = res
verts[res] = newResourceVertex(res)
}
// Now walk the list of resources and connect them to their dependencies.
for _, res := range resources {
m := res.Moniker()
urn := res.URN()
fromv := verts[res]
for ref := range res.Properties().AllResources() {
to := mks[ref]
contract.Assertf(to != nil, "Missing resource for target; from=%v to=%v", m, ref)
for dep := range res.Properties().AllResources() {
to := urns[dep]
contract.Assertf(to != nil, "Missing resource for target; from=%v to=%v", urn, dep)
tov := verts[to]
contract.Assertf(tov != nil, "Missing vertex entry for target; from=%v to=%v", m, ref)
contract.Assertf(tov != nil, "Missing vertex entry for target; from=%v to=%v", urn, dep)
fromv.connectTo(tov)
}
}

View file

@ -14,12 +14,12 @@ import (
// MarshalOptions controls the marshaling of RPC structures.
type MarshalOptions struct {
PermitOlds bool // true to permit old monikers in the properties (e.g., for pre-update).
RawMonikers bool // true to marshal monikers "as-is"; often used when ID mappings aren't known yet.
PermitOlds bool // true to permit old URNs in the properties (e.g., for pre-update).
RawURNs bool // true to marshal URNs "as-is"; often used when ID mappings aren't known yet.
}
// MarshalProperties marshals a resource's property map as a "JSON-like" protobuf structure. Any monikers are replaced
// with their resource IDs during marshaling; it is an error to marshal a moniker for a resource without an ID.
// MarshalProperties marshals a resource's property map as a "JSON-like" protobuf structure. Any URNs are replaced
// with their resource IDs during marshaling; it is an error to marshal a URN for a resource without an ID.
func MarshalProperties(ctx *Context, props PropertyMap, opts MarshalOptions) *structpb.Struct {
result := &structpb.Struct{
Fields: make(map[string]*structpb.Value),
@ -79,21 +79,21 @@ func MarshalPropertyValue(ctx *Context, v PropertyValue, opts MarshalOptions) (*
} else if v.IsResource() {
var wire string
m := v.ResourceValue()
if opts.RawMonikers {
if opts.RawURNs {
wire = string(m)
} else {
var id ID
if res, has := ctx.MksRes[m]; has {
if res, has := ctx.URNRes[m]; has {
id = res.ID() // found a new resource with this ID, use it.
} else if oldid, has := ctx.MksOldIDs[m]; opts.PermitOlds && has {
} else if oldid, has := ctx.URNOldIDs[m]; opts.PermitOlds && has {
id = oldid // found an old resource, maybe deleted, so use that.
} else {
contract.Failf("Expected resource moniker '%v' to exist at marshal time", m)
contract.Failf("Expected resource URN '%v' to exist at marshal time", m)
}
contract.Assertf(id != "", "Expected resource moniker '%v' to have an ID at marshal time", m)
contract.Assertf(id != "", "Expected resource URN '%v' to have an ID at marshal time", m)
wire = string(id)
}
glog.V(7).Infof("Serializing resource moniker '%v' as '%v' (raw=%v)", m, wire, opts.RawMonikers)
glog.V(7).Infof("Serializing resource URN '%v' as '%v' (raw=%v)", m, wire, opts.RawURNs)
return &structpb.Value{
Kind: &structpb.Value_StringValue{
StringValue: wire,

View file

@ -24,7 +24,7 @@ type Snapshot interface {
Args() core.Args // the arguments used to compile this package.
Resources() []Resource // a topologically sorted list of resources (based on dependencies).
ResourceByID(id ID, t tokens.Type) Resource // looks up a resource by ID and type.
ResourceByMoniker(m Moniker) Resource // looks up a resource by its moniker.
ResourceByURN(urn URN) Resource // looks up a resource by its URN.
ResourceByObject(obj *rt.Object) Resource // looks up a resource by its object.
}
@ -43,7 +43,7 @@ func NewGraphSnapshot(ctx *Context, ns tokens.QName, pkg tokens.Package, args co
if old != nil {
for _, res := range old.Resources() {
contract.Assert(res.HasID())
ctx.MksOldIDs[res.Moniker()] = res.ID()
ctx.URNOldIDs[res.URN()] = res.ID()
}
}
@ -53,8 +53,8 @@ func NewGraphSnapshot(ctx *Context, ns tokens.QName, pkg tokens.Package, args co
return nil, err
}
// Next, name all resources, create their monikers and objects, and maps that we will use. Note that we must do
// this in DAG order (guaranteed by our topological sort above), so that referenced monikers are available.
// Next, name all resources, create their URNs and objects, and maps that we will use. Note that we must do
// this in DAG order (guaranteed by our topological sort above), so that referenced URNs are available.
resources, err := createResources(ctx, ns, heap, resobjs)
if err != nil {
return nil, err
@ -82,15 +82,15 @@ func (s *snapshot) ResourceByID(id ID, t tokens.Type) Resource {
return nil
}
func (s *snapshot) ResourceByMoniker(m Moniker) Resource { return s.ctx.MksRes[m] }
func (s *snapshot) ResourceByURN(urn URN) Resource { return s.ctx.URNRes[urn] }
func (s *snapshot) ResourceByObject(obj *rt.Object) Resource { return s.ctx.ObjRes[obj] }
// createResources uses a graph to create monikers and resource objects for every resource within. It
// returns two maps for further use: a map of vertex to its new resource object, and a map of vertex to its moniker.
// createResources uses a graph to create URNs and resource objects for every resource within. It
// returns two maps for further use: a map of vertex to its new resource object, and a map of vertex to its URN.
func createResources(ctx *Context, husk tokens.QName, heap *heapstate.Heap, resobjs []*rt.Object) ([]Resource, error) {
var resources []Resource
for _, resobj := range resobjs {
// Create an object resource without a moniker.
// Create an object resource without a URN.
res := NewObjectResource(ctx, resobj)
// Now fetch this resource's name by looking up its provider and doing an RPC.
@ -104,20 +104,20 @@ func createResources(ctx *Context, husk tokens.QName, heap *heapstate.Heap, reso
return nil, err
}
// Now compute a unique moniker for this object and ensure we haven't had any collisions.
// Now compute a unique URN for this object and ensure we haven't had any collisions.
alloc := heap.Alloc(resobj)
moniker := NewMoniker(husk, alloc.Mod.Tok, t, name)
glog.V(7).Infof("Resource moniker computed: %v", moniker)
if _, exists := ctx.MksRes[moniker]; exists {
// If this moniker is already in use, issue an error, ignore this one, and break. The break is necessary
// because subsequent resources might contain references to this moniker and would fail to find it.
ctx.Diag.Errorf(errors.ErrorDuplicateMonikerNames.At(alloc.Loc), moniker)
urn := NewURN(husk, alloc.Mod.Tok, t, name)
glog.V(7).Infof("Resource URN computed: %v", urn)
if _, exists := ctx.URNRes[urn]; exists {
// If this URN is already in use, issue an error, ignore this one, and break. The break is necessary
// because subsequent resources might contain references to this URN and would fail to find it.
ctx.Diag.Errorf(errors.ErrorDuplicateURNNames.At(alloc.Loc), urn)
break
} else {
res.SetMoniker(moniker)
res.SetURN(urn)
ctx.ObjRes[resobj] = res
ctx.MksRes[moniker] = res
ctx.ObjMks[resobj] = moniker
ctx.URNRes[urn] = res
ctx.ObjURN[resobj] = urn
}
resources = append(resources, res)
}

77
pkg/resource/urn.go Normal file
View file

@ -0,0 +1,77 @@
// Copyright 2016 Pulumi, Inc. All rights reserved.
package resource
import (
"strings"
"github.com/pulumi/coconut/pkg/tokens"
"github.com/pulumi/coconut/pkg/util/contract"
)
// URN is a friendly, but unique, URN for a resource, most often auto-assigned by Coconut. These are
// used as unique IDs for objects, and help us to perform graph diffing and resolution of resource objects.
//
// In theory, we could support manually assigned URIs in the future. For the time being, however, we have opted to
// simplify developers' lives by mostly automating the generation of them algorithmically. The one caveat where it
// isn't truly automatic is that a developer -- or resource provider -- must provide a semi-unique name part.
//
// Each resource URN is of the form:
//
// urn:coconut:<Namespace>::<AllocModule>::<Type>::<Name>
//
// wherein each element is the following:
//
// <Namespace> The namespace being deployed into
// <AllocModule> The module token in which the object was allocated
// <Type> The object type's full type token
// <Name> The human-friendly name identifier assigned by the developer or provider
//
// In the future, we may add elements to the URN; it is more important that it is unique than it is human-typable.
type URN string
const (
URNPrefix = "urn:" + URNNamespaceID + ":" // the standard URN prefix
URNNamespaceID = "coconut" // the URN namespace
URNNameDelimiter = "::" // the delimiter between URN name elements
)
// NewURN creates a unique resource URN for the given resource object.
func NewURN(ns tokens.QName, alloc tokens.Module, t tokens.Type, name tokens.QName) URN {
urn := URN(
URNPrefix +
string(ns) +
URNNameDelimiter + string(alloc) +
URNNameDelimiter + string(t) +
URNNameDelimiter + string(name),
)
contract.Assert(!urn.Replacement())
return urn
}
// replaceURNSuffix is the suffix for URNs referring to resources that are being replaced.
const replaceURNSuffix = URN("#<new-id(replace)>")
// Name returns the name part of a URN.
func (urn URN) Name() string {
urns := string(urn)
contract.Assert(strings.HasPrefix(urns, URNPrefix))
return urns[len(URNPrefix)+1:]
}
// Replace returns a new, modified replacement URN (used to tag resources that are meant to be replaced).
func (urn URN) Replace() URN {
contract.Assert(!urn.Replacement())
return urn + replaceURNSuffix
}
// Unreplace returns the underlying replacement's URN.
func (urn URN) Unreplace() URN {
contract.Assert(urn.Replacement())
return urn[:len(urn)-len(replaceURNSuffix)]
}
// Replacement returns true if this URN refers to a resource that is meant to be replaced.
func (urn URN) Replacement() bool {
return strings.HasSuffix(string(urn), string(replaceURNSuffix))
}