terraform: switch to Context for stop, Stoppable provisioners

This switches to the Go "context" package for cancellation and threads
the context through all the way to evaluation to allow behavior based on
stopping deep within graph execution.

This also adds the Stop API to provisioners so they can quickly exit
when stop is called.
This commit is contained in:
Mitchell Hashimoto 2016-12-22 11:33:26 -08:00
parent a9d799cce6
commit f8c7b639c9
No known key found for this signature in database
GPG Key ID: 744E147AA52F5B0A
12 changed files with 252 additions and 66 deletions

View File

@ -1,6 +1,7 @@
package terraform package terraform
import ( import (
"context"
"fmt" "fmt"
"log" "log"
"sort" "sort"
@ -91,8 +92,9 @@ type Context struct {
l sync.Mutex // Lock acquired during any task l sync.Mutex // Lock acquired during any task
parallelSem Semaphore parallelSem Semaphore
providerInputConfig map[string]map[string]interface{} providerInputConfig map[string]map[string]interface{}
runCh <-chan struct{} runLock sync.Mutex
stopCh chan struct{} runContext context.Context
runContextCancel context.CancelFunc
shadowErr error shadowErr error
} }
@ -339,8 +341,7 @@ func (c *Context) Interpolater() *Interpolater {
// This modifies the configuration in-place, so asking for Input twice // This modifies the configuration in-place, so asking for Input twice
// may result in different UI output showing different current values. // may result in different UI output showing different current values.
func (c *Context) Input(mode InputMode) error { func (c *Context) Input(mode InputMode) error {
v := c.acquireRun("input") defer c.acquireRun("input")()
defer c.releaseRun(v)
if mode&InputModeVar != 0 { if mode&InputModeVar != 0 {
// Walk the variables first for the root module. We walk them in // Walk the variables first for the root module. We walk them in
@ -459,8 +460,7 @@ func (c *Context) Input(mode InputMode) error {
// In addition to returning the resulting state, this context is updated // In addition to returning the resulting state, this context is updated
// with the latest state. // with the latest state.
func (c *Context) Apply() (*State, error) { func (c *Context) Apply() (*State, error) {
v := c.acquireRun("apply") defer c.acquireRun("apply")()
defer c.releaseRun(v)
// Copy our own state // Copy our own state
c.state = c.state.DeepCopy() c.state = c.state.DeepCopy()
@ -504,8 +504,7 @@ func (c *Context) Apply() (*State, error) {
// Plan also updates the diff of this context to be the diff generated // Plan also updates the diff of this context to be the diff generated
// by the plan, so Apply can be called after. // by the plan, so Apply can be called after.
func (c *Context) Plan() (*Plan, error) { func (c *Context) Plan() (*Plan, error) {
v := c.acquireRun("plan") defer c.acquireRun("plan")()
defer c.releaseRun(v)
p := &Plan{ p := &Plan{
Module: c.module, Module: c.module,
@ -600,8 +599,7 @@ func (c *Context) Plan() (*Plan, error) {
// Even in the case an error is returned, the state will be returned and // Even in the case an error is returned, the state will be returned and
// will potentially be partially updated. // will potentially be partially updated.
func (c *Context) Refresh() (*State, error) { func (c *Context) Refresh() (*State, error) {
v := c.acquireRun("refresh") defer c.acquireRun("refresh")()
defer c.releaseRun(v)
// Copy our own state // Copy our own state
c.state = c.state.DeepCopy() c.state = c.state.DeepCopy()
@ -635,29 +633,32 @@ func (c *Context) Refresh() (*State, error) {
// Stop will block until the task completes. // Stop will block until the task completes.
func (c *Context) Stop() { func (c *Context) Stop() {
c.l.Lock() c.l.Lock()
ch := c.runCh
// If we aren't running, then just return // If we're running, then stop
if ch == nil { if c.runContextCancel != nil {
c.l.Unlock() // Tell the hook we want to stop
return c.sh.Stop()
// Stop the context
c.runContextCancel()
c.runContextCancel = nil
} }
// Tell the hook we want to stop // Grab the context before we unlock
c.sh.Stop() ctx := c.runContext
// Close the stop channel // Unlock
close(c.stopCh)
// Wait for us to stop
c.l.Unlock() c.l.Unlock()
<-ch
// Wait if we have a context
if ctx != nil {
<-ctx.Done()
}
} }
// Validate validates the configuration and returns any warnings or errors. // Validate validates the configuration and returns any warnings or errors.
func (c *Context) Validate() ([]string, []error) { func (c *Context) Validate() ([]string, []error) {
v := c.acquireRun("validate") defer c.acquireRun("validate")()
defer c.releaseRun(v)
var errs error var errs error
@ -718,26 +719,26 @@ func (c *Context) SetVariable(k string, v interface{}) {
c.variables[k] = v c.variables[k] = v
} }
func (c *Context) acquireRun(phase string) chan<- struct{} { func (c *Context) acquireRun(phase string) func() {
// Acquire the runlock first. This is the lock that is held for
// the duration of a run to prevent multiple runs.
c.runLock.Lock()
// With the run lock held, grab the context lock to make changes
// to the run context.
c.l.Lock() c.l.Lock()
defer c.l.Unlock() defer c.l.Unlock()
// Setup debugging
dbug.SetPhase(phase) dbug.SetPhase(phase)
// Wait for no channel to exist // runContext should never be non-nil, check that here
for c.runCh != nil { if c.runContext != nil {
c.l.Unlock() panic("acquireRun called with runContext != nil")
ch := c.runCh
<-ch
c.l.Lock()
} }
// Create the new channel // Create a new run context
ch := make(chan struct{}) c.runContext, c.runContextCancel = context.WithCancel(context.Background())
c.runCh = ch
// Reset the stop channel so we can watch that
c.stopCh = make(chan struct{})
// Reset the stop hook so we're not stopped // Reset the stop hook so we're not stopped
c.sh.Reset() c.sh.Reset()
@ -745,10 +746,11 @@ func (c *Context) acquireRun(phase string) chan<- struct{} {
// Reset the shadow errors // Reset the shadow errors
c.shadowErr = nil c.shadowErr = nil
return ch return c.releaseRun
} }
func (c *Context) releaseRun(ch chan<- struct{}) { func (c *Context) releaseRun() {
// Grab the context lock so that we can make modifications to fields
c.l.Lock() c.l.Lock()
defer c.l.Unlock() defer c.l.Unlock()
@ -757,9 +759,17 @@ func (c *Context) releaseRun(ch chan<- struct{}) {
// phase // phase
dbug.SetPhase("INVALID") dbug.SetPhase("INVALID")
close(ch) // End our run. We check if runContext is non-nil because it can be
c.runCh = nil // set to nil if it was cancelled via Stop()
c.stopCh = nil if c.runContextCancel != nil {
c.runContextCancel()
}
// Unset the context
c.runContext = nil
// Unlock the run lock
c.runLock.Unlock()
} }
func (c *Context) walk( func (c *Context) walk(
@ -791,13 +801,14 @@ func (c *Context) walk(
log.Printf("[DEBUG] Starting graph walk: %s", operation.String()) log.Printf("[DEBUG] Starting graph walk: %s", operation.String())
walker := &ContextGraphWalker{ walker := &ContextGraphWalker{
Context: realCtx, Context: realCtx,
Operation: operation, Operation: operation,
StopContext: c.runContext,
} }
// Watch for a stop so we can call the provider Stop() API. // Watch for a stop so we can call the provider Stop() API.
doneCh := make(chan struct{}) doneCh := make(chan struct{})
go c.watchStop(walker, c.stopCh, doneCh) go c.watchStop(walker, doneCh)
// Walk the real graph, this will block until it completes // Walk the real graph, this will block until it completes
realErr := graph.Walk(walker) realErr := graph.Walk(walker)
@ -892,7 +903,15 @@ func (c *Context) walk(
return walker, realErr return walker, realErr
} }
func (c *Context) watchStop(walker *ContextGraphWalker, stopCh, doneCh <-chan struct{}) { func (c *Context) watchStop(walker *ContextGraphWalker, doneCh <-chan struct{}) {
// Get the stop channel. runContext might be nil only during tests.
// If this is called during a proper run operation, this will never
// be nil.
var stopCh <-chan struct{}
if ctx := c.runContext; ctx != nil {
stopCh = ctx.Done()
}
// Wait for a stop or completion // Wait for a stop or completion
select { select {
case <-stopCh: case <-stopCh:
@ -904,20 +923,39 @@ func (c *Context) watchStop(walker *ContextGraphWalker, stopCh, doneCh <-chan st
// If we're here, we're stopped, trigger the call. // If we're here, we're stopped, trigger the call.
// Copy the providers so that a misbehaved blocking Stop doesn't {
// completely hang Terraform. // Copy the providers so that a misbehaved blocking Stop doesn't
walker.providerLock.Lock() // completely hang Terraform.
ps := make([]ResourceProvider, 0, len(walker.providerCache)) walker.providerLock.Lock()
for _, p := range walker.providerCache { ps := make([]ResourceProvider, 0, len(walker.providerCache))
ps = append(ps, p) for _, p := range walker.providerCache {
} ps = append(ps, p)
defer walker.providerLock.Unlock() }
defer walker.providerLock.Unlock()
for _, p := range ps { for _, p := range ps {
// We ignore the error for now since there isn't any reasonable // We ignore the error for now since there isn't any reasonable
// action to take if there is an error here, since the stop is still // action to take if there is an error here, since the stop is still
// advisory: Terraform will exit once the graph node completes. // advisory: Terraform will exit once the graph node completes.
p.Stop() p.Stop()
}
}
{
// Call stop on all the provisioners
walker.provisionerLock.Lock()
ps := make([]ResourceProvisioner, 0, len(walker.provisionerCache))
for _, p := range walker.provisionerCache {
ps = append(ps, p)
}
defer walker.provisionerLock.Unlock()
for _, p := range ps {
// We ignore the error for now since there isn't any reasonable
// action to take if there is an error here, since the stop is still
// advisory: Terraform will exit once the graph node completes.
p.Stop()
}
} }
} }

View File

@ -1720,6 +1720,69 @@ func TestContext2Apply_cancel(t *testing.T) {
} }
} }
func TestContext2Apply_cancelProvisioner(t *testing.T) {
m := testModule(t, "apply-cancel-provisioner")
p := testProvider("aws")
p.ApplyFn = testApplyFn
p.DiffFn = testDiffFn
pr := testProvisioner()
ctx := testContext2(t, &ContextOpts{
Module: m,
Providers: map[string]ResourceProviderFactory{
"aws": testProviderFuncFixed(p),
},
Provisioners: map[string]ResourceProvisionerFactory{
"shell": testProvisionerFuncFixed(pr),
},
})
prStopped := make(chan struct{})
pr.ApplyFn = func(rs *InstanceState, c *ResourceConfig) error {
// Start the stop process
go ctx.Stop()
<-prStopped
return nil
}
pr.StopFn = func() error {
close(prStopped)
return nil
}
if _, err := ctx.Plan(); err != nil {
t.Fatalf("err: %s", err)
}
// Start the Apply in a goroutine
var applyErr error
stateCh := make(chan *State)
go func() {
state, err := ctx.Apply()
if err != nil {
applyErr = err
}
stateCh <- state
}()
// Wait for completion
state := <-stateCh
if applyErr != nil {
t.Fatalf("err: %s", applyErr)
}
checkStateString(t, state, `
aws_instance.foo: (tainted)
ID = foo
num = 2
type = aws_instance
`)
if !pr.StopCalled {
t.Fatal("stop should be called")
}
}
func TestContext2Apply_compute(t *testing.T) { func TestContext2Apply_compute(t *testing.T) {
m := testModule(t, "apply-compute") m := testModule(t, "apply-compute")
p := testProvider("aws") p := testProvider("aws")

View File

@ -40,8 +40,7 @@ type ImportTarget struct {
// imported. // imported.
func (c *Context) Import(opts *ImportOpts) (*State, error) { func (c *Context) Import(opts *ImportOpts) (*State, error) {
// Hold a lock since we can modify our own state here // Hold a lock since we can modify our own state here
v := c.acquireRun("import") defer c.acquireRun("import")()
defer c.releaseRun(v)
// Copy our own state // Copy our own state
c.state = c.state.DeepCopy() c.state = c.state.DeepCopy()

View File

@ -8,6 +8,10 @@ import (
// EvalContext is the interface that is given to eval nodes to execute. // EvalContext is the interface that is given to eval nodes to execute.
type EvalContext interface { type EvalContext interface {
// Stopped returns a channel that is closed when evaluation is stopped
// via Terraform.Context.Stop()
Stopped() <-chan struct{}
// Path is the current module path. // Path is the current module path.
Path() []string Path() []string

View File

@ -1,6 +1,7 @@
package terraform package terraform
import ( import (
"context"
"fmt" "fmt"
"log" "log"
"strings" "strings"
@ -12,6 +13,9 @@ import (
// BuiltinEvalContext is an EvalContext implementation that is used by // BuiltinEvalContext is an EvalContext implementation that is used by
// Terraform by default. // Terraform by default.
type BuiltinEvalContext struct { type BuiltinEvalContext struct {
// StopContext is the context used to track whether we're complete
StopContext context.Context
// PathValue is the Path that this context is operating within. // PathValue is the Path that this context is operating within.
PathValue []string PathValue []string
@ -43,6 +47,15 @@ type BuiltinEvalContext struct {
once sync.Once once sync.Once
} }
func (ctx *BuiltinEvalContext) Stopped() <-chan struct{} {
// This can happen during tests. During tests, we just block forever.
if ctx.StopContext == nil {
return nil
}
return ctx.StopContext.Done()
}
func (ctx *BuiltinEvalContext) Hook(fn func(Hook) (HookAction, error)) error { func (ctx *BuiltinEvalContext) Hook(fn func(Hook) (HookAction, error)) error {
for _, h := range ctx.Hooks { for _, h := range ctx.Hooks {
action, err := fn(h) action, err := fn(h)

View File

@ -9,6 +9,9 @@ import (
// MockEvalContext is a mock version of EvalContext that can be used // MockEvalContext is a mock version of EvalContext that can be used
// for tests. // for tests.
type MockEvalContext struct { type MockEvalContext struct {
StoppedCalled bool
StoppedValue <-chan struct{}
HookCalled bool HookCalled bool
HookHook Hook HookHook Hook
HookError error HookError error
@ -85,6 +88,11 @@ type MockEvalContext struct {
StateLock *sync.RWMutex StateLock *sync.RWMutex
} }
func (c *MockEvalContext) Stopped() <-chan struct{} {
c.StoppedCalled = true
return c.StoppedValue
}
func (c *MockEvalContext) Hook(fn func(Hook) (HookAction, error)) error { func (c *MockEvalContext) Hook(fn func(Hook) (HookAction, error)) error {
c.HookCalled = true c.HookCalled = true
if c.HookHook != nil { if c.HookHook != nil {

View File

@ -1,6 +1,7 @@
package terraform package terraform
import ( import (
"context"
"fmt" "fmt"
"log" "log"
"sync" "sync"
@ -15,8 +16,9 @@ type ContextGraphWalker struct {
NullGraphWalker NullGraphWalker
// Configurable values // Configurable values
Context *Context Context *Context
Operation walkOperation Operation walkOperation
StopContext context.Context
// Outputs, do not set these. Do not read these while the graph // Outputs, do not set these. Do not read these while the graph
// is being walked. // is being walked.
@ -65,6 +67,7 @@ func (w *ContextGraphWalker) EnterPath(path []string) EvalContext {
w.interpolaterVarLock.Unlock() w.interpolaterVarLock.Unlock()
ctx := &BuiltinEvalContext{ ctx := &BuiltinEvalContext{
StopContext: w.StopContext,
PathValue: path, PathValue: path,
Hooks: w.Context.hooks, Hooks: w.Context.hooks,
InputValue: w.Context.uiInput, InputValue: w.Context.uiInput,

View File

@ -21,6 +21,26 @@ type ResourceProvisioner interface {
// is provided since provisioners only run after a resource has been // is provided since provisioners only run after a resource has been
// newly created. // newly created.
Apply(UIOutput, *InstanceState, *ResourceConfig) error Apply(UIOutput, *InstanceState, *ResourceConfig) error
// Stop is called when the provisioner should halt any in-flight actions.
//
// This can be used to make a nicer Ctrl-C experience for Terraform.
// Even if this isn't implemented to do anything (just returns nil),
// Terraform will still cleanly stop after the currently executing
// graph node is complete. However, this API can be used to make more
// efficient halts.
//
// Stop doesn't have to and shouldn't block waiting for in-flight actions
// to complete. It should take any action it wants and return immediately
// acknowledging it has received the stop request. Terraform core will
// automatically not make any further API calls to the provider soon
// after Stop is called (technically exactly once the currently executing
// graph nodes are complete).
//
// The error returned, if non-nil, is assumed to mean that signaling the
// stop somehow failed and that the user should expect potentially waiting
// a longer period of time.
Stop() error
} }
// ResourceProvisionerCloser is an interface that provisioners that can close // ResourceProvisionerCloser is an interface that provisioners that can close

View File

@ -21,6 +21,10 @@ type MockResourceProvisioner struct {
ValidateFn func(c *ResourceConfig) ([]string, []error) ValidateFn func(c *ResourceConfig) ([]string, []error)
ValidateReturnWarns []string ValidateReturnWarns []string
ValidateReturnErrors []error ValidateReturnErrors []error
StopCalled bool
StopFn func() error
StopReturnError error
} }
func (p *MockResourceProvisioner) Validate(c *ResourceConfig) ([]string, []error) { func (p *MockResourceProvisioner) Validate(c *ResourceConfig) ([]string, []error) {
@ -40,14 +44,29 @@ func (p *MockResourceProvisioner) Apply(
state *InstanceState, state *InstanceState,
c *ResourceConfig) error { c *ResourceConfig) error {
p.Lock() p.Lock()
defer p.Unlock()
p.ApplyCalled = true p.ApplyCalled = true
p.ApplyOutput = output p.ApplyOutput = output
p.ApplyState = state p.ApplyState = state
p.ApplyConfig = c p.ApplyConfig = c
if p.ApplyFn != nil { if p.ApplyFn != nil {
return p.ApplyFn(state, c) fn := p.ApplyFn
p.Unlock()
return fn(state, c)
} }
defer p.Unlock()
return p.ApplyReturnError return p.ApplyReturnError
} }
func (p *MockResourceProvisioner) Stop() error {
p.Lock()
defer p.Unlock()
p.StopCalled = true
if p.StopFn != nil {
return p.StopFn()
}
return p.StopReturnError
}

View File

@ -88,7 +88,8 @@ func newShadowContext(c *Context) (*Context, *Context, Shadow) {
// l - no copy // l - no copy
parallelSem: c.parallelSem, parallelSem: c.parallelSem,
providerInputConfig: c.providerInputConfig, providerInputConfig: c.providerInputConfig,
runCh: c.runCh, runContext: c.runContext,
runContextCancel: c.runContextCancel,
shadowErr: c.shadowErr, shadowErr: c.shadowErr,
} }

View File

@ -112,6 +112,10 @@ func (p *shadowResourceProvisionerReal) Apply(
return err return err
} }
func (p *shadowResourceProvisionerReal) Stop() error {
return p.ResourceProvisioner.Stop()
}
// shadowResourceProvisionerShadow is the shadow resource provisioner. Function // shadowResourceProvisionerShadow is the shadow resource provisioner. Function
// calls never affect real resources. This is paired with the "real" side // calls never affect real resources. This is paired with the "real" side
// which must be called properly to enable recording. // which must be called properly to enable recording.
@ -228,6 +232,13 @@ func (p *shadowResourceProvisionerShadow) Apply(
return result.ResultErr return result.ResultErr
} }
func (p *shadowResourceProvisionerShadow) Stop() error {
// For the shadow, we always just return nil since a Stop indicates
// that we were interrupted and shadows are disabled during interrupts
// anyways.
return nil
}
// The structs for the various function calls are put below. These structs // The structs for the various function calls are put below. These structs
// are used to carry call information across the real/shadow boundaries. // are used to carry call information across the real/shadow boundaries.

View File

@ -0,0 +1,7 @@
resource "aws_instance" "foo" {
num = "2"
provisioner "shell" {
foo = "bar"
}
}