mirror of
https://github.com/opentffoundation/opentf.git
synced 2025-12-19 17:59:05 -05:00
tofu: Explicitly preload provider schemas
Previously we used an API design that suggested that provider schemas were treated as a read-through cache with providers loaded only on demand. However, that's really a fiction: during graph construction the AttachSchemaTransformer would always end up loading all of the schemas up front anyway, and so everything after that point would always be pulling from that cache. To make the situation a little more explicit so that the system is easier to follow -- particularly now that we'll be exposing work like this in OpenTelemetry tracing -- we'll switch to a model where populating the cache is an explicit step and then all downstream calls just assume the cache is already populated. This means that there's now one obvious place where the provider schema lookups are triggered, but we'll also do the real work in the background and thus we can make some progress on the graph construction (CPU bound) concurrently with the schema fetching (I/O bound) so that the schema is more likely to be available at or soon after the start of the execution of AttachSchemaTransformer, which will then block until the schema loading has completed. We have some unit tests that were previously depending on the "load schemas on first request, wherever it happens" behavior and so those are now updated here to either explicitly load the schemas themselves or to rely on a higher-level helper to do it for them, but for normal code and for context tests we expect that the schema loading will always be triggered at the first entry into an exported method of Context, with subsequent calls able to reuse that cache as long as they are still working with the same providers. Signed-off-by: Martin Atkins <mart@degeneration.co.uk>
This commit is contained in:
@@ -12,6 +12,7 @@ import (
|
||||
"github.com/zclconf/go-cty/cty"
|
||||
|
||||
"github.com/opentofu/opentofu/internal/addrs"
|
||||
"github.com/opentofu/opentofu/internal/collections"
|
||||
"github.com/opentofu/opentofu/internal/getproviders"
|
||||
)
|
||||
|
||||
@@ -318,6 +319,19 @@ func (s *State) LocalValue(addr addrs.AbsLocalValue) cty.Value {
|
||||
return ms.LocalValues[addr.LocalValue.Name]
|
||||
}
|
||||
|
||||
func (s *State) ProviderTypes() collections.Set[addrs.Provider] {
|
||||
if s == nil {
|
||||
return nil
|
||||
}
|
||||
ret := collections.NewSet[addrs.Provider]()
|
||||
for _, ms := range s.Modules {
|
||||
for _, rc := range ms.Resources {
|
||||
ret[rc.ProviderConfig.Provider] = struct{}{}
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// ProviderAddrs returns a list of all of the provider configuration addresses
|
||||
// referenced throughout the receiving state.
|
||||
//
|
||||
|
||||
@@ -154,9 +154,26 @@ func NewContext(opts *ContextOpts) (*Context, tfdiags.Diagnostics) {
|
||||
}, diags
|
||||
}
|
||||
|
||||
// ensureProviderSchemasLoaded starts a background task to collect the schemas
|
||||
// for any providers used in the given configuration or state (either of which
|
||||
// can be nil) so that they'll become available for later use.
|
||||
//
|
||||
// This function returns immediately and so cannot signal whether the schema
|
||||
// loading was successful. Instead, any errors are reported once individual
|
||||
// schemas are requested.
|
||||
func (c *Context) ensureProviderSchemasLoaded(ctx context.Context, config *configs.Config, state *states.State) {
|
||||
c.plugins.LoadProviderSchemas(ctx, config, state)
|
||||
}
|
||||
|
||||
func (c *Context) Schemas(ctx context.Context, config *configs.Config, state *states.State) (*Schemas, tfdiags.Diagnostics) {
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
// This uses an older codepath that predates our async-loading provider
|
||||
// schema mechanism from [Context.ensureProviderSchemasLoaded], but
|
||||
// internally it uses the same cache and so this will be fast if
|
||||
// [Context.ensureProviderSchemasLoaded] was previously called with
|
||||
// the same config and state, or will internally cause the same
|
||||
// effect as that function if it had not already been called.
|
||||
ret, err := loadSchemas(ctx, config, state, c.plugins)
|
||||
if err != nil {
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
|
||||
@@ -33,6 +33,12 @@ import (
|
||||
func (c *Context) Apply(ctx context.Context, plan *plans.Plan, config *configs.Config) (*states.State, tfdiags.Diagnostics) {
|
||||
defer c.acquireRun("apply")()
|
||||
|
||||
// We'll get this started as soon as possible so that this I/O bound work
|
||||
// can run concurrently with some CPU-bound work we're about to do. The
|
||||
// next attempt to access schemas will block until the background task
|
||||
// started by this call has completed.
|
||||
c.plugins.LoadProviderSchemas(ctx, config, plan.PriorState)
|
||||
|
||||
log.Printf("[DEBUG] Building and walking apply graph for %s plan", plan.UIMode)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
@@ -240,6 +246,8 @@ func (c *Context) ApplyGraphForUI(plan *plans.Plan, config *configs.Config) (*Gr
|
||||
// For now though, this really is just the internal graph, confusing
|
||||
// implementation details and all.
|
||||
|
||||
c.ensureProviderSchemasLoaded(context.TODO(), config, plan.PriorState)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
graph, _, moreDiags := c.applyGraph(context.TODO(), plan, config, make(ProviderFunctionMapping))
|
||||
|
||||
@@ -44,6 +44,12 @@ func (c *Context) Eval(ctx context.Context, config *configs.Config, state *state
|
||||
// command. Internally, we create an evaluator in c.walk before walking
|
||||
// the graph, and create scopes in ContextGraphWalker.
|
||||
|
||||
// We'll get this started as soon as possible so that this I/O bound work
|
||||
// can run concurrently with some CPU-bound work we're about to do. The
|
||||
// next attempt to access schemas will block until the background task
|
||||
// started by this call has completed.
|
||||
c.plugins.LoadProviderSchemas(ctx, config, state)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
defer c.acquireRun("eval")()
|
||||
|
||||
|
||||
@@ -242,6 +242,12 @@ func (ri *ImportResolver) GetImport(address addrs.AbsResourceInstance) *Evaluate
|
||||
func (c *Context) Import(ctx context.Context, config *configs.Config, prevRunState *states.State, opts *ImportOpts) (*states.State, tfdiags.Diagnostics) {
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
// We'll get this started as soon as possible so that this I/O bound work
|
||||
// can run concurrently with some CPU-bound work we're about to do. The
|
||||
// next attempt to access schemas will block until the background task
|
||||
// started by this call has completed.
|
||||
c.plugins.LoadProviderSchemas(ctx, config, prevRunState)
|
||||
|
||||
// Hold a lock since we can modify our own state here
|
||||
defer c.acquireRun("import")()
|
||||
|
||||
|
||||
@@ -48,6 +48,8 @@ func (c *Context) Input(ctx context.Context, config *configs.Config, mode InputM
|
||||
// CLI layer too in order to avoid this odd situation where core code
|
||||
// produces UI input prompts.)
|
||||
|
||||
c.plugins.LoadProviderSchemas(ctx, config, nil)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
defer c.acquireRun("input")()
|
||||
|
||||
|
||||
@@ -131,6 +131,12 @@ func (c *Context) Plan(ctx context.Context, config *configs.Config, prevRunState
|
||||
defer c.acquireRun("plan")()
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
// We'll get this started as soon as possible so that this I/O bound work
|
||||
// can run concurrently with some CPU-bound work we're about to do. The
|
||||
// next attempt to access schemas will block until the background task
|
||||
// started by this call has completed.
|
||||
c.plugins.LoadProviderSchemas(ctx, config, prevRunState)
|
||||
|
||||
// Save the downstream functions from needing to deal with these broken situations.
|
||||
// No real callers should rely on these, but we have a bunch of old and
|
||||
// sloppy tests that don't always populate arguments properly.
|
||||
@@ -1075,6 +1081,8 @@ func (c *Context) PlanGraphForUI(config *configs.Config, prevRunState *states.St
|
||||
// For now though, this really is just the internal graph, confusing
|
||||
// implementation details and all.
|
||||
|
||||
c.ensureProviderSchemasLoaded(context.TODO(), config, prevRunState)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
opts := &PlanOpts{Mode: mode}
|
||||
|
||||
@@ -11,9 +11,12 @@ import (
|
||||
"log"
|
||||
|
||||
"github.com/opentofu/opentofu/internal/addrs"
|
||||
"github.com/opentofu/opentofu/internal/configs"
|
||||
"github.com/opentofu/opentofu/internal/configs/configschema"
|
||||
"github.com/opentofu/opentofu/internal/providers"
|
||||
"github.com/opentofu/opentofu/internal/provisioners"
|
||||
"github.com/opentofu/opentofu/internal/states"
|
||||
"github.com/opentofu/opentofu/internal/tfdiags"
|
||||
)
|
||||
|
||||
// contextPlugins represents a library of available plugins (providers and
|
||||
@@ -23,6 +26,12 @@ import (
|
||||
type contextPlugins struct {
|
||||
providerFactories map[addrs.Provider]providers.Factory
|
||||
provisionerFactories map[string]provisioners.Factory
|
||||
|
||||
// In cache we retain results from certain operations that we expect
|
||||
// should be constants for a particular version of a plugin, such as
|
||||
// a provider's schema, so that we can avoid the cost of re-fetching the
|
||||
// same data.
|
||||
cache contextPluginsCache
|
||||
}
|
||||
|
||||
func newContextPlugins(providerFactories map[addrs.Provider]providers.Factory, provisionerFactories map[string]provisioners.Factory) *contextPlugins {
|
||||
@@ -61,71 +70,58 @@ func (cp *contextPlugins) NewProvisionerInstance(typ string) (provisioners.Inter
|
||||
return f()
|
||||
}
|
||||
|
||||
// ProviderSchema uses a temporary instance of the provider with the given
|
||||
// address to obtain the full schema for all aspects of that provider.
|
||||
// LoadProviderSchemas starts a background task to load the schemas for any
|
||||
// providers used by the given configuration and state, either of which may
|
||||
// be nil to represent their absence.
|
||||
//
|
||||
// ProviderSchema memoizes results by unique provider address, so it's fine
|
||||
// to repeatedly call this method with the same address if various different
|
||||
// parts of OpenTofu all need the same schema information.
|
||||
func (cp *contextPlugins) ProviderSchema(ctx context.Context, addr addrs.Provider) (providers.ProviderSchema, error) {
|
||||
var schemas providers.ProviderSchema
|
||||
// This function returns immediately but subsequent calls to access provider
|
||||
// schemas will then block until the background work has completed, so it's
|
||||
// better to call this function as early as possible and then delay accessing
|
||||
// provider schema information for as long as possible after that to achieve
|
||||
// the biggest concurrency benefit.
|
||||
func (cp *contextPlugins) LoadProviderSchemas(ctx context.Context, config *configs.Config, state *states.State) {
|
||||
cp.cache.LoadProviderSchemas(ctx, config, state, cp.providerFactories)
|
||||
}
|
||||
|
||||
log.Printf("[TRACE] tofu.contextPlugins: Initializing provider %q to read its schema", addr)
|
||||
provider, err := cp.NewProviderInstance(addr)
|
||||
if err != nil {
|
||||
return schemas, fmt.Errorf("failed to instantiate provider %q to obtain schema: %w", addr, err)
|
||||
}
|
||||
defer provider.Close(ctx)
|
||||
// ProviderSchema returns the schema information for the given provider
|
||||
// from a cache previously populated by call to
|
||||
// [contextPlugins.LoadProviderSchemas].
|
||||
//
|
||||
// If the background work started by an earlier
|
||||
// [contextPlugins.LoadProviderSchemas] is still in progress then this function
|
||||
// blocks until that work is complete. However, this function never makes any
|
||||
// provider calls directly itself.
|
||||
//
|
||||
// If the requested provider was not included in a previous call to
|
||||
// [contextPlugins.LoadProviderSchemas] then this returns diagnostics.
|
||||
func (cp *contextPlugins) ProviderSchema(addr addrs.Provider) (providers.ProviderSchema, tfdiags.Diagnostics) {
|
||||
resp := cp.cache.GetProviderSchemaResponse(addr)
|
||||
|
||||
resp := provider.GetProviderSchema(ctx)
|
||||
if resp.Diagnostics.HasErrors() {
|
||||
return resp, fmt.Errorf("failed to retrieve schema from provider %q: %w", addr, resp.Diagnostics.Err())
|
||||
}
|
||||
|
||||
if resp.Provider.Version < 0 {
|
||||
// We're not using the version numbers here yet, but we'll check
|
||||
// for validity anyway in case we start using them in future.
|
||||
return resp, fmt.Errorf("provider %s has invalid negative schema version for its configuration blocks,which is a bug in the provider ", addr)
|
||||
}
|
||||
|
||||
for t, r := range resp.ResourceTypes {
|
||||
if err := r.Block.InternalValidate(); err != nil {
|
||||
return resp, fmt.Errorf("provider %s has invalid schema for managed resource type %q, which is a bug in the provider: %w", addr, t, err)
|
||||
}
|
||||
if r.Version < 0 {
|
||||
return resp, fmt.Errorf("provider %s has invalid negative schema version for managed resource type %q, which is a bug in the provider", addr, t)
|
||||
}
|
||||
}
|
||||
|
||||
for t, d := range resp.DataSources {
|
||||
if err := d.Block.InternalValidate(); err != nil {
|
||||
return resp, fmt.Errorf("provider %s has invalid schema for data resource type %q, which is a bug in the provider: %w", addr, t, err)
|
||||
}
|
||||
if d.Version < 0 {
|
||||
// We're not using the version numbers here yet, but we'll check
|
||||
// for validity anyway in case we start using them in future.
|
||||
return resp, fmt.Errorf("provider %s has invalid negative schema version for data resource type %q, which is a bug in the provider", addr, t)
|
||||
}
|
||||
}
|
||||
|
||||
return resp, nil
|
||||
// The underlying provider API includes diagnostics inline in the response
|
||||
// due to quirks of the mapping to gRPC, but we'll adapt that here to be
|
||||
// more like how we conventionally treat diagnostics so that our caller
|
||||
// can follow the usual diagnostics-handling patterns.
|
||||
//
|
||||
// GetProviderSchemaResponse is guaranteed to always return a non-nil
|
||||
// result, since it'll synthesize an error response itself if there is
|
||||
// not already a cached entry for this provider.
|
||||
return *resp, resp.Diagnostics
|
||||
}
|
||||
|
||||
// ProviderConfigSchema is a helper wrapper around ProviderSchema which first
|
||||
// reads the full schema of the given provider and then extracts just the
|
||||
// retrieves the full schema of the given provider and then extracts just the
|
||||
// provider's configuration schema, which defines what's expected in a
|
||||
// "provider" block in the configuration when configuring this provider.
|
||||
func (cp *contextPlugins) ProviderConfigSchema(ctx context.Context, providerAddr addrs.Provider) (*configschema.Block, error) {
|
||||
providerSchema, err := cp.ProviderSchema(ctx, providerAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
func (cp *contextPlugins) ProviderConfigSchema(providerAddr addrs.Provider) (*configschema.Block, tfdiags.Diagnostics) {
|
||||
providerSchema, diags := cp.ProviderSchema(providerAddr)
|
||||
if diags.HasErrors() {
|
||||
return nil, diags
|
||||
}
|
||||
|
||||
return providerSchema.Provider.Block, nil
|
||||
return providerSchema.Provider.Block, diags
|
||||
}
|
||||
|
||||
// ResourceTypeSchema is a helper wrapper around ProviderSchema which first
|
||||
// reads the schema of the given provider and then tries to find the schema
|
||||
// retrieves the schema of the given provider and then tries to find the schema
|
||||
// for the resource type of the given resource mode in that provider.
|
||||
//
|
||||
// ResourceTypeSchema will return an error if the provider schema lookup
|
||||
@@ -135,10 +131,10 @@ func (cp *contextPlugins) ProviderConfigSchema(ctx context.Context, providerAddr
|
||||
// Managed resource types have versioned schemas, so the second return value
|
||||
// is the current schema version number for the requested resource. The version
|
||||
// is irrelevant for other resource modes.
|
||||
func (cp *contextPlugins) ResourceTypeSchema(ctx context.Context, providerAddr addrs.Provider, resourceMode addrs.ResourceMode, resourceType string) (*configschema.Block, uint64, error) {
|
||||
providerSchema, err := cp.ProviderSchema(ctx, providerAddr)
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
func (cp *contextPlugins) ResourceTypeSchema(providerAddr addrs.Provider, resourceMode addrs.ResourceMode, resourceType string) (*configschema.Block, uint64, tfdiags.Diagnostics) {
|
||||
providerSchema, diags := cp.ProviderSchema(providerAddr)
|
||||
if diags.HasErrors() {
|
||||
return nil, 0, diags
|
||||
}
|
||||
|
||||
schema, version := providerSchema.SchemaForResourceType(resourceMode, resourceType)
|
||||
@@ -148,9 +144,12 @@ func (cp *contextPlugins) ResourceTypeSchema(ctx context.Context, providerAddr a
|
||||
// ProvisionerSchema uses a temporary instance of the provisioner with the
|
||||
// given type name to obtain the schema for that provisioner's configuration.
|
||||
//
|
||||
// ProvisionerSchema memoizes results by provisioner type name, so it's fine
|
||||
// to repeatedly call this method with the same name if various different
|
||||
// parts of OpenTofu all need the same schema information.
|
||||
// Provisioner schemas are currently not cached because we assume that it's
|
||||
// rare to use any except those compiled directly into OpenTofu, and therefore
|
||||
// we're usually just retrieving an already-resident data structure from a
|
||||
// different part of the program. This could potentially be slow for those
|
||||
// using the legacy support for plugin-based provisioners, if they have many
|
||||
// instances of such provisioners.
|
||||
func (cp *contextPlugins) ProvisionerSchema(typ string) (*configschema.Block, error) {
|
||||
log.Printf("[TRACE] tofu.contextPlugins: Initializing provisioner %q to read its schema", typ)
|
||||
provisioner, err := cp.NewProvisionerInstance(typ)
|
||||
|
||||
259
internal/tofu/context_plugins_cache.go
Normal file
259
internal/tofu/context_plugins_cache.go
Normal file
@@ -0,0 +1,259 @@
|
||||
// Copyright (c) The OpenTofu Authors
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
// Copyright (c) 2023 HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
package tofu
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"iter"
|
||||
"log"
|
||||
"maps"
|
||||
"slices"
|
||||
"sync"
|
||||
|
||||
"github.com/opentofu/opentofu/internal/addrs"
|
||||
"github.com/opentofu/opentofu/internal/configs"
|
||||
"github.com/opentofu/opentofu/internal/providers"
|
||||
"github.com/opentofu/opentofu/internal/states"
|
||||
"github.com/opentofu/opentofu/internal/tfdiags"
|
||||
)
|
||||
|
||||
// contextPluginsCache is the overall container for various information about
|
||||
// provider and provisioner plugins that we expect should remain constant
|
||||
// for the whole life of a [Context] object.
|
||||
//
|
||||
// Instances of this type are part of a [contextPlugins] object, and so the
|
||||
// cached data is valid only for the plugins used by that object. This should
|
||||
// be accessed only indirectly through the [contextPlugins] API.
|
||||
type contextPluginsCache struct {
|
||||
// providerSchemas is a cache of previously-fetched provider schema
|
||||
// responses. We currently populate this all at once across all
|
||||
// providers so that subsequent code can assume that the schema for
|
||||
// any provider is always cheaply available in RAM.
|
||||
//
|
||||
// Access to this map must be coordinated through providerSchemasMu.
|
||||
providerSchemas map[addrs.Provider]*providers.GetProviderSchemaResponse
|
||||
providerSchemasMu sync.RWMutex
|
||||
}
|
||||
|
||||
// GetProviderSchemaResponse returns the full schema response from the given
|
||||
// provider's "GetProviderSchema" function.
|
||||
//
|
||||
// The requested provider must have been included in an earlier call to
|
||||
// [contextPluginsCache.LoadProviderSchemas] on the same cache object, or
|
||||
// this will return a synthetic response containing an error diagnostic.
|
||||
func (c *contextPluginsCache) GetProviderSchemaResponse(providerAddr addrs.Provider) *providers.GetProviderSchemaResponse {
|
||||
// This shared lock is mainly just to force us to block until any
|
||||
// background LoadProviderSchemas operation has completed. It's fine
|
||||
// for multiple readers to access the results of such a call concurrently.
|
||||
c.providerSchemasMu.RLock()
|
||||
ret, ok := c.providerSchemas[providerAddr]
|
||||
c.providerSchemasMu.RUnlock()
|
||||
if !ok {
|
||||
var diags tfdiags.Diagnostics
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
"Provider schema unavailable",
|
||||
fmt.Sprintf("OpenTofu needs to use the schema of provider %s, but it was not previously loaded from the provider. This is a bug in OpenTofu.", providerAddr),
|
||||
))
|
||||
ret = &providers.GetProviderSchemaResponse{
|
||||
Diagnostics: diags,
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// LoadProviderSchemas starts some background work to collect schema information
|
||||
// for all of the providers used by the given configuration and state, both
|
||||
// of which are optional and can be nil.
|
||||
//
|
||||
// This function returns immediately while schema loading work continues in the
|
||||
// background. The other methods of this type related to provider schemas will
|
||||
// block until all of the background work has completed, and so callers can
|
||||
// assume that any future schema lookups will be handled only after the
|
||||
// requests for all providers included in the given config and state have
|
||||
// completed, but those later calls may take longer to return in order to
|
||||
// satisfy that constraint.
|
||||
//
|
||||
// As an exception to the above rule, LoadProviderSchemas does _not_ return
|
||||
// immediately if there is still a background task running from a previous
|
||||
// call to the same function, and will instead block until that previous
|
||||
// operation is complete to find out whether there's any new work left to do.
|
||||
func (c *contextPluginsCache) LoadProviderSchemas(ctx context.Context, config *configs.Config, state *states.State, factories map[addrs.Provider]providers.Factory) {
|
||||
if config == nil && state == nil {
|
||||
// Nothing to do then. This is a silly situation but we'll handle it
|
||||
// here because it's easy to handle here.
|
||||
return
|
||||
}
|
||||
// We acquire the lock immediately here, before entering the goroutine
|
||||
// to make sure we always uphold the guarantee that any subsequent call
|
||||
// to fetch a provider schema will block until the background work is
|
||||
// complete. Otherwise it's possible for an early call to sneak in
|
||||
// before our goroutine begins running.
|
||||
log.Printf("[TRACE] contextPluginsCache.LoadProviderSchemas waiting for exclusive lock")
|
||||
c.providerSchemasMu.Lock()
|
||||
go func() {
|
||||
// We hold an exclusive lock on c.providerSchemas throughout our work
|
||||
// here so that subsequent lookups will block until we've had a change
|
||||
// to try to load everything.
|
||||
defer c.providerSchemasMu.Unlock()
|
||||
log.Printf("[TRACE] contextPluginsCache.LoadProviderSchemas begins")
|
||||
if c.providerSchemas == nil {
|
||||
c.providerSchemas = make(map[addrs.Provider]*providers.GetProviderSchemaResponse)
|
||||
}
|
||||
|
||||
if config != nil {
|
||||
// We'll be loading schemas from multiple providers concurrently, but
|
||||
// we need to make sure our updates of c.providerSchemas happen only
|
||||
// sequentially and so we'll use a channel to gather the results.
|
||||
results := make(chan providerSchemaLoadResult)
|
||||
go c.loadSchemasForProviders(ctx, slices.Values(config.ProviderTypes()), factories, results)
|
||||
for result := range results {
|
||||
c.providerSchemas[result.providerAddr] = result.response
|
||||
}
|
||||
}
|
||||
|
||||
if state != nil {
|
||||
// Whenever we have both a config and a state it's pretty rare for
|
||||
// there to be providers in the state that aren't in the config, so
|
||||
// we'll just deal with the state-only ones separately to avoid the
|
||||
// extra complexity of negotiating between two sets of concurrent
|
||||
// loads. Most of the time this will make no additional provider
|
||||
// schema requests at all, because we'll have already fetched
|
||||
// everything in the previous loop.
|
||||
results := make(chan providerSchemaLoadResult)
|
||||
go c.loadSchemasForProviders(ctx, maps.Keys(state.ProviderTypes()), factories, results)
|
||||
for result := range results {
|
||||
c.providerSchemas[result.providerAddr] = result.response
|
||||
}
|
||||
}
|
||||
|
||||
log.Printf("[TRACE] contextPluginsCache.LoadProviderSchemas ends")
|
||||
|
||||
// We could potentially choose to now throw away schema entries
|
||||
// for specific items that are not in config or state, but that
|
||||
// would mean we'd no longer be able to rely only on the direct
|
||||
// presence of keys in c.providerSchemas to detect if we can
|
||||
// reuse our existing cache entry, so we'll keep this simple for
|
||||
// now at the expense of keeping some unneeded data in RAM.
|
||||
}()
|
||||
}
|
||||
|
||||
// providerSchemaLoadResult is an implementation detail of
|
||||
// [contextPluginsCache.LoadProviderSchemas] and should not be used in any
|
||||
// other way.
|
||||
type providerSchemaLoadResult struct {
|
||||
providerAddr addrs.Provider
|
||||
response *providers.GetProviderSchemaResponse
|
||||
}
|
||||
|
||||
// loadProviderSchemas fetches the schema for any provider in providerAddrs
|
||||
// that isn't already in c.providerSchemas and writes the result to the
|
||||
// channel given in into, and then closes that channel when the work is
|
||||
// all done.
|
||||
//
|
||||
// This may be called only from [contextPluginsCache.LoadProviderSchemas]
|
||||
// while holding an exclusive lock on the providerSchemas map, and the
|
||||
// providerSchemas must not be concurrently modified until at least one
|
||||
// item has been written to the channel or the channel has been closed.
|
||||
func (c *contextPluginsCache) loadSchemasForProviders(ctx context.Context, providerAddrs iter.Seq[addrs.Provider], factories map[addrs.Provider]providers.Factory, into chan<- providerSchemaLoadResult) {
|
||||
// To allow the caller to modify c.providerSchemas each time we
|
||||
// emit a result we'll first collect a copy of the set of providers
|
||||
// we already alreadyHave cached results for, which we'll then use instead
|
||||
// of directly accessing the map in our main loop below.
|
||||
alreadyHave := make(map[addrs.Provider]struct{}, len(c.providerSchemas))
|
||||
for providerAddr := range c.providerSchemas {
|
||||
alreadyHave[providerAddr] = struct{}{}
|
||||
}
|
||||
// After this point we must not access c.providerSchemas anymore.
|
||||
|
||||
log.Printf("[TRACE] contextPluginsCache.loadSchemasForProviders starting loop")
|
||||
var wg sync.WaitGroup
|
||||
for providerAddr := range providerAddrs {
|
||||
if _, exists := alreadyHave[providerAddr]; exists {
|
||||
continue // we already have this one
|
||||
}
|
||||
|
||||
wg.Add(1)
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
resp := loadProviderSchema(ctx, providerAddr, factories)
|
||||
into <- providerSchemaLoadResult{
|
||||
providerAddr: providerAddr,
|
||||
response: resp,
|
||||
}
|
||||
}()
|
||||
}
|
||||
log.Printf("[TRACE] contextPluginsCache.loadSchemasForProviders waiting for goroutines to complete")
|
||||
wg.Wait()
|
||||
log.Printf("[TRACE] contextPluginsCache.loadSchemasForProviders goroutines are complete, so closing channel")
|
||||
close(into) // all done!
|
||||
log.Printf("[TRACE] contextPluginsCache.loadSchemasForProviders all done")
|
||||
}
|
||||
|
||||
// loadProviderSchema does the main work of retrieving the schema for a
|
||||
// particular provider, called from one of the loops in
|
||||
// [contextPluginsCache.LoadProviderSchemas].
|
||||
//
|
||||
// This function intentionally has no direct access to the [contextPluginsCache]
|
||||
// object that it was called for so that it can safely run concurrently with
|
||||
// other calls fetching other providers without any data races. The caller
|
||||
// is responsible for whatever synchronization is needed to safely store the
|
||||
// result once this function returns.
|
||||
//
|
||||
// This function always returns a response object, but in some cases that object
|
||||
// is synthetically-constructed within the function as a way to report
|
||||
// diagnostics about problems that prevented interacting with the provider at
|
||||
// all.
|
||||
func loadProviderSchema(ctx context.Context, providerAddr addrs.Provider, factories map[addrs.Provider]providers.Factory) *providers.GetProviderSchemaResponse {
|
||||
log.Printf("[TRACE] contextPluginsCache.loadProviderSchema for %s", providerAddr)
|
||||
|
||||
factory, ok := factories[providerAddr]
|
||||
if !ok {
|
||||
// Should not get here: this means that we are trying to load a schema
|
||||
// from a provider we don't have, which suggests a bug in whatever
|
||||
// called tofu.NewContext and one of the methods on the result.
|
||||
// We'll still return a reasonable diagnostic for it though, for
|
||||
// robustness.
|
||||
log.Printf("[TRACE] contextPluginsCache.loadProviderSchema no factory for %s", providerAddr)
|
||||
var diags tfdiags.Diagnostics
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
"Schema request for unavailable provider",
|
||||
fmt.Sprintf("OpenTofu needs to load a schema for provider %s, but that provider is not available in this execution context. This is a bug in OpenTofu.", providerAddr),
|
||||
))
|
||||
return &providers.GetProviderSchemaResponse{
|
||||
Diagnostics: diags,
|
||||
}
|
||||
}
|
||||
|
||||
provider, err := factory()
|
||||
if err != nil {
|
||||
log.Printf("[TRACE] contextPluginsCache.loadProviderSchema failed to start %s", providerAddr)
|
||||
var diags tfdiags.Diagnostics
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
"Failed to start provider",
|
||||
fmt.Sprintf("Unable to start provider %q to fetch its schema: %s.", providerAddr, tfdiags.FormatError(err)),
|
||||
))
|
||||
return &providers.GetProviderSchemaResponse{
|
||||
Diagnostics: diags,
|
||||
}
|
||||
}
|
||||
defer provider.Close(ctx)
|
||||
|
||||
log.Printf("[TRACE] contextPluginsCache.loadProviderSchema GetProviderSchema starting for %s", providerAddr)
|
||||
resp := provider.GetProviderSchema(ctx)
|
||||
log.Printf("[TRACE] contextPluginsCache.loadProviderSchema GetProviderSchema completed for %s", providerAddr)
|
||||
// We'll also add any schema validation errors into the response, so that
|
||||
// callers have only one place to check for all possible errors. This
|
||||
// does mean that the errors about the response are embedded in that same
|
||||
// response, which is a little weird but also true for any errors that
|
||||
// could be returned by the provider itself, and so callers need to be
|
||||
// prepared for that situation anyway.
|
||||
resp.Diagnostics = append(resp.Diagnostics, validateProviderSchemaResponse(providerAddr, &resp)...)
|
||||
return &resp
|
||||
}
|
||||
43
internal/tofu/context_plugins_cache_test.go
Normal file
43
internal/tofu/context_plugins_cache_test.go
Normal file
@@ -0,0 +1,43 @@
|
||||
// Copyright (c) The OpenTofu Authors
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
// Copyright (c) 2023 HashiCorp, Inc.
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
package tofu
|
||||
|
||||
import (
|
||||
"maps"
|
||||
"testing"
|
||||
|
||||
"github.com/opentofu/opentofu/internal/addrs"
|
||||
"github.com/opentofu/opentofu/internal/providers"
|
||||
)
|
||||
|
||||
// preloadAllProviderSchemasForUnitTest is a unit-testing-only helper method
|
||||
// that simulates the effect of calling [contextPluginsCache.LoadProviderSchemas]
|
||||
// with a configuration that makes use of all of the providers that are
|
||||
// available in this [contextPlugins] object.
|
||||
//
|
||||
// Unlike the real LoadProviderSchemas method, this one does its work in the
|
||||
// foreground and blocks until all schemas have been loaded.
|
||||
//
|
||||
// This is only for use in unit tests for components that typically expect
|
||||
// that some other part of the system will have preloaded the schemas they
|
||||
// need. It should not be used in context tests because the exported entrypoints
|
||||
// of [Context] are supposed to arrange themselves for schemas to be loaded.
|
||||
func (c *contextPluginsCache) preloadAllProviderSchemasForUnitTest(t *testing.T, factories map[addrs.Provider]providers.Factory) {
|
||||
c.providerSchemasMu.Lock()
|
||||
defer c.providerSchemasMu.Unlock()
|
||||
|
||||
if c.providerSchemas == nil {
|
||||
c.providerSchemas = make(map[addrs.Provider]*providers.GetProviderSchemaResponse)
|
||||
}
|
||||
|
||||
// Since we're reusing the same concurrent-loading helper the normal
|
||||
// load method uses, we'll still do the channel-related ceremony here.
|
||||
results := make(chan providerSchemaLoadResult)
|
||||
go c.loadSchemasForProviders(t.Context(), maps.Keys(factories), factories, results)
|
||||
for result := range results {
|
||||
c.providerSchemas[result.providerAddr] = result.response
|
||||
}
|
||||
}
|
||||
@@ -6,6 +6,8 @@
|
||||
package tofu
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/zclconf/go-cty/cty"
|
||||
|
||||
"github.com/opentofu/opentofu/internal/addrs"
|
||||
@@ -24,7 +26,7 @@ import (
|
||||
// Each call to this function produces an entirely-separate set of objects,
|
||||
// so the caller can feel free to modify the returned value to further
|
||||
// customize the mocks contained within.
|
||||
func simpleMockPluginLibrary() *contextPlugins {
|
||||
func simpleMockPluginLibrary(t *testing.T) *contextPlugins {
|
||||
// We create these out here, rather than in the factory functions below,
|
||||
// because we want each call to the factory to return the _same_ instance,
|
||||
// so that test code can customize it before passing this component
|
||||
@@ -43,6 +45,7 @@ func simpleMockPluginLibrary() *contextPlugins {
|
||||
},
|
||||
},
|
||||
}
|
||||
ret.preloadAllProviderSchemasForUnitTest(t)
|
||||
return ret
|
||||
}
|
||||
|
||||
@@ -85,3 +88,16 @@ func simpleTestSchema() *configschema.Block {
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// preloadAllProviderSchemasForUnitTest is a unit-testing-only helper method
|
||||
// that simulates the effect of calling [contextPlugins.LoadProviderSchemas]
|
||||
// with a configuration that makes use of all of the providers that are
|
||||
// available in this [contextPlugins] object.
|
||||
//
|
||||
// This is only for use in unit tests for components that typically expect
|
||||
// that some other part of the system will have preloaded the schemas they
|
||||
// need. It should not be used in context tests because the exported entrypoints
|
||||
// of [Context] are supposed to arrange themselves for schemas to be loaded.
|
||||
func (cp *contextPlugins) preloadAllProviderSchemasForUnitTest(t *testing.T) {
|
||||
cp.cache.preloadAllProviderSchemasForUnitTest(t, cp.providerFactories)
|
||||
}
|
||||
|
||||
@@ -31,6 +31,12 @@ import (
|
||||
func (c *Context) Validate(ctx context.Context, config *configs.Config) tfdiags.Diagnostics {
|
||||
defer c.acquireRun("validate")()
|
||||
|
||||
// We'll get this started as soon as possible so that this I/O bound work
|
||||
// can run concurrently with some CPU-bound work we're about to do. The
|
||||
// next attempt to access schemas will block until the background task
|
||||
// started by this call has completed.
|
||||
c.plugins.LoadProviderSchemas(ctx, config, nil)
|
||||
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
ctx, span := tracing.Tracer().Start(
|
||||
|
||||
@@ -183,7 +183,12 @@ func (c *BuiltinEvalContext) Provider(addr addrs.AbsProviderConfig, key addrs.In
|
||||
}
|
||||
|
||||
func (c *BuiltinEvalContext) ProviderSchema(ctx context.Context, addr addrs.AbsProviderConfig) (providers.ProviderSchema, error) {
|
||||
return c.Plugins.ProviderSchema(ctx, addr.Provider)
|
||||
// FIXME: Change the signature of this function to return diagnostics
|
||||
// instead of an error. For now we just turn the diagnostics into an
|
||||
// error, which means we'd discard any warnings that are not also
|
||||
// accompanied by at least one error.
|
||||
schema, diags := c.Plugins.ProviderSchema(addr.Provider)
|
||||
return schema, diags.Err()
|
||||
}
|
||||
|
||||
func (c *BuiltinEvalContext) CloseProvider(addr addrs.AbsProviderConfig) error {
|
||||
|
||||
@@ -6,7 +6,6 @@
|
||||
package tofu
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
@@ -926,8 +925,8 @@ func (d *evaluationStateData) GetResource(addr addrs.Resource, rng tfdiags.Sourc
|
||||
|
||||
func (d *evaluationStateData) getResourceSchema(addr addrs.Resource, providerAddr addrs.Provider) *configschema.Block {
|
||||
// TODO: Plumb a useful context.Context through to here.
|
||||
schema, _, err := d.Evaluator.Plugins.ResourceTypeSchema(context.TODO(), providerAddr, addr.Mode, addr.Type)
|
||||
if err != nil {
|
||||
schema, _, diags := d.Evaluator.Plugins.ResourceTypeSchema(providerAddr, addr.Mode, addr.Type)
|
||||
if diags.HasErrors() {
|
||||
// We have plenty of other codepaths that will detect and report
|
||||
// schema lookup errors before we'd reach this point, so we'll just
|
||||
// treat a failure here the same as having no schema.
|
||||
|
||||
@@ -6,7 +6,6 @@
|
||||
package tofu
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
|
||||
@@ -238,19 +237,14 @@ func (d *evaluationStateData) staticValidateResourceReference(modCfg *configs.Co
|
||||
})
|
||||
}
|
||||
|
||||
// TODO: Plugin a suitable context.Context through to here.
|
||||
providerFqn := modCfg.Module.ProviderForLocalConfig(cfg.ProviderConfigAddr())
|
||||
schema, _, err := d.Evaluator.Plugins.ResourceTypeSchema(context.TODO(), providerFqn, addr.Mode, addr.Type)
|
||||
if err != nil {
|
||||
schema, _, schemaDiags := d.Evaluator.Plugins.ResourceTypeSchema(providerFqn, addr.Mode, addr.Type)
|
||||
diags = diags.Append(schemaDiags)
|
||||
if schemaDiags.HasErrors() {
|
||||
// Prior validation should've taken care of a schema lookup error,
|
||||
// so we should never get here but we'll handle it here anyway for
|
||||
// robustness.
|
||||
diags = diags.Append(&hcl.Diagnostic{
|
||||
Severity: hcl.DiagError,
|
||||
Summary: `Failed provider schema lookup`,
|
||||
Detail: fmt.Sprintf(`Couldn't load schema for %s resource type %q in %s: %s.`, modeAdjective, addr.Type, providerFqn.String(), err),
|
||||
Subject: rng.ToHCL().Ptr(),
|
||||
})
|
||||
return diags
|
||||
}
|
||||
|
||||
if schema == nil {
|
||||
|
||||
@@ -56,7 +56,7 @@ func TestApplyGraphBuilder(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-basic"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
}
|
||||
|
||||
g, err := b.Build(t.Context(), addrs.RootModuleInstance)
|
||||
@@ -120,7 +120,7 @@ func TestApplyGraphBuilder_depCbd(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-dep-cbd"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -193,7 +193,7 @@ func TestApplyGraphBuilder_doubleCBD(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-double-cbd"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
}
|
||||
|
||||
g, err := b.Build(t.Context(), addrs.RootModuleInstance)
|
||||
@@ -289,7 +289,7 @@ func TestApplyGraphBuilder_destroyStateOnly(t *testing.T) {
|
||||
Config: testModule(t, "empty"),
|
||||
Changes: changes,
|
||||
State: state,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
}
|
||||
|
||||
g, diags := b.Build(t.Context(), addrs.RootModuleInstance)
|
||||
@@ -352,7 +352,7 @@ func TestApplyGraphBuilder_destroyCount(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-count"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -416,7 +416,7 @@ func TestApplyGraphBuilder_moduleDestroy(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-module-destroy"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -453,7 +453,7 @@ func TestApplyGraphBuilder_targetModule(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-target-module"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
Targets: []addrs.Targetable{
|
||||
addrs.RootModuleInstance.Child("child2", addrs.NoKey),
|
||||
},
|
||||
@@ -488,7 +488,7 @@ func TestApplyGraphBuilder_excludeModule(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-target-module"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
Excludes: []addrs.Targetable{
|
||||
addrs.RootModuleInstance.Child("child2", addrs.NoKey),
|
||||
},
|
||||
@@ -586,7 +586,7 @@ func TestApplyGraphBuilder_updateFromOrphan(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-orphan-update"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -688,7 +688,7 @@ func TestApplyGraphBuilder_updateFromCBDOrphan(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-apply-orphan-update"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -739,7 +739,7 @@ func TestApplyGraphBuilder_orphanedWithProvider(t *testing.T) {
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-orphan-alias"),
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
|
||||
@@ -783,6 +783,7 @@ func TestApplyGraphBuilder_withChecks(t *testing.T) {
|
||||
plugins := newContextPlugins(map[addrs.Provider]providers.Factory{
|
||||
addrs.NewDefaultProvider("aws"): providers.FactoryFixed(awsProvider),
|
||||
}, nil)
|
||||
plugins.preloadAllProviderSchemasForUnitTest(t)
|
||||
|
||||
b := &ApplyGraphBuilder{
|
||||
Config: testModule(t, "apply-with-checks"),
|
||||
|
||||
@@ -37,6 +37,7 @@ func TestPlanGraphBuilder(t *testing.T) {
|
||||
addrs.NewDefaultProvider("aws"): providers.FactoryFixed(awsProvider),
|
||||
addrs.NewDefaultProvider("openstack"): providers.FactoryFixed(openstackProvider),
|
||||
}, nil)
|
||||
plugins.preloadAllProviderSchemasForUnitTest(t)
|
||||
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-plan-basic"),
|
||||
@@ -80,6 +81,7 @@ func TestPlanGraphBuilder_dynamicBlock(t *testing.T) {
|
||||
plugins := newContextPlugins(map[addrs.Provider]providers.Factory{
|
||||
addrs.NewDefaultProvider("test"): providers.FactoryFixed(provider),
|
||||
}, nil)
|
||||
plugins.preloadAllProviderSchemasForUnitTest(t)
|
||||
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-plan-dynblock"),
|
||||
@@ -136,6 +138,7 @@ func TestPlanGraphBuilder_attrAsBlocks(t *testing.T) {
|
||||
plugins := newContextPlugins(map[addrs.Provider]providers.Factory{
|
||||
addrs.NewDefaultProvider("test"): providers.FactoryFixed(provider),
|
||||
}, nil)
|
||||
plugins.preloadAllProviderSchemasForUnitTest(t)
|
||||
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-plan-attr-as-blocks"),
|
||||
@@ -177,7 +180,7 @@ test_thing.b (expand)
|
||||
func TestPlanGraphBuilder_targetModule(t *testing.T) {
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-plan-target-module-provider"),
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
Targets: []addrs.Targetable{
|
||||
addrs.RootModuleInstance.Child("child2", addrs.NoKey),
|
||||
},
|
||||
@@ -198,7 +201,7 @@ func TestPlanGraphBuilder_targetModule(t *testing.T) {
|
||||
func TestPlanGraphBuilder_excludeModule(t *testing.T) {
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "graph-builder-plan-target-module-provider"),
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
Excludes: []addrs.Targetable{
|
||||
addrs.RootModuleInstance.Child("child1", addrs.NoKey),
|
||||
},
|
||||
@@ -222,6 +225,7 @@ func TestPlanGraphBuilder_forEach(t *testing.T) {
|
||||
plugins := newContextPlugins(map[addrs.Provider]providers.Factory{
|
||||
addrs.NewDefaultProvider("aws"): providers.FactoryFixed(awsProvider),
|
||||
}, nil)
|
||||
plugins.preloadAllProviderSchemasForUnitTest(t)
|
||||
|
||||
b := &PlanGraphBuilder{
|
||||
Config: testModule(t, "plan-for-each"),
|
||||
|
||||
@@ -92,6 +92,15 @@ func loadSchemas(ctx context.Context, config *configs.Config, state *states.Stat
|
||||
func loadProviderSchemas(ctx context.Context, schemas map[addrs.Provider]providers.ProviderSchema, config *configs.Config, state *states.State, plugins *contextPlugins) tfdiags.Diagnostics {
|
||||
var diags tfdiags.Diagnostics
|
||||
|
||||
// This function is used by some callers that predate our current design
|
||||
// of separating the loading of provider schemas from their later use,
|
||||
// and so to avoid disrupting them too much we'll start a background load
|
||||
// here (which will complete quickly if something has already made a
|
||||
// similar request earlier) and then immediately block until we can
|
||||
// collect the results into our own structure below. This is odd and we
|
||||
// should eventually rework this.
|
||||
plugins.LoadProviderSchemas(ctx, config, state)
|
||||
|
||||
ensure := func(fqn addrs.Provider) {
|
||||
name := fqn.String()
|
||||
|
||||
@@ -100,7 +109,7 @@ func loadProviderSchemas(ctx context.Context, schemas map[addrs.Provider]provide
|
||||
}
|
||||
|
||||
log.Printf("[TRACE] LoadSchemas: retrieving schema for provider type %q", name)
|
||||
schema, err := plugins.ProviderSchema(ctx, fqn)
|
||||
schema, err := plugins.ProviderSchema(fqn)
|
||||
if err != nil {
|
||||
// We'll put a stub in the map so we won't re-attempt this on
|
||||
// future calls, which would then repeat the same error message
|
||||
@@ -176,3 +185,64 @@ func loadProvisionerSchemas(ctx context.Context, schemas map[string]*configschem
|
||||
|
||||
return diags
|
||||
}
|
||||
|
||||
// validateProviderSchemaResponse verifies that the given provider schema
|
||||
// response is valid, returning error diagnostics if not.
|
||||
func validateProviderSchemaResponse(providerAddr addrs.Provider, resp *providers.GetProviderSchemaResponse) tfdiags.Diagnostics {
|
||||
var diags tfdiags.Diagnostics
|
||||
const summary = "Invalid provider schema"
|
||||
|
||||
if resp.Provider.Version < 0 {
|
||||
// We're not using the version numbers here yet, but we'll check
|
||||
// for validity anyway in case we start using them in future.
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
summary,
|
||||
fmt.Sprintf("Provider %s has an invalid negative schema version for its configuration blocks, which is a bug in the provider.", providerAddr),
|
||||
))
|
||||
}
|
||||
|
||||
for t, r := range resp.ResourceTypes {
|
||||
if err := r.Block.InternalValidate(); err != nil {
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
summary,
|
||||
fmt.Sprintf(
|
||||
"Provider %s has an invalid schema for managed resource type %q, which is a bug in the provider: %s.",
|
||||
providerAddr, t, tfdiags.FormatError(err),
|
||||
),
|
||||
))
|
||||
}
|
||||
if r.Version < 0 {
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
summary,
|
||||
fmt.Sprintf("Provider %s has an invalid negative schema version for managed resource type %q, which is a bug in the provider.", providerAddr, t),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
for t, d := range resp.DataSources {
|
||||
if err := d.Block.InternalValidate(); err != nil {
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
summary,
|
||||
fmt.Sprintf(
|
||||
"Provider %s has an invalid schema for data resource type %q, which is a bug in the provider: %s.",
|
||||
providerAddr, t, tfdiags.FormatError(err),
|
||||
),
|
||||
))
|
||||
}
|
||||
if d.Version < 0 {
|
||||
// We're not using the version numbers here yet, but we'll check
|
||||
// for validity anyway in case we start using them in future.
|
||||
diags = diags.Append(tfdiags.Sourceless(
|
||||
tfdiags.Error,
|
||||
summary,
|
||||
fmt.Sprintf("Provider %s has an invalid negative schema version for data resource type %q, which is a bug in the provider.", providerAddr, t),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
return diags
|
||||
}
|
||||
|
||||
@@ -36,6 +36,14 @@ func simpleTestSchemas() *Schemas {
|
||||
// The intended use for this is in testing components that use schemas to
|
||||
// drive other behavior, such as reference analysis during graph construction,
|
||||
// but that don't actually need to interact with providers otherwise.
|
||||
//
|
||||
// This function automatically preloads the schemas from the providers into
|
||||
// the cache of the returned object, and so this function is appropriate for
|
||||
// unit testing of individual components that typically expect someone else
|
||||
// to have already loaded the schemas, but should not be used for context
|
||||
// tests where schemas are supposed to be loaded on entry to the Context
|
||||
// methods, or any other situation where the correctness of the code under
|
||||
// test includes that it explicitly arranges for schemas to be loaded.
|
||||
func schemaOnlyProvidersForTesting(schemas map[addrs.Provider]providers.ProviderSchema, t *testing.T) *contextPlugins {
|
||||
factories := make(map[addrs.Provider]providers.Factory, len(schemas))
|
||||
|
||||
@@ -51,5 +59,7 @@ func schemaOnlyProvidersForTesting(schemas map[addrs.Provider]providers.Provider
|
||||
}
|
||||
}
|
||||
|
||||
return newContextPlugins(factories, nil)
|
||||
ret := newContextPlugins(factories, nil)
|
||||
ret.preloadAllProviderSchemasForUnitTest(t)
|
||||
return ret
|
||||
}
|
||||
|
||||
@@ -60,6 +60,10 @@ func (c *Context) TestContext(config *configs.Config, state *states.State, plan
|
||||
// function, but no data or changes from the embedded plan is referenced in
|
||||
// this function.
|
||||
func (ctx *TestContext) EvaluateAgainstState(run *moduletest.Run) {
|
||||
// We'll start loading the schemas we'll need in the background now so
|
||||
// that they are most likely to be ready when we eventually need them.
|
||||
ctx.Context.plugins.LoadProviderSchemas(context.TODO(), ctx.Config, ctx.State)
|
||||
|
||||
defer ctx.acquireRun("evaluate")()
|
||||
ctx.evaluate(ctx.State.SyncWrapper(), plans.NewChanges().SyncWrapper(), run, walkApply)
|
||||
}
|
||||
@@ -67,6 +71,10 @@ func (ctx *TestContext) EvaluateAgainstState(run *moduletest.Run) {
|
||||
// EvaluateAgainstPlan processes the assertions inside the provided
|
||||
// configs.TestRun against the embedded plan and state.
|
||||
func (ctx *TestContext) EvaluateAgainstPlan(run *moduletest.Run) {
|
||||
// We'll start loading the schemas we'll need in the background now so
|
||||
// that they are most likely to be ready when we eventually need them.
|
||||
ctx.Context.plugins.LoadProviderSchemas(context.TODO(), ctx.Config, ctx.Plan.PriorState)
|
||||
|
||||
defer ctx.acquireRun("evaluate")()
|
||||
ctx.evaluate(ctx.State.SyncWrapper(), ctx.Plan.Changes.SyncWrapper(), run, walkPlan)
|
||||
}
|
||||
|
||||
@@ -48,6 +48,11 @@ type GraphNodeAttachProvisionerSchema interface {
|
||||
// GraphNodeAttachResourceSchema, GraphNodeAttachProviderConfigSchema, or
|
||||
// GraphNodeAttachProvisionerSchema, looks up the needed schemas for each
|
||||
// and then passes them to a method implemented by the node.
|
||||
//
|
||||
// Some other part of the system must have called
|
||||
// [contextplugins.LoadProviderSchemas] with the same configuration prior to
|
||||
// executing this graph transformer, so that the needed schemas will become
|
||||
// available for this transformer to use.
|
||||
type AttachSchemaTransformer struct {
|
||||
Plugins *contextPlugins
|
||||
Config *configs.Config
|
||||
@@ -68,10 +73,9 @@ func (t *AttachSchemaTransformer) Transform(ctx context.Context, g *Graph) error
|
||||
typeName := addr.Resource.Type
|
||||
providerFqn := tv.Provider()
|
||||
|
||||
// TODO: Plumb a useful context.Context through to here.
|
||||
schema, version, err := t.Plugins.ResourceTypeSchema(ctx, providerFqn, mode, typeName)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to read schema for %s in %s: %w", addr, providerFqn, err)
|
||||
schema, version, diags := t.Plugins.ResourceTypeSchema(providerFqn, mode, typeName)
|
||||
if diags.HasErrors() {
|
||||
return diags.Err()
|
||||
}
|
||||
if schema == nil {
|
||||
log.Printf("[ERROR] AttachSchemaTransformer: No resource schema available for %s", addr)
|
||||
@@ -83,10 +87,9 @@ func (t *AttachSchemaTransformer) Transform(ctx context.Context, g *Graph) error
|
||||
|
||||
if tv, ok := v.(GraphNodeAttachProviderConfigSchema); ok {
|
||||
providerAddr := tv.ProviderAddr()
|
||||
// TODO: Plumb a useful context.Context through to here.
|
||||
schema, err := t.Plugins.ProviderConfigSchema(ctx, providerAddr.Provider)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to read provider configuration schema for %s: %w", providerAddr.Provider, err)
|
||||
schema, diags := t.Plugins.ProviderConfigSchema(providerAddr.Provider)
|
||||
if diags.HasErrors() {
|
||||
return diags.Err()
|
||||
}
|
||||
if schema == nil {
|
||||
log.Printf("[ERROR] AttachSchemaTransformer: No provider config schema available for %s", providerAddr)
|
||||
|
||||
@@ -22,7 +22,7 @@ func cbdTestGraph(t *testing.T, mod string, changes *plans.Changes, state *state
|
||||
applyBuilder := &ApplyGraphBuilder{
|
||||
Config: module,
|
||||
Changes: changes,
|
||||
Plugins: simpleMockPluginLibrary(),
|
||||
Plugins: simpleMockPluginLibrary(t),
|
||||
State: state,
|
||||
}
|
||||
g, err := (&BasicGraphBuilder{
|
||||
|
||||
Reference in New Issue
Block a user