Merge pull request #1010 from hashicorp/f-ast-graph
core: formalize internals to do graph transformation steps and walking over an AST tree for execution
This commit is contained in:
commit
ed115f495b
8
Makefile
8
Makefile
|
@ -40,6 +40,14 @@ updatedeps:
|
|||
| sort -u \
|
||||
| xargs go get -f -u -v
|
||||
|
||||
cover:
|
||||
@go tool cover 2>/dev/null; if [ $$? -eq 3 ]; then \
|
||||
go get -u golang.org/x/tools/cmd/cover; \
|
||||
fi
|
||||
go test $(TEST) -coverprofile=coverage.out
|
||||
go tool cover -html=coverage.out
|
||||
rm coverage.out
|
||||
|
||||
# vet runs the Go source code static analysis tool `vet` to find
|
||||
# any common errors.
|
||||
vet:
|
||||
|
|
|
@ -58,11 +58,7 @@ func (c *GraphCommand) Run(args []string) int {
|
|||
return 1
|
||||
}
|
||||
|
||||
opts := &terraform.GraphDotOpts{
|
||||
ModuleDepth: moduleDepth,
|
||||
}
|
||||
|
||||
c.Ui.Output(terraform.GraphDot(g, opts))
|
||||
c.Ui.Output(terraform.GraphDot(g, nil))
|
||||
|
||||
return 0
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ func TestGraph(t *testing.T) {
|
|||
}
|
||||
|
||||
output := ui.OutputWriter.String()
|
||||
if !strings.Contains(output, "digraph {") {
|
||||
if !strings.Contains(output, "provider.test") {
|
||||
t.Fatalf("doesn't look like digraph: %s", output)
|
||||
}
|
||||
}
|
||||
|
@ -73,7 +73,7 @@ func TestGraph_noArgs(t *testing.T) {
|
|||
}
|
||||
|
||||
output := ui.OutputWriter.String()
|
||||
if !strings.Contains(output, "digraph {") {
|
||||
if !strings.Contains(output, "provider.test") {
|
||||
t.Fatalf("doesn't look like digraph: %s", output)
|
||||
}
|
||||
}
|
||||
|
@ -99,7 +99,7 @@ func TestGraph_plan(t *testing.T) {
|
|||
}
|
||||
|
||||
output := ui.OutputWriter.String()
|
||||
if !strings.Contains(output, "digraph {") {
|
||||
if !strings.Contains(output, "provider.test") {
|
||||
t.Fatalf("doesn't look like digraph: %s", output)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -92,6 +92,51 @@ func (r *RawConfig) Interpolate(vs map[string]ast.Variable) error {
|
|||
})
|
||||
}
|
||||
|
||||
// Merge merges another RawConfig into this one (overriding any conflicting
|
||||
// values in this config) and returns a new config. The original config
|
||||
// is not modified.
|
||||
func (r *RawConfig) Merge(other *RawConfig) *RawConfig {
|
||||
// Merge the raw configurations
|
||||
raw := make(map[string]interface{})
|
||||
for k, v := range r.Raw {
|
||||
raw[k] = v
|
||||
}
|
||||
for k, v := range other.Raw {
|
||||
raw[k] = v
|
||||
}
|
||||
|
||||
// Create the result
|
||||
result, err := NewRawConfig(raw)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// Merge the interpolated results
|
||||
result.config = make(map[string]interface{})
|
||||
for k, v := range r.config {
|
||||
result.config[k] = v
|
||||
}
|
||||
for k, v := range other.config {
|
||||
result.config[k] = v
|
||||
}
|
||||
|
||||
// Build the unknown keys
|
||||
unknownKeys := make(map[string]struct{})
|
||||
for _, k := range r.unknownKeys {
|
||||
unknownKeys[k] = struct{}{}
|
||||
}
|
||||
for _, k := range other.unknownKeys {
|
||||
unknownKeys[k] = struct{}{}
|
||||
}
|
||||
|
||||
result.unknownKeys = make([]string, 0, len(unknownKeys))
|
||||
for k, _ := range unknownKeys {
|
||||
result.unknownKeys = append(result.unknownKeys, k)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func (r *RawConfig) init() error {
|
||||
r.config = r.Raw
|
||||
r.Interpolations = nil
|
||||
|
|
|
@ -114,6 +114,87 @@ func TestRawConfig_double(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestRawConfig_merge(t *testing.T) {
|
||||
raw1 := map[string]interface{}{
|
||||
"foo": "${var.foo}",
|
||||
"bar": "${var.bar}",
|
||||
}
|
||||
|
||||
rc1, err := NewRawConfig(raw1)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
{
|
||||
vars := map[string]ast.Variable{
|
||||
"var.foo": ast.Variable{
|
||||
Value: "foovalue",
|
||||
Type: ast.TypeString,
|
||||
},
|
||||
"var.bar": ast.Variable{
|
||||
Value: "nope",
|
||||
Type: ast.TypeString,
|
||||
},
|
||||
}
|
||||
if err := rc1.Interpolate(vars); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
raw2 := map[string]interface{}{
|
||||
"bar": "${var.bar}",
|
||||
"baz": "${var.baz}",
|
||||
}
|
||||
|
||||
rc2, err := NewRawConfig(raw2)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
{
|
||||
vars := map[string]ast.Variable{
|
||||
"var.bar": ast.Variable{
|
||||
Value: "barvalue",
|
||||
Type: ast.TypeString,
|
||||
},
|
||||
"var.baz": ast.Variable{
|
||||
Value: UnknownVariableValue,
|
||||
Type: ast.TypeString,
|
||||
},
|
||||
}
|
||||
if err := rc2.Interpolate(vars); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Merge the two
|
||||
rc3 := rc1.Merge(rc2)
|
||||
|
||||
// Raw should be merged
|
||||
raw3 := map[string]interface{}{
|
||||
"foo": "${var.foo}",
|
||||
"bar": "${var.bar}",
|
||||
"baz": "${var.baz}",
|
||||
}
|
||||
if !reflect.DeepEqual(rc3.Raw, raw3) {
|
||||
t.Fatalf("bad: %#v", rc3.Raw)
|
||||
}
|
||||
|
||||
actual := rc3.Config()
|
||||
expected := map[string]interface{}{
|
||||
"foo": "foovalue",
|
||||
"bar": "barvalue",
|
||||
}
|
||||
if !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
|
||||
expectedKeys := []string{"baz"}
|
||||
if !reflect.DeepEqual(rc3.UnknownKeys(), expectedKeys) {
|
||||
t.Fatalf("bad: %#v", rc3.UnknownKeys())
|
||||
}
|
||||
}
|
||||
|
||||
func TestRawConfig_syntax(t *testing.T) {
|
||||
raw := map[string]interface{}{
|
||||
"foo": "${var",
|
||||
|
|
|
@ -0,0 +1,163 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
)
|
||||
|
||||
// AcyclicGraph is a specialization of Graph that cannot have cycles. With
|
||||
// this property, we get the property of sane graph traversal.
|
||||
type AcyclicGraph struct {
|
||||
Graph
|
||||
}
|
||||
|
||||
// WalkFunc is the callback used for walking the graph.
|
||||
type WalkFunc func(Vertex) error
|
||||
|
||||
// Root returns the root of the DAG, or an error.
|
||||
//
|
||||
// Complexity: O(V)
|
||||
func (g *AcyclicGraph) Root() (Vertex, error) {
|
||||
roots := make([]Vertex, 0, 1)
|
||||
for _, v := range g.Vertices() {
|
||||
if g.UpEdges(v).Len() == 0 {
|
||||
roots = append(roots, v)
|
||||
}
|
||||
}
|
||||
|
||||
if len(roots) > 1 {
|
||||
// TODO(mitchellh): make this error message a lot better
|
||||
return nil, fmt.Errorf("multiple roots: %#v", roots)
|
||||
}
|
||||
|
||||
if len(roots) == 0 {
|
||||
return nil, fmt.Errorf("no roots found")
|
||||
}
|
||||
|
||||
return roots[0], nil
|
||||
}
|
||||
|
||||
// Validate validates the DAG. A DAG is valid if it has a single root
|
||||
// with no cycles.
|
||||
func (g *AcyclicGraph) Validate() error {
|
||||
if _, err := g.Root(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Look for cycles of more than 1 component
|
||||
var err error
|
||||
var cycles [][]Vertex
|
||||
for _, cycle := range StronglyConnected(&g.Graph) {
|
||||
if len(cycle) > 1 {
|
||||
cycles = append(cycles, cycle)
|
||||
}
|
||||
}
|
||||
if len(cycles) > 0 {
|
||||
for _, cycle := range cycles {
|
||||
cycleStr := make([]string, len(cycle))
|
||||
for j, vertex := range cycle {
|
||||
cycleStr[j] = VertexName(vertex)
|
||||
}
|
||||
|
||||
err = multierror.Append(err, fmt.Errorf(
|
||||
"Cycle: %s", strings.Join(cycleStr, ", ")))
|
||||
}
|
||||
}
|
||||
|
||||
// Look for cycles to self
|
||||
for _, e := range g.Edges() {
|
||||
if e.Source() == e.Target() {
|
||||
err = multierror.Append(err, fmt.Errorf(
|
||||
"Self reference: %s", VertexName(e.Source())))
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// Walk walks the graph, calling your callback as each node is visited.
|
||||
// This will walk nodes in parallel if it can. Because the walk is done
|
||||
// in parallel, the error returned will be a multierror.
|
||||
func (g *AcyclicGraph) Walk(cb WalkFunc) error {
|
||||
// Cache the vertices since we use it multiple times
|
||||
vertices := g.Vertices()
|
||||
|
||||
// Build the waitgroup that signals when we're done
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(len(vertices))
|
||||
doneCh := make(chan struct{})
|
||||
go func() {
|
||||
defer close(doneCh)
|
||||
wg.Wait()
|
||||
}()
|
||||
|
||||
// The map of channels to watch to wait for vertices to finish
|
||||
vertMap := make(map[Vertex]chan struct{})
|
||||
for _, v := range vertices {
|
||||
vertMap[v] = make(chan struct{})
|
||||
}
|
||||
|
||||
// The map of whether a vertex errored or not during the walk
|
||||
var errLock sync.Mutex
|
||||
var errs error
|
||||
errMap := make(map[Vertex]bool)
|
||||
for _, v := range vertices {
|
||||
// Build our list of dependencies and the list of channels to
|
||||
// wait on until we start executing for this vertex.
|
||||
depsRaw := g.DownEdges(v).List()
|
||||
deps := make([]Vertex, len(depsRaw))
|
||||
depChs := make([]<-chan struct{}, len(deps))
|
||||
for i, raw := range depsRaw {
|
||||
deps[i] = raw.(Vertex)
|
||||
depChs[i] = vertMap[deps[i]]
|
||||
}
|
||||
|
||||
// Get our channel so that we can close it when we're done
|
||||
ourCh := vertMap[v]
|
||||
|
||||
// Start the goroutine to wait for our dependencies
|
||||
readyCh := make(chan bool)
|
||||
go func(deps []Vertex, chs []<-chan struct{}, readyCh chan<- bool) {
|
||||
// First wait for all the dependencies
|
||||
for _, ch := range chs {
|
||||
<-ch
|
||||
}
|
||||
|
||||
// Then, check the map to see if any of our dependencies failed
|
||||
errLock.Lock()
|
||||
defer errLock.Unlock()
|
||||
for _, dep := range deps {
|
||||
if errMap[dep] {
|
||||
readyCh <- false
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
readyCh <- true
|
||||
}(deps, depChs, readyCh)
|
||||
|
||||
// Start the goroutine that executes
|
||||
go func(v Vertex, doneCh chan<- struct{}, readyCh <-chan bool) {
|
||||
defer close(doneCh)
|
||||
defer wg.Done()
|
||||
|
||||
var err error
|
||||
if ready := <-readyCh; ready {
|
||||
err = cb(v)
|
||||
}
|
||||
|
||||
errLock.Lock()
|
||||
defer errLock.Unlock()
|
||||
if err != nil {
|
||||
errMap[v] = true
|
||||
errs = multierror.Append(errs, err)
|
||||
}
|
||||
}(v, ourCh, readyCh)
|
||||
}
|
||||
|
||||
<-doneCh
|
||||
return errs
|
||||
}
|
|
@ -0,0 +1,158 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sync"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestAcyclicGraphRoot(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
|
||||
if root, err := g.Root(); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
} else if root != 3 {
|
||||
t.Fatalf("bad: %#v", root)
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphRoot_cycle(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 3))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
|
||||
if _, err := g.Root(); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphRoot_multiple(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
|
||||
if _, err := g.Root(); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphValidate(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
|
||||
if err := g.Validate(); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphValidate_cycle(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 1))
|
||||
|
||||
if err := g.Validate(); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphValidate_cycleSelf(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Connect(BasicEdge(1, 1))
|
||||
|
||||
if err := g.Validate(); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestAcyclicGraphWalk(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
|
||||
var visits []Vertex
|
||||
var lock sync.Mutex
|
||||
err := g.Walk(func(v Vertex) error {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
visits = append(visits, v)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
expected := [][]Vertex{
|
||||
{1, 2, 3},
|
||||
{2, 1, 3},
|
||||
}
|
||||
for _, e := range expected {
|
||||
if reflect.DeepEqual(visits, e) {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
t.Fatalf("bad: %#v", visits)
|
||||
}
|
||||
|
||||
func TestAcyclicGraphWalk_error(t *testing.T) {
|
||||
var g AcyclicGraph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(3, 2))
|
||||
g.Connect(BasicEdge(3, 1))
|
||||
|
||||
var visits []Vertex
|
||||
var lock sync.Mutex
|
||||
err := g.Walk(func(v Vertex) error {
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
if v == 2 {
|
||||
return fmt.Errorf("error")
|
||||
}
|
||||
|
||||
visits = append(visits, v)
|
||||
return nil
|
||||
})
|
||||
if err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
|
||||
expected := [][]Vertex{
|
||||
{1},
|
||||
}
|
||||
for _, e := range expected {
|
||||
if reflect.DeepEqual(visits, e) {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
t.Fatalf("bad: %#v", visits)
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// Edge represents an edge in the graph, with a source and target vertex.
|
||||
type Edge interface {
|
||||
Source() Vertex
|
||||
Target() Vertex
|
||||
|
||||
Hashable
|
||||
}
|
||||
|
||||
// BasicEdge returns an Edge implementation that simply tracks the source
|
||||
// and target given as-is.
|
||||
func BasicEdge(source, target Vertex) Edge {
|
||||
return &basicEdge{S: source, T: target}
|
||||
}
|
||||
|
||||
// basicEdge is a basic implementation of Edge that has the source and
|
||||
// target vertex.
|
||||
type basicEdge struct {
|
||||
S, T Vertex
|
||||
}
|
||||
|
||||
func (e *basicEdge) Hashcode() interface{} {
|
||||
return fmt.Sprintf("%p-%p", e.S, e.T)
|
||||
}
|
||||
|
||||
func (e *basicEdge) Source() Vertex {
|
||||
return e.S
|
||||
}
|
||||
|
||||
func (e *basicEdge) Target() Vertex {
|
||||
return e.T
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestBasicEdgeHashcode(t *testing.T) {
|
||||
e1 := BasicEdge(1, 2)
|
||||
e2 := BasicEdge(1, 2)
|
||||
if e1.Hashcode() != e2.Hashcode() {
|
||||
t.Fatalf("bad")
|
||||
}
|
||||
}
|
||||
|
||||
func TestBasicEdgeHashcode_pointer(t *testing.T) {
|
||||
type test struct {
|
||||
Value string
|
||||
}
|
||||
|
||||
v1, v2 := &test{"foo"}, &test{"bar"}
|
||||
e1 := BasicEdge(v1, v2)
|
||||
e2 := BasicEdge(v1, v2)
|
||||
if e1.Hashcode() != e2.Hashcode() {
|
||||
t.Fatalf("bad")
|
||||
}
|
||||
}
|
|
@ -0,0 +1,219 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"sort"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// Graph is used to represent a dependency graph.
|
||||
type Graph struct {
|
||||
vertices *Set
|
||||
edges *Set
|
||||
downEdges map[Vertex]*Set
|
||||
upEdges map[Vertex]*Set
|
||||
once sync.Once
|
||||
}
|
||||
|
||||
// Vertex of the graph.
|
||||
type Vertex interface{}
|
||||
|
||||
// NamedVertex is an optional interface that can be implemented by Vertex
|
||||
// to give it a human-friendly name that is used for outputting the graph.
|
||||
type NamedVertex interface {
|
||||
Vertex
|
||||
Name() string
|
||||
}
|
||||
|
||||
// Vertices returns the list of all the vertices in the graph.
|
||||
func (g *Graph) Vertices() []Vertex {
|
||||
list := g.vertices.List()
|
||||
result := make([]Vertex, len(list))
|
||||
for i, v := range list {
|
||||
result[i] = v.(Vertex)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// Edges returns the list of all the edges in the graph.
|
||||
func (g *Graph) Edges() []Edge {
|
||||
list := g.edges.List()
|
||||
result := make([]Edge, len(list))
|
||||
for i, v := range list {
|
||||
result[i] = v.(Edge)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// Add adds a vertex to the graph. This is safe to call multiple time with
|
||||
// the same Vertex.
|
||||
func (g *Graph) Add(v Vertex) Vertex {
|
||||
g.once.Do(g.init)
|
||||
g.vertices.Add(v)
|
||||
return v
|
||||
}
|
||||
|
||||
// Remove removes a vertex from the graph. This will also remove any
|
||||
// edges with this vertex as a source or target.
|
||||
func (g *Graph) Remove(v Vertex) Vertex {
|
||||
// Delete the vertex itself
|
||||
g.vertices.Delete(v)
|
||||
|
||||
// Delete the edges to non-existent things
|
||||
for _, target := range g.DownEdges(v).List() {
|
||||
g.RemoveEdge(BasicEdge(v, target))
|
||||
}
|
||||
for _, source := range g.UpEdges(v).List() {
|
||||
g.RemoveEdge(BasicEdge(source, v))
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Replace replaces the original Vertex with replacement. If the original
|
||||
// does not exist within the graph, then false is returned. Otherwise, true
|
||||
// is returned.
|
||||
func (g *Graph) Replace(original, replacement Vertex) bool {
|
||||
// If we don't have the original, we can't do anything
|
||||
if !g.vertices.Include(original) {
|
||||
return false
|
||||
}
|
||||
|
||||
// Add our new vertex, then copy all the edges
|
||||
g.Add(replacement)
|
||||
for _, target := range g.DownEdges(original).List() {
|
||||
g.Connect(BasicEdge(replacement, target))
|
||||
}
|
||||
for _, source := range g.UpEdges(original).List() {
|
||||
g.Connect(BasicEdge(source, replacement))
|
||||
}
|
||||
|
||||
// Remove our old vertex, which will also remove all the edges
|
||||
g.Remove(original)
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// RemoveEdge removes an edge from the graph.
|
||||
func (g *Graph) RemoveEdge(edge Edge) {
|
||||
g.once.Do(g.init)
|
||||
|
||||
// Delete the edge from the set
|
||||
g.edges.Delete(edge)
|
||||
|
||||
// Delete the up/down edges
|
||||
if s, ok := g.downEdges[edge.Source()]; ok {
|
||||
s.Delete(edge.Target())
|
||||
}
|
||||
if s, ok := g.upEdges[edge.Target()]; ok {
|
||||
s.Delete(edge.Source())
|
||||
}
|
||||
}
|
||||
|
||||
// DownEdges returns the outward edges from the source Vertex v.
|
||||
func (g *Graph) DownEdges(v Vertex) *Set {
|
||||
g.once.Do(g.init)
|
||||
return g.downEdges[v]
|
||||
}
|
||||
|
||||
// UpEdges returns the inward edges to the destination Vertex v.
|
||||
func (g *Graph) UpEdges(v Vertex) *Set {
|
||||
g.once.Do(g.init)
|
||||
return g.upEdges[v]
|
||||
}
|
||||
|
||||
// Connect adds an edge with the given source and target. This is safe to
|
||||
// call multiple times with the same value. Note that the same value is
|
||||
// verified through pointer equality of the vertices, not through the
|
||||
// value of the edge itself.
|
||||
func (g *Graph) Connect(edge Edge) {
|
||||
g.once.Do(g.init)
|
||||
|
||||
source := edge.Source()
|
||||
target := edge.Target()
|
||||
|
||||
// Do we have this already? If so, don't add it again.
|
||||
if s, ok := g.downEdges[source]; ok && s.Include(target) {
|
||||
return
|
||||
}
|
||||
|
||||
// Add the edge to the set
|
||||
g.edges.Add(edge)
|
||||
|
||||
// Add the down edge
|
||||
s, ok := g.downEdges[source]
|
||||
if !ok {
|
||||
s = new(Set)
|
||||
g.downEdges[source] = s
|
||||
}
|
||||
s.Add(target)
|
||||
|
||||
// Add the up edge
|
||||
s, ok = g.upEdges[target]
|
||||
if !ok {
|
||||
s = new(Set)
|
||||
g.upEdges[target] = s
|
||||
}
|
||||
s.Add(source)
|
||||
}
|
||||
|
||||
// String outputs some human-friendly output for the graph structure.
|
||||
func (g *Graph) String() string {
|
||||
var buf bytes.Buffer
|
||||
|
||||
// Build the list of node names and a mapping so that we can more
|
||||
// easily alphabetize the output to remain deterministic.
|
||||
vertices := g.Vertices()
|
||||
names := make([]string, 0, len(vertices))
|
||||
mapping := make(map[string]Vertex, len(vertices))
|
||||
for _, v := range vertices {
|
||||
name := VertexName(v)
|
||||
names = append(names, name)
|
||||
mapping[name] = v
|
||||
}
|
||||
sort.Strings(names)
|
||||
|
||||
// Write each node in order...
|
||||
for _, name := range names {
|
||||
v := mapping[name]
|
||||
targets := g.downEdges[v]
|
||||
|
||||
buf.WriteString(fmt.Sprintf("%s\n", name))
|
||||
|
||||
// Alphabetize dependencies
|
||||
deps := make([]string, 0, targets.Len())
|
||||
for _, target := range targets.List() {
|
||||
deps = append(deps, VertexName(target))
|
||||
}
|
||||
sort.Strings(deps)
|
||||
|
||||
// Write dependencies
|
||||
for _, d := range deps {
|
||||
buf.WriteString(fmt.Sprintf(" %s\n", d))
|
||||
}
|
||||
}
|
||||
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
func (g *Graph) init() {
|
||||
g.vertices = new(Set)
|
||||
g.edges = new(Set)
|
||||
g.downEdges = make(map[Vertex]*Set)
|
||||
g.upEdges = make(map[Vertex]*Set)
|
||||
}
|
||||
|
||||
// VertexName returns the name of a vertex.
|
||||
func VertexName(raw Vertex) string {
|
||||
switch v := raw.(type) {
|
||||
case NamedVertex:
|
||||
return v.Name()
|
||||
case fmt.Stringer:
|
||||
return fmt.Sprintf("%s", v)
|
||||
default:
|
||||
return fmt.Sprintf("%v", v)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestGraph_empty(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphEmptyStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraph_basic(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(1, 3))
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraph_remove(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(1, 3))
|
||||
g.Remove(3)
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphRemoveStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraph_replace(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Add(3)
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 3))
|
||||
g.Replace(2, 42)
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphReplaceStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
const testGraphBasicStr = `
|
||||
1
|
||||
3
|
||||
2
|
||||
3
|
||||
`
|
||||
|
||||
const testGraphEmptyStr = `
|
||||
1
|
||||
2
|
||||
3
|
||||
`
|
||||
|
||||
const testGraphRemoveStr = `
|
||||
1
|
||||
2
|
||||
`
|
||||
|
||||
const testGraphReplaceStr = `
|
||||
1
|
||||
42
|
||||
3
|
||||
42
|
||||
3
|
||||
`
|
|
@ -0,0 +1,72 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"sync"
|
||||
)
|
||||
|
||||
// Set is a set data structure.
|
||||
type Set struct {
|
||||
m map[interface{}]interface{}
|
||||
once sync.Once
|
||||
}
|
||||
|
||||
// Hashable is the interface used by set to get the hash code of a value.
|
||||
// If this isn't given, then the value of the item being added to the set
|
||||
// itself is used as the comparison value.
|
||||
type Hashable interface {
|
||||
Hashcode() interface{}
|
||||
}
|
||||
|
||||
// Add adds an item to the set
|
||||
func (s *Set) Add(v interface{}) {
|
||||
s.once.Do(s.init)
|
||||
s.m[s.code(v)] = v
|
||||
}
|
||||
|
||||
// Delete removes an item from the set.
|
||||
func (s *Set) Delete(v interface{}) {
|
||||
s.once.Do(s.init)
|
||||
delete(s.m, s.code(v))
|
||||
}
|
||||
|
||||
// Include returns true/false of whether a value is in the set.
|
||||
func (s *Set) Include(v interface{}) bool {
|
||||
s.once.Do(s.init)
|
||||
_, ok := s.m[s.code(v)]
|
||||
return ok
|
||||
}
|
||||
|
||||
// Len is the number of items in the set.
|
||||
func (s *Set) Len() int {
|
||||
if s == nil {
|
||||
return 0
|
||||
}
|
||||
|
||||
return len(s.m)
|
||||
}
|
||||
|
||||
// List returns the list of set elements.
|
||||
func (s *Set) List() []interface{} {
|
||||
if s == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
r := make([]interface{}, 0, len(s.m))
|
||||
for _, v := range s.m {
|
||||
r = append(r, v)
|
||||
}
|
||||
|
||||
return r
|
||||
}
|
||||
|
||||
func (s *Set) code(v interface{}) interface{} {
|
||||
if h, ok := v.(Hashable); ok {
|
||||
return h.Hashcode()
|
||||
}
|
||||
|
||||
return v
|
||||
}
|
||||
|
||||
func (s *Set) init() {
|
||||
s.m = make(map[interface{}]interface{})
|
||||
}
|
|
@ -0,0 +1,107 @@
|
|||
package dag
|
||||
|
||||
// StronglyConnected returns the list of strongly connected components
|
||||
// within the Graph g. This information is primarily used by this package
|
||||
// for cycle detection, but strongly connected components have widespread
|
||||
// use.
|
||||
func StronglyConnected(g *Graph) [][]Vertex {
|
||||
vs := g.Vertices()
|
||||
acct := sccAcct{
|
||||
NextIndex: 1,
|
||||
VertexIndex: make(map[Vertex]int, len(vs)),
|
||||
}
|
||||
for _, v := range vs {
|
||||
// Recurse on any non-visited nodes
|
||||
if acct.VertexIndex[v] == 0 {
|
||||
stronglyConnected(&acct, g, v)
|
||||
}
|
||||
}
|
||||
return acct.SCC
|
||||
}
|
||||
|
||||
func stronglyConnected(acct *sccAcct, g *Graph, v Vertex) int {
|
||||
// Initial vertex visit
|
||||
index := acct.visit(v)
|
||||
minIdx := index
|
||||
|
||||
for _, raw := range g.DownEdges(v).List() {
|
||||
target := raw.(Vertex)
|
||||
targetIdx := acct.VertexIndex[target]
|
||||
|
||||
// Recurse on successor if not yet visited
|
||||
if targetIdx == 0 {
|
||||
minIdx = min(minIdx, stronglyConnected(acct, g, target))
|
||||
} else if acct.inStack(target) {
|
||||
// Check if the vertex is in the stack
|
||||
minIdx = min(minIdx, targetIdx)
|
||||
}
|
||||
}
|
||||
|
||||
// Pop the strongly connected components off the stack if
|
||||
// this is a root vertex
|
||||
if index == minIdx {
|
||||
var scc []Vertex
|
||||
for {
|
||||
v2 := acct.pop()
|
||||
scc = append(scc, v2)
|
||||
if v2 == v {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
acct.SCC = append(acct.SCC, scc)
|
||||
}
|
||||
|
||||
return minIdx
|
||||
}
|
||||
|
||||
func min(a, b int) int {
|
||||
if a <= b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
// sccAcct is used ot pass around accounting information for
|
||||
// the StronglyConnectedComponents algorithm
|
||||
type sccAcct struct {
|
||||
NextIndex int
|
||||
VertexIndex map[Vertex]int
|
||||
Stack []Vertex
|
||||
SCC [][]Vertex
|
||||
}
|
||||
|
||||
// visit assigns an index and pushes a vertex onto the stack
|
||||
func (s *sccAcct) visit(v Vertex) int {
|
||||
idx := s.NextIndex
|
||||
s.VertexIndex[v] = idx
|
||||
s.NextIndex++
|
||||
s.push(v)
|
||||
return idx
|
||||
}
|
||||
|
||||
// push adds a vertex to the stack
|
||||
func (s *sccAcct) push(n Vertex) {
|
||||
s.Stack = append(s.Stack, n)
|
||||
}
|
||||
|
||||
// pop removes a vertex from the stack
|
||||
func (s *sccAcct) pop() Vertex {
|
||||
n := len(s.Stack)
|
||||
if n == 0 {
|
||||
return nil
|
||||
}
|
||||
vertex := s.Stack[n-1]
|
||||
s.Stack = s.Stack[:n-1]
|
||||
return vertex
|
||||
}
|
||||
|
||||
// inStack checks if a vertex is in the stack
|
||||
func (s *sccAcct) inStack(needle Vertex) bool {
|
||||
for _, n := range s.Stack {
|
||||
if n == needle {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package dag
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"strings"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestGraphStronglyConnected(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 1))
|
||||
|
||||
actual := strings.TrimSpace(testSCCStr(StronglyConnected(&g)))
|
||||
expected := strings.TrimSpace(testGraphStronglyConnectedStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraphStronglyConnected_two(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 1))
|
||||
g.Add(3)
|
||||
|
||||
actual := strings.TrimSpace(testSCCStr(StronglyConnected(&g)))
|
||||
expected := strings.TrimSpace(testGraphStronglyConnectedTwoStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraphStronglyConnected_three(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Connect(BasicEdge(1, 2))
|
||||
g.Connect(BasicEdge(2, 1))
|
||||
g.Add(3)
|
||||
g.Add(4)
|
||||
g.Add(5)
|
||||
g.Add(6)
|
||||
g.Connect(BasicEdge(4, 5))
|
||||
g.Connect(BasicEdge(5, 6))
|
||||
g.Connect(BasicEdge(6, 4))
|
||||
|
||||
actual := strings.TrimSpace(testSCCStr(StronglyConnected(&g)))
|
||||
expected := strings.TrimSpace(testGraphStronglyConnectedThreeStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func testSCCStr(list [][]Vertex) string {
|
||||
var lines []string
|
||||
for _, vs := range list {
|
||||
result := make([]string, len(vs))
|
||||
for i, v := range vs {
|
||||
result[i] = VertexName(v)
|
||||
}
|
||||
|
||||
sort.Strings(result)
|
||||
lines = append(lines, strings.Join(result, ","))
|
||||
}
|
||||
|
||||
sort.Strings(lines)
|
||||
return strings.Join(lines, "\n")
|
||||
}
|
||||
|
||||
const testGraphStronglyConnectedStr = `1,2`
|
||||
|
||||
const testGraphStronglyConnectedTwoStr = `
|
||||
1,2
|
||||
3
|
||||
`
|
||||
|
||||
const testGraphStronglyConnectedThreeStr = `
|
||||
1,2
|
||||
3
|
||||
4,5,6
|
||||
`
|
1805
terraform/context.go
1805
terraform/context.go
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -333,6 +333,10 @@ func (d *InstanceDiff) Empty() bool {
|
|||
return !d.Destroy && len(d.Attributes) == 0
|
||||
}
|
||||
|
||||
func (d *InstanceDiff) GoString() string {
|
||||
return fmt.Sprintf("*%#v", *d)
|
||||
}
|
||||
|
||||
// RequiresNew returns true if the diff requires the creation of a new
|
||||
// resource (implying the destruction of the old).
|
||||
func (d *InstanceDiff) RequiresNew() bool {
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"log"
|
||||
"strings"
|
||||
)
|
||||
|
||||
// EvalNode is the interface that must be implemented by graph nodes to
|
||||
// evaluate/execute.
|
||||
type EvalNode interface {
|
||||
// Eval evaluates this node with the given context. The second parameter
|
||||
// are the argument values. These will match in order and 1-1 with the
|
||||
// results of the Args() return value.
|
||||
Eval(EvalContext) (interface{}, error)
|
||||
}
|
||||
|
||||
// GraphNodeEvalable is the interface that graph nodes must implement
|
||||
// to enable valuation.
|
||||
type GraphNodeEvalable interface {
|
||||
EvalTree() EvalNode
|
||||
}
|
||||
|
||||
// EvalEarlyExitError is a special error return value that can be returned
|
||||
// by eval nodes that does an early exit.
|
||||
type EvalEarlyExitError struct{}
|
||||
|
||||
func (EvalEarlyExitError) Error() string { return "early exit" }
|
||||
|
||||
// Eval evaluates the given EvalNode with the given context, properly
|
||||
// evaluating all args in the correct order.
|
||||
func Eval(n EvalNode, ctx EvalContext) (interface{}, error) {
|
||||
// Call the lower level eval which doesn't understand early exit,
|
||||
// and if we early exit, it isn't an error.
|
||||
result, err := EvalRaw(n, ctx)
|
||||
if err != nil {
|
||||
if _, ok := err.(EvalEarlyExitError); ok {
|
||||
return nil, nil
|
||||
}
|
||||
}
|
||||
|
||||
return result, err
|
||||
}
|
||||
|
||||
// EvalRaw is like Eval except that it returns all errors, even if they
|
||||
// signal something normal such as EvalEarlyExitError.
|
||||
func EvalRaw(n EvalNode, ctx EvalContext) (interface{}, error) {
|
||||
path := "unknown"
|
||||
if ctx != nil {
|
||||
path = strings.Join(ctx.Path(), ".")
|
||||
}
|
||||
|
||||
log.Printf("[DEBUG] %s: eval: %T", path, n)
|
||||
output, err := n.Eval(ctx)
|
||||
if err != nil {
|
||||
log.Printf("[ERROR] %s: eval: %T, err: %s", path, n, err)
|
||||
}
|
||||
|
||||
return output, err
|
||||
}
|
|
@ -0,0 +1,297 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"strconv"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalApply is an EvalNode implementation that writes the diff to
|
||||
// the full diff.
|
||||
type EvalApply struct {
|
||||
Info *InstanceInfo
|
||||
State **InstanceState
|
||||
Diff **InstanceDiff
|
||||
Provider *ResourceProvider
|
||||
Output **InstanceState
|
||||
CreateNew *bool
|
||||
Error *error
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalApply) Eval(ctx EvalContext) (interface{}, error) {
|
||||
diff := *n.Diff
|
||||
provider := *n.Provider
|
||||
state := *n.State
|
||||
|
||||
// If we have no diff, we have nothing to do!
|
||||
if diff.Empty() {
|
||||
log.Printf(
|
||||
"[DEBUG] apply: %s: diff is empty, doing nothing.", n.Info.Id)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Remove any output values from the diff
|
||||
for k, ad := range diff.Attributes {
|
||||
if ad.Type == DiffAttrOutput {
|
||||
delete(diff.Attributes, k)
|
||||
}
|
||||
}
|
||||
|
||||
// If the state is nil, make it non-nil
|
||||
if state == nil {
|
||||
state = new(InstanceState)
|
||||
}
|
||||
state.init()
|
||||
|
||||
// Flag if we're creating a new instance
|
||||
if n.CreateNew != nil {
|
||||
*n.CreateNew = (state.ID == "" && !diff.Destroy) || diff.RequiresNew()
|
||||
}
|
||||
|
||||
{
|
||||
// Call pre-apply hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreApply(n.Info, state, diff)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// With the completed diff, apply!
|
||||
log.Printf("[DEBUG] apply: %s: executing Apply", n.Info.Id)
|
||||
state, err := provider.Apply(n.Info, state, diff)
|
||||
if state == nil {
|
||||
state = new(InstanceState)
|
||||
}
|
||||
state.init()
|
||||
|
||||
// Force the "id" attribute to be our ID
|
||||
if state.ID != "" {
|
||||
state.Attributes["id"] = state.ID
|
||||
}
|
||||
|
||||
// If the value is the unknown variable value, then it is an error.
|
||||
// In this case we record the error and remove it from the state
|
||||
for ak, av := range state.Attributes {
|
||||
if av == config.UnknownVariableValue {
|
||||
err = multierror.Append(err, fmt.Errorf(
|
||||
"Attribute with unknown value: %s", ak))
|
||||
delete(state.Attributes, ak)
|
||||
}
|
||||
}
|
||||
|
||||
// Write the final state
|
||||
if n.Output != nil {
|
||||
*n.Output = state
|
||||
}
|
||||
|
||||
// If there are no errors, then we append it to our output error
|
||||
// if we have one, otherwise we just output it.
|
||||
if err != nil {
|
||||
if n.Error != nil {
|
||||
*n.Error = multierror.Append(*n.Error, err)
|
||||
} else {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalApplyPost is an EvalNode implementation that does the post-Apply work
|
||||
type EvalApplyPost struct {
|
||||
Info *InstanceInfo
|
||||
State **InstanceState
|
||||
Error *error
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalApplyPost) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state := *n.State
|
||||
|
||||
{
|
||||
// Call post-apply hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostApply(n.Info, state, *n.Error)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return nil, *n.Error
|
||||
}
|
||||
|
||||
// EvalApplyProvisioners is an EvalNode implementation that executes
|
||||
// the provisioners for a resource.
|
||||
//
|
||||
// TODO(mitchellh): This should probably be split up into a more fine-grained
|
||||
// ApplyProvisioner (single) that is looped over.
|
||||
type EvalApplyProvisioners struct {
|
||||
Info *InstanceInfo
|
||||
State **InstanceState
|
||||
Resource *config.Resource
|
||||
InterpResource *Resource
|
||||
CreateNew *bool
|
||||
Tainted *bool
|
||||
Error *error
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalApplyProvisioners) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state := *n.State
|
||||
|
||||
if !*n.CreateNew {
|
||||
// If we're not creating a new resource, then don't run provisioners
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if len(n.Resource.Provisioners) == 0 {
|
||||
// We have no provisioners, so don't do anything
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if n.Error != nil && *n.Error != nil {
|
||||
// We're already errored creating, so mark as tainted and continue
|
||||
if n.Tainted != nil {
|
||||
*n.Tainted = true
|
||||
}
|
||||
|
||||
// We're already tainted, so just return out
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
{
|
||||
// Call pre hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreProvisionResource(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
// If there are no errors, then we append it to our output error
|
||||
// if we have one, otherwise we just output it.
|
||||
err := n.apply(ctx)
|
||||
if n.Tainted != nil {
|
||||
*n.Tainted = err != nil
|
||||
}
|
||||
if err != nil {
|
||||
if n.Error != nil {
|
||||
*n.Error = multierror.Append(*n.Error, err)
|
||||
} else {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
// Call post hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostProvisionResource(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (n *EvalApplyProvisioners) apply(ctx EvalContext) error {
|
||||
state := *n.State
|
||||
|
||||
// Store the original connection info, restore later
|
||||
origConnInfo := state.Ephemeral.ConnInfo
|
||||
defer func() {
|
||||
state.Ephemeral.ConnInfo = origConnInfo
|
||||
}()
|
||||
|
||||
for _, prov := range n.Resource.Provisioners {
|
||||
// Get the provisioner
|
||||
provisioner := ctx.Provisioner(prov.Type)
|
||||
|
||||
// Interpolate the provisioner config
|
||||
provConfig, err := ctx.Interpolate(prov.RawConfig, n.InterpResource)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Interpolate the conn info, since it may contain variables
|
||||
connInfo, err := ctx.Interpolate(prov.ConnInfo, n.InterpResource)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Merge the connection information
|
||||
overlay := make(map[string]string)
|
||||
if origConnInfo != nil {
|
||||
for k, v := range origConnInfo {
|
||||
overlay[k] = v
|
||||
}
|
||||
}
|
||||
for k, v := range connInfo.Config {
|
||||
switch vt := v.(type) {
|
||||
case string:
|
||||
overlay[k] = vt
|
||||
case int64:
|
||||
overlay[k] = strconv.FormatInt(vt, 10)
|
||||
case int32:
|
||||
overlay[k] = strconv.FormatInt(int64(vt), 10)
|
||||
case int:
|
||||
overlay[k] = strconv.FormatInt(int64(vt), 10)
|
||||
case float32:
|
||||
overlay[k] = strconv.FormatFloat(float64(vt), 'f', 3, 32)
|
||||
case float64:
|
||||
overlay[k] = strconv.FormatFloat(vt, 'f', 3, 64)
|
||||
case bool:
|
||||
overlay[k] = strconv.FormatBool(vt)
|
||||
default:
|
||||
overlay[k] = fmt.Sprintf("%v", vt)
|
||||
}
|
||||
}
|
||||
state.Ephemeral.ConnInfo = overlay
|
||||
|
||||
{
|
||||
// Call pre hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreProvision(n.Info, prov.Type)
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// The output function
|
||||
outputFn := func(msg string) {
|
||||
ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
h.ProvisionOutput(n.Info, prov.Type, msg)
|
||||
return HookActionContinue, nil
|
||||
})
|
||||
}
|
||||
|
||||
// Invoke the Provisioner
|
||||
output := CallbackUIOutput{OutputFn: outputFn}
|
||||
if err := provisioner.Apply(&output, state, provConfig); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
{
|
||||
// Call post hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostProvision(n.Info, prov.Type)
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
|
||||
}
|
|
@ -0,0 +1,224 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalContext is the interface that is given to eval nodes to execute.
|
||||
type EvalContext interface {
|
||||
// Path is the current module path.
|
||||
Path() []string
|
||||
|
||||
// Hook is used to call hook methods. The callback is called for each
|
||||
// hook and should return the hook action to take and the error.
|
||||
Hook(func(Hook) (HookAction, error)) error
|
||||
|
||||
// Input is the UIInput object for interacting with the UI.
|
||||
Input() UIInput
|
||||
|
||||
// InitProvider initializes the provider with the given name and
|
||||
// returns the implementation of the resource provider or an error.
|
||||
//
|
||||
// It is an error to initialize the same provider more than once.
|
||||
InitProvider(string) (ResourceProvider, error)
|
||||
|
||||
// Provider gets the provider instance with the given name (already
|
||||
// initialized) or returns nil if the provider isn't initialized.
|
||||
Provider(string) ResourceProvider
|
||||
|
||||
// ConfigureProvider configures the provider with the given
|
||||
// configuration. This is a separate context call because this call
|
||||
// is used to store the provider configuration for inheritance lookups
|
||||
// with ParentProviderConfig().
|
||||
ConfigureProvider(string, *ResourceConfig) error
|
||||
ParentProviderConfig(string) *ResourceConfig
|
||||
|
||||
// ProviderInput and SetProviderInput are used to configure providers
|
||||
// from user input.
|
||||
ProviderInput(string) map[string]interface{}
|
||||
SetProviderInput(string, map[string]interface{})
|
||||
|
||||
// InitProvisioner initializes the provisioner with the given name and
|
||||
// returns the implementation of the resource provisioner or an error.
|
||||
//
|
||||
// It is an error to initialize the same provisioner more than once.
|
||||
InitProvisioner(string) (ResourceProvisioner, error)
|
||||
|
||||
// Provisioner gets the provisioner instance with the given name (already
|
||||
// initialized) or returns nil if the provisioner isn't initialized.
|
||||
Provisioner(string) ResourceProvisioner
|
||||
|
||||
// Interpolate takes the given raw configuration and completes
|
||||
// the interpolations, returning the processed ResourceConfig.
|
||||
//
|
||||
// The resource argument is optional. If given, it is the resource
|
||||
// that is currently being acted upon.
|
||||
Interpolate(*config.RawConfig, *Resource) (*ResourceConfig, error)
|
||||
|
||||
// SetVariables sets the variables for interpolation. These variables
|
||||
// should not have a "var." prefix. For example: "var.foo" should be
|
||||
// "foo" as the key.
|
||||
SetVariables(map[string]string)
|
||||
|
||||
// Diff returns the global diff as well as the lock that should
|
||||
// be used to modify that diff.
|
||||
Diff() (*Diff, *sync.RWMutex)
|
||||
|
||||
// State returns the global state as well as the lock that should
|
||||
// be used to modify that state.
|
||||
State() (*State, *sync.RWMutex)
|
||||
}
|
||||
|
||||
// MockEvalContext is a mock version of EvalContext that can be used
|
||||
// for tests.
|
||||
type MockEvalContext struct {
|
||||
HookCalled bool
|
||||
HookError error
|
||||
|
||||
InputCalled bool
|
||||
InputInput UIInput
|
||||
|
||||
InitProviderCalled bool
|
||||
InitProviderName string
|
||||
InitProviderProvider ResourceProvider
|
||||
InitProviderError error
|
||||
|
||||
ProviderCalled bool
|
||||
ProviderName string
|
||||
ProviderProvider ResourceProvider
|
||||
|
||||
ProviderInputCalled bool
|
||||
ProviderInputName string
|
||||
ProviderInputConfig map[string]interface{}
|
||||
|
||||
SetProviderInputCalled bool
|
||||
SetProviderInputName string
|
||||
SetProviderInputConfig map[string]interface{}
|
||||
|
||||
ConfigureProviderCalled bool
|
||||
ConfigureProviderName string
|
||||
ConfigureProviderConfig *ResourceConfig
|
||||
ConfigureProviderError error
|
||||
|
||||
ParentProviderConfigCalled bool
|
||||
ParentProviderConfigName string
|
||||
ParentProviderConfigConfig *ResourceConfig
|
||||
|
||||
InitProvisionerCalled bool
|
||||
InitProvisionerName string
|
||||
InitProvisionerProvisioner ResourceProvisioner
|
||||
InitProvisionerError error
|
||||
|
||||
ProvisionerCalled bool
|
||||
ProvisionerName string
|
||||
ProvisionerProvisioner ResourceProvisioner
|
||||
|
||||
InterpolateCalled bool
|
||||
InterpolateConfig *config.RawConfig
|
||||
InterpolateResource *Resource
|
||||
InterpolateConfigResult *ResourceConfig
|
||||
InterpolateError error
|
||||
|
||||
PathCalled bool
|
||||
PathPath []string
|
||||
|
||||
SetVariablesCalled bool
|
||||
SetVariablesVariables map[string]string
|
||||
|
||||
DiffCalled bool
|
||||
DiffDiff *Diff
|
||||
DiffLock *sync.RWMutex
|
||||
|
||||
StateCalled bool
|
||||
StateState *State
|
||||
StateLock *sync.RWMutex
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Hook(fn func(Hook) (HookAction, error)) error {
|
||||
c.HookCalled = true
|
||||
return c.HookError
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Input() UIInput {
|
||||
c.InputCalled = true
|
||||
return c.InputInput
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) InitProvider(n string) (ResourceProvider, error) {
|
||||
c.InitProviderCalled = true
|
||||
c.InitProviderName = n
|
||||
return c.InitProviderProvider, c.InitProviderError
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Provider(n string) ResourceProvider {
|
||||
c.ProviderCalled = true
|
||||
c.ProviderName = n
|
||||
return c.ProviderProvider
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) ConfigureProvider(n string, cfg *ResourceConfig) error {
|
||||
c.ConfigureProviderCalled = true
|
||||
c.ConfigureProviderName = n
|
||||
c.ConfigureProviderConfig = cfg
|
||||
return c.ConfigureProviderError
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) ParentProviderConfig(n string) *ResourceConfig {
|
||||
c.ParentProviderConfigCalled = true
|
||||
c.ParentProviderConfigName = n
|
||||
return c.ParentProviderConfigConfig
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) ProviderInput(n string) map[string]interface{} {
|
||||
c.ProviderInputCalled = true
|
||||
c.ProviderInputName = n
|
||||
return c.ProviderInputConfig
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) SetProviderInput(n string, cfg map[string]interface{}) {
|
||||
c.SetProviderInputCalled = true
|
||||
c.SetProviderInputName = n
|
||||
c.SetProviderInputConfig = cfg
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) InitProvisioner(n string) (ResourceProvisioner, error) {
|
||||
c.InitProvisionerCalled = true
|
||||
c.InitProvisionerName = n
|
||||
return c.InitProvisionerProvisioner, c.InitProvisionerError
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Provisioner(n string) ResourceProvisioner {
|
||||
c.ProvisionerCalled = true
|
||||
c.ProvisionerName = n
|
||||
return c.ProvisionerProvisioner
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Interpolate(
|
||||
config *config.RawConfig, resource *Resource) (*ResourceConfig, error) {
|
||||
c.InterpolateCalled = true
|
||||
c.InterpolateConfig = config
|
||||
c.InterpolateResource = resource
|
||||
return c.InterpolateConfigResult, c.InterpolateError
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Path() []string {
|
||||
c.PathCalled = true
|
||||
return c.PathPath
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) SetVariables(vs map[string]string) {
|
||||
c.SetVariablesCalled = true
|
||||
c.SetVariablesVariables = vs
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) Diff() (*Diff, *sync.RWMutex) {
|
||||
c.DiffCalled = true
|
||||
return c.DiffDiff, c.DiffLock
|
||||
}
|
||||
|
||||
func (c *MockEvalContext) State() (*State, *sync.RWMutex) {
|
||||
c.StateCalled = true
|
||||
return c.StateState, c.StateLock
|
||||
}
|
|
@ -0,0 +1,230 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// BuiltinEvalContext is an EvalContext implementation that is used by
|
||||
// Terraform by default.
|
||||
type BuiltinEvalContext struct {
|
||||
PathValue []string
|
||||
Interpolater *Interpolater
|
||||
Hooks []Hook
|
||||
InputValue UIInput
|
||||
Providers map[string]ResourceProviderFactory
|
||||
ProviderCache map[string]ResourceProvider
|
||||
ProviderConfigCache map[string]*ResourceConfig
|
||||
ProviderInputConfig map[string]map[string]interface{}
|
||||
ProviderLock *sync.Mutex
|
||||
Provisioners map[string]ResourceProvisionerFactory
|
||||
ProvisionerCache map[string]ResourceProvisioner
|
||||
ProvisionerLock *sync.Mutex
|
||||
DiffValue *Diff
|
||||
DiffLock *sync.RWMutex
|
||||
StateValue *State
|
||||
StateLock *sync.RWMutex
|
||||
|
||||
once sync.Once
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Hook(fn func(Hook) (HookAction, error)) error {
|
||||
for _, h := range ctx.Hooks {
|
||||
action, err := fn(h)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
switch action {
|
||||
case HookActionContinue:
|
||||
continue
|
||||
case HookActionHalt:
|
||||
// Return an early exit error to trigger an early exit
|
||||
log.Printf("[WARN] Early exit triggered by hook: %T", h)
|
||||
return EvalEarlyExitError{}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Input() UIInput {
|
||||
return ctx.InputValue
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) InitProvider(n string) (ResourceProvider, error) {
|
||||
ctx.once.Do(ctx.init)
|
||||
|
||||
// If we already initialized, it is an error
|
||||
if p := ctx.Provider(n); p != nil {
|
||||
return nil, fmt.Errorf("Provider '%s' already initialized", n)
|
||||
}
|
||||
|
||||
// Warning: make sure to acquire these locks AFTER the call to Provider
|
||||
// above, since it also acquires locks.
|
||||
ctx.ProviderLock.Lock()
|
||||
defer ctx.ProviderLock.Unlock()
|
||||
|
||||
f, ok := ctx.Providers[n]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("Provider '%s' not found", n)
|
||||
}
|
||||
|
||||
p, err := f()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
providerPath := make([]string, len(ctx.Path())+1)
|
||||
copy(providerPath, ctx.Path())
|
||||
providerPath[len(providerPath)-1] = n
|
||||
|
||||
ctx.ProviderCache[PathCacheKey(providerPath)] = p
|
||||
return p, nil
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Provider(n string) ResourceProvider {
|
||||
ctx.once.Do(ctx.init)
|
||||
|
||||
ctx.ProviderLock.Lock()
|
||||
defer ctx.ProviderLock.Unlock()
|
||||
|
||||
providerPath := make([]string, len(ctx.Path())+1)
|
||||
copy(providerPath, ctx.Path())
|
||||
providerPath[len(providerPath)-1] = n
|
||||
|
||||
return ctx.ProviderCache[PathCacheKey(providerPath)]
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) ConfigureProvider(
|
||||
n string, cfg *ResourceConfig) error {
|
||||
p := ctx.Provider(n)
|
||||
if p == nil {
|
||||
return fmt.Errorf("Provider '%s' not initialized", n)
|
||||
}
|
||||
|
||||
// Save the configuration
|
||||
ctx.ProviderLock.Lock()
|
||||
ctx.ProviderConfigCache[PathCacheKey(ctx.Path())] = cfg
|
||||
ctx.ProviderLock.Unlock()
|
||||
|
||||
return p.Configure(cfg)
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) ProviderInput(n string) map[string]interface{} {
|
||||
ctx.ProviderLock.Lock()
|
||||
defer ctx.ProviderLock.Unlock()
|
||||
|
||||
return ctx.ProviderInputConfig[n]
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) SetProviderInput(n string, c map[string]interface{}) {
|
||||
ctx.ProviderLock.Lock()
|
||||
defer ctx.ProviderLock.Unlock()
|
||||
|
||||
ctx.ProviderInputConfig[n] = c
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) ParentProviderConfig(n string) *ResourceConfig {
|
||||
ctx.ProviderLock.Lock()
|
||||
defer ctx.ProviderLock.Unlock()
|
||||
|
||||
path := ctx.Path()
|
||||
for i := len(path) - 1; i >= 1; i-- {
|
||||
k := PathCacheKey(path[:i])
|
||||
if v, ok := ctx.ProviderConfigCache[k]; ok {
|
||||
return v
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) InitProvisioner(
|
||||
n string) (ResourceProvisioner, error) {
|
||||
ctx.once.Do(ctx.init)
|
||||
|
||||
// If we already initialized, it is an error
|
||||
if p := ctx.Provisioner(n); p != nil {
|
||||
return nil, fmt.Errorf("Provisioner '%s' already initialized", n)
|
||||
}
|
||||
|
||||
// Warning: make sure to acquire these locks AFTER the call to Provisioner
|
||||
// above, since it also acquires locks.
|
||||
ctx.ProvisionerLock.Lock()
|
||||
defer ctx.ProvisionerLock.Unlock()
|
||||
|
||||
f, ok := ctx.Provisioners[n]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("Provisioner '%s' not found", n)
|
||||
}
|
||||
|
||||
p, err := f()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ctx.ProvisionerCache[PathCacheKey(ctx.Path())] = p
|
||||
return p, nil
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Provisioner(n string) ResourceProvisioner {
|
||||
ctx.once.Do(ctx.init)
|
||||
|
||||
ctx.ProvisionerLock.Lock()
|
||||
defer ctx.ProvisionerLock.Unlock()
|
||||
|
||||
return ctx.ProvisionerCache[PathCacheKey(ctx.Path())]
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Interpolate(
|
||||
cfg *config.RawConfig, r *Resource) (*ResourceConfig, error) {
|
||||
if cfg != nil {
|
||||
scope := &InterpolationScope{
|
||||
Path: ctx.Path(),
|
||||
Resource: r,
|
||||
}
|
||||
vs, err := ctx.Interpolater.Values(scope, cfg.Variables)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Do the interpolation
|
||||
if err := cfg.Interpolate(vs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
result := NewResourceConfig(cfg)
|
||||
result.interpolateForce()
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Path() []string {
|
||||
return ctx.PathValue
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) SetVariables(vs map[string]string) {
|
||||
for k, v := range vs {
|
||||
ctx.Interpolater.Variables[k] = v
|
||||
}
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) Diff() (*Diff, *sync.RWMutex) {
|
||||
return ctx.DiffValue, ctx.DiffLock
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) State() (*State, *sync.RWMutex) {
|
||||
return ctx.StateValue, ctx.StateLock
|
||||
}
|
||||
|
||||
func (ctx *BuiltinEvalContext) init() {
|
||||
// We nil-check the things below because they're meant to be configured,
|
||||
// and we just default them to non-nil.
|
||||
if ctx.Providers == nil {
|
||||
ctx.Providers = make(map[string]ResourceProviderFactory)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalCountFixZeroOneBoundary is an EvalNode that fixes up the state
|
||||
// when there is a resource count with zero/one boundary, i.e. fixing
|
||||
// a resource named "aws_instance.foo" to "aws_instance.foo.0" and vice-versa.
|
||||
type EvalCountFixZeroOneBoundary struct {
|
||||
Resource *config.Resource
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalCountFixZeroOneBoundary) Eval(ctx EvalContext) (interface{}, error) {
|
||||
// Get the count, important for knowing whether we're supposed to
|
||||
// be adding the zero, or trimming it.
|
||||
count, err := n.Resource.Count()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Figure what to look for and what to replace it with
|
||||
hunt := n.Resource.Id()
|
||||
replace := hunt + ".0"
|
||||
if count < 2 {
|
||||
hunt, replace = replace, hunt
|
||||
}
|
||||
|
||||
state, lock := ctx.State()
|
||||
|
||||
// Get a lock so we can access this instance and potentially make
|
||||
// changes to it.
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Look for the module state. If we don't have one, then it doesn't matter.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Look for the resource state. If we don't have one, then it is okay.
|
||||
if rs, ok := mod.Resources[hunt]; ok {
|
||||
mod.Resources[replace] = rs
|
||||
delete(mod.Resources, hunt)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,299 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
)
|
||||
|
||||
// EvalCompareDiff is an EvalNode implementation that compares two diffs
|
||||
// and errors if the diffs are not equal.
|
||||
type EvalCompareDiff struct {
|
||||
Info *InstanceInfo
|
||||
One, Two **InstanceDiff
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalCompareDiff) Eval(ctx EvalContext) (interface{}, error) {
|
||||
one, two := *n.One, *n.Two
|
||||
|
||||
// If either are nil, let them be empty
|
||||
if one == nil {
|
||||
one = new(InstanceDiff)
|
||||
one.init()
|
||||
}
|
||||
if two == nil {
|
||||
two = new(InstanceDiff)
|
||||
two.init()
|
||||
}
|
||||
oneId := one.Attributes["id"]
|
||||
twoId := two.Attributes["id"]
|
||||
delete(one.Attributes, "id")
|
||||
delete(two.Attributes, "id")
|
||||
defer func() {
|
||||
if oneId != nil {
|
||||
one.Attributes["id"] = oneId
|
||||
}
|
||||
if twoId != nil {
|
||||
two.Attributes["id"] = twoId
|
||||
}
|
||||
}()
|
||||
|
||||
if !one.Same(two) {
|
||||
log.Printf("[ERROR] %s: diff's didn't match", n.Info.Id)
|
||||
log.Printf("[ERROR] %s: diff one: %#v", n.Info.Id, one)
|
||||
log.Printf("[ERROR] %s: diff two: %#v", n.Info.Id, two)
|
||||
return nil, fmt.Errorf(
|
||||
"%s: diffs didn't match during apply. This is a bug with "+
|
||||
"Terraform and should be reported.", n.Info.Id)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalDiff is an EvalNode implementation that does a refresh for
|
||||
// a resource.
|
||||
type EvalDiff struct {
|
||||
Info *InstanceInfo
|
||||
Config **ResourceConfig
|
||||
Provider *ResourceProvider
|
||||
State **InstanceState
|
||||
Output **InstanceDiff
|
||||
OutputState **InstanceState
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalDiff) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state := *n.State
|
||||
config := *n.Config
|
||||
provider := *n.Provider
|
||||
|
||||
// Call pre-diff hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreDiff(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// The state for the diff must never be nil
|
||||
diffState := state
|
||||
if diffState == nil {
|
||||
diffState = new(InstanceState)
|
||||
}
|
||||
diffState.init()
|
||||
|
||||
// Diff!
|
||||
diff, err := provider.Diff(n.Info, diffState, config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if diff == nil {
|
||||
diff = new(InstanceDiff)
|
||||
}
|
||||
|
||||
// Require a destroy if there is no ID and it requires new.
|
||||
if diff.RequiresNew() && state != nil && state.ID != "" {
|
||||
diff.Destroy = true
|
||||
}
|
||||
|
||||
// If we're creating a new resource, compute its ID
|
||||
if diff.RequiresNew() || state == nil || state.ID == "" {
|
||||
var oldID string
|
||||
if state != nil {
|
||||
oldID = state.Attributes["id"]
|
||||
}
|
||||
|
||||
// Add diff to compute new ID
|
||||
diff.init()
|
||||
diff.Attributes["id"] = &ResourceAttrDiff{
|
||||
Old: oldID,
|
||||
NewComputed: true,
|
||||
RequiresNew: true,
|
||||
Type: DiffAttrOutput,
|
||||
}
|
||||
}
|
||||
|
||||
// Call post-refresh hook
|
||||
err = ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostDiff(n.Info, diff)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Update our output
|
||||
*n.Output = diff
|
||||
|
||||
// Update the state if we care
|
||||
if n.OutputState != nil {
|
||||
*n.OutputState = state
|
||||
|
||||
// Merge our state so that the state is updated with our plan
|
||||
if !diff.Empty() && n.OutputState != nil {
|
||||
*n.OutputState = state.MergeDiff(diff)
|
||||
}
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalDiffDestroy is an EvalNode implementation that returns a plain
|
||||
// destroy diff.
|
||||
type EvalDiffDestroy struct {
|
||||
Info *InstanceInfo
|
||||
State **InstanceState
|
||||
Output **InstanceDiff
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalDiffDestroy) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state := *n.State
|
||||
|
||||
// If there is no state or we don't have an ID, we're already destroyed
|
||||
if state == nil || state.ID == "" {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Call pre-diff hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreDiff(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// The diff
|
||||
diff := &InstanceDiff{Destroy: true}
|
||||
|
||||
// Call post-diff hook
|
||||
err = ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostDiff(n.Info, diff)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Update our output
|
||||
*n.Output = diff
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalDiffDestroyModule is an EvalNode implementation that writes the diff to
|
||||
// the full diff.
|
||||
type EvalDiffDestroyModule struct {
|
||||
Path []string
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalDiffDestroyModule) Eval(ctx EvalContext) (interface{}, error) {
|
||||
diff, lock := ctx.Diff()
|
||||
|
||||
// Acquire the lock so that we can do this safely concurrently
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Write the diff
|
||||
modDiff := diff.ModuleByPath(n.Path)
|
||||
if modDiff == nil {
|
||||
modDiff = diff.AddModule(n.Path)
|
||||
}
|
||||
modDiff.Destroy = true
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalDiffTainted is an EvalNode implementation that writes the diff to
|
||||
// the full diff.
|
||||
type EvalDiffTainted struct {
|
||||
Name string
|
||||
Diff **InstanceDiff
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalDiffTainted) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state, lock := ctx.State()
|
||||
|
||||
// Get a read lock so we can access this instance
|
||||
lock.RLock()
|
||||
defer lock.RUnlock()
|
||||
|
||||
// Look for the module state. If we don't have one, then it doesn't matter.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Look for the resource state. If we don't have one, then it is okay.
|
||||
rs := mod.Resources[n.Name]
|
||||
if rs == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// If we have tainted, then mark it on the diff
|
||||
if len(rs.Tainted) > 0 {
|
||||
(*n.Diff).DestroyTainted = true
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalReadDiff is an EvalNode implementation that writes the diff to
|
||||
// the full diff.
|
||||
type EvalReadDiff struct {
|
||||
Name string
|
||||
Diff **InstanceDiff
|
||||
}
|
||||
|
||||
func (n *EvalReadDiff) Eval(ctx EvalContext) (interface{}, error) {
|
||||
diff, lock := ctx.Diff()
|
||||
|
||||
// Acquire the lock so that we can do this safely concurrently
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Write the diff
|
||||
modDiff := diff.ModuleByPath(ctx.Path())
|
||||
if modDiff == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
*n.Diff = modDiff.Resources[n.Name]
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalWriteDiff is an EvalNode implementation that writes the diff to
|
||||
// the full diff.
|
||||
type EvalWriteDiff struct {
|
||||
Name string
|
||||
Diff **InstanceDiff
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalWriteDiff) Eval(ctx EvalContext) (interface{}, error) {
|
||||
diff, lock := ctx.Diff()
|
||||
|
||||
// The diff to write, if its empty it should write nil
|
||||
diffVal := *n.Diff
|
||||
if diffVal.Empty() {
|
||||
diffVal = nil
|
||||
}
|
||||
|
||||
// Acquire the lock so that we can do this safely concurrently
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Write the diff
|
||||
modDiff := diff.ModuleByPath(ctx.Path())
|
||||
if modDiff == nil {
|
||||
modDiff = diff.AddModule(ctx.Path())
|
||||
}
|
||||
if diffVal != nil {
|
||||
modDiff.Resources[n.Name] = diffVal
|
||||
} else {
|
||||
delete(modDiff.Resources, n.Name)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,25 @@
|
|||
package terraform
|
||||
|
||||
// EvalNodeFilterFunc is the callback used to replace a node with
|
||||
// another to node. To not do the replacement, just return the input node.
|
||||
type EvalNodeFilterFunc func(EvalNode) EvalNode
|
||||
|
||||
// EvalNodeFilterable is an interface that can be implemented by
|
||||
// EvalNodes to allow filtering of sub-elements. Note that this isn't
|
||||
// a common thing to implement and you probably don't need it.
|
||||
type EvalNodeFilterable interface {
|
||||
EvalNode
|
||||
Filter(EvalNodeFilterFunc)
|
||||
}
|
||||
|
||||
// EvalFilter runs the filter on the given node and returns the
|
||||
// final filtered value. This should be called rather than checking
|
||||
// the EvalNode directly since this will properly handle EvalNodeFilterables.
|
||||
func EvalFilter(node EvalNode, fn EvalNodeFilterFunc) EvalNode {
|
||||
if f, ok := node.(EvalNodeFilterable); ok {
|
||||
f.Filter(fn)
|
||||
return node
|
||||
}
|
||||
|
||||
return fn(node)
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
package terraform
|
||||
|
||||
// EvalNodeOpFilterable is an interface that EvalNodes can implement
|
||||
// to be filterable by the operation that is being run on Terraform.
|
||||
type EvalNodeOpFilterable interface {
|
||||
IncludeInOp(walkOperation) bool
|
||||
}
|
||||
|
||||
// EvalNodeFilterOp returns a filter function that filters nodes that
|
||||
// include themselves in specific operations.
|
||||
func EvalNodeFilterOp(op walkOperation) EvalNodeFilterFunc {
|
||||
return func(n EvalNode) EvalNode {
|
||||
include := true
|
||||
if of, ok := n.(EvalNodeOpFilterable); ok {
|
||||
include = of.IncludeInOp(op)
|
||||
}
|
||||
if include {
|
||||
return n
|
||||
}
|
||||
|
||||
return EvalNoop{}
|
||||
}
|
||||
}
|
||||
|
||||
// EvalOpFilter is an EvalNode implementation that is a proxy to
|
||||
// another node but filters based on the operation.
|
||||
type EvalOpFilter struct {
|
||||
// Ops is the list of operations to include this node in.
|
||||
Ops []walkOperation
|
||||
|
||||
// Node is the node to execute
|
||||
Node EvalNode
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalOpFilter) Eval(ctx EvalContext) (interface{}, error) {
|
||||
return EvalRaw(n.Node, ctx)
|
||||
}
|
||||
|
||||
// EvalNodeOpFilterable impl.
|
||||
func (n *EvalOpFilter) IncludeInOp(op walkOperation) bool {
|
||||
for _, v := range n.Ops {
|
||||
if v == op {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
package terraform
|
||||
|
||||
// EvalIf is an EvalNode that is a conditional.
|
||||
type EvalIf struct {
|
||||
If func(EvalContext) (bool, error)
|
||||
Node EvalNode
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalIf) Eval(ctx EvalContext) (interface{}, error) {
|
||||
yes, err := n.If(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if yes {
|
||||
return EvalRaw(n.Node, ctx)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalInterpolate is an EvalNode implementation that takes a raw
|
||||
// configuration and interpolates it.
|
||||
type EvalInterpolate struct {
|
||||
Config *config.RawConfig
|
||||
Resource *Resource
|
||||
Output **ResourceConfig
|
||||
}
|
||||
|
||||
func (n *EvalInterpolate) Eval(ctx EvalContext) (interface{}, error) {
|
||||
rc, err := ctx.Interpolate(n.Config, n.Resource)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if n.Output != nil {
|
||||
*n.Output = rc
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
func TestEvalInterpolate_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalInterpolate)
|
||||
}
|
||||
|
||||
func TestEvalInterpolate(t *testing.T) {
|
||||
config, err := config.NewRawConfig(map[string]interface{}{})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
var actual *ResourceConfig
|
||||
n := &EvalInterpolate{Config: config, Output: &actual}
|
||||
result := testResourceConfig(t, map[string]interface{}{})
|
||||
ctx := &MockEvalContext{InterpolateConfigResult: result}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
if actual != result {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
|
||||
if !ctx.InterpolateCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if !reflect.DeepEqual(ctx.InterpolateConfig, config) {
|
||||
t.Fatalf("bad: %#v", ctx.InterpolateConfig)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,8 @@
|
|||
package terraform
|
||||
|
||||
// EvalNoop is an EvalNode that does nothing.
|
||||
type EvalNoop struct{}
|
||||
|
||||
func (EvalNoop) Eval(EvalContext) (interface{}, error) {
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalWriteOutput is an EvalNode implementation that writes the output
|
||||
// for the given name to the current state.
|
||||
type EvalWriteOutput struct {
|
||||
Name string
|
||||
Value *config.RawConfig
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalWriteOutput) Eval(ctx EvalContext) (interface{}, error) {
|
||||
cfg, err := ctx.Interpolate(n.Value, nil)
|
||||
if err != nil {
|
||||
// Ignore it
|
||||
}
|
||||
|
||||
state, lock := ctx.State()
|
||||
if state == nil {
|
||||
return nil, fmt.Errorf("cannot write state to nil state")
|
||||
}
|
||||
|
||||
// Get a write lock so we can access this instance
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Look for the module state. If we don't have one, create it.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
mod = state.AddModule(ctx.Path())
|
||||
}
|
||||
|
||||
// Get the value from the config
|
||||
var valueRaw interface{} = config.UnknownVariableValue
|
||||
if cfg != nil {
|
||||
var ok bool
|
||||
valueRaw, ok = cfg.Get("value")
|
||||
if !ok {
|
||||
valueRaw = ""
|
||||
}
|
||||
if cfg.IsComputed("value") {
|
||||
valueRaw = config.UnknownVariableValue
|
||||
}
|
||||
}
|
||||
|
||||
// If it is a list of values, get the first one
|
||||
if list, ok := valueRaw.([]interface{}); ok {
|
||||
valueRaw = list[0]
|
||||
}
|
||||
if _, ok := valueRaw.(string); !ok {
|
||||
return nil, fmt.Errorf("output %s is not a string", n.Name)
|
||||
}
|
||||
|
||||
// Write the output
|
||||
mod.Outputs[n.Name] = valueRaw.(string)
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,108 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalConfigProvider is an EvalNode implementation that configures
|
||||
// a provider that is already initialized and retrieved.
|
||||
type EvalConfigProvider struct {
|
||||
Provider string
|
||||
Config **ResourceConfig
|
||||
}
|
||||
|
||||
func (n *EvalConfigProvider) Eval(ctx EvalContext) (interface{}, error) {
|
||||
cfg := *n.Config
|
||||
|
||||
// If we have a configuration set, then use that
|
||||
if input := ctx.ProviderInput(n.Provider); input != nil {
|
||||
rc, err := config.NewRawConfig(input)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
merged := cfg.raw.Merge(rc)
|
||||
cfg = NewResourceConfig(merged)
|
||||
}
|
||||
|
||||
// Get the parent configuration if there is one
|
||||
if parent := ctx.ParentProviderConfig(n.Provider); parent != nil {
|
||||
merged := cfg.raw.Merge(parent.raw)
|
||||
cfg = NewResourceConfig(merged)
|
||||
}
|
||||
|
||||
return nil, ctx.ConfigureProvider(n.Provider, cfg)
|
||||
}
|
||||
|
||||
// EvalInitProvider is an EvalNode implementation that initializes a provider
|
||||
// and returns nothing. The provider can be retrieved again with the
|
||||
// EvalGetProvider node.
|
||||
type EvalInitProvider struct {
|
||||
Name string
|
||||
}
|
||||
|
||||
func (n *EvalInitProvider) Eval(ctx EvalContext) (interface{}, error) {
|
||||
return ctx.InitProvider(n.Name)
|
||||
}
|
||||
|
||||
// EvalGetProvider is an EvalNode implementation that retrieves an already
|
||||
// initialized provider instance for the given name.
|
||||
type EvalGetProvider struct {
|
||||
Name string
|
||||
Output *ResourceProvider
|
||||
}
|
||||
|
||||
func (n *EvalGetProvider) Eval(ctx EvalContext) (interface{}, error) {
|
||||
result := ctx.Provider(n.Name)
|
||||
if result == nil {
|
||||
return nil, fmt.Errorf("provider %s not initialized", n.Name)
|
||||
}
|
||||
|
||||
if n.Output != nil {
|
||||
*n.Output = result
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalInputProvider is an EvalNode implementation that asks for input
|
||||
// for the given provider configurations.
|
||||
type EvalInputProvider struct {
|
||||
Name string
|
||||
Provider *ResourceProvider
|
||||
Config *config.RawConfig
|
||||
}
|
||||
|
||||
func (n *EvalInputProvider) Eval(ctx EvalContext) (interface{}, error) {
|
||||
// If we already configured this provider, then don't do this again
|
||||
if v := ctx.ProviderInput(n.Name); v != nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
rc := NewResourceConfig(n.Config)
|
||||
rc.Config = make(map[string]interface{})
|
||||
|
||||
// Wrap the input into a namespace
|
||||
input := &PrefixUIInput{
|
||||
IdPrefix: fmt.Sprintf("provider.%s", n.Name),
|
||||
QueryPrefix: fmt.Sprintf("provider.%s.", n.Name),
|
||||
UIInput: ctx.Input(),
|
||||
}
|
||||
|
||||
// Go through each provider and capture the input necessary
|
||||
// to satisfy it.
|
||||
config, err := (*n.Provider).Input(input, rc)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf(
|
||||
"Error configuring %s: %s", n.Name, err)
|
||||
}
|
||||
|
||||
if config != nil && len(config.Config) > 0 {
|
||||
// Set the configuration
|
||||
ctx.SetProviderInput(n.Name, config.Config)
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestEvalConfigProvider_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalConfigProvider)
|
||||
}
|
||||
|
||||
func TestEvalConfigProvider(t *testing.T) {
|
||||
config := testResourceConfig(t, map[string]interface{}{})
|
||||
provider := &MockResourceProvider{}
|
||||
n := &EvalConfigProvider{Config: &config}
|
||||
|
||||
ctx := &MockEvalContext{ProviderProvider: provider}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
if !ctx.ConfigureProviderCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if !reflect.DeepEqual(ctx.ConfigureProviderConfig, config) {
|
||||
t.Fatalf("bad: %#v", ctx.ConfigureProviderConfig)
|
||||
}
|
||||
}
|
||||
|
||||
func TestEvalInitProvider_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalInitProvider)
|
||||
}
|
||||
|
||||
func TestEvalInitProvider(t *testing.T) {
|
||||
n := &EvalInitProvider{Name: "foo"}
|
||||
provider := &MockResourceProvider{}
|
||||
ctx := &MockEvalContext{InitProviderProvider: provider}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
if !ctx.InitProviderCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if ctx.InitProviderName != "foo" {
|
||||
t.Fatalf("bad: %#v", ctx.InitProviderName)
|
||||
}
|
||||
}
|
||||
|
||||
func TestEvalGetProvider_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalGetProvider)
|
||||
}
|
||||
|
||||
func TestEvalGetProvider(t *testing.T) {
|
||||
var actual ResourceProvider
|
||||
n := &EvalGetProvider{Name: "foo", Output: &actual}
|
||||
provider := &MockResourceProvider{}
|
||||
ctx := &MockEvalContext{ProviderProvider: provider}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
if actual != provider {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
|
||||
if !ctx.ProviderCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if ctx.ProviderName != "foo" {
|
||||
t.Fatalf("bad: %#v", ctx.ProviderName)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// EvalInitProvisioner is an EvalNode implementation that initializes a provisioner
|
||||
// and returns nothing. The provisioner can be retrieved again with the
|
||||
// EvalGetProvisioner node.
|
||||
type EvalInitProvisioner struct {
|
||||
Name string
|
||||
}
|
||||
|
||||
func (n *EvalInitProvisioner) Eval(ctx EvalContext) (interface{}, error) {
|
||||
return ctx.InitProvisioner(n.Name)
|
||||
}
|
||||
|
||||
// EvalGetProvisioner is an EvalNode implementation that retrieves an already
|
||||
// initialized provisioner instance for the given name.
|
||||
type EvalGetProvisioner struct {
|
||||
Name string
|
||||
Output *ResourceProvisioner
|
||||
}
|
||||
|
||||
func (n *EvalGetProvisioner) Eval(ctx EvalContext) (interface{}, error) {
|
||||
result := ctx.Provisioner(n.Name)
|
||||
if result == nil {
|
||||
return nil, fmt.Errorf("provisioner %s not initialized", n.Name)
|
||||
}
|
||||
|
||||
if n.Output != nil {
|
||||
*n.Output = result
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestEvalInitProvisioner_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalInitProvisioner)
|
||||
}
|
||||
|
||||
func TestEvalInitProvisioner(t *testing.T) {
|
||||
n := &EvalInitProvisioner{Name: "foo"}
|
||||
provisioner := &MockResourceProvisioner{}
|
||||
ctx := &MockEvalContext{InitProvisionerProvisioner: provisioner}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
if !ctx.InitProvisionerCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if ctx.InitProvisionerName != "foo" {
|
||||
t.Fatalf("bad: %#v", ctx.InitProvisionerName)
|
||||
}
|
||||
}
|
||||
|
||||
func TestEvalGetProvisioner_impl(t *testing.T) {
|
||||
var _ EvalNode = new(EvalGetProvisioner)
|
||||
}
|
||||
|
||||
func TestEvalGetProvisioner(t *testing.T) {
|
||||
var actual ResourceProvisioner
|
||||
n := &EvalGetProvisioner{Name: "foo", Output: &actual}
|
||||
provisioner := &MockResourceProvisioner{}
|
||||
ctx := &MockEvalContext{ProvisionerProvisioner: provisioner}
|
||||
if _, err := n.Eval(ctx); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
if actual != provisioner {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
|
||||
if !ctx.ProvisionerCalled {
|
||||
t.Fatal("should be called")
|
||||
}
|
||||
if ctx.ProvisionerName != "foo" {
|
||||
t.Fatalf("bad: %#v", ctx.ProvisionerName)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"log"
|
||||
)
|
||||
|
||||
// EvalRefresh is an EvalNode implementation that does a refresh for
|
||||
// a resource.
|
||||
type EvalRefresh struct {
|
||||
Provider *ResourceProvider
|
||||
State **InstanceState
|
||||
Info *InstanceInfo
|
||||
Output **InstanceState
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalRefresh) Eval(ctx EvalContext) (interface{}, error) {
|
||||
provider := *n.Provider
|
||||
state := *n.State
|
||||
|
||||
// If we have no state, we don't do any refreshing
|
||||
if state == nil {
|
||||
log.Printf("[DEBUG] refresh: %s: no state, not refreshing", n.Info.Id)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Call pre-refresh hook
|
||||
err := ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PreRefresh(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Refresh!
|
||||
state, err = provider.Refresh(n.Info, state)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Call post-refresh hook
|
||||
err = ctx.Hook(func(h Hook) (HookAction, error) {
|
||||
return h.PostRefresh(n.Info, state)
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if n.Output != nil {
|
||||
*n.Output = state
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,13 @@
|
|||
package terraform
|
||||
|
||||
// EvalInstanceInfo is an EvalNode implementation that fills in the
|
||||
// InstanceInfo as much as it can.
|
||||
type EvalInstanceInfo struct {
|
||||
Info *InstanceInfo
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalInstanceInfo) Eval(ctx EvalContext) (interface{}, error) {
|
||||
n.Info.ModulePath = ctx.Path()
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
package terraform
|
||||
|
||||
// EvalSequence is an EvalNode that evaluates in sequence.
|
||||
type EvalSequence struct {
|
||||
Nodes []EvalNode
|
||||
}
|
||||
|
||||
func (n *EvalSequence) Eval(ctx EvalContext) (interface{}, error) {
|
||||
for _, n := range n.Nodes {
|
||||
if _, err := EvalRaw(n, ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalNodeFilterable impl.
|
||||
func (n *EvalSequence) Filter(fn EvalNodeFilterFunc) {
|
||||
for i, node := range n.Nodes {
|
||||
n.Nodes[i] = fn(node)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,9 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestEvalSequence_impl(t *testing.T) {
|
||||
var _ EvalNodeFilterable = new(EvalSequence)
|
||||
}
|
|
@ -0,0 +1,196 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
// EvalReadState is an EvalNode implementation that reads the
|
||||
// InstanceState for a specific resource out of the state.
|
||||
type EvalReadState struct {
|
||||
Name string
|
||||
Tainted bool
|
||||
TaintedIndex int
|
||||
Output **InstanceState
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalReadState) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state, lock := ctx.State()
|
||||
|
||||
// Get a read lock so we can access this instance
|
||||
lock.RLock()
|
||||
defer lock.RUnlock()
|
||||
|
||||
// Look for the module state. If we don't have one, then it doesn't matter.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Look for the resource state. If we don't have one, then it is okay.
|
||||
rs := mod.Resources[n.Name]
|
||||
if rs == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
var result *InstanceState
|
||||
if !n.Tainted {
|
||||
// Return the primary
|
||||
result = rs.Primary
|
||||
} else {
|
||||
// Get the index. If it is negative, then we get the last one
|
||||
idx := n.TaintedIndex
|
||||
if idx < 0 {
|
||||
idx = len(rs.Tainted) - 1
|
||||
}
|
||||
|
||||
if idx < len(rs.Tainted) {
|
||||
// Return the proper tainted resource
|
||||
result = rs.Tainted[n.TaintedIndex]
|
||||
}
|
||||
}
|
||||
|
||||
// Write the result to the output pointer
|
||||
if n.Output != nil {
|
||||
*n.Output = result
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// EvalWriteState is an EvalNode implementation that reads the
|
||||
// InstanceState for a specific resource out of the state.
|
||||
type EvalWriteState struct {
|
||||
Name string
|
||||
ResourceType string
|
||||
Dependencies []string
|
||||
State **InstanceState
|
||||
Tainted *bool
|
||||
TaintedIndex int
|
||||
TaintedClearPrimary bool
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalWriteState) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state, lock := ctx.State()
|
||||
if state == nil {
|
||||
return nil, fmt.Errorf("cannot write state to nil state")
|
||||
}
|
||||
|
||||
// Get a write lock so we can access this instance
|
||||
lock.Lock()
|
||||
defer lock.Unlock()
|
||||
|
||||
// Look for the module state. If we don't have one, create it.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
mod = state.AddModule(ctx.Path())
|
||||
}
|
||||
|
||||
// Look for the resource state.
|
||||
rs := mod.Resources[n.Name]
|
||||
if rs == nil {
|
||||
rs = &ResourceState{}
|
||||
rs.init()
|
||||
mod.Resources[n.Name] = rs
|
||||
}
|
||||
rs.Type = n.ResourceType
|
||||
rs.Dependencies = n.Dependencies
|
||||
|
||||
if n.Tainted != nil && *n.Tainted {
|
||||
if n.TaintedIndex != -1 {
|
||||
rs.Tainted[n.TaintedIndex] = *n.State
|
||||
} else {
|
||||
rs.Tainted = append(rs.Tainted, *n.State)
|
||||
}
|
||||
|
||||
if n.TaintedClearPrimary {
|
||||
rs.Primary = nil
|
||||
}
|
||||
} else {
|
||||
// Set the primary state
|
||||
rs.Primary = *n.State
|
||||
}
|
||||
println(fmt.Sprintf("%#v", rs))
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalDeposeState is an EvalNode implementation that takes the primary
|
||||
// out of a state and makes it tainted. This is done at the beggining of
|
||||
// create-before-destroy calls so that the create can create while preserving
|
||||
// the old state of the to-be-destroyed resource.
|
||||
type EvalDeposeState struct {
|
||||
Name string
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalDeposeState) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state, lock := ctx.State()
|
||||
|
||||
// Get a read lock so we can access this instance
|
||||
lock.RLock()
|
||||
defer lock.RUnlock()
|
||||
|
||||
// Look for the module state. If we don't have one, then it doesn't matter.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Look for the resource state. If we don't have one, then it is okay.
|
||||
rs := mod.Resources[n.Name]
|
||||
if rs == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// If we don't have a primary, we have nothing to depose
|
||||
if rs.Primary == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Depose to the tainted
|
||||
rs.Tainted = append(rs.Tainted, rs.Primary)
|
||||
rs.Primary = nil
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalUndeposeState is an EvalNode implementation that reads the
|
||||
// InstanceState for a specific resource out of the state.
|
||||
type EvalUndeposeState struct {
|
||||
Name string
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalUndeposeState) Eval(ctx EvalContext) (interface{}, error) {
|
||||
state, lock := ctx.State()
|
||||
|
||||
// Get a read lock so we can access this instance
|
||||
lock.RLock()
|
||||
defer lock.RUnlock()
|
||||
|
||||
// Look for the module state. If we don't have one, then it doesn't matter.
|
||||
mod := state.ModuleByPath(ctx.Path())
|
||||
if mod == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Look for the resource state. If we don't have one, then it is okay.
|
||||
rs := mod.Resources[n.Name]
|
||||
if rs == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// If we don't have any tainted, then we don't have anything to do
|
||||
if len(rs.Tainted) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Undepose to the tainted
|
||||
idx := len(rs.Tainted) - 1
|
||||
rs.Primary = rs.Tainted[idx]
|
||||
rs.Tainted[idx] = nil
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestMockEvalContext_impl(t *testing.T) {
|
||||
var _ EvalContext = new(MockEvalContext)
|
||||
}
|
||||
|
||||
func TestEval(t *testing.T) {
|
||||
var result int
|
||||
n := &testEvalAdd{
|
||||
Items: []int{10, 32},
|
||||
Result: &result,
|
||||
}
|
||||
|
||||
if _, err := Eval(n, nil); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
if result != 42 {
|
||||
t.Fatalf("bad: %#v", result)
|
||||
}
|
||||
}
|
||||
|
||||
type testEvalAdd struct {
|
||||
Items []int
|
||||
Result *int
|
||||
}
|
||||
|
||||
func (n *testEvalAdd) Eval(ctx EvalContext) (interface{}, error) {
|
||||
result := 0
|
||||
for _, item := range n.Items {
|
||||
result += item
|
||||
}
|
||||
|
||||
*n.Result = result
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,141 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalValidateError is the error structure returned if there were
|
||||
// validation errors.
|
||||
type EvalValidateError struct {
|
||||
Warnings []string
|
||||
Errors []error
|
||||
}
|
||||
|
||||
func (e *EvalValidateError) Error() string {
|
||||
return fmt.Sprintf("Warnings: %s. Errors: %s", e.Warnings, e.Errors)
|
||||
}
|
||||
|
||||
// EvalValidateCount is an EvalNode implementation that validates
|
||||
// the count of a resource.
|
||||
type EvalValidateCount struct {
|
||||
Resource *config.Resource
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalValidateCount) Eval(ctx EvalContext) (interface{}, error) {
|
||||
var count int
|
||||
var errs []error
|
||||
var err error
|
||||
if _, err := ctx.Interpolate(n.Resource.RawCount, nil); err != nil {
|
||||
errs = append(errs, fmt.Errorf(
|
||||
"Failed to interpolate count: %s", err))
|
||||
goto RETURN
|
||||
}
|
||||
|
||||
count, err = n.Resource.Count()
|
||||
if err != nil {
|
||||
// If we can't get the count during validation, then
|
||||
// just replace it with the number 1.
|
||||
c := n.Resource.RawCount.Config()
|
||||
c[n.Resource.RawCount.Key] = "1"
|
||||
count = 1
|
||||
}
|
||||
|
||||
if count < 0 {
|
||||
errs = append(errs, fmt.Errorf(
|
||||
"Count is less than zero: %d", count))
|
||||
}
|
||||
|
||||
RETURN:
|
||||
return nil, &EvalValidateError{
|
||||
Errors: errs,
|
||||
}
|
||||
}
|
||||
|
||||
// EvalValidateProvider is an EvalNode implementation that validates
|
||||
// the configuration of a resource.
|
||||
type EvalValidateProvider struct {
|
||||
ProviderName string
|
||||
Provider *ResourceProvider
|
||||
Config **ResourceConfig
|
||||
}
|
||||
|
||||
func (n *EvalValidateProvider) Eval(ctx EvalContext) (interface{}, error) {
|
||||
provider := *n.Provider
|
||||
config := *n.Config
|
||||
|
||||
// Get the parent configuration if there is one
|
||||
if parent := ctx.ParentProviderConfig(n.ProviderName); parent != nil {
|
||||
merged := parent.raw.Merge(config.raw)
|
||||
config = NewResourceConfig(merged)
|
||||
}
|
||||
|
||||
warns, errs := provider.Validate(config)
|
||||
if len(warns) == 0 && len(errs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return nil, &EvalValidateError{
|
||||
Warnings: warns,
|
||||
Errors: errs,
|
||||
}
|
||||
}
|
||||
|
||||
// EvalValidateProvisioner is an EvalNode implementation that validates
|
||||
// the configuration of a resource.
|
||||
type EvalValidateProvisioner struct {
|
||||
Provisioner *ResourceProvisioner
|
||||
Config **ResourceConfig
|
||||
}
|
||||
|
||||
func (n *EvalValidateProvisioner) Eval(ctx EvalContext) (interface{}, error) {
|
||||
provisioner := *n.Provisioner
|
||||
config := *n.Config
|
||||
warns, errs := provisioner.Validate(config)
|
||||
if len(warns) == 0 && len(errs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return nil, &EvalValidateError{
|
||||
Warnings: warns,
|
||||
Errors: errs,
|
||||
}
|
||||
}
|
||||
|
||||
// EvalValidateResource is an EvalNode implementation that validates
|
||||
// the configuration of a resource.
|
||||
type EvalValidateResource struct {
|
||||
Provider *ResourceProvider
|
||||
Config **ResourceConfig
|
||||
ResourceName string
|
||||
ResourceType string
|
||||
}
|
||||
|
||||
func (n *EvalValidateResource) Eval(ctx EvalContext) (interface{}, error) {
|
||||
// TODO: test
|
||||
|
||||
provider := *n.Provider
|
||||
cfg := *n.Config
|
||||
warns, errs := provider.ValidateResource(n.ResourceType, cfg)
|
||||
|
||||
// If the resouce name doesn't match the name regular
|
||||
// expression, show a warning.
|
||||
if !config.NameRegexp.Match([]byte(n.ResourceName)) {
|
||||
warns = append(warns, fmt.Sprintf(
|
||||
"%s: resource name can only contain letters, numbers, "+
|
||||
"dashes, and underscores.\n"+
|
||||
"This will be an error in Terraform 0.4",
|
||||
n.ResourceName))
|
||||
}
|
||||
|
||||
if len(warns) == 0 && len(errs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return nil, &EvalValidateError{
|
||||
Warnings: warns,
|
||||
Errors: errs,
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// EvalSetVariables is an EvalNode implementation that sets the variables
|
||||
// explicitly for interpolation later.
|
||||
type EvalSetVariables struct {
|
||||
Variables map[string]string
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalSetVariables) Eval(ctx EvalContext) (interface{}, error) {
|
||||
ctx.SetVariables(n.Variables)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// EvalVariableBlock is an EvalNode implementation that evaluates the
|
||||
// given configuration, and uses the final values as a way to set the
|
||||
// mapping.
|
||||
type EvalVariableBlock struct {
|
||||
Config **ResourceConfig
|
||||
Variables map[string]string
|
||||
}
|
||||
|
||||
// TODO: test
|
||||
func (n *EvalVariableBlock) Eval(ctx EvalContext) (interface{}, error) {
|
||||
// Clear out the existing mapping
|
||||
for k, _ := range n.Variables {
|
||||
delete(n.Variables, k)
|
||||
}
|
||||
|
||||
// Get our configuration
|
||||
rc := *n.Config
|
||||
for k, v := range rc.Config {
|
||||
n.Variables[k] = v.(string)
|
||||
}
|
||||
for k, _ := range rc.Raw {
|
||||
if _, ok := n.Variables[k]; !ok {
|
||||
n.Variables[k] = config.UnknownVariableValue
|
||||
}
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
|
@ -0,0 +1,61 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/config"
|
||||
)
|
||||
|
||||
// ProviderEvalTree returns the evaluation tree for initializing and
|
||||
// configuring providers.
|
||||
func ProviderEvalTree(n string, config *config.RawConfig) EvalNode {
|
||||
var provider ResourceProvider
|
||||
var resourceConfig *ResourceConfig
|
||||
|
||||
seq := make([]EvalNode, 0, 5)
|
||||
seq = append(seq, &EvalInitProvider{Name: n})
|
||||
|
||||
// Input stuff
|
||||
seq = append(seq, &EvalOpFilter{
|
||||
Ops: []walkOperation{walkInput},
|
||||
Node: &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalGetProvider{
|
||||
Name: n,
|
||||
Output: &provider,
|
||||
},
|
||||
&EvalInputProvider{
|
||||
Name: n,
|
||||
Provider: &provider,
|
||||
Config: config,
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
// Apply stuff
|
||||
seq = append(seq, &EvalOpFilter{
|
||||
Ops: []walkOperation{walkValidate, walkRefresh, walkPlan, walkApply},
|
||||
Node: &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalGetProvider{
|
||||
Name: n,
|
||||
Output: &provider,
|
||||
},
|
||||
&EvalInterpolate{
|
||||
Config: config,
|
||||
Output: &resourceConfig,
|
||||
},
|
||||
&EvalValidateProvider{
|
||||
ProviderName: n,
|
||||
Provider: &provider,
|
||||
Config: &resourceConfig,
|
||||
},
|
||||
&EvalConfigProvider{
|
||||
Provider: n,
|
||||
Config: &resourceConfig,
|
||||
},
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
return &EvalSequence{Nodes: seq}
|
||||
}
|
2115
terraform/graph.go
2115
terraform/graph.go
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,113 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"log"
|
||||
|
||||
"github.com/hashicorp/terraform/config/module"
|
||||
)
|
||||
|
||||
// GraphBuilder is an interface that can be implemented and used with
|
||||
// Terraform to build the graph that Terraform walks.
|
||||
type GraphBuilder interface {
|
||||
// Build builds the graph for the given module path. It is up to
|
||||
// the interface implementation whether this build should expand
|
||||
// the graph or not.
|
||||
Build(path []string) (*Graph, error)
|
||||
}
|
||||
|
||||
// BasicGraphBuilder is a GraphBuilder that builds a graph out of a
|
||||
// series of transforms and validates the graph is a valid structure.
|
||||
type BasicGraphBuilder struct {
|
||||
Steps []GraphTransformer
|
||||
}
|
||||
|
||||
func (b *BasicGraphBuilder) Build(path []string) (*Graph, error) {
|
||||
g := &Graph{Path: path}
|
||||
for _, step := range b.Steps {
|
||||
if err := step.Transform(g); err != nil {
|
||||
return g, err
|
||||
}
|
||||
|
||||
log.Printf("[TRACE] Graph after step %T:\n\n%s", step, g.String())
|
||||
}
|
||||
|
||||
// Validate the graph structure
|
||||
if err := g.Validate(); err != nil {
|
||||
log.Printf("[ERROR] Graph validation failed. Graph:\n\n%s", g.String())
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return g, nil
|
||||
}
|
||||
|
||||
// BuiltinGraphBuilder is responsible for building the complete graph that
|
||||
// Terraform uses for execution. It is an opinionated builder that defines
|
||||
// the step order required to build a complete graph as is used and expected
|
||||
// by Terraform.
|
||||
//
|
||||
// If you require a custom graph, you'll have to build it up manually
|
||||
// on your own by building a new GraphBuilder implementation.
|
||||
type BuiltinGraphBuilder struct {
|
||||
// Root is the root module of the graph to build.
|
||||
Root *module.Tree
|
||||
|
||||
// Diff is the diff. The proper module diffs will be looked up.
|
||||
Diff *Diff
|
||||
|
||||
// State is the global state. The proper module states will be looked
|
||||
// up by graph path.
|
||||
State *State
|
||||
|
||||
// Providers is the list of providers supported.
|
||||
Providers []string
|
||||
|
||||
// Provisioners is the list of provisioners supported.
|
||||
Provisioners []string
|
||||
}
|
||||
|
||||
// Build builds the graph according to the steps returned by Steps.
|
||||
func (b *BuiltinGraphBuilder) Build(path []string) (*Graph, error) {
|
||||
basic := &BasicGraphBuilder{
|
||||
Steps: b.Steps(),
|
||||
}
|
||||
|
||||
return basic.Build(path)
|
||||
}
|
||||
|
||||
// Steps returns the ordered list of GraphTransformers that must be executed
|
||||
// to build a complete graph.
|
||||
func (b *BuiltinGraphBuilder) Steps() []GraphTransformer {
|
||||
return []GraphTransformer{
|
||||
// Create all our resources from the configuration and state
|
||||
&ConfigTransformer{Module: b.Root},
|
||||
&OrphanTransformer{State: b.State, Module: b.Root},
|
||||
|
||||
// Provider-related transformations
|
||||
&MissingProviderTransformer{Providers: b.Providers},
|
||||
&ProviderTransformer{},
|
||||
&PruneProviderTransformer{},
|
||||
|
||||
// Provisioner-related transformations
|
||||
&MissingProvisionerTransformer{Provisioners: b.Provisioners},
|
||||
&ProvisionerTransformer{},
|
||||
&PruneProvisionerTransformer{},
|
||||
|
||||
// Run our vertex-level transforms
|
||||
&VertexTransformer{
|
||||
Transforms: []GraphVertexTransformer{
|
||||
// Expand any statically expanded nodes, such as module graphs
|
||||
&ExpandTransform{
|
||||
Builder: b,
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
// Create the destruction nodes
|
||||
&DestroyTransformer{},
|
||||
&CreateBeforeDestroyTransformer{},
|
||||
&PruneDestroyTransformer{Diff: b.Diff, State: b.State},
|
||||
|
||||
// Make sure we create one root
|
||||
&RootTransformer{},
|
||||
}
|
||||
}
|
|
@ -0,0 +1,156 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
func TestBasicGraphBuilder_impl(t *testing.T) {
|
||||
var _ GraphBuilder = new(BasicGraphBuilder)
|
||||
}
|
||||
|
||||
func TestBasicGraphBuilder(t *testing.T) {
|
||||
b := &BasicGraphBuilder{
|
||||
Steps: []GraphTransformer{
|
||||
&testBasicGraphBuilderTransform{1},
|
||||
},
|
||||
}
|
||||
|
||||
g, err := b.Build(RootModulePath)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(g.Path, RootModulePath) {
|
||||
t.Fatalf("bad: %#v", g.Path)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testBasicGraphBuilderStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestBasicGraphBuilder_validate(t *testing.T) {
|
||||
b := &BasicGraphBuilder{
|
||||
Steps: []GraphTransformer{
|
||||
&testBasicGraphBuilderTransform{1},
|
||||
&testBasicGraphBuilderTransform{2},
|
||||
},
|
||||
}
|
||||
|
||||
_, err := b.Build(RootModulePath)
|
||||
if err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestBuiltinGraphBuilder_impl(t *testing.T) {
|
||||
var _ GraphBuilder = new(BuiltinGraphBuilder)
|
||||
}
|
||||
|
||||
// This test is not meant to test all the transforms but rather just
|
||||
// to verify we get some basic sane graph out. Special tests to ensure
|
||||
// specific ordering of steps should be added in other tests.
|
||||
func TestBuiltinGraphBuilder(t *testing.T) {
|
||||
b := &BuiltinGraphBuilder{
|
||||
Root: testModule(t, "graph-builder-basic"),
|
||||
}
|
||||
|
||||
g, err := b.Build(RootModulePath)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testBuiltinGraphBuilderBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
// This tests a cycle we got when a CBD resource depends on a non-CBD
|
||||
// resource. This cycle shouldn't happen in the general case anymore.
|
||||
func TestBuiltinGraphBuilder_cbdDepNonCbd(t *testing.T) {
|
||||
b := &BuiltinGraphBuilder{
|
||||
Root: testModule(t, "graph-builder-cbd-non-cbd"),
|
||||
}
|
||||
|
||||
_, err := b.Build(RootModulePath)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
TODO: This exposes a really bad bug we need to fix after we merge
|
||||
the f-ast-branch. This bug still exists in master.
|
||||
|
||||
// This test tests that the graph builder properly expands modules.
|
||||
func TestBuiltinGraphBuilder_modules(t *testing.T) {
|
||||
b := &BuiltinGraphBuilder{
|
||||
Root: testModule(t, "graph-builder-modules"),
|
||||
}
|
||||
|
||||
g, err := b.Build(RootModulePath)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testBuiltinGraphBuilderModuleStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
type testBasicGraphBuilderTransform struct {
|
||||
V dag.Vertex
|
||||
}
|
||||
|
||||
func (t *testBasicGraphBuilderTransform) Transform(g *Graph) error {
|
||||
g.Add(t.V)
|
||||
return nil
|
||||
}
|
||||
|
||||
const testBasicGraphBuilderStr = `
|
||||
1
|
||||
`
|
||||
|
||||
const testBuiltinGraphBuilderBasicStr = `
|
||||
aws_instance.db
|
||||
aws_instance.db (destroy tainted)
|
||||
provider.aws
|
||||
aws_instance.db (destroy tainted)
|
||||
aws_instance.web (destroy tainted)
|
||||
provider.aws
|
||||
aws_instance.web
|
||||
aws_instance.db
|
||||
aws_instance.web (destroy tainted)
|
||||
provider.aws
|
||||
aws_instance.web (destroy tainted)
|
||||
provider.aws
|
||||
provider.aws
|
||||
`
|
||||
|
||||
const testBuiltinGraphBuilderModuleStr = `
|
||||
aws_instance.web
|
||||
aws_instance.web (destroy)
|
||||
aws_instance.web (destroy)
|
||||
aws_security_group.firewall
|
||||
module.consul (expanded)
|
||||
provider.aws
|
||||
aws_security_group.firewall
|
||||
aws_security_group.firewall (destroy)
|
||||
aws_security_group.firewall (destroy)
|
||||
provider.aws
|
||||
module.consul (expanded)
|
||||
aws_security_group.firewall
|
||||
provider.aws
|
||||
provider.aws
|
||||
`
|
|
@ -0,0 +1,546 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/config/module"
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// graphNodeConfig is an interface that all graph nodes for the
|
||||
// configuration graph need to implement in order to build the variable
|
||||
// dependencies properly.
|
||||
type graphNodeConfig interface {
|
||||
dag.NamedVertex
|
||||
|
||||
// All graph nodes should be dependent on other things, and able to
|
||||
// be depended on.
|
||||
GraphNodeDependable
|
||||
GraphNodeDependent
|
||||
}
|
||||
|
||||
// GraphNodeConfigModule represents a module within the configuration graph.
|
||||
type GraphNodeConfigModule struct {
|
||||
Path []string
|
||||
Module *config.Module
|
||||
Tree *module.Tree
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigModule) DependableName() []string {
|
||||
return []string{n.Name()}
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigModule) DependentOn() []string {
|
||||
vars := n.Module.RawConfig.Variables
|
||||
result := make([]string, 0, len(vars))
|
||||
for _, v := range vars {
|
||||
if vn := varNameForVar(v); vn != "" {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigModule) Name() string {
|
||||
return fmt.Sprintf("module.%s", n.Module.Name)
|
||||
}
|
||||
|
||||
// GraphNodeExpandable
|
||||
func (n *GraphNodeConfigModule) Expand(b GraphBuilder) (GraphNodeSubgraph, error) {
|
||||
// Build the graph first
|
||||
graph, err := b.Build(n.Path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Add the parameters node to the module
|
||||
t := &ModuleInputTransformer{Variables: make(map[string]string)}
|
||||
if err := t.Transform(graph); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Build the actual subgraph node
|
||||
return &graphNodeModuleExpanded{
|
||||
Original: n,
|
||||
Graph: graph,
|
||||
InputConfig: n.Module.RawConfig,
|
||||
Variables: t.Variables,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// GraphNodeExpandable
|
||||
func (n *GraphNodeConfigModule) ProvidedBy() []string {
|
||||
// Build up the list of providers by simply going over our configuration
|
||||
// to find the providers that are configured there as well as the
|
||||
// providers that the resources use.
|
||||
config := n.Tree.Config()
|
||||
providers := make(map[string]struct{})
|
||||
for _, p := range config.ProviderConfigs {
|
||||
providers[p.Name] = struct{}{}
|
||||
}
|
||||
for _, r := range config.Resources {
|
||||
providers[resourceProvider(r.Type)] = struct{}{}
|
||||
}
|
||||
|
||||
// Turn the map into a string. This makes sure that the list is
|
||||
// de-dupped since we could be going over potentially many resources.
|
||||
result := make([]string, 0, len(providers))
|
||||
for p, _ := range providers {
|
||||
result = append(result, p)
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// GraphNodeConfigOutput represents an output configured within the
|
||||
// configuration.
|
||||
type GraphNodeConfigOutput struct {
|
||||
Output *config.Output
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigOutput) Name() string {
|
||||
return fmt.Sprintf("output.%s", n.Output.Name)
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigOutput) DependableName() []string {
|
||||
return []string{n.Name()}
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigOutput) DependentOn() []string {
|
||||
vars := n.Output.RawConfig.Variables
|
||||
result := make([]string, 0, len(vars))
|
||||
for _, v := range vars {
|
||||
if vn := varNameForVar(v); vn != "" {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// GraphNodeEvalable impl.
|
||||
func (n *GraphNodeConfigOutput) EvalTree() EvalNode {
|
||||
return &EvalOpFilter{
|
||||
Ops: []walkOperation{walkRefresh, walkPlan, walkApply},
|
||||
Node: &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalWriteOutput{
|
||||
Name: n.Output.Name,
|
||||
Value: n.Output.RawConfig,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// GraphNodeConfigProvider represents a configured provider within the
|
||||
// configuration graph. These are only immediately in the graph when an
|
||||
// explicit `provider` configuration block is in the configuration.
|
||||
type GraphNodeConfigProvider struct {
|
||||
Provider *config.ProviderConfig
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigProvider) Name() string {
|
||||
return fmt.Sprintf("provider.%s", n.Provider.Name)
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigProvider) DependableName() []string {
|
||||
return []string{n.Name()}
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigProvider) DependentOn() []string {
|
||||
vars := n.Provider.RawConfig.Variables
|
||||
result := make([]string, 0, len(vars))
|
||||
for _, v := range vars {
|
||||
if vn := varNameForVar(v); vn != "" {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// GraphNodeEvalable impl.
|
||||
func (n *GraphNodeConfigProvider) EvalTree() EvalNode {
|
||||
return ProviderEvalTree(n.Provider.Name, n.Provider.RawConfig)
|
||||
}
|
||||
|
||||
// GraphNodeProvider implementation
|
||||
func (n *GraphNodeConfigProvider) ProviderName() string {
|
||||
return n.Provider.Name
|
||||
}
|
||||
|
||||
// GraphNodeDotter impl.
|
||||
func (n *GraphNodeConfigProvider) Dot(name string) string {
|
||||
return fmt.Sprintf(
|
||||
"\"%s\" [\n"+
|
||||
"\tlabel=\"%s\"\n"+
|
||||
"\tshape=diamond\n"+
|
||||
"];",
|
||||
name,
|
||||
n.Name())
|
||||
}
|
||||
|
||||
// GraphNodeConfigResource represents a resource within the config graph.
|
||||
type GraphNodeConfigResource struct {
|
||||
Resource *config.Resource
|
||||
|
||||
// If this is set to anything other than destroyModeNone, then this
|
||||
// resource represents a resource that will be destroyed in some way.
|
||||
DestroyMode GraphNodeDestroyMode
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigResource) DependableName() []string {
|
||||
return []string{n.Resource.Id()}
|
||||
}
|
||||
|
||||
// GraphNodeDependent impl.
|
||||
func (n *GraphNodeConfigResource) DependentOn() []string {
|
||||
result := make([]string, len(n.Resource.DependsOn),
|
||||
(len(n.Resource.RawCount.Variables)+
|
||||
len(n.Resource.RawConfig.Variables)+
|
||||
len(n.Resource.DependsOn))*2)
|
||||
copy(result, n.Resource.DependsOn)
|
||||
|
||||
for _, v := range n.Resource.RawCount.Variables {
|
||||
if vn := varNameForVar(v); vn != "" {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
for _, v := range n.Resource.RawConfig.Variables {
|
||||
if vn := varNameForVar(v); vn != "" {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
for _, p := range n.Resource.Provisioners {
|
||||
for _, v := range p.ConnInfo.Variables {
|
||||
if vn := varNameForVar(v); vn != "" && vn != n.Resource.Id() {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
for _, v := range p.RawConfig.Variables {
|
||||
if vn := varNameForVar(v); vn != "" && vn != n.Resource.Id() {
|
||||
result = append(result, vn)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
func (n *GraphNodeConfigResource) Name() string {
|
||||
result := n.Resource.Id()
|
||||
switch n.DestroyMode {
|
||||
case DestroyNone:
|
||||
case DestroyPrimary:
|
||||
result += " (destroy)"
|
||||
case DestroyTainted:
|
||||
result += " (destroy tainted)"
|
||||
default:
|
||||
result += " (unknown destroy type)"
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// GraphNodeDotter impl.
|
||||
func (n *GraphNodeConfigResource) Dot(name string) string {
|
||||
if n.DestroyMode != DestroyNone {
|
||||
return ""
|
||||
}
|
||||
|
||||
return fmt.Sprintf(
|
||||
"\"%s\" [\n"+
|
||||
"\tlabel=\"%s\"\n"+
|
||||
"\tshape=box\n"+
|
||||
"];",
|
||||
name,
|
||||
n.Name())
|
||||
}
|
||||
|
||||
// GraphNodeDynamicExpandable impl.
|
||||
func (n *GraphNodeConfigResource) DynamicExpand(ctx EvalContext) (*Graph, error) {
|
||||
state, lock := ctx.State()
|
||||
lock.RLock()
|
||||
defer lock.RUnlock()
|
||||
|
||||
// Start creating the steps
|
||||
steps := make([]GraphTransformer, 0, 5)
|
||||
|
||||
// Primary and non-destroy modes are responsible for creating/destroying
|
||||
// all the nodes, expanding counts.
|
||||
switch n.DestroyMode {
|
||||
case DestroyNone:
|
||||
fallthrough
|
||||
case DestroyPrimary:
|
||||
steps = append(steps, &ResourceCountTransformer{
|
||||
Resource: n.Resource,
|
||||
Destroy: n.DestroyMode != DestroyNone,
|
||||
})
|
||||
}
|
||||
|
||||
// Additional destroy modifications.
|
||||
switch n.DestroyMode {
|
||||
case DestroyPrimary:
|
||||
// If we're destroying the primary instance, then we want to
|
||||
// expand orphans, which have all the same semantics in a destroy
|
||||
// as a primary.
|
||||
steps = append(steps, &OrphanTransformer{
|
||||
State: state,
|
||||
View: n.Resource.Id(),
|
||||
})
|
||||
|
||||
// If we're only destroying tainted resources, then we only
|
||||
// want to find tainted resources and destroy them here.
|
||||
steps = append(steps, &TaintedTransformer{
|
||||
State: state,
|
||||
View: n.Resource.Id(),
|
||||
Deposed: n.Resource.Lifecycle.CreateBeforeDestroy,
|
||||
DeposedInclude: true,
|
||||
})
|
||||
case DestroyTainted:
|
||||
// If we're only destroying tainted resources, then we only
|
||||
// want to find tainted resources and destroy them here.
|
||||
steps = append(steps, &TaintedTransformer{
|
||||
State: state,
|
||||
View: n.Resource.Id(),
|
||||
Deposed: n.Resource.Lifecycle.CreateBeforeDestroy,
|
||||
DeposedInclude: false,
|
||||
})
|
||||
}
|
||||
|
||||
// Always end with the root being added
|
||||
steps = append(steps, &RootTransformer{})
|
||||
|
||||
// Build the graph
|
||||
b := &BasicGraphBuilder{Steps: steps}
|
||||
return b.Build(ctx.Path())
|
||||
}
|
||||
|
||||
// GraphNodeEvalable impl.
|
||||
func (n *GraphNodeConfigResource) EvalTree() EvalNode {
|
||||
return &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalInterpolate{Config: n.Resource.RawCount},
|
||||
&EvalOpFilter{
|
||||
Ops: []walkOperation{walkValidate},
|
||||
Node: &EvalValidateCount{Resource: n.Resource},
|
||||
},
|
||||
&EvalCountFixZeroOneBoundary{Resource: n.Resource},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// GraphNodeProviderConsumer
|
||||
func (n *GraphNodeConfigResource) ProvidedBy() []string {
|
||||
return []string{resourceProvider(n.Resource.Type)}
|
||||
}
|
||||
|
||||
// GraphNodeProvisionerConsumer
|
||||
func (n *GraphNodeConfigResource) ProvisionedBy() []string {
|
||||
result := make([]string, len(n.Resource.Provisioners))
|
||||
for i, p := range n.Resource.Provisioners {
|
||||
result[i] = p.Type
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// GraphNodeDestroyable
|
||||
func (n *GraphNodeConfigResource) DestroyNode(mode GraphNodeDestroyMode) GraphNodeDestroy {
|
||||
// If we're already a destroy node, then don't do anything
|
||||
if n.DestroyMode != DestroyNone {
|
||||
return nil
|
||||
}
|
||||
|
||||
result := &graphNodeResourceDestroy{
|
||||
GraphNodeConfigResource: *n,
|
||||
Original: n,
|
||||
}
|
||||
result.DestroyMode = mode
|
||||
return result
|
||||
}
|
||||
|
||||
// graphNodeResourceDestroy represents the logical destruction of a
|
||||
// resource. This node doesn't mean it will be destroyed for sure, but
|
||||
// instead that if a destroy were to happen, it must happen at this point.
|
||||
type graphNodeResourceDestroy struct {
|
||||
GraphNodeConfigResource
|
||||
Original *GraphNodeConfigResource
|
||||
}
|
||||
|
||||
func (n *graphNodeResourceDestroy) CreateBeforeDestroy() bool {
|
||||
// CBD is enabled if the resource enables it in addition to us
|
||||
// being responsible for destroying the primary state. The primary
|
||||
// state destroy node is the only destroy node that needs to be
|
||||
// "shuffled" according to the CBD rules, since tainted resources
|
||||
// don't have the same inverse dependencies.
|
||||
return n.Original.Resource.Lifecycle.CreateBeforeDestroy &&
|
||||
n.DestroyMode == DestroyPrimary
|
||||
}
|
||||
|
||||
func (n *graphNodeResourceDestroy) CreateNode() dag.Vertex {
|
||||
return n.Original
|
||||
}
|
||||
|
||||
func (n *graphNodeResourceDestroy) DestroyInclude(d *ModuleDiff, s *ModuleState) bool {
|
||||
// Always include anything other than the primary destroy
|
||||
if n.DestroyMode != DestroyPrimary {
|
||||
return true
|
||||
}
|
||||
|
||||
// Get the count, and specifically the raw value of the count
|
||||
// (with interpolations and all). If the count is NOT a static "1",
|
||||
// then we keep the destroy node no matter what.
|
||||
//
|
||||
// The reasoning for this is complicated and not intuitively obvious,
|
||||
// but I attempt to explain it below.
|
||||
//
|
||||
// The destroy transform works by generating the worst case graph,
|
||||
// with worst case being the case that every resource already exists
|
||||
// and needs to be destroy/created (force-new). There is a single important
|
||||
// edge case where this actually results in a real-life cycle: if a
|
||||
// create-before-destroy (CBD) resource depends on a non-CBD resource.
|
||||
// Imagine a EC2 instance "foo" with CBD depending on a security
|
||||
// group "bar" without CBD, and conceptualize the worst case destroy
|
||||
// order:
|
||||
//
|
||||
// 1.) SG must be destroyed (non-CBD)
|
||||
// 2.) SG must be created/updated
|
||||
// 3.) EC2 instance must be created (CBD, requires the SG be made)
|
||||
// 4.) EC2 instance must be destroyed (requires SG be destroyed)
|
||||
//
|
||||
// Except, #1 depends on #4, since the SG can't be destroyed while
|
||||
// an EC2 instance is using it (AWS API requirements). As you can see,
|
||||
// this is a real life cycle that can't be automatically reconciled
|
||||
// except under two conditions:
|
||||
//
|
||||
// 1.) SG is also CBD. This doesn't work 100% of the time though
|
||||
// since the non-CBD resource might not support CBD. To make matters
|
||||
// worse, the entire transitive closure of dependencies must be
|
||||
// CBD (if the SG depends on a VPC, you have the same problem).
|
||||
// 2.) EC2 must not CBD. This can't happen automatically because CBD
|
||||
// is used as a way to ensure zero (or minimal) downtime Terraform
|
||||
// applies, and it isn't acceptable for TF to ignore this request,
|
||||
// since it can result in unexpected downtime.
|
||||
//
|
||||
// Therefore, we compromise with this edge case here: if there is
|
||||
// a static count of "1", we prune the diff to remove cycles during a
|
||||
// graph optimization path if we don't see the resource in the diff.
|
||||
// If the count is set to ANYTHING other than a static "1" (variable,
|
||||
// computed attribute, static number greater than 1), then we keep the
|
||||
// destroy, since it is required for dynamic graph expansion to find
|
||||
// orphan/tainted count objects.
|
||||
//
|
||||
// This isn't ideal logic, but its strictly better without introducing
|
||||
// new impossibilities. It breaks the cycle in practical cases, and the
|
||||
// cycle comes back in no cases we've found to be practical, but just
|
||||
// as the cycle would already exist without this anyways.
|
||||
count := n.Original.Resource.RawCount
|
||||
if raw := count.Raw[count.Key]; raw != "1" {
|
||||
return true
|
||||
}
|
||||
|
||||
// Okay, we're dealing with a static count. There are a few ways
|
||||
// to include this resource.
|
||||
prefix := n.Original.Resource.Id()
|
||||
|
||||
// If we're present in the diff proper, then keep it.
|
||||
if d != nil {
|
||||
for k, _ := range d.Resources {
|
||||
if strings.HasPrefix(k, prefix) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If we're in the state as a primary in any form, then keep it.
|
||||
// This does a prefix check so it will also catch orphans on count
|
||||
// decreases to "1".
|
||||
if s != nil {
|
||||
for k, v := range s.Resources {
|
||||
if !strings.HasPrefix(k, prefix) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Ignore exact matches and the 0'th index. We only care
|
||||
// about if there is a decrease in count.
|
||||
if k == prefix {
|
||||
continue
|
||||
}
|
||||
if k == prefix+".0" {
|
||||
continue
|
||||
}
|
||||
|
||||
if v.Primary != nil {
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// graphNodeModuleExpanded represents a module where the graph has
|
||||
// been expanded. It stores the graph of the module as well as a reference
|
||||
// to the map of variables.
|
||||
type graphNodeModuleExpanded struct {
|
||||
Original dag.Vertex
|
||||
Graph *Graph
|
||||
InputConfig *config.RawConfig
|
||||
|
||||
// Variables is a map of the input variables. This reference should
|
||||
// be shared with ModuleInputTransformer in order to create a connection
|
||||
// where the variables are set properly.
|
||||
Variables map[string]string
|
||||
}
|
||||
|
||||
func (n *graphNodeModuleExpanded) Name() string {
|
||||
return fmt.Sprintf("%s (expanded)", dag.VertexName(n.Original))
|
||||
}
|
||||
|
||||
// GraphNodeDotter impl.
|
||||
func (n *graphNodeModuleExpanded) Dot(name string) string {
|
||||
return fmt.Sprintf(
|
||||
"\"%s\" [\n"+
|
||||
"\tlabel=\"%s\"\n"+
|
||||
"\tshape=component\n"+
|
||||
"];",
|
||||
name,
|
||||
dag.VertexName(n.Original))
|
||||
}
|
||||
|
||||
// GraphNodeEvalable impl.
|
||||
func (n *graphNodeModuleExpanded) EvalTree() EvalNode {
|
||||
var resourceConfig *ResourceConfig
|
||||
return &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalInterpolate{
|
||||
Config: n.InputConfig,
|
||||
Output: &resourceConfig,
|
||||
},
|
||||
|
||||
&EvalVariableBlock{
|
||||
Config: &resourceConfig,
|
||||
Variables: n.Variables,
|
||||
},
|
||||
|
||||
&EvalOpFilter{
|
||||
Ops: []walkOperation{walkPlanDestroy},
|
||||
Node: &EvalSequence{
|
||||
Nodes: []EvalNode{
|
||||
&EvalDiffDestroyModule{Path: n.Graph.Path},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// GraphNodeSubgraph impl.
|
||||
func (n *graphNodeModuleExpanded) Subgraph() *Graph {
|
||||
return n.Graph
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
func TestGraphNodeConfigModule_impl(t *testing.T) {
|
||||
var _ dag.Vertex = new(GraphNodeConfigModule)
|
||||
var _ dag.NamedVertex = new(GraphNodeConfigModule)
|
||||
var _ graphNodeConfig = new(GraphNodeConfigModule)
|
||||
var _ GraphNodeExpandable = new(GraphNodeConfigModule)
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigModuleExpand(t *testing.T) {
|
||||
mod := testModule(t, "graph-node-module-expand")
|
||||
|
||||
node := &GraphNodeConfigModule{
|
||||
Path: []string{RootModuleName, "child"},
|
||||
Module: &config.Module{},
|
||||
Tree: nil,
|
||||
}
|
||||
|
||||
g, err := node.Expand(&BasicGraphBuilder{
|
||||
Steps: []GraphTransformer{
|
||||
&ConfigTransformer{Module: mod},
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.Subgraph().String())
|
||||
expected := strings.TrimSpace(testGraphNodeModuleExpandStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigProvider_impl(t *testing.T) {
|
||||
var _ dag.Vertex = new(GraphNodeConfigProvider)
|
||||
var _ dag.NamedVertex = new(GraphNodeConfigProvider)
|
||||
var _ graphNodeConfig = new(GraphNodeConfigProvider)
|
||||
var _ GraphNodeProvider = new(GraphNodeConfigProvider)
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigProvider_ProviderName(t *testing.T) {
|
||||
n := &GraphNodeConfigProvider{
|
||||
Provider: &config.ProviderConfig{Name: "foo"},
|
||||
}
|
||||
|
||||
if v := n.ProviderName(); v != "foo" {
|
||||
t.Fatalf("bad: %#v", v)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigResource_impl(t *testing.T) {
|
||||
var _ dag.Vertex = new(GraphNodeConfigResource)
|
||||
var _ dag.NamedVertex = new(GraphNodeConfigResource)
|
||||
var _ graphNodeConfig = new(GraphNodeConfigResource)
|
||||
var _ GraphNodeProviderConsumer = new(GraphNodeConfigResource)
|
||||
var _ GraphNodeProvisionerConsumer = new(GraphNodeConfigResource)
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigResource_ProvidedBy(t *testing.T) {
|
||||
n := &GraphNodeConfigResource{
|
||||
Resource: &config.Resource{Type: "aws_instance"},
|
||||
}
|
||||
|
||||
if v := n.ProvidedBy(); v[0] != "aws" {
|
||||
t.Fatalf("bad: %#v", v)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGraphNodeConfigResource_ProvisionedBy(t *testing.T) {
|
||||
n := &GraphNodeConfigResource{
|
||||
Resource: &config.Resource{
|
||||
Type: "aws_instance",
|
||||
Provisioners: []*config.Provisioner{
|
||||
&config.Provisioner{Type: "foo"},
|
||||
&config.Provisioner{Type: "bar"},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
expected := []string{"foo", "bar"}
|
||||
actual := n.ProvisionedBy()
|
||||
if !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
}
|
||||
|
||||
const testGraphNodeModuleExpandStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.foo
|
||||
aws_instance.foo
|
||||
module inputs
|
||||
module inputs
|
||||
`
|
|
@ -6,371 +6,66 @@ import (
|
|||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/hashicorp/terraform/depgraph"
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// GraphDotOpts are options for turning a graph into dot format.
|
||||
type GraphDotOpts struct {
|
||||
// ModuleDepth is the depth of modules to expand. Zero is no expansion,
|
||||
// one expands the first set of modules, etc. If this is set to -1, then
|
||||
// all modules are expanded.
|
||||
ModuleDepth int
|
||||
|
||||
// Depth is an internal track of what depth we're at within
|
||||
// the graph, used to control indentation and other such things.
|
||||
depth int
|
||||
// GraphNodeDotter can be implemented by a node to cause it to be included
|
||||
// in the dot graph. The Dot method will be called which is expected to
|
||||
// return a representation of this node.
|
||||
type GraphNodeDotter interface {
|
||||
// Dot is called to return the dot formatting for the node.
|
||||
// The parameter must be the title of the node.
|
||||
Dot(string) string
|
||||
}
|
||||
|
||||
// GraphDotOpts are the options for generating a dot formatted Graph.
|
||||
type GraphDotOpts struct{}
|
||||
|
||||
// GraphDot returns the dot formatting of a visual representation of
|
||||
// the given Terraform graph.
|
||||
func GraphDot(g *depgraph.Graph, opts *GraphDotOpts) string {
|
||||
func GraphDot(g *Graph, opts *GraphDotOpts) string {
|
||||
buf := new(bytes.Buffer)
|
||||
|
||||
if opts.depth == 0 {
|
||||
// Start the graph
|
||||
buf.WriteString("digraph {\n")
|
||||
buf.WriteString("\tcompound = true;\n")
|
||||
|
||||
// Go through all the vertices and draw it
|
||||
vertices := g.Vertices()
|
||||
dotVertices := make(map[dag.Vertex]struct{}, len(vertices))
|
||||
for _, v := range vertices {
|
||||
if dn, ok := v.(GraphNodeDotter); !ok {
|
||||
continue
|
||||
} else if dn.Dot("fake") == "" {
|
||||
continue
|
||||
}
|
||||
|
||||
// Determine and add the title
|
||||
// graphDotTitle(buf, g)
|
||||
dotVertices[v] = struct{}{}
|
||||
}
|
||||
|
||||
// Add all the resource.
|
||||
graphDotAddResources(buf, g, opts)
|
||||
for v, _ := range dotVertices {
|
||||
dn := v.(GraphNodeDotter)
|
||||
scanner := bufio.NewScanner(strings.NewReader(
|
||||
dn.Dot(dag.VertexName(v))))
|
||||
for scanner.Scan() {
|
||||
buf.WriteString("\t" + scanner.Text() + "\n")
|
||||
}
|
||||
|
||||
// Add all the resource providers
|
||||
graphDotAddResourceProviders(buf, g, opts)
|
||||
// Draw all the edges
|
||||
for _, t := range g.DownEdges(v).List() {
|
||||
target := t.(dag.Vertex)
|
||||
if _, ok := dotVertices[target]; !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// Add all the modules
|
||||
graphDotAddModules(buf, g, opts)
|
||||
buf.WriteString(fmt.Sprintf(
|
||||
"\t\"%s\" -> \"%s\";\n",
|
||||
dag.VertexName(v),
|
||||
dag.VertexName(target)))
|
||||
}
|
||||
}
|
||||
|
||||
if opts.depth == 0 {
|
||||
// End the graph
|
||||
buf.WriteString("}\n")
|
||||
}
|
||||
|
||||
return buf.String()
|
||||
}
|
||||
|
||||
func graphDotAddRoot(buf *bytes.Buffer, n *depgraph.Noun) {
|
||||
buf.WriteString(fmt.Sprintf("\t\"%s\" [shape=circle];\n", "root"))
|
||||
|
||||
for _, e := range n.Edges() {
|
||||
target := e.Tail()
|
||||
buf.WriteString(fmt.Sprintf(
|
||||
"\t\"%s\" -> \"%s\";\n",
|
||||
"root",
|
||||
target))
|
||||
}
|
||||
}
|
||||
|
||||
func graphDotAddModules(buf *bytes.Buffer, g *depgraph.Graph, opts *GraphDotOpts) {
|
||||
for _, n := range g.Nouns {
|
||||
_, ok := n.Meta.(*GraphNodeModule)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
if graphExpand(opts) {
|
||||
// We're expanding
|
||||
graphDotAddModuleExpand(buf, n, opts)
|
||||
} else {
|
||||
// We're not expanding, so just add the module on its own
|
||||
graphDotAddModuleSingle(buf, n, opts)
|
||||
}
|
||||
|
||||
graphWriteEdges(buf, n, opts)
|
||||
}
|
||||
}
|
||||
|
||||
func graphDotAddModuleExpand(
|
||||
buf *bytes.Buffer, n *depgraph.Noun, opts *GraphDotOpts) {
|
||||
m := n.Meta.(*GraphNodeModule)
|
||||
tab := strings.Repeat("\t", opts.depth+1)
|
||||
uniqueName := graphUniqueName(n, opts)
|
||||
|
||||
// Wrap ourselves in a subgraph
|
||||
buf.WriteString(fmt.Sprintf("%ssubgraph \"cluster_%s\" {\n", tab, uniqueName))
|
||||
defer buf.WriteString(fmt.Sprintf("%s}\n", tab))
|
||||
|
||||
// Add our label so that we have the proper name.
|
||||
buf.WriteString(fmt.Sprintf("%s\tlabel = \"%s\";\n", tab, n))
|
||||
|
||||
// Add a hidden name for edges to point from/to
|
||||
buf.WriteString(fmt.Sprintf("%s\t\"%s_hidden\" [fixedsize=true,width=0,height=0,label=\"\",style=invisible];\n", tab, uniqueName))
|
||||
|
||||
// Graph the subgraph just as we would any other graph
|
||||
subOpts := *opts
|
||||
subOpts.depth++
|
||||
subStr := GraphDot(m.Graph, &subOpts)
|
||||
|
||||
// Tab all the lines of the subgraph
|
||||
s := bufio.NewScanner(strings.NewReader(subStr))
|
||||
for s.Scan() {
|
||||
buf.WriteString(fmt.Sprintf("%s%s\n", tab, s.Text()))
|
||||
}
|
||||
}
|
||||
|
||||
func graphDotAddModuleSingle(
|
||||
buf *bytes.Buffer, n *depgraph.Noun, opts *GraphDotOpts) {
|
||||
tab := strings.Repeat("\t", opts.depth+1)
|
||||
uniqueName := graphUniqueName(n, opts)
|
||||
|
||||
// Create this node.
|
||||
buf.WriteString(fmt.Sprintf("%s\"%s\" [\n", tab, uniqueName))
|
||||
buf.WriteString(fmt.Sprintf("%s\tlabel=\"%s\"\n", tab, n))
|
||||
buf.WriteString(fmt.Sprintf("%s\tshape=component\n", tab))
|
||||
buf.WriteString(fmt.Sprintf("%s];\n", tab))
|
||||
}
|
||||
|
||||
func graphDotAddResources(
|
||||
buf *bytes.Buffer, g *depgraph.Graph, opts *GraphDotOpts) {
|
||||
// Determine if we have diffs. If we do, then we're graphing a
|
||||
// plan, which alters our graph a bit.
|
||||
hasDiff := false
|
||||
for _, n := range g.Nouns {
|
||||
rn, ok := n.Meta.(*GraphNodeResource)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if rn.Resource.Diff != nil && !rn.Resource.Diff.Empty() {
|
||||
hasDiff = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
var edgeBuf bytes.Buffer
|
||||
// Do all the non-destroy resources
|
||||
buf.WriteString("\tsubgraph {\n")
|
||||
for _, n := range g.Nouns {
|
||||
rn, ok := n.Meta.(*GraphNodeResource)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if rn.Resource.Diff != nil && rn.Resource.Diff.Destroy {
|
||||
continue
|
||||
}
|
||||
|
||||
// If we have diffs then we're graphing a plan. If we don't have
|
||||
// have a diff on this resource, don't graph anything, since the
|
||||
// plan wouldn't do anything to this resource.
|
||||
if hasDiff {
|
||||
if rn.Resource.Diff == nil || rn.Resource.Diff.Empty() {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
// Determine the colors. White = no change, yellow = change,
|
||||
// green = create. Destroy is in the next section.
|
||||
var color, fillColor string
|
||||
if rn.Resource.Diff != nil && !rn.Resource.Diff.Empty() {
|
||||
if rn.Resource.State != nil && rn.Resource.State.ID != "" {
|
||||
color = "#FFFF00"
|
||||
fillColor = "#FFFF94"
|
||||
} else {
|
||||
color = "#00FF00"
|
||||
fillColor = "#9EFF9E"
|
||||
}
|
||||
}
|
||||
|
||||
uniqueName := fmt.Sprintf("%d_%s", opts.depth, n)
|
||||
|
||||
// Create this node.
|
||||
buf.WriteString(fmt.Sprintf("\t\t\"%s\" [\n", uniqueName))
|
||||
buf.WriteString(fmt.Sprintf("\t\t\tlabel=\"%s\"\n", n))
|
||||
buf.WriteString("\t\t\tshape=box\n")
|
||||
if color != "" {
|
||||
buf.WriteString("\t\t\tstyle=filled\n")
|
||||
buf.WriteString(fmt.Sprintf("\t\t\tcolor=\"%s\"\n", color))
|
||||
buf.WriteString(fmt.Sprintf("\t\t\tfillcolor=\"%s\"\n", fillColor))
|
||||
}
|
||||
buf.WriteString("\t\t];\n")
|
||||
|
||||
// Build up all the edges in a separate buffer so they're not in the
|
||||
// subgraph.
|
||||
graphWriteEdges(&edgeBuf, n, opts)
|
||||
}
|
||||
buf.WriteString("\t}\n\n")
|
||||
if edgeBuf.Len() > 0 {
|
||||
buf.WriteString(edgeBuf.String())
|
||||
buf.WriteString("\n")
|
||||
}
|
||||
|
||||
// Do all the destroy resources
|
||||
edgeBuf.Reset()
|
||||
buf.WriteString("\tsubgraph {\n")
|
||||
for _, n := range g.Nouns {
|
||||
rn, ok := n.Meta.(*GraphNodeResource)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if rn.Resource.Diff == nil || !rn.Resource.Diff.Destroy {
|
||||
continue
|
||||
}
|
||||
|
||||
uniqueName := fmt.Sprintf("%d_%s", opts.depth, n)
|
||||
|
||||
buf.WriteString(fmt.Sprintf(
|
||||
"\t\t\"%s\" [label=\"%s\",shape=box,style=filled,color=\"#FF0000\",fillcolor=\"#FF9494\"];\n", uniqueName, n))
|
||||
|
||||
graphWriteEdges(&edgeBuf, n, opts)
|
||||
}
|
||||
buf.WriteString("\t}\n\n")
|
||||
if edgeBuf.Len() > 0 {
|
||||
buf.WriteString(edgeBuf.String())
|
||||
buf.WriteString("\n")
|
||||
}
|
||||
|
||||
// Handle the meta resources
|
||||
/*
|
||||
edgeBuf.Reset()
|
||||
for _, n := range g.Nouns {
|
||||
_, ok := n.Meta.(*GraphNodeResourceMeta)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// Determine which edges to add
|
||||
var edges []digraph.Edge
|
||||
if hasDiff {
|
||||
for _, e := range n.Edges() {
|
||||
rn, ok := e.Tail().(*depgraph.Noun).Meta.(*GraphNodeResource)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if rn.Resource.Diff == nil || rn.Resource.Diff.Empty() {
|
||||
continue
|
||||
}
|
||||
edges = append(edges, e)
|
||||
}
|
||||
} else {
|
||||
edges = n.Edges()
|
||||
}
|
||||
|
||||
// Do not draw if we have no edges
|
||||
if len(edges) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
uniqueName := fmt.Sprintf("%d_%s", opts.depth, n)
|
||||
for _, e := range edges {
|
||||
target := e.Tail()
|
||||
uniqueTarget := fmt.Sprintf("%d_%s", opts.depth, target)
|
||||
edgeBuf.WriteString(fmt.Sprintf(
|
||||
"\t\"%s\" -> \"%s\";\n",
|
||||
uniqueName,
|
||||
uniqueTarget))
|
||||
}
|
||||
}
|
||||
if edgeBuf.Len() > 0 {
|
||||
buf.WriteString(edgeBuf.String())
|
||||
buf.WriteString("\n")
|
||||
}
|
||||
*/
|
||||
}
|
||||
|
||||
func graphDotAddResourceProviders(
|
||||
buf *bytes.Buffer, g *depgraph.Graph, opts *GraphDotOpts) {
|
||||
var edgeBuf bytes.Buffer
|
||||
buf.WriteString("\tsubgraph {\n")
|
||||
for _, n := range g.Nouns {
|
||||
_, ok := n.Meta.(*GraphNodeResourceProvider)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
uniqueName := fmt.Sprintf("%d_%s", opts.depth, n)
|
||||
|
||||
// Create this node.
|
||||
buf.WriteString(fmt.Sprintf("\t\t\"%s\" [\n", uniqueName))
|
||||
buf.WriteString(fmt.Sprintf("\t\t\tlabel=\"%s\"\n", n))
|
||||
buf.WriteString("\t\t\tshape=diamond\n")
|
||||
buf.WriteString("\t\t];\n")
|
||||
|
||||
// Build up all the edges in a separate buffer so they're not in the
|
||||
// subgraph.
|
||||
graphWriteEdges(&edgeBuf, n, opts)
|
||||
}
|
||||
buf.WriteString("\t}\n\n")
|
||||
if edgeBuf.Len() > 0 {
|
||||
buf.WriteString(edgeBuf.String())
|
||||
buf.WriteString("\n")
|
||||
}
|
||||
}
|
||||
|
||||
func graphDotTitle(buf *bytes.Buffer, g *depgraph.Graph) {
|
||||
// Determine if we have diffs. If we do, then we're graphing a
|
||||
// plan, which alters our graph a bit.
|
||||
hasDiff := false
|
||||
for _, n := range g.Nouns {
|
||||
rn, ok := n.Meta.(*GraphNodeResource)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
if rn.Resource.Diff != nil && !rn.Resource.Diff.Empty() {
|
||||
hasDiff = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
graphType := "Configuration"
|
||||
if hasDiff {
|
||||
graphType = "Plan"
|
||||
}
|
||||
title := fmt.Sprintf("Terraform %s Resource Graph", graphType)
|
||||
|
||||
buf.WriteString(fmt.Sprintf("\tlabel=\"%s\\n\\n\\n\";\n", title))
|
||||
buf.WriteString("\tlabelloc=\"t\";\n\n")
|
||||
}
|
||||
|
||||
func graphExpand(opts *GraphDotOpts) bool {
|
||||
return opts.ModuleDepth > opts.depth || opts.ModuleDepth == -1
|
||||
}
|
||||
|
||||
func graphUniqueName(n *depgraph.Noun, opts *GraphDotOpts) string {
|
||||
return fmt.Sprintf("%d_%s", opts.depth, n)
|
||||
}
|
||||
|
||||
func graphWriteEdges(
|
||||
buf *bytes.Buffer, n *depgraph.Noun, opts *GraphDotOpts) {
|
||||
tab := strings.Repeat("\t", opts.depth+1)
|
||||
|
||||
uniqueName := graphUniqueName(n, opts)
|
||||
var ltail string
|
||||
if _, ok := n.Meta.(*GraphNodeModule); ok && graphExpand(opts) {
|
||||
ltail = "cluster_" + uniqueName
|
||||
uniqueName = uniqueName + "_hidden"
|
||||
}
|
||||
|
||||
for _, e := range n.Edges() {
|
||||
target := e.Tail()
|
||||
targetN := target.(*depgraph.Noun)
|
||||
uniqueTarget := graphUniqueName(targetN, opts)
|
||||
|
||||
var lhead string
|
||||
if _, ok := targetN.Meta.(*GraphNodeModule); ok && graphExpand(opts) {
|
||||
lhead = "cluster_" + uniqueTarget
|
||||
uniqueTarget = uniqueTarget + "_hidden"
|
||||
}
|
||||
|
||||
var attrs string
|
||||
if lhead != "" || ltail != "" {
|
||||
var attrList []string
|
||||
if lhead != "" {
|
||||
attrList = append(attrList, fmt.Sprintf(
|
||||
"lhead=\"%s\"", lhead))
|
||||
}
|
||||
if ltail != "" {
|
||||
attrList = append(attrList, fmt.Sprintf(
|
||||
"ltail=\"%s\"", ltail))
|
||||
}
|
||||
|
||||
attrs = fmt.Sprintf(" [%s]", strings.Join(attrList, ","))
|
||||
}
|
||||
|
||||
buf.WriteString(fmt.Sprintf(
|
||||
"%s\"%s\" -> \"%s\"%s;\n",
|
||||
tab,
|
||||
uniqueName,
|
||||
uniqueTarget,
|
||||
attrs))
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,30 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// GraphWalker is an interface that can be implemented that when used
|
||||
// with Graph.Walk will invoke the given callbacks under certain events.
|
||||
type GraphWalker interface {
|
||||
EnterGraph(*Graph) EvalContext
|
||||
ExitGraph(*Graph)
|
||||
EnterVertex(dag.Vertex)
|
||||
ExitVertex(dag.Vertex, error)
|
||||
EnterEvalTree(dag.Vertex, EvalNode) EvalNode
|
||||
ExitEvalTree(dag.Vertex, interface{}, error) error
|
||||
}
|
||||
|
||||
// NullGraphWalker is a GraphWalker implementation that does nothing.
|
||||
// This can be embedded within other GraphWalker implementations for easily
|
||||
// implementing all the required functions.
|
||||
type NullGraphWalker struct{}
|
||||
|
||||
func (NullGraphWalker) EnterGraph(*Graph) EvalContext { return nil }
|
||||
func (NullGraphWalker) ExitGraph(*Graph) {}
|
||||
func (NullGraphWalker) EnterVertex(dag.Vertex) {}
|
||||
func (NullGraphWalker) ExitVertex(dag.Vertex, error) {}
|
||||
func (NullGraphWalker) EnterEvalTree(v dag.Vertex, n EvalNode) EvalNode { return n }
|
||||
func (NullGraphWalker) ExitEvalTree(dag.Vertex, interface{}, error) error {
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,134 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/errwrap"
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// ContextGraphWalker is the GraphWalker implementation used with the
|
||||
// Context struct to walk and evaluate the graph.
|
||||
type ContextGraphWalker struct {
|
||||
NullGraphWalker
|
||||
|
||||
// Configurable values
|
||||
Context *Context
|
||||
Operation walkOperation
|
||||
|
||||
// Outputs, do not set these. Do not read these while the graph
|
||||
// is being walked.
|
||||
ValidationWarnings []string
|
||||
ValidationErrors []error
|
||||
|
||||
errorLock sync.Mutex
|
||||
once sync.Once
|
||||
contexts map[string]*BuiltinEvalContext
|
||||
contextLock sync.Mutex
|
||||
providerCache map[string]ResourceProvider
|
||||
providerConfigCache map[string]*ResourceConfig
|
||||
providerLock sync.Mutex
|
||||
provisionerCache map[string]ResourceProvisioner
|
||||
provisionerLock sync.Mutex
|
||||
}
|
||||
|
||||
func (w *ContextGraphWalker) EnterGraph(g *Graph) EvalContext {
|
||||
w.once.Do(w.init)
|
||||
|
||||
w.contextLock.Lock()
|
||||
defer w.contextLock.Unlock()
|
||||
|
||||
// If we already have a context for this path cached, use that
|
||||
key := PathCacheKey(g.Path)
|
||||
if ctx, ok := w.contexts[key]; ok {
|
||||
return ctx
|
||||
}
|
||||
|
||||
// Variables should be our context variables, but these only apply
|
||||
// to the root module. As we enter subgraphs, we don't want to set
|
||||
// variables, which is set by the SetVariables EvalContext function.
|
||||
variables := w.Context.variables
|
||||
if len(g.Path) > 1 {
|
||||
// We're in a submodule, the variables should be empty
|
||||
variables = make(map[string]string)
|
||||
}
|
||||
|
||||
ctx := &BuiltinEvalContext{
|
||||
PathValue: g.Path,
|
||||
Hooks: w.Context.hooks,
|
||||
InputValue: w.Context.uiInput,
|
||||
Providers: w.Context.providers,
|
||||
ProviderCache: w.providerCache,
|
||||
ProviderConfigCache: w.providerConfigCache,
|
||||
ProviderInputConfig: w.Context.providerInputConfig,
|
||||
ProviderLock: &w.providerLock,
|
||||
Provisioners: w.Context.provisioners,
|
||||
ProvisionerCache: w.provisionerCache,
|
||||
ProvisionerLock: &w.provisionerLock,
|
||||
DiffValue: w.Context.diff,
|
||||
DiffLock: &w.Context.diffLock,
|
||||
StateValue: w.Context.state,
|
||||
StateLock: &w.Context.stateLock,
|
||||
Interpolater: &Interpolater{
|
||||
Operation: w.Operation,
|
||||
Module: w.Context.module,
|
||||
State: w.Context.state,
|
||||
StateLock: &w.Context.stateLock,
|
||||
Variables: variables,
|
||||
},
|
||||
}
|
||||
|
||||
w.contexts[key] = ctx
|
||||
return ctx
|
||||
}
|
||||
|
||||
func (w *ContextGraphWalker) EnterEvalTree(v dag.Vertex, n EvalNode) EvalNode {
|
||||
// Acquire a lock on the semaphore
|
||||
w.Context.parallelSem.Acquire()
|
||||
|
||||
// We want to filter the evaluation tree to only include operations
|
||||
// that belong in this operation.
|
||||
return EvalFilter(n, EvalNodeFilterOp(w.Operation))
|
||||
}
|
||||
|
||||
func (w *ContextGraphWalker) ExitEvalTree(
|
||||
v dag.Vertex, output interface{}, err error) error {
|
||||
// Release the semaphore
|
||||
w.Context.parallelSem.Release()
|
||||
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Acquire the lock because anything is going to require a lock.
|
||||
w.errorLock.Lock()
|
||||
defer w.errorLock.Unlock()
|
||||
|
||||
// Try to get a validation error out of it. If its not a validation
|
||||
// error, then just record the normal error.
|
||||
verr, ok := err.(*EvalValidateError)
|
||||
if !ok {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, msg := range verr.Warnings {
|
||||
w.ValidationWarnings = append(
|
||||
w.ValidationWarnings,
|
||||
fmt.Sprintf("%s: %s", dag.VertexName(v), msg))
|
||||
}
|
||||
for _, e := range verr.Errors {
|
||||
w.ValidationErrors = append(
|
||||
w.ValidationErrors,
|
||||
errwrap.Wrapf(fmt.Sprintf("%s: {{err}}", dag.VertexName(v)), e))
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *ContextGraphWalker) init() {
|
||||
w.contexts = make(map[string]*BuiltinEvalContext, 5)
|
||||
w.providerCache = make(map[string]ResourceProvider, 5)
|
||||
w.providerConfigCache = make(map[string]*ResourceConfig, 5)
|
||||
w.provisionerCache = make(map[string]ResourceProvisioner, 5)
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
package terraform
|
||||
|
||||
//go:generate stringer -type=walkOperation graph_walk_operation.go
|
||||
|
||||
// walkOperation is an enum which tells the walkContext what to do.
|
||||
type walkOperation byte
|
||||
|
||||
const (
|
||||
walkInvalid walkOperation = iota
|
||||
walkInput
|
||||
walkApply
|
||||
walkPlan
|
||||
walkPlanDestroy
|
||||
walkRefresh
|
||||
walkValidate
|
||||
)
|
|
@ -0,0 +1,9 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestNullGraphWalker_impl(t *testing.T) {
|
||||
var _ GraphWalker = NullGraphWalker{}
|
||||
}
|
|
@ -0,0 +1,440 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/config/lang/ast"
|
||||
"github.com/hashicorp/terraform/config/module"
|
||||
)
|
||||
|
||||
// Interpolater is the structure responsible for determining the values
|
||||
// for interpolations such as `aws_instance.foo.bar`.
|
||||
type Interpolater struct {
|
||||
Operation walkOperation
|
||||
Module *module.Tree
|
||||
State *State
|
||||
StateLock *sync.RWMutex
|
||||
Variables map[string]string
|
||||
}
|
||||
|
||||
// InterpolationScope is the current scope of execution. This is required
|
||||
// since some variables which are interpolated are dependent on what we're
|
||||
// operating on and where we are.
|
||||
type InterpolationScope struct {
|
||||
Path []string
|
||||
Resource *Resource
|
||||
}
|
||||
|
||||
// Values returns the values for all the variables in the given map.
|
||||
func (i *Interpolater) Values(
|
||||
scope *InterpolationScope,
|
||||
vars map[string]config.InterpolatedVariable) (map[string]ast.Variable, error) {
|
||||
result := make(map[string]ast.Variable, len(vars))
|
||||
|
||||
// Copy the default variables
|
||||
if i.Module != nil && scope != nil {
|
||||
mod := i.Module
|
||||
if len(scope.Path) > 1 {
|
||||
mod = i.Module.Child(scope.Path[1:])
|
||||
}
|
||||
for _, v := range mod.Config().Variables {
|
||||
for k, val := range v.DefaultsMap() {
|
||||
result[k] = ast.Variable{
|
||||
Value: val,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for n, rawV := range vars {
|
||||
var err error
|
||||
switch v := rawV.(type) {
|
||||
case *config.CountVariable:
|
||||
err = i.valueCountVar(scope, n, v, result)
|
||||
case *config.ModuleVariable:
|
||||
err = i.valueModuleVar(scope, n, v, result)
|
||||
case *config.PathVariable:
|
||||
err = i.valuePathVar(scope, n, v, result)
|
||||
case *config.ResourceVariable:
|
||||
err = i.valueResourceVar(scope, n, v, result)
|
||||
case *config.UserVariable:
|
||||
err = i.valueUserVar(scope, n, v, result)
|
||||
default:
|
||||
err = fmt.Errorf("%s: unknown variable type: %T", n, rawV)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func (i *Interpolater) valueCountVar(
|
||||
scope *InterpolationScope,
|
||||
n string,
|
||||
v *config.CountVariable,
|
||||
result map[string]ast.Variable) error {
|
||||
switch v.Type {
|
||||
case config.CountValueIndex:
|
||||
result[n] = ast.Variable{
|
||||
Value: scope.Resource.CountIndex,
|
||||
Type: ast.TypeInt,
|
||||
}
|
||||
return nil
|
||||
default:
|
||||
return fmt.Errorf("%s: unknown count type: %#v", n, v.Type)
|
||||
}
|
||||
}
|
||||
|
||||
func (i *Interpolater) valueModuleVar(
|
||||
scope *InterpolationScope,
|
||||
n string,
|
||||
v *config.ModuleVariable,
|
||||
result map[string]ast.Variable) error {
|
||||
// If we're computing all dynamic fields, then module vars count
|
||||
// and we mark it as computed.
|
||||
if i.Operation == walkValidate {
|
||||
result[n] = ast.Variable{
|
||||
Value: config.UnknownVariableValue,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Build the path to the child module we want
|
||||
path := make([]string, len(scope.Path), len(scope.Path)+1)
|
||||
copy(path, scope.Path)
|
||||
path = append(path, v.Name)
|
||||
|
||||
// Grab the lock so that if other interpolations are running or
|
||||
// state is being modified, we'll be safe.
|
||||
i.StateLock.RLock()
|
||||
defer i.StateLock.RUnlock()
|
||||
|
||||
// Get the module where we're looking for the value
|
||||
var value string
|
||||
mod := i.State.ModuleByPath(path)
|
||||
if mod == nil {
|
||||
// If the module doesn't exist, then we can return an empty string.
|
||||
// This happens usually only in Refresh() when we haven't populated
|
||||
// a state. During validation, we semantically verify that all
|
||||
// modules reference other modules, and graph ordering should
|
||||
// ensure that the module is in the state, so if we reach this
|
||||
// point otherwise it really is a panic.
|
||||
value = config.UnknownVariableValue
|
||||
} else {
|
||||
// Get the value from the outputs
|
||||
var ok bool
|
||||
value, ok = mod.Outputs[v.Field]
|
||||
if !ok {
|
||||
// Same reasons as the comment above.
|
||||
value = config.UnknownVariableValue
|
||||
}
|
||||
}
|
||||
|
||||
result[n] = ast.Variable{
|
||||
Value: value,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *Interpolater) valuePathVar(
|
||||
scope *InterpolationScope,
|
||||
n string,
|
||||
v *config.PathVariable,
|
||||
result map[string]ast.Variable) error {
|
||||
switch v.Type {
|
||||
case config.PathValueCwd:
|
||||
wd, err := os.Getwd()
|
||||
if err != nil {
|
||||
return fmt.Errorf(
|
||||
"Couldn't get cwd for var %s: %s",
|
||||
v.FullKey(), err)
|
||||
}
|
||||
|
||||
result[n] = ast.Variable{
|
||||
Value: wd,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
case config.PathValueModule:
|
||||
if t := i.Module.Child(scope.Path[1:]); t != nil {
|
||||
result[n] = ast.Variable{
|
||||
Value: t.Config().Dir,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
}
|
||||
case config.PathValueRoot:
|
||||
result[n] = ast.Variable{
|
||||
Value: i.Module.Config().Dir,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("%s: unknown path type: %#v", n, v.Type)
|
||||
}
|
||||
|
||||
return nil
|
||||
|
||||
}
|
||||
|
||||
func (i *Interpolater) valueResourceVar(
|
||||
scope *InterpolationScope,
|
||||
n string,
|
||||
v *config.ResourceVariable,
|
||||
result map[string]ast.Variable) error {
|
||||
// If we're computing all dynamic fields, then module vars count
|
||||
// and we mark it as computed.
|
||||
if i.Operation == walkValidate || i.Operation == walkRefresh {
|
||||
result[n] = ast.Variable{
|
||||
Value: config.UnknownVariableValue,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var attr string
|
||||
var err error
|
||||
if v.Multi && v.Index == -1 {
|
||||
attr, err = i.computeResourceMultiVariable(scope, v)
|
||||
} else {
|
||||
attr, err = i.computeResourceVariable(scope, v)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
result[n] = ast.Variable{
|
||||
Value: attr,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *Interpolater) valueUserVar(
|
||||
scope *InterpolationScope,
|
||||
n string,
|
||||
v *config.UserVariable,
|
||||
result map[string]ast.Variable) error {
|
||||
val, ok := i.Variables[v.Name]
|
||||
if ok {
|
||||
result[n] = ast.Variable{
|
||||
Value: val,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
if _, ok := result[n]; !ok && i.Operation == walkValidate {
|
||||
result[n] = ast.Variable{
|
||||
Value: config.UnknownVariableValue,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Look up if we have any variables with this prefix because
|
||||
// those are map overrides. Include those.
|
||||
for k, val := range i.Variables {
|
||||
if strings.HasPrefix(k, v.Name+".") {
|
||||
result["var."+k] = ast.Variable{
|
||||
Value: val,
|
||||
Type: ast.TypeString,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *Interpolater) computeResourceVariable(
|
||||
scope *InterpolationScope,
|
||||
v *config.ResourceVariable) (string, error) {
|
||||
id := v.ResourceId()
|
||||
if v.Multi {
|
||||
id = fmt.Sprintf("%s.%d", id, v.Index)
|
||||
}
|
||||
|
||||
i.StateLock.RLock()
|
||||
defer i.StateLock.RUnlock()
|
||||
|
||||
// Get the information about this resource variable, and verify
|
||||
// that it exists and such.
|
||||
module, _, err := i.resourceVariableInfo(scope, v)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
// If we have no module in the state yet or count, return empty
|
||||
if module == nil || len(module.Resources) == 0 {
|
||||
return "", nil
|
||||
}
|
||||
|
||||
// Get the resource out from the state. We know the state exists
|
||||
// at this point and if there is a state, we expect there to be a
|
||||
// resource with the given name.
|
||||
r, ok := module.Resources[id]
|
||||
if !ok && v.Multi && v.Index == 0 {
|
||||
r, ok = module.Resources[v.ResourceId()]
|
||||
}
|
||||
if !ok {
|
||||
r = nil
|
||||
}
|
||||
if r == nil {
|
||||
return "", fmt.Errorf(
|
||||
"Resource '%s' not found for variable '%s'",
|
||||
id,
|
||||
v.FullKey())
|
||||
}
|
||||
|
||||
if r.Primary == nil {
|
||||
goto MISSING
|
||||
}
|
||||
|
||||
if attr, ok := r.Primary.Attributes[v.Field]; ok {
|
||||
return attr, nil
|
||||
}
|
||||
|
||||
// At apply time, we can't do the "maybe has it" check below
|
||||
// that we need for plans since parent elements might be computed.
|
||||
// Therefore, it is an error and we're missing the key.
|
||||
//
|
||||
// TODO: test by creating a state and configuration that is referencing
|
||||
// a non-existent variable "foo.bar" where the state only has "foo"
|
||||
// and verify plan works, but apply doesn't.
|
||||
if i.Operation == walkApply {
|
||||
goto MISSING
|
||||
}
|
||||
|
||||
// We didn't find the exact field, so lets separate the dots
|
||||
// and see if anything along the way is a computed set. i.e. if
|
||||
// we have "foo.0.bar" as the field, check to see if "foo" is
|
||||
// a computed list. If so, then the whole thing is computed.
|
||||
if parts := strings.Split(v.Field, "."); len(parts) > 1 {
|
||||
for i := 1; i < len(parts); i++ {
|
||||
// Lists and sets make this
|
||||
key := fmt.Sprintf("%s.#", strings.Join(parts[:i], "."))
|
||||
if attr, ok := r.Primary.Attributes[key]; ok {
|
||||
return attr, nil
|
||||
}
|
||||
|
||||
// Maps make this
|
||||
key = fmt.Sprintf("%s", strings.Join(parts[:i], "."))
|
||||
if attr, ok := r.Primary.Attributes[key]; ok {
|
||||
return attr, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
MISSING:
|
||||
return "", fmt.Errorf(
|
||||
"Resource '%s' does not have attribute '%s' "+
|
||||
"for variable '%s'",
|
||||
id,
|
||||
v.Field,
|
||||
v.FullKey())
|
||||
}
|
||||
|
||||
func (i *Interpolater) computeResourceMultiVariable(
|
||||
scope *InterpolationScope,
|
||||
v *config.ResourceVariable) (string, error) {
|
||||
i.StateLock.RLock()
|
||||
defer i.StateLock.RUnlock()
|
||||
|
||||
// Get the information about this resource variable, and verify
|
||||
// that it exists and such.
|
||||
module, cr, err := i.resourceVariableInfo(scope, v)
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
|
||||
// Get the count so we know how many to iterate over
|
||||
count, err := cr.Count()
|
||||
if err != nil {
|
||||
return "", fmt.Errorf(
|
||||
"Error reading %s count: %s",
|
||||
v.ResourceId(),
|
||||
err)
|
||||
}
|
||||
|
||||
// If we have no module in the state yet or count, return empty
|
||||
if module == nil || len(module.Resources) == 0 || count == 0 {
|
||||
return "", nil
|
||||
}
|
||||
|
||||
var values []string
|
||||
for i := 0; i < count; i++ {
|
||||
id := fmt.Sprintf("%s.%d", v.ResourceId(), i)
|
||||
|
||||
// If we're dealing with only a single resource, then the
|
||||
// ID doesn't have a trailing index.
|
||||
if count == 1 {
|
||||
id = v.ResourceId()
|
||||
}
|
||||
|
||||
r, ok := module.Resources[id]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
if r.Primary == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
attr, ok := r.Primary.Attributes[v.Field]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
values = append(values, attr)
|
||||
}
|
||||
|
||||
if len(values) == 0 {
|
||||
return "", fmt.Errorf(
|
||||
"Resource '%s' does not have attribute '%s' "+
|
||||
"for variable '%s'",
|
||||
v.ResourceId(),
|
||||
v.Field,
|
||||
v.FullKey())
|
||||
}
|
||||
|
||||
return strings.Join(values, config.InterpSplitDelim), nil
|
||||
}
|
||||
|
||||
func (i *Interpolater) resourceVariableInfo(
|
||||
scope *InterpolationScope,
|
||||
v *config.ResourceVariable) (*ModuleState, *config.Resource, error) {
|
||||
// Get the module tree that contains our current path. This is
|
||||
// either the current module (path is empty) or a child.
|
||||
modTree := i.Module
|
||||
if len(scope.Path) > 1 {
|
||||
modTree = i.Module.Child(scope.Path[1:])
|
||||
}
|
||||
|
||||
// Get the resource from the configuration so we can verify
|
||||
// that the resource is in the configuration and so we can access
|
||||
// the configuration if we need to.
|
||||
var cr *config.Resource
|
||||
for _, r := range modTree.Config().Resources {
|
||||
if r.Id() == v.ResourceId() {
|
||||
cr = r
|
||||
break
|
||||
}
|
||||
}
|
||||
if cr == nil {
|
||||
return nil, nil, fmt.Errorf(
|
||||
"Resource '%s' not found for variable '%s'",
|
||||
v.ResourceId(),
|
||||
v.FullKey())
|
||||
}
|
||||
|
||||
// Get the relevant module
|
||||
module := i.State.ModuleByPath(scope.Path)
|
||||
return module, cr, nil
|
||||
}
|
|
@ -0,0 +1,135 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"os"
|
||||
"reflect"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/config/lang/ast"
|
||||
)
|
||||
|
||||
func TestInterpolater_countIndex(t *testing.T) {
|
||||
i := &Interpolater{}
|
||||
|
||||
scope := &InterpolationScope{
|
||||
Path: rootModulePath,
|
||||
Resource: &Resource{CountIndex: 42},
|
||||
}
|
||||
|
||||
testInterpolate(t, i, scope, "count.index", ast.Variable{
|
||||
Value: 42,
|
||||
Type: ast.TypeInt,
|
||||
})
|
||||
}
|
||||
|
||||
func TestInterpolater_moduleVariable(t *testing.T) {
|
||||
lock := new(sync.RWMutex)
|
||||
state := &State{
|
||||
Modules: []*ModuleState{
|
||||
&ModuleState{
|
||||
Path: rootModulePath,
|
||||
Resources: map[string]*ResourceState{
|
||||
"aws_instance.web": &ResourceState{
|
||||
Type: "aws_instance",
|
||||
Primary: &InstanceState{
|
||||
ID: "bar",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
&ModuleState{
|
||||
Path: []string{RootModuleName, "child"},
|
||||
Outputs: map[string]string{
|
||||
"foo": "bar",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
i := &Interpolater{
|
||||
State: state,
|
||||
StateLock: lock,
|
||||
}
|
||||
|
||||
scope := &InterpolationScope{
|
||||
Path: rootModulePath,
|
||||
}
|
||||
|
||||
testInterpolate(t, i, scope, "module.child.foo", ast.Variable{
|
||||
Value: "bar",
|
||||
Type: ast.TypeString,
|
||||
})
|
||||
}
|
||||
|
||||
func TestInterpolater_pathCwd(t *testing.T) {
|
||||
i := &Interpolater{}
|
||||
scope := &InterpolationScope{}
|
||||
|
||||
expected, err := os.Getwd()
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
testInterpolate(t, i, scope, "path.cwd", ast.Variable{
|
||||
Value: expected,
|
||||
Type: ast.TypeString,
|
||||
})
|
||||
}
|
||||
|
||||
func TestInterpolater_pathModule(t *testing.T) {
|
||||
mod := testModule(t, "interpolate-path-module")
|
||||
i := &Interpolater{
|
||||
Module: mod,
|
||||
}
|
||||
scope := &InterpolationScope{
|
||||
Path: []string{RootModuleName, "child"},
|
||||
}
|
||||
|
||||
path := mod.Child([]string{"child"}).Config().Dir
|
||||
testInterpolate(t, i, scope, "path.module", ast.Variable{
|
||||
Value: path,
|
||||
Type: ast.TypeString,
|
||||
})
|
||||
}
|
||||
|
||||
func TestInterpolater_pathRoot(t *testing.T) {
|
||||
mod := testModule(t, "interpolate-path-module")
|
||||
i := &Interpolater{
|
||||
Module: mod,
|
||||
}
|
||||
scope := &InterpolationScope{
|
||||
Path: []string{RootModuleName, "child"},
|
||||
}
|
||||
|
||||
path := mod.Config().Dir
|
||||
testInterpolate(t, i, scope, "path.root", ast.Variable{
|
||||
Value: path,
|
||||
Type: ast.TypeString,
|
||||
})
|
||||
}
|
||||
|
||||
func testInterpolate(
|
||||
t *testing.T, i *Interpolater,
|
||||
scope *InterpolationScope,
|
||||
n string, expectedVar ast.Variable) {
|
||||
v, err := config.NewInterpolatedVariable(n)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual, err := i.Values(scope, map[string]config.InterpolatedVariable{
|
||||
"foo": v,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
expected := map[string]ast.Variable{
|
||||
"foo": expectedVar,
|
||||
}
|
||||
if !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"crypto/md5"
|
||||
"encoding/hex"
|
||||
)
|
||||
|
||||
// PathCacheKey returns a cache key for a module path.
|
||||
//
|
||||
// TODO: test
|
||||
func PathCacheKey(path []string) string {
|
||||
// There is probably a better way to do this, but this is working for now.
|
||||
// We just create an MD5 hash of all the MD5 hashes of all the path
|
||||
// elements. This gets us the property that it is unique per ordering.
|
||||
hash := md5.New()
|
||||
for _, p := range path {
|
||||
single := md5.Sum([]byte(p))
|
||||
if _, err := hash.Write(single[:]); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
return hex.EncodeToString(hash.Sum(nil))
|
||||
}
|
|
@ -93,7 +93,7 @@ type ResourceConfig struct {
|
|||
// NewResourceConfig creates a new ResourceConfig from a config.RawConfig.
|
||||
func NewResourceConfig(c *config.RawConfig) *ResourceConfig {
|
||||
result := &ResourceConfig{raw: c}
|
||||
result.interpolate(nil, nil)
|
||||
result.interpolateForce()
|
||||
return result
|
||||
}
|
||||
|
||||
|
@ -201,28 +201,19 @@ func (c *ResourceConfig) get(
|
|||
return current, true
|
||||
}
|
||||
|
||||
func (c *ResourceConfig) interpolate(
|
||||
ctx *walkContext, r *Resource) error {
|
||||
if c == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if ctx != nil {
|
||||
if err := ctx.computeVars(c.raw, r); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// interpolateForce is a temporary thing. We want to get rid of interpolate
|
||||
// above and likewise this, but it can only be done after the f-ast-graph
|
||||
// refactor is complete.
|
||||
func (c *ResourceConfig) interpolateForce() {
|
||||
if c.raw == nil {
|
||||
var err error
|
||||
c.raw, err = config.NewRawConfig(make(map[string]interface{}))
|
||||
if err != nil {
|
||||
return err
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
c.ComputedKeys = c.raw.UnknownKeys()
|
||||
c.Raw = c.raw.Raw
|
||||
c.Config = c.raw.Config()
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -5,6 +5,7 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/config/lang/ast"
|
||||
)
|
||||
|
||||
func TestInstanceInfo(t *testing.T) {
|
||||
|
@ -99,20 +100,33 @@ func TestResourceConfigGet(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
rc := NewResourceConfig(rawC)
|
||||
if tc.Vars != nil {
|
||||
ctx := NewContext(&ContextOpts{Variables: tc.Vars})
|
||||
err := rc.interpolate(
|
||||
ctx.walkContext(walkInvalid, rootModulePath),
|
||||
nil)
|
||||
if err != nil {
|
||||
vs := make(map[string]ast.Variable)
|
||||
for k, v := range tc.Vars {
|
||||
vs["var."+k] = ast.Variable{Value: v, Type: ast.TypeString}
|
||||
}
|
||||
|
||||
if err := rawC.Interpolate(vs); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
rc := NewResourceConfig(rawC)
|
||||
rc.interpolateForce()
|
||||
|
||||
v, _ := rc.Get(tc.Key)
|
||||
if !reflect.DeepEqual(v, tc.Value) {
|
||||
t.Fatalf("%d bad: %#v", i, v)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testResourceConfig(
|
||||
t *testing.T, c map[string]interface{}) *ResourceConfig {
|
||||
raw, err := config.NewRawConfig(c)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
return NewResourceConfig(raw)
|
||||
}
|
||||
|
|
|
@ -3,9 +3,70 @@ package terraform
|
|||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// GraphSemanticChecker is the interface that semantic checks across
|
||||
// the entire Terraform graph implement.
|
||||
//
|
||||
// The graph should NOT be modified by the semantic checker.
|
||||
type GraphSemanticChecker interface {
|
||||
Check(*dag.Graph) error
|
||||
}
|
||||
|
||||
// UnorderedSemanticCheckRunner is an implementation of GraphSemanticChecker
|
||||
// that runs a list of SemanticCheckers against the vertices of the graph
|
||||
// in no specified order.
|
||||
type UnorderedSemanticCheckRunner struct {
|
||||
Checks []SemanticChecker
|
||||
}
|
||||
|
||||
func (sc *UnorderedSemanticCheckRunner) Check(g *dag.Graph) error {
|
||||
var err error
|
||||
for _, v := range g.Vertices() {
|
||||
for _, check := range sc.Checks {
|
||||
if e := check.Check(g, v); e != nil {
|
||||
err = multierror.Append(err, e)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// SemanticChecker is the interface that semantic checks across the
|
||||
// Terraform graph implement. Errors are accumulated. Even after an error
|
||||
// is returned, child vertices in the graph will still be visited.
|
||||
//
|
||||
// The graph should NOT be modified by the semantic checker.
|
||||
//
|
||||
// The order in which vertices are visited is left unspecified, so the
|
||||
// semantic checks should not rely on that.
|
||||
type SemanticChecker interface {
|
||||
Check(*dag.Graph, dag.Vertex) error
|
||||
}
|
||||
|
||||
// SemanticCheckModulesExist is an implementation of SemanticChecker that
|
||||
// verifies that all the modules that are referenced in the graph exist.
|
||||
type SemanticCheckModulesExist struct{}
|
||||
|
||||
// TODO: test
|
||||
func (*SemanticCheckModulesExist) Check(g *dag.Graph, v dag.Vertex) error {
|
||||
mn, ok := v.(*GraphNodeConfigModule)
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
if mn.Tree == nil {
|
||||
return fmt.Errorf(
|
||||
"module '%s' not found", mn.Module.Name)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// smcUserVariables does all the semantic checks to verify that the
|
||||
// variables given satisfy the configuration itself.
|
||||
func smcUserVariables(c *config.Config, vs map[string]string) []error {
|
||||
|
|
|
@ -100,6 +100,31 @@ func (s *State) ModuleByPath(path []string) *ModuleState {
|
|||
return nil
|
||||
}
|
||||
|
||||
// ModuleOrphans returns all the module orphans in this state by
|
||||
// returning their full paths. These paths can be used with ModuleByPath
|
||||
// to return the actual state.
|
||||
func (s *State) ModuleOrphans(path []string, c *config.Config) [][]string {
|
||||
childrenKeys := make(map[string]struct{})
|
||||
if c != nil {
|
||||
for _, m := range c.Modules {
|
||||
childrenKeys[m.Name] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
// Go over the direct children and find any that aren't in our
|
||||
// keys.
|
||||
var orphans [][]string
|
||||
for _, m := range s.Children(path) {
|
||||
if _, ok := childrenKeys[m.Path[len(m.Path)-1]]; ok {
|
||||
continue
|
||||
}
|
||||
|
||||
orphans = append(orphans, m.Path)
|
||||
}
|
||||
|
||||
return orphans
|
||||
}
|
||||
|
||||
// RootModule returns the ModuleState for the root module
|
||||
func (s *State) RootModule() *ModuleState {
|
||||
root := s.ModuleByPath(rootModulePath)
|
||||
|
@ -278,6 +303,7 @@ func (m *ModuleState) Orphans(c *config.Config) []string {
|
|||
keys[k] = struct{}{}
|
||||
}
|
||||
|
||||
if c != nil {
|
||||
for _, r := range c.Resources {
|
||||
delete(keys, r.Id())
|
||||
|
||||
|
@ -287,6 +313,7 @@ func (m *ModuleState) Orphans(c *config.Config) []string {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
result := make([]string, 0, len(keys))
|
||||
for k, _ := range keys {
|
||||
|
@ -346,10 +373,17 @@ func (m *ModuleState) deepcopy() *ModuleState {
|
|||
func (m *ModuleState) prune() {
|
||||
for k, v := range m.Resources {
|
||||
v.prune()
|
||||
|
||||
if (v.Primary == nil || v.Primary.ID == "") && len(v.Tainted) == 0 {
|
||||
delete(m.Resources, k)
|
||||
}
|
||||
}
|
||||
|
||||
for k, v := range m.Outputs {
|
||||
if v == config.UnknownVariableValue {
|
||||
delete(m.Outputs, k)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (m *ModuleState) sort() {
|
||||
|
@ -517,12 +551,14 @@ func (r *ResourceState) prune() {
|
|||
n := len(r.Tainted)
|
||||
for i := 0; i < n; i++ {
|
||||
inst := r.Tainted[i]
|
||||
if inst.ID == "" {
|
||||
if inst == nil || inst.ID == "" {
|
||||
copy(r.Tainted[i:], r.Tainted[i+1:])
|
||||
r.Tainted[n-1] = nil
|
||||
n--
|
||||
i--
|
||||
}
|
||||
}
|
||||
|
||||
r.Tainted = r.Tainted[:n]
|
||||
}
|
||||
|
||||
|
|
|
@ -59,6 +59,58 @@ func TestStateAddModule(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestStateModuleOrphans(t *testing.T) {
|
||||
state := &State{
|
||||
Modules: []*ModuleState{
|
||||
&ModuleState{
|
||||
Path: RootModulePath,
|
||||
},
|
||||
&ModuleState{
|
||||
Path: []string{RootModuleName, "foo"},
|
||||
},
|
||||
&ModuleState{
|
||||
Path: []string{RootModuleName, "bar"},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
config := testModule(t, "state-module-orphans").Config()
|
||||
actual := state.ModuleOrphans(RootModulePath, config)
|
||||
expected := [][]string{
|
||||
[]string{RootModuleName, "foo"},
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestStateModuleOrphans_nilConfig(t *testing.T) {
|
||||
state := &State{
|
||||
Modules: []*ModuleState{
|
||||
&ModuleState{
|
||||
Path: RootModulePath,
|
||||
},
|
||||
&ModuleState{
|
||||
Path: []string{RootModuleName, "foo"},
|
||||
},
|
||||
&ModuleState{
|
||||
Path: []string{RootModuleName, "bar"},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
actual := state.ModuleOrphans(RootModulePath, nil)
|
||||
expected := [][]string{
|
||||
[]string{RootModuleName, "foo"},
|
||||
[]string{RootModuleName, "bar"},
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(actual, expected) {
|
||||
t.Fatalf("bad: %#v", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestInstanceState_MergeDiff(t *testing.T) {
|
||||
is := InstanceState{
|
||||
ID: "foo",
|
||||
|
|
|
@ -200,6 +200,13 @@ aws_instance.bar:
|
|||
type = aws_instance
|
||||
`
|
||||
|
||||
const testTerraformApplyCreateBeforeUpdateStr = `
|
||||
aws_instance.bar:
|
||||
ID = foo
|
||||
foo = baz
|
||||
type = aws_instance
|
||||
`
|
||||
|
||||
const testTerraformApplyCancelStr = `
|
||||
aws_instance.foo:
|
||||
ID = foo
|
||||
|
@ -233,7 +240,7 @@ aws_instance.foo.1:
|
|||
`
|
||||
|
||||
const testTerraformApplyCountDecToOneStr = `
|
||||
aws_instance.foo.0:
|
||||
aws_instance.foo:
|
||||
ID = bar
|
||||
foo = foo
|
||||
type = aws_instance
|
||||
|
@ -278,6 +285,17 @@ module.child:
|
|||
type = aws_instance
|
||||
`
|
||||
|
||||
const testTerraformApplyMultiProviderStr = `
|
||||
aws_instance.bar:
|
||||
ID = foo
|
||||
foo = bar
|
||||
type = aws_instance
|
||||
do_instance.foo:
|
||||
ID = foo
|
||||
num = 2
|
||||
type = do_instance
|
||||
`
|
||||
|
||||
const testTerraformApplyProvisionerStr = `
|
||||
aws_instance.bar:
|
||||
ID = foo
|
||||
|
|
|
@ -0,0 +1,7 @@
|
|||
resource "aws_instance" "bar" {
|
||||
foo = "baz"
|
||||
|
||||
lifecycle {
|
||||
create_before_destroy = true
|
||||
}
|
||||
}
|
|
@ -0,0 +1,7 @@
|
|||
resource "do_instance" "foo" {
|
||||
num = "2"
|
||||
}
|
||||
|
||||
resource "aws_instance" "bar" {
|
||||
foo = "bar"
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
provider "aws" {}
|
||||
resource "aws_instance" "db" {}
|
||||
resource "aws_instance" "web" {
|
||||
foo = "${aws_instance.db.id}"
|
||||
}
|
|
@ -0,0 +1,9 @@
|
|||
provider "aws" {}
|
||||
|
||||
resource "aws_lc" "foo" {}
|
||||
|
||||
resource "aws_asg" "foo" {
|
||||
lc = "${aws_lc.foo.id}"
|
||||
|
||||
lifecycle { create_before_destroy = true }
|
||||
}
|
|
@ -0,0 +1,2 @@
|
|||
provider "aws" {}
|
||||
resource "aws_instance" "server" {}
|
|
@ -0,0 +1,16 @@
|
|||
module "consul" {
|
||||
foo = "${aws_security_group.firewall.foo}"
|
||||
source = "./consul"
|
||||
}
|
||||
|
||||
provider "aws" {}
|
||||
|
||||
resource "aws_security_group" "firewall" {}
|
||||
|
||||
resource "aws_instance" "web" {
|
||||
security_groups = [
|
||||
"foo",
|
||||
"${aws_security_group.firewall.foo}",
|
||||
"${module.consul.security_group}"
|
||||
]
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
resource "aws_instance" "db" {}
|
||||
|
||||
resource "aws_instance" "web" {
|
||||
foo = "${aws_instance.lb.id}"
|
||||
}
|
|
@ -0,0 +1,4 @@
|
|||
resource "aws_instance" "foo" {}
|
||||
resource "aws_instance" "bar" {
|
||||
var = "${aws_instance.foo.whatever}"
|
||||
}
|
|
@ -0,0 +1,3 @@
|
|||
module "child" {
|
||||
source = "./child"
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
resource "aws_instance" "foo" {}
|
||||
|
||||
output "foo" {
|
||||
value = "${aws_instance.foo.value}"
|
||||
}
|
|
@ -0,0 +1,3 @@
|
|||
module "child" {
|
||||
source = "./child"
|
||||
}
|
|
@ -0,0 +1 @@
|
|||
# Nothing
|
|
@ -0,0 +1,3 @@
|
|||
module "bar" {
|
||||
source = "./bar"
|
||||
}
|
|
@ -0,0 +1,9 @@
|
|||
resource "aws_instance" "web" {
|
||||
lifecycle {
|
||||
create_before_destroy = true
|
||||
}
|
||||
}
|
||||
|
||||
resource "aws_load_balancer" "lb" {
|
||||
member = "${aws_instance.web.id}"
|
||||
}
|
|
@ -0,0 +1,9 @@
|
|||
resource "aws_lc" "foo" {
|
||||
lifecycle { create_before_destroy = true }
|
||||
}
|
||||
|
||||
resource "aws_autoscale" "bar" {
|
||||
lc = "${aws_lc.foo.id}"
|
||||
|
||||
lifecycle { create_before_destroy = true }
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
resource "aws_instance" "foo" {}
|
||||
|
||||
resource "aws_instance" "bar" {
|
||||
value = "${aws_instance.foo.value}"
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
resource "aws_lc" "foo" {}
|
||||
|
||||
resource "aws_asg" "bar" {
|
||||
lc = "${aws_lc.foo.id}"
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
resource "aws_instance" "foo" {}
|
||||
|
||||
resource "aws_instance" "bar" {
|
||||
value = "${aws_instance.foo.value}"
|
||||
count = "5"
|
||||
}
|
|
@ -0,0 +1 @@
|
|||
resource "aws_instance" "web" {}
|
|
@ -0,0 +1 @@
|
|||
resource "aws_instance" "foo" {}
|
|
@ -0,0 +1,2 @@
|
|||
provider "aws" {}
|
||||
resource "aws_instance" "web" {}
|
|
@ -0,0 +1,2 @@
|
|||
provider "aws" {}
|
||||
resource "foo_instance" "web" {}
|
|
@ -0,0 +1,3 @@
|
|||
resource "aws_instance" "web" {
|
||||
provisioner "shell" {}
|
||||
}
|
|
@ -0,0 +1,3 @@
|
|||
resource "aws_instance" "web" {
|
||||
provisioner "foo" {}
|
||||
}
|
|
@ -0,0 +1,4 @@
|
|||
resource "aws_instance" "foo" {
|
||||
count = 3
|
||||
value = "${aws_instance.foo.0.value}"
|
||||
}
|
|
@ -0,0 +1,4 @@
|
|||
resource "aws_instance" "foo" {
|
||||
count = -5
|
||||
value = "${aws_instance.foo.0.value}"
|
||||
}
|
|
@ -0,0 +1,5 @@
|
|||
provider "aws" {}
|
||||
resource "aws_instance" "foo" {}
|
||||
|
||||
provider "do" {}
|
||||
resource "do_droplet" "bar" {}
|
|
@ -0,0 +1 @@
|
|||
resource "aws_instance" "web" {}
|
|
@ -3,5 +3,7 @@ provider "aws" {
|
|||
}
|
||||
|
||||
resource "aws_instance" "test" {
|
||||
provisioner "shell" {}
|
||||
provisioner "shell" {
|
||||
command = "foo"
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,21 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// GraphTransformer is the interface that transformers implement. This
|
||||
// interface is only for transforms that need entire graph visibility.
|
||||
type GraphTransformer interface {
|
||||
Transform(*Graph) error
|
||||
}
|
||||
|
||||
// GraphVertexTransformer is an interface that transforms a single
|
||||
// Vertex within with graph. This is a specialization of GraphTransformer
|
||||
// that makes it easy to do vertex replacement.
|
||||
//
|
||||
// The GraphTransformer that runs through the GraphVertexTransformers is
|
||||
// VertexTransformer.
|
||||
type GraphVertexTransformer interface {
|
||||
Transform(dag.Vertex) (dag.Vertex, error)
|
||||
}
|
|
@ -0,0 +1,105 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/go-multierror"
|
||||
"github.com/hashicorp/terraform/config"
|
||||
"github.com/hashicorp/terraform/config/module"
|
||||
)
|
||||
|
||||
// ConfigTransformer is a GraphTransformer that adds the configuration
|
||||
// to the graph. The module used to configure this transformer must be
|
||||
// the root module. We'll look up the child module by the Path in the
|
||||
// Graph.
|
||||
type ConfigTransformer struct {
|
||||
Module *module.Tree
|
||||
}
|
||||
|
||||
func (t *ConfigTransformer) Transform(g *Graph) error {
|
||||
// A module is required and also must be completely loaded.
|
||||
if t.Module == nil {
|
||||
return errors.New("module must not be nil")
|
||||
}
|
||||
if !t.Module.Loaded() {
|
||||
return errors.New("module must be loaded")
|
||||
}
|
||||
|
||||
// Get the module we care about
|
||||
module := t.Module.Child(g.Path[1:])
|
||||
if module == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Get the configuration for this module
|
||||
config := module.Config()
|
||||
|
||||
// Create the node list we'll use for the graph
|
||||
nodes := make([]graphNodeConfig, 0,
|
||||
(len(config.ProviderConfigs)+len(config.Modules)+len(config.Resources))*2)
|
||||
|
||||
// Write all the provider configs out
|
||||
for _, pc := range config.ProviderConfigs {
|
||||
nodes = append(nodes, &GraphNodeConfigProvider{Provider: pc})
|
||||
}
|
||||
|
||||
// Write all the resources out
|
||||
for _, r := range config.Resources {
|
||||
nodes = append(nodes, &GraphNodeConfigResource{Resource: r})
|
||||
}
|
||||
|
||||
// Write all the modules out
|
||||
children := module.Children()
|
||||
for _, m := range config.Modules {
|
||||
path := make([]string, len(g.Path), len(g.Path)+1)
|
||||
copy(path, g.Path)
|
||||
path = append(path, m.Name)
|
||||
|
||||
nodes = append(nodes, &GraphNodeConfigModule{
|
||||
Path: path,
|
||||
Module: m,
|
||||
Tree: children[m.Name],
|
||||
})
|
||||
}
|
||||
|
||||
// Write all the outputs out
|
||||
for _, o := range config.Outputs {
|
||||
nodes = append(nodes, &GraphNodeConfigOutput{Output: o})
|
||||
}
|
||||
|
||||
// Err is where the final error value will go if there is one
|
||||
var err error
|
||||
|
||||
// Build the graph vertices
|
||||
for _, n := range nodes {
|
||||
g.Add(n)
|
||||
}
|
||||
|
||||
// Build up the dependencies. We have to do this outside of the above
|
||||
// loop since the nodes need to be in place for us to build the deps.
|
||||
for _, n := range nodes {
|
||||
if missing := g.ConnectDependent(n); len(missing) > 0 {
|
||||
for _, m := range missing {
|
||||
err = multierror.Append(err, fmt.Errorf(
|
||||
"%s: missing dependency: %s", n.Name(), m))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// varNameForVar returns the VarName value for an interpolated variable.
|
||||
// This value is compared to the VarName() value for the nodes within the
|
||||
// graph to build the graph edges.
|
||||
func varNameForVar(raw config.InterpolatedVariable) string {
|
||||
switch v := raw.(type) {
|
||||
case *config.ModuleVariable:
|
||||
return fmt.Sprintf("module.%s", v.Name)
|
||||
case *config.ResourceVariable:
|
||||
return v.ResourceId()
|
||||
default:
|
||||
return ""
|
||||
}
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/config/module"
|
||||
)
|
||||
|
||||
func TestConfigTransformer_nilModule(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{}
|
||||
if err := tf.Transform(&g); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer_unloadedModule(t *testing.T) {
|
||||
mod, err := module.NewTreeModule(
|
||||
"", filepath.Join(fixtureDir, "graph-basic"))
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err == nil {
|
||||
t.Fatal("should error")
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: testModule(t, "graph-basic")}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer_dependsOn(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: testModule(t, "graph-depends-on")}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphDependsOnStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer_modules(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: testModule(t, "graph-modules")}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphModulesStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer_outputs(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: testModule(t, "graph-outputs")}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testGraphOutputsStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestConfigTransformer_errMissingDeps(t *testing.T) {
|
||||
g := Graph{Path: RootModulePath}
|
||||
tf := &ConfigTransformer{Module: testModule(t, "graph-missing-deps")}
|
||||
if err := tf.Transform(&g); err == nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
const testGraphBasicStr = `
|
||||
aws_instance.web
|
||||
aws_security_group.firewall
|
||||
aws_load_balancer.weblb
|
||||
aws_instance.web
|
||||
aws_security_group.firewall
|
||||
openstack_floating_ip.random
|
||||
provider.aws
|
||||
openstack_floating_ip.random
|
||||
`
|
||||
|
||||
const testGraphDependsOnStr = `
|
||||
aws_instance.db
|
||||
aws_instance.web
|
||||
aws_instance.web
|
||||
`
|
||||
|
||||
const testGraphModulesStr = `
|
||||
aws_instance.web
|
||||
aws_security_group.firewall
|
||||
module.consul
|
||||
aws_security_group.firewall
|
||||
module.consul
|
||||
aws_security_group.firewall
|
||||
provider.aws
|
||||
`
|
||||
|
||||
const testGraphOutputsStr = `
|
||||
aws_instance.foo
|
||||
output.foo
|
||||
aws_instance.foo
|
||||
`
|
|
@ -0,0 +1,230 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
type GraphNodeDestroyMode byte
|
||||
|
||||
const (
|
||||
DestroyNone GraphNodeDestroyMode = 0
|
||||
DestroyPrimary GraphNodeDestroyMode = 1 << iota
|
||||
DestroyTainted
|
||||
)
|
||||
|
||||
// GraphNodeDestroyable is the interface that nodes that can be destroyed
|
||||
// must implement. This is used to automatically handle the creation of
|
||||
// destroy nodes in the graph and the dependency ordering of those destroys.
|
||||
type GraphNodeDestroyable interface {
|
||||
// DestroyNode returns the node used for the destroy with the given
|
||||
// mode. If this returns nil, then a destroy node for that mode
|
||||
// will not be added.
|
||||
DestroyNode(GraphNodeDestroyMode) GraphNodeDestroy
|
||||
}
|
||||
|
||||
// GraphNodeDestroy is the interface that must implemented by
|
||||
// nodes that destroy.
|
||||
type GraphNodeDestroy interface {
|
||||
dag.Vertex
|
||||
|
||||
// CreateBeforeDestroy is called to check whether this node
|
||||
// should be created before it is destroyed. The CreateBeforeDestroy
|
||||
// transformer uses this information to setup the graph.
|
||||
CreateBeforeDestroy() bool
|
||||
|
||||
// CreateNode returns the node used for the create side of this
|
||||
// destroy. This must already exist within the graph.
|
||||
CreateNode() dag.Vertex
|
||||
}
|
||||
|
||||
// GraphNodeDestroyPrunable is the interface that can be implemented to
|
||||
// signal that this node can be pruned depending on state.
|
||||
type GraphNodeDestroyPrunable interface {
|
||||
// DestroyInclude is called to check if this node should be included
|
||||
// with the given state. The state and diff must NOT be modified.
|
||||
DestroyInclude(*ModuleDiff, *ModuleState) bool
|
||||
}
|
||||
|
||||
// DestroyTransformer is a GraphTransformer that creates the destruction
|
||||
// nodes for things that _might_ be destroyed.
|
||||
type DestroyTransformer struct{}
|
||||
|
||||
func (t *DestroyTransformer) Transform(g *Graph) error {
|
||||
var connect, remove []dag.Edge
|
||||
|
||||
modes := []GraphNodeDestroyMode{DestroyPrimary, DestroyTainted}
|
||||
for _, m := range modes {
|
||||
connectMode, removeMode, err := t.transform(g, m)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
connect = append(connect, connectMode...)
|
||||
remove = append(remove, removeMode...)
|
||||
}
|
||||
|
||||
// Atomatically add/remove the edges
|
||||
for _, e := range connect {
|
||||
g.Connect(e)
|
||||
}
|
||||
for _, e := range remove {
|
||||
g.RemoveEdge(e)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *DestroyTransformer) transform(
|
||||
g *Graph, mode GraphNodeDestroyMode) ([]dag.Edge, []dag.Edge, error) {
|
||||
var connect, remove []dag.Edge
|
||||
nodeToCn := make(map[dag.Vertex]dag.Vertex, len(g.Vertices()))
|
||||
nodeToDn := make(map[dag.Vertex]dag.Vertex, len(g.Vertices()))
|
||||
for _, v := range g.Vertices() {
|
||||
// If it is not a destroyable, we don't care
|
||||
cn, ok := v.(GraphNodeDestroyable)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// Grab the destroy side of the node and connect it through
|
||||
n := cn.DestroyNode(mode)
|
||||
if n == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Store it
|
||||
nodeToCn[n] = cn
|
||||
nodeToDn[cn] = n
|
||||
|
||||
// Add it to the graph
|
||||
g.Add(n)
|
||||
|
||||
// Inherit all the edges from the old node
|
||||
downEdges := g.DownEdges(v).List()
|
||||
for _, edgeRaw := range downEdges {
|
||||
g.Connect(dag.BasicEdge(n, edgeRaw.(dag.Vertex)))
|
||||
}
|
||||
|
||||
// Add a new edge to connect the node to be created to
|
||||
// the destroy node.
|
||||
connect = append(connect, dag.BasicEdge(v, n))
|
||||
}
|
||||
|
||||
// Go through the nodes we added and determine if they depend
|
||||
// on any nodes with a destroy node. If so, depend on that instead.
|
||||
for n, _ := range nodeToCn {
|
||||
for _, downRaw := range g.DownEdges(n).List() {
|
||||
target := downRaw.(dag.Vertex)
|
||||
cn2, ok := target.(GraphNodeDestroyable)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
newTarget := nodeToDn[cn2]
|
||||
if newTarget == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Make the new edge and transpose
|
||||
connect = append(connect, dag.BasicEdge(newTarget, n))
|
||||
|
||||
// Remove the old edge
|
||||
remove = append(remove, dag.BasicEdge(n, target))
|
||||
}
|
||||
}
|
||||
|
||||
return connect, remove, nil
|
||||
}
|
||||
|
||||
// CreateBeforeDestroyTransformer is a GraphTransformer that modifies
|
||||
// the destroys of some nodes so that the creation happens before the
|
||||
// destroy.
|
||||
type CreateBeforeDestroyTransformer struct{}
|
||||
|
||||
func (t *CreateBeforeDestroyTransformer) Transform(g *Graph) error {
|
||||
// We "stage" the edge connections/destroys in these slices so that
|
||||
// while we're doing the edge transformations (transpositions) in
|
||||
// the graph, we're not affecting future edge transpositions. These
|
||||
// slices let us stage ALL the changes that WILL happen so that all
|
||||
// of the transformations happen atomically.
|
||||
var connect, destroy []dag.Edge
|
||||
|
||||
for _, v := range g.Vertices() {
|
||||
// We only care to use the destroy nodes
|
||||
dn, ok := v.(GraphNodeDestroy)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// If the node doesn't need to create before destroy, then continue
|
||||
if !dn.CreateBeforeDestroy() {
|
||||
continue
|
||||
}
|
||||
|
||||
// Get the creation side of this node
|
||||
cn := dn.CreateNode()
|
||||
|
||||
// Take all the things which depend on the creation node and
|
||||
// make them dependencies on the destruction. Clarifying this
|
||||
// with an example: if you have a web server and a load balancer
|
||||
// and the load balancer depends on the web server, then when we
|
||||
// do a create before destroy, we want to make sure the steps are:
|
||||
//
|
||||
// 1.) Create new web server
|
||||
// 2.) Update load balancer
|
||||
// 3.) Delete old web server
|
||||
//
|
||||
// This ensures that.
|
||||
for _, sourceRaw := range g.UpEdges(cn).List() {
|
||||
source := sourceRaw.(dag.Vertex)
|
||||
connect = append(connect, dag.BasicEdge(dn, source))
|
||||
}
|
||||
|
||||
// Swap the edge so that the destroy depends on the creation
|
||||
// happening...
|
||||
connect = append(connect, dag.BasicEdge(dn, cn))
|
||||
destroy = append(destroy, dag.BasicEdge(cn, dn))
|
||||
}
|
||||
|
||||
for _, edge := range connect {
|
||||
g.Connect(edge)
|
||||
}
|
||||
for _, edge := range destroy {
|
||||
g.RemoveEdge(edge)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// PruneDestroyTransformer is a GraphTransformer that removes the destroy
|
||||
// nodes that aren't in the diff.
|
||||
type PruneDestroyTransformer struct {
|
||||
Diff *Diff
|
||||
State *State
|
||||
}
|
||||
|
||||
func (t *PruneDestroyTransformer) Transform(g *Graph) error {
|
||||
var modDiff *ModuleDiff
|
||||
var modState *ModuleState
|
||||
if t.Diff != nil {
|
||||
modDiff = t.Diff.ModuleByPath(g.Path)
|
||||
}
|
||||
if t.State != nil {
|
||||
modState = t.State.ModuleByPath(g.Path)
|
||||
}
|
||||
|
||||
for _, v := range g.Vertices() {
|
||||
// If it is not a destroyer, we don't care
|
||||
dn, ok := v.(GraphNodeDestroyPrunable)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// Remove it if we should
|
||||
if !dn.DestroyInclude(modDiff, modState) {
|
||||
g.Remove(v)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -0,0 +1,411 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestDestroyTransformer(t *testing.T) {
|
||||
mod := testModule(t, "transform-destroy-basic")
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformDestroyBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCreateBeforeDestroyTransformer(t *testing.T) {
|
||||
mod := testModule(t, "transform-create-before-destroy-basic")
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &CreateBeforeDestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformCreateBeforeDestroyBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestCreateBeforeDestroyTransformer_twice(t *testing.T) {
|
||||
mod := testModule(t, "transform-create-before-destroy-twice")
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &CreateBeforeDestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformCreateBeforeDestroyTwiceStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPruneDestroyTransformer(t *testing.T) {
|
||||
var diff *Diff
|
||||
mod := testModule(t, "transform-destroy-basic")
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &PruneDestroyTransformer{Diff: diff}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformPruneDestroyBasicStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPruneDestroyTransformer_diff(t *testing.T) {
|
||||
mod := testModule(t, "transform-destroy-basic")
|
||||
|
||||
diff := &Diff{
|
||||
Modules: []*ModuleDiff{
|
||||
&ModuleDiff{
|
||||
Path: RootModulePath,
|
||||
Resources: map[string]*InstanceDiff{
|
||||
"aws_instance.bar": &InstanceDiff{},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &PruneDestroyTransformer{Diff: diff}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformPruneDestroyBasicDiffStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPruneDestroyTransformer_count(t *testing.T) {
|
||||
mod := testModule(t, "transform-destroy-prune-count")
|
||||
|
||||
diff := &Diff{}
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &PruneDestroyTransformer{Diff: diff}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformPruneDestroyCountStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPruneDestroyTransformer_countDec(t *testing.T) {
|
||||
mod := testModule(t, "transform-destroy-basic")
|
||||
|
||||
diff := &Diff{}
|
||||
state := &State{
|
||||
Modules: []*ModuleState{
|
||||
&ModuleState{
|
||||
Path: RootModulePath,
|
||||
Resources: map[string]*ResourceState{
|
||||
"aws_instance.bar.1": &ResourceState{
|
||||
Primary: &InstanceState{},
|
||||
},
|
||||
"aws_instance.bar.2": &ResourceState{
|
||||
Primary: &InstanceState{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &PruneDestroyTransformer{Diff: diff, State: state}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformPruneDestroyCountDecStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestPruneDestroyTransformer_countState(t *testing.T) {
|
||||
mod := testModule(t, "transform-destroy-basic")
|
||||
|
||||
diff := &Diff{}
|
||||
state := &State{
|
||||
Modules: []*ModuleState{
|
||||
&ModuleState{
|
||||
Path: RootModulePath,
|
||||
Resources: map[string]*ResourceState{
|
||||
"aws_instance.bar": &ResourceState{
|
||||
Primary: &InstanceState{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
g := Graph{Path: RootModulePath}
|
||||
{
|
||||
tf := &ConfigTransformer{Module: mod}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &DestroyTransformer{}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
tf := &PruneDestroyTransformer{Diff: diff, State: state}
|
||||
if err := tf.Transform(&g); err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(g.String())
|
||||
expected := strings.TrimSpace(testTransformPruneDestroyCountStateStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad:\n\n%s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
const testTransformDestroyBasicStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.foo (destroy)
|
||||
aws_instance.bar (destroy)
|
||||
`
|
||||
|
||||
const testTransformPruneDestroyBasicStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
`
|
||||
|
||||
const testTransformPruneDestroyBasicDiffStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
`
|
||||
|
||||
const testTransformPruneDestroyCountStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
`
|
||||
|
||||
const testTransformPruneDestroyCountDecStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.bar (destroy)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
`
|
||||
|
||||
const testTransformPruneDestroyCountStateStr = `
|
||||
aws_instance.bar
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.foo
|
||||
aws_instance.bar (destroy tainted)
|
||||
aws_instance.foo
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.foo (destroy tainted)
|
||||
aws_instance.bar (destroy tainted)
|
||||
`
|
||||
|
||||
const testTransformCreateBeforeDestroyBasicStr = `
|
||||
aws_instance.web
|
||||
aws_instance.web (destroy tainted)
|
||||
aws_instance.web (destroy tainted)
|
||||
aws_load_balancer.lb (destroy tainted)
|
||||
aws_instance.web (destroy)
|
||||
aws_instance.web
|
||||
aws_load_balancer.lb
|
||||
aws_load_balancer.lb (destroy)
|
||||
aws_load_balancer.lb
|
||||
aws_instance.web
|
||||
aws_load_balancer.lb (destroy tainted)
|
||||
aws_load_balancer.lb (destroy)
|
||||
aws_load_balancer.lb (destroy tainted)
|
||||
aws_load_balancer.lb (destroy)
|
||||
`
|
||||
|
||||
const testTransformCreateBeforeDestroyTwiceStr = `
|
||||
aws_autoscale.bar
|
||||
aws_autoscale.bar (destroy tainted)
|
||||
aws_lc.foo
|
||||
aws_autoscale.bar (destroy tainted)
|
||||
aws_autoscale.bar (destroy)
|
||||
aws_autoscale.bar
|
||||
aws_lc.foo
|
||||
aws_lc.foo (destroy tainted)
|
||||
aws_lc.foo (destroy tainted)
|
||||
aws_autoscale.bar (destroy tainted)
|
||||
aws_lc.foo (destroy)
|
||||
aws_autoscale.bar
|
||||
aws_autoscale.bar (destroy)
|
||||
aws_lc.foo
|
||||
`
|
|
@ -0,0 +1,61 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"log"
|
||||
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
// GraphNodeExapndable is an interface that nodes can implement to
|
||||
// signal that they can be expanded. Expanded nodes turn into
|
||||
// GraphNodeSubgraph nodes within the graph.
|
||||
type GraphNodeExpandable interface {
|
||||
Expand(GraphBuilder) (GraphNodeSubgraph, error)
|
||||
}
|
||||
|
||||
// GraphNodeDynamicExpandable is an interface that nodes can implement
|
||||
// to signal that they can be expanded at eval-time (hence dynamic).
|
||||
// These nodes are given the eval context and are expected to return
|
||||
// a new subgraph.
|
||||
type GraphNodeDynamicExpandable interface {
|
||||
DynamicExpand(EvalContext) (*Graph, error)
|
||||
}
|
||||
|
||||
// GraphNodeSubgraph is an interface a node can implement if it has
|
||||
// a larger subgraph that should be walked.
|
||||
type GraphNodeSubgraph interface {
|
||||
Subgraph() *Graph
|
||||
}
|
||||
|
||||
// ExpandTransform is a transformer that does a subgraph expansion
|
||||
// at graph transform time (vs. at eval time). The benefit of earlier
|
||||
// subgraph expansion is that errors with the graph build can be detected
|
||||
// at an earlier stage.
|
||||
type ExpandTransform struct {
|
||||
Builder GraphBuilder
|
||||
}
|
||||
|
||||
func (t *ExpandTransform) Transform(v dag.Vertex) (dag.Vertex, error) {
|
||||
ev, ok := v.(GraphNodeExpandable)
|
||||
if !ok {
|
||||
// This isn't an expandable vertex, so just ignore it.
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// Expand the subgraph!
|
||||
log.Printf("[DEBUG] vertex %s: static expanding", dag.VertexName(ev))
|
||||
return ev.Expand(t.Builder)
|
||||
}
|
||||
|
||||
type GraphNodeBasicSubgraph struct {
|
||||
NameValue string
|
||||
Graph *Graph
|
||||
}
|
||||
|
||||
func (n *GraphNodeBasicSubgraph) Name() string {
|
||||
return n.NameValue
|
||||
}
|
||||
|
||||
func (n *GraphNodeBasicSubgraph) Subgraph() *Graph {
|
||||
return n.Graph
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
package terraform
|
||||
|
||||
import (
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/hashicorp/terraform/dag"
|
||||
)
|
||||
|
||||
func TestExpandTransform_impl(t *testing.T) {
|
||||
var _ GraphVertexTransformer = new(ExpandTransform)
|
||||
}
|
||||
|
||||
func TestExpandTransform(t *testing.T) {
|
||||
var g Graph
|
||||
g.Add(1)
|
||||
g.Add(2)
|
||||
g.Connect(dag.BasicEdge(1, 2))
|
||||
|
||||
tf := &ExpandTransform{}
|
||||
out, err := tf.Transform(&testExpandable{
|
||||
Result: &g,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
|
||||
sn, ok := out.(GraphNodeSubgraph)
|
||||
if !ok {
|
||||
t.Fatalf("not subgraph: %#v", out)
|
||||
}
|
||||
|
||||
actual := strings.TrimSpace(sn.Subgraph().String())
|
||||
expected := strings.TrimSpace(testExpandTransformStr)
|
||||
if actual != expected {
|
||||
t.Fatalf("bad: %s", actual)
|
||||
}
|
||||
}
|
||||
|
||||
func TestExpandTransform_nonExpandable(t *testing.T) {
|
||||
tf := &ExpandTransform{}
|
||||
out, err := tf.Transform(42)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %s", err)
|
||||
}
|
||||
if out != 42 {
|
||||
t.Fatalf("bad: %#v", out)
|
||||
}
|
||||
}
|
||||
|
||||
type testExpandable struct {
|
||||
// Inputs
|
||||
Result *Graph
|
||||
ResultError error
|
||||
|
||||
// Outputs
|
||||
Builder GraphBuilder
|
||||
}
|
||||
|
||||
func (n *testExpandable) Expand(b GraphBuilder) (GraphNodeSubgraph, error) {
|
||||
n.Builder = b
|
||||
return &testSubgraph{n.Result}, n.ResultError
|
||||
}
|
||||
|
||||
type testSubgraph struct {
|
||||
Graph *Graph
|
||||
}
|
||||
|
||||
func (n *testSubgraph) Subgraph() *Graph {
|
||||
return n.Graph
|
||||
}
|
||||
|
||||
const testExpandTransformStr = `
|
||||
1
|
||||
2
|
||||
2
|
||||
`
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue