engine, lang: Modern exported resources

I've been waiting to write this patch for a long time. I firmly believe
that the idea of "exported resources" was truly a brilliant one, but
which was never even properly understood by its original inventors! This
patch set aims to show how it should have been done.

The main differences are:

* Real-time modelling, since "once per run" makes no sense.
* Filter with code/functions not language syntax.
* Directed exporting to limit the intended recipients.

The next step is to add more "World" reading and filtering functions to
make it easy and expressive to make your selection of resources to
collect!
This commit is contained in:
James Shubin
2025-03-24 18:54:06 -04:00
parent 955112f64f
commit 045b29291e
24 changed files with 2367 additions and 312 deletions

View File

@@ -181,6 +181,18 @@ func (obj *Engine) Process(ctx context.Context, vertex pgraph.Vertex) error {
refreshableRes.SetRefresh(refresh) // tell the resource
}
// Run the exported resource exporter!
var exportOK bool
var exportErr error
wg := &sync.WaitGroup{}
wg.Add(1)
// (Run this concurrently with the CheckApply related stuff below...)
go func() {
defer wg.Done()
// doesn't really need to be in parallel, but we can...
exportOK, exportErr = obj.Exporter.Export(ctx, res)
}()
// Check cached state, to skip CheckApply, but can't skip if refreshing!
// If the resource doesn't implement refresh, skip the refresh test.
// FIXME: if desired, check that we pass through refresh notifications!
@@ -190,6 +202,13 @@ func (obj *Engine) Process(ctx context.Context, vertex pgraph.Vertex) error {
} else if noop && (refresh && isRefreshableRes) { // had a refresh to do w/ noop!
checkOK, err = false, nil // therefore the state is wrong
} else if res.MetaParams().Hidden {
// We're not running CheckApply
if obj.Debug {
obj.Logf("%s: Hidden", res)
}
checkOK, err = true, nil // default
} else {
// run the CheckApply!
if obj.Debug {
@@ -201,6 +220,13 @@ func (obj *Engine) Process(ctx context.Context, vertex pgraph.Vertex) error {
obj.Logf("%s: CheckApply(%t): Return(%t, %s)", res, !noop, checkOK, engineUtil.CleanError(err))
}
}
wg.Wait()
checkOK = checkOK && exportOK // always combine
if err == nil { // If CheckApply didn't error, look at exportOK.
// This is because if CheckApply errors we don't need to care or
// tell anyone about an exporting error.
err = exportErr
}
if checkOK && err != nil { // should never return this way
return fmt.Errorf("%s: resource programming error: CheckApply(%t): %t, %+v", res, !noop, checkOK, err)
@@ -304,14 +330,24 @@ func (obj *Engine) Worker(vertex pgraph.Vertex) error {
}
// initialize or reinitialize the meta state for this resource uid
obj.mlock.Lock()
if _, exists := obj.metas[engine.PtrUID(res)]; !exists || res.MetaParams().Reset {
obj.metas[engine.PtrUID(res)] = &engine.MetaState{
CheckApplyRetry: res.MetaParams().Retry, // lookup the retry value
}
// if we're using a Hidden resource, we don't support this feature
// TODO: should we consider supporting it? is it really necessary?
// XXX: to support this for Hidden, we'd need to handle dupe names
metas := &engine.MetaState{
CheckApplyRetry: res.MetaParams().Retry, // lookup the retry value
}
if !res.MetaParams().Hidden {
// Skip this if Hidden since we can have a hidden res that has
// the same kind+name as a regular res, and this would conflict.
obj.mlock.Lock()
if _, exists := obj.metas[engine.PtrUID(res)]; !exists || res.MetaParams().Reset {
obj.metas[engine.PtrUID(res)] = &engine.MetaState{
CheckApplyRetry: res.MetaParams().Retry, // lookup the retry value
}
}
metas = obj.metas[engine.PtrUID(res)] // handle
obj.mlock.Unlock()
}
metas := obj.metas[engine.PtrUID(res)] // handle
obj.mlock.Unlock()
//defer close(obj.state[vertex].stopped) // done signal
@@ -376,10 +412,21 @@ func (obj *Engine) Worker(vertex pgraph.Vertex) error {
delay = 0 // reset
continue
}
} else if res.MetaParams().Hidden {
// We're not running Watch
if obj.Debug {
obj.Logf("%s: Hidden", res)
}
obj.state[vertex].cuid.StartTimer() // TODO: Should we do this?
err = obj.state[vertex].hidden(obj.state[vertex].doneCtx)
obj.state[vertex].cuid.StopTimer() // TODO: Should we do this?
} else if interval := res.MetaParams().Poll; interval > 0 { // poll instead of watching :(
obj.state[vertex].cuid.StartTimer()
err = obj.state[vertex].poll(obj.state[vertex].doneCtx, interval)
obj.state[vertex].cuid.StopTimer() // clean up nicely
} else {
obj.state[vertex].cuid.StartTimer()
if obj.Debug {

View File

@@ -59,9 +59,12 @@ type Engine struct {
Version string
Hostname string
// Break off separate logical pieces into chunks where possible.
Converger *converger.Coordinator
Local *local.API
World engine.World
Exporter *Exporter
Local *local.API
World engine.World
// Prefix is a unique directory prefix which can be used. It should be
// created if needed.
@@ -85,6 +88,7 @@ type Engine struct {
paused bool // are we paused?
fastPause bool
isClosing bool // are we shutting down?
}
// Init initializes the internal structures and starts this the graph running.
@@ -116,7 +120,7 @@ func (obj *Engine) Init() error {
obj.wlock = &sync.Mutex{}
obj.mlock = &sync.Mutex{}
obj.metas = make(map[engine.ResPtrUID]*engine.MetaState)
obj.metas = make(map[engine.ResPtrUID]*engine.MetaState) // don't include .Hidden res
obj.slock = &sync.Mutex{}
obj.semas = make(map[string]*semaphore.Semaphore)
@@ -125,6 +129,18 @@ func (obj *Engine) Init() error {
obj.paused = true // start off true, so we can Resume after first Commit
obj.Exporter = &Exporter{
World: obj.World,
Debug: obj.Debug,
Logf: func(format string, v ...interface{}) {
// TODO: is this a sane prefix to use here?
obj.Logf("export: "+format, v...)
},
}
if err := obj.Exporter.Init(); err != nil {
return err
}
return nil
}
@@ -188,6 +204,12 @@ func (obj *Engine) Commit() error {
if !ok { // should not happen, previously validated
return fmt.Errorf("not a Res")
}
// Skip this if Hidden since we can have a hidden res that has
// the same kind+name as a regular res, and this would conflict.
if res.MetaParams().Hidden {
continue
}
activeMetas[engine.PtrUID(res)] = struct{}{} // add
}
@@ -208,7 +230,11 @@ func (obj *Engine) Commit() error {
return fmt.Errorf("the Res state already exists")
}
activeMetas[engine.PtrUID(res)] = struct{}{} // add
// Skip this if Hidden since we can have a hidden res that has
// the same kind+name as a regular res, and this would conflict.
if !res.MetaParams().Hidden {
activeMetas[engine.PtrUID(res)] = struct{}{} // add
}
if obj.Debug {
obj.Logf("Validate(%s)", res)
@@ -299,7 +325,12 @@ func (obj *Engine) Commit() error {
if !ok { // should not happen, previously validated
return fmt.Errorf("not a Res")
}
delete(activeMetas, engine.PtrUID(res))
// Skip this if Hidden since we can have a hidden res that has
// the same kind+name as a regular res, and this would conflict.
if !res.MetaParams().Hidden {
delete(activeMetas, engine.PtrUID(res))
}
// wait for exit before starting new graph!
close(obj.state[vertex].removeDone) // causes doneCtx to cancel
@@ -501,6 +532,7 @@ func (obj *Engine) Pause(fastPause bool) error {
// actually just a Load of an empty graph and a Commit. It waits for all the
// resources to exit before returning.
func (obj *Engine) Shutdown() error {
obj.isClosing = true
emptyGraph, reterr := pgraph.NewGraph("empty")
// this is a graph switch (graph sync) that switches to an empty graph!
@@ -517,6 +549,15 @@ func (obj *Engine) Shutdown() error {
return reterr
}
// IsClosing tells the caller if a Shutdown() was run. This is helpful so that
// the graph can behave slightly differently when receiving the final empty
// graph. This is because it's empty because we passed one to unload everything,
// not because the user actually removed all resources. We may want to preserve
// the exported state for example, and not purge it.
func (obj *Engine) IsClosing() bool {
return obj.isClosing
}
// Graph returns the running graph.
func (obj *Engine) Graph() *pgraph.Graph {
return obj.graph

349
engine/graph/exporter.go Normal file
View File

@@ -0,0 +1,349 @@
// Mgmt
// Copyright (C) James Shubin and the project contributors
// Written by James Shubin <james@shubin.ca> and the project contributors
//
// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.
//
// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.
//
// You should have received a copy of the GNU General Public License
// along with this program. If not, see <https://www.gnu.org/licenses/>.
//
// Additional permission under GNU GPL version 3 section 7
//
// If you modify this program, or any covered work, by linking or combining it
// with embedded mcl code and modules (and that the embedded mcl code and
// modules which link with this program, contain a copy of their source code in
// the authoritative form) containing parts covered by the terms of any other
// license, the licensors of this program grant you additional permission to
// convey the resulting work. Furthermore, the licensors of this program grant
// the original author, James Shubin, additional permission to update this
// additional permission if he deems it necessary to achieve the goals of this
// additional permission.
package graph
import (
"context"
"fmt"
"sync"
"github.com/purpleidea/mgmt/engine"
"github.com/purpleidea/mgmt/pgraph"
//"github.com/purpleidea/mgmt/pgraph"
engineUtil "github.com/purpleidea/mgmt/engine/util"
)
// Exporter is the main engine mechanism that sends the exported resource data
// to the World database. The code is relatively succinct, but slightly subtle.
type Exporter struct {
// Watch specifies if we want to enable the additional watch feature. It
// should probably be left off unless we're debugging something or using
// weird environments where we expect someone to mess with our res data.
Watch bool
World engine.World
Debug bool
Logf func(format string, v ...interface{})
state map[engine.ResDelete]bool // key NOT a pointer for it to be unique
prev map[engine.ResDelete]pgraph.Vertex
mutex *sync.Mutex
// watch specific variables
workerRunning bool
workerWg *sync.WaitGroup
workerCtx context.Context
workerCancel func()
}
// Init performs some initialization before first use. This is required.
func (obj *Exporter) Init() error {
obj.state = make(map[engine.ResDelete]bool)
obj.prev = make(map[engine.ResDelete]pgraph.Vertex)
obj.mutex = &sync.Mutex{}
obj.workerRunning = false
obj.workerWg = &sync.WaitGroup{}
obj.workerCtx, obj.workerCancel = context.WithCancel(context.Background())
return nil
}
// Export performs the worldly export, and then stores the resource unique ID in
// our in-memory data store. Exported resources use this tracking to know when
// to run their cleanups. If this function encounters an error, it returns
// (false, err). If it does nothing it returns (true, nil). If it does work it
// return (false, nil). These return codes match how CheckApply returns. This
// may run concurrently by multiple different resources, so as a result it must
// stay thread safe.
func (obj *Exporter) Export(ctx context.Context, res engine.Res) (bool, error) {
// As a result of running this operation in roughly the same places that
// the usual CheckApply step would run, we end up with a more nuanced
// and mature "exported resources" model than what was ever possible
// with other tools. We can now "wait" (via the resource graph
// dependencies) to run an export until an earlier resource dependency
// step has run. We can also programmatically "un-export" a resource by
// publishing a subsequent resource graph which either removes that
// Export flag or the entire resource. The one downside is that
// exporting to the database happens in multiple transactions rather
// than a batched bolus, but this is more appropriate because we're now
// more accurately modelling real-time systems, and this bandwidth is
// not a significant amount anyways. Lastly, we make sure to not run the
// purge when we ^C, since it should be safe to shutdown without killing
// all the data we left there.
if res.MetaParams().Noop {
return true, nil // did nothing
}
exports := res.MetaParams().Export
if len(exports) == 0 {
return true, nil // did nothing
}
// It's OK to check the cache here instead of re-sending via the World
// API and so on, because the only way the Res data would change in
// World is if (1) someone messed with etcd, which we'd see with Watch,
// or (2) if the Res data changed because we have a new resource graph.
// If we have a new resource graph, then any changed elements will get
// pruned from this state cache via the Prune method, which helps us.
// If send/recv or any other weird resource method changes things, then
// we also want to invalidate the state cache.
state := true
// TODO: This recv code is untested!
if r, ok := res.(engine.RecvableRes); ok {
for _, v := range r.Recv() { // map[string]*Send
// XXX: After we read the changed value, will it persist?
state = state && !v.Changed
}
}
obj.mutex.Lock()
for _, ptrUID := range obj.ptrUID(res) {
b := obj.state[*ptrUID] // no need to check if exists
state = state && b // if any are false, it's all false
}
obj.mutex.Unlock()
if state {
return true, nil // state OK!
}
// XXX: Do we want to change any metaparams when we export?
// XXX: Do we want to change any metaparams when we collect?
b64, err := obj.resToB64(res)
if err != nil {
return false, err
}
resourceExports := []*engine.ResExport{}
duplicates := make(map[string]struct{})
for _, export := range exports {
//ptrUID := engine.ResDelete{
// Kind: res.Kind(),
// Name: res.Name(),
// Host: export,
//}
if export == "*" {
export = "" // XXX: use whatever means "all"
}
if _, exists := duplicates[export]; exists {
continue
}
duplicates[export] = struct{}{}
// skip this check since why race it or split the resource...
//if stateOK := obj.state[ptrUID]; stateOK {
// // rare that we'd have a split of some of these from a
// // single resource updated and others already fine, but
// // might as well do the check since it's cheap...
// continue
//}
resExport := &engine.ResExport{
Kind: res.Kind(),
Name: res.Name(),
Host: export,
Data: b64, // encoded res data
}
resourceExports = append(resourceExports, resExport)
}
// The fact that we Watch the write-only-by-us values at all, is a
// luxury that allows us to handle mischievous actors that overwrote an
// exported value. It really isn't necessary. It's the consumers that
// really need to watch.
if err := obj.worker(); err != nil {
return false, err // big error
}
// TODO: Do we want to log more information about where this exports to?
obj.Logf("%s", res)
// XXX: Add a TTL if requested
b, err := obj.World.ResExport(ctx, resourceExports) // do it!
if err != nil {
return false, err
}
obj.mutex.Lock()
defer obj.mutex.Unlock()
// NOTE: The Watch() method *must* invalidate this state if it changes.
// This is only pertinent if we're using the luxury Watch add-ons.
for _, ptrUID := range obj.ptrUID(res) {
obj.state[*ptrUID] = true // state OK!
}
return b, nil
}
// Prune removes any exports which are no longer actively being presented in the
// resource graph. This cleans things up between graph swaps. This should NOT
// run if we're shutting down cleanly. Keep in mind that this must act on the
// new graph which is available by "Commit", not before we're ready to "Commit".
func (obj *Exporter) Prune(ctx context.Context, graph *pgraph.Graph) error {
// mutex should be optional since this should only run when graph paused
obj.mutex.Lock()
defer obj.mutex.Unlock()
// make searching faster by initially storing it all in a map
m := make(map[engine.ResDelete]pgraph.Vertex) // key is NOT a pointer
for _, v := range graph.Vertices() {
res, ok := v.(engine.Res)
if !ok { // should not happen
return fmt.Errorf("not a Res")
}
for _, ptrUID := range obj.ptrUID(res) { // skips non-export things
m[*ptrUID] = v
}
}
resourceDeletes := []*engine.ResDelete{}
for k := range obj.state {
v, exists := m[k] // exists means it's in the graph
prev := obj.prev[k]
obj.prev[k] = v // may be nil
if exists && v != prev { // pointer compare to old vertex
// Here we have a Res that previously existed under the
// same kind/name/host. We need to invalidate the state
// only if it's a different Res than the previous one!
// If we do this erroneously, it causes extra traffic.
obj.state[k] = false // do this only if the Res is NEW
continue // skip it, it's staying
}
delete(obj.state, k) // it's gone!
resourceDeletes = append(resourceDeletes, &k)
}
if len(resourceDeletes) == 0 {
return nil
}
obj.Logf("prune: %d exports", len(resourceDeletes))
for _, x := range resourceDeletes {
obj.Logf("prune: %s to %s", engine.Repr(x.Kind, x.Name), x.Host)
}
// XXX: this function could optimize the grouping since we split the
// list of host entries out from the kind/name since we can't have a
// unique map key with a struct that contains a slice.
if _, err := obj.World.ResDelete(ctx, resourceDeletes); err != nil {
return err
}
return nil
}
// resToB64 is a helper to refactor out this method.
func (obj *Exporter) resToB64(res engine.Res) (string, error) {
if r, ok := res.(engine.ExportableRes); ok {
return r.ToB64()
}
return engineUtil.ResToB64(res)
}
// ptrUID is a helper for this repetitive code.
func (obj *Exporter) ptrUID(res engine.Res) []*engine.ResDelete {
a := []*engine.ResDelete{}
for _, export := range res.MetaParams().Export {
if export == "*" {
export = "" // XXX: use whatever means "all"
}
ptrUID := &engine.ResDelete{
Kind: res.Kind(),
Name: res.Name(),
Host: export,
}
a = append(a, ptrUID)
}
return a
}
// worker is a helper to kick off the optional Watch workers.
func (obj *Exporter) worker() error {
if !obj.Watch {
return nil // feature is disabled
}
obj.mutex.Lock()
defer obj.mutex.Unlock()
if obj.workerRunning {
return nil // already running
}
kind := "" // watch everything
ch, err := obj.World.ResWatch(obj.workerCtx, kind) // (chan error, error)
if err != nil {
return err // big error
}
obj.workerRunning = true
obj.workerWg.Add(1)
go func() {
defer func() {
obj.mutex.Lock()
obj.workerRunning = false
obj.mutex.Unlock()
}()
defer obj.workerWg.Done()
Loop:
for {
var e error
var ok bool
select {
case e, ok = <-ch:
if !ok {
// chan closed
break Loop
}
case <-obj.workerCtx.Done():
break Loop
}
if e != nil {
// something errored... shutdown coming!
}
// event!
obj.mutex.Lock()
for k := range obj.state {
obj.state[k] = false // reset it all
}
obj.mutex.Unlock()
}
}()
return nil
}
// Shutdown cancels any running workers and waits for them to finish.
func (obj *Exporter) Shutdown() {
obj.workerCancel()
obj.workerWg.Wait()
}

View File

@@ -430,3 +430,13 @@ func (obj *State) poll(ctx context.Context, interval uint32) error {
obj.init.Event() // notify engine of an event (this can block)
}
}
// hidden is a replacement for Watch when the Hidden metaparameter is used.
func (obj *State) hidden(ctx context.Context) error {
obj.init.Running() // when started, notify engine that we're running
select {
case <-ctx.Done(): // signal for shutdown request
return nil
}
}

View File

@@ -53,6 +53,8 @@ var DefaultMetaParams = &MetaParams{
Rewatch: false,
Realize: false, // true would be more awesome, but unexpected for users
Dollar: false,
Hidden: false,
Export: []string{},
}
// MetaRes is the interface a resource must implement to support meta params.
@@ -140,6 +142,32 @@ type MetaParams struct {
// interpolate a variable name. In the rare case when it's needed, you
// can disable that check with this meta param.
Dollar bool `yaml:"dollar"`
// Hidden means that this resource will not get executed on the resource
// graph on which it is defined. This can be used as a simple boolean
// switch, or, more commonly in combination with the Export meta param
// which specifies that the resource params are exported into the shared
// database. When this is true, it does not prevent export. In fact, it
// is commonly used in combination with Export. Using this option will
// still include it in the resource graph, but it will exist there in a
// special "mode" where it will not conflict with any other identically
// named resources. It can even be used as part of an edge or via a
// send/recv receiver. It can NOT be a sending vertex. These properties
// differentiate the use of this instead of simply wrapping a resource
// in an "if" statement.
Hidden bool `yaml:"hidden"`
// Export is a list of hostnames (and/or the special "*" entry) which if
// set, will mark this resource data as intended for export to those
// hosts. This does not prevent any users of the shared data storage
// from reading these values, so if you want to guarantee secrecy, use
// the encryption primitives. This only labels the data accordingly, so
// that other hosts can know what data is available for them to collect.
// The (kind, name, host) export triple must be unique from any given
// exporter. In other words, you may not export two different instances
// of a kind+name to the same host, the exports must not conflict. On
// resource collect, this parameter is not preserved.
Export []string `yaml:"export"`
}
// Cmp compares two AutoGroupMeta structs and determines if they're equivalent.
@@ -189,6 +217,12 @@ func (obj *MetaParams) Cmp(meta *MetaParams) error {
if obj.Dollar != meta.Dollar {
return fmt.Errorf("values for Dollar are different")
}
if obj.Hidden != meta.Hidden {
return fmt.Errorf("values for Hidden are different")
}
if err := util.SortedStrSliceCompare(obj.Export, meta.Export); err != nil {
return errwrap.Wrapf(err, "values for Export are different")
}
return nil
}
@@ -208,6 +242,13 @@ func (obj *MetaParams) Validate() error {
}
}
for _, s := range obj.Export {
if s == "" {
return fmt.Errorf("export is empty")
}
}
// TODO: Should we validate the export patterns?
return nil
}
@@ -218,6 +259,11 @@ func (obj *MetaParams) Copy() *MetaParams {
sema = make([]string, len(obj.Sema))
copy(sema, obj.Sema)
}
export := []string{}
if obj.Export != nil {
export = make([]string, len(obj.Export))
copy(export, obj.Export)
}
return &MetaParams{
Noop: obj.Noop,
Retry: obj.Retry,
@@ -230,6 +276,8 @@ func (obj *MetaParams) Copy() *MetaParams {
Rewatch: obj.Rewatch,
Realize: obj.Realize,
Dollar: obj.Dollar,
Hidden: obj.Hidden,
Export: export,
}
}

View File

@@ -376,6 +376,22 @@ type CompatibleRes interface {
Merge(CompatibleRes) (CompatibleRes, error)
}
// ExportableRes allows the resource to have its own implementation of resource
// encoding, so that it can send data over the wire differently. It's unlikely
// that you will want to implement this interface for most scenarios. It may be
// useful to limit private data exposure, large data sizes, and to add more info
// to what would normally be shared.
type ExportableRes interface {
Res
// ToB64 lets the resource provide an alternative implementation of the
// usual ResToB64 method. This lets the resource omit, add, or modify
// the parameter data before it goes out over the wire.
ToB64() (string, error)
// TODO: Do we want to add a FromB64 method for decoding the Resource?
}
// YAMLRes is a resource that supports creation by unmarshalling.
type YAMLRes interface {
Res

View File

@@ -31,6 +31,7 @@ package engine
import (
"context"
"fmt"
"github.com/purpleidea/mgmt/etcd/interfaces"
"github.com/purpleidea/mgmt/etcd/scheduler"
@@ -117,11 +118,103 @@ type StrWorld interface {
// ResWorld is a world interface that lets us store, pull and watch resources in
// a distributed database.
// XXX: These API's are likely to change.
// XXX: Add optional TTL's to these API's, maybe use WithTTL(...) type options.
type ResWorld interface {
ResWatch(context.Context) (chan error, error)
ResExport(context.Context, []Res) error
// FIXME: should this method take a "filter" data struct instead of many args?
ResCollect(ctx context.Context, hostnameFilter, kindFilter []string) ([]Res, error)
// ResWatch returns a channel which produces a new value once on startup
// as soon as it is successfully connected, and once for every time it
// sees that a resource that has been exported for this hostname is
// added, deleted, or modified. If kind is specified, the watch will
// attempt to only send events relating to that resource kind. We always
// intended to only show events for resources which the watching host is
// allowed to see.
ResWatch(ctx context.Context, kind string) (chan error, error)
// ResCollect does a lookup for resource entries that have previously
// been stored for us. It returns a subset of these based on the input
// filter. It does not return a Res, since while that would be useful,
// and logical, it turns out we usually want to transport the Res data
// onwards through the function graph, and using a native string is what
// is already supported. (A native res type would just be encoded as a
// string anyways.) While it might be more "correct" to do the work to
// decode the string into a Res, the user of this function would just
// encode it back to a string anyways, and this is not very efficient.
ResCollect(ctx context.Context, filters []*ResFilter) ([]*ResOutput, error)
// ResExport stores a number of resources in the world storage system.
// The individual records should not be updated if they are identical to
// what is already present. (This is to prevent unnecessary events.) If
// this makes no changes, it returns (true, nil). If it makes a change,
// then it returns (false, nil). On any error we return (false, err).
ResExport(ctx context.Context, resourceExports []*ResExport) (bool, error)
// ResDelete deletes a number of resources in the world storage system.
// If this doesn't delete, it returns (true, nil). If it makes a delete,
// then it returns (false, nil). On any error we return (false, err).
ResDelete(ctx context.Context, resourceDeletes []*ResDelete) (bool, error)
}
// ResFilter specifies that we want to match an item with this three tuple. If
// any of these are empty, then it means to match an item with any value for
// that field.
// TODO: Future secure implementations must verify that the exported made a
// value available to that hostname. It's not enough for a host to request it.
// We can enforce this with public key encryption eventually.
type ResFilter struct {
Kind string
Name string
Host string // from this host
}
// Match returns nil on a successful match.
func (obj *ResFilter) Match(kind, name, host string) error {
if obj.Kind != "" && obj.Kind != kind {
return fmt.Errorf("kind did not match")
}
if obj.Name != "" && obj.Name != name {
return fmt.Errorf("name did not match")
}
if obj.Host != "" && obj.Host != host {
return fmt.Errorf("host did not match")
}
return nil // match!
}
// ResOutput represents a record of exported resource data which we have read
// out from the world storage system. The Data field contains an encoded version
// of the resource, and even though decoding it will get you a Kind and Name, we
// still store those values here in duplicate for them to be available before
// decoding.
type ResOutput struct {
Kind string
Name string
Host string // from this host
Data string // encoded res data
}
// ResExport represents a record of exported resource data which we want to save
// to the world storage system. The Data field contains an encoded version of
// the resource, and even though decoding it will get you a Kind and Name, we
// still store those values here in duplicate for them to be available before
// decoding. If Host is specified, then only the node with that hostname may
// access this resource. If it's empty than it may be collected by anyone. If we
// want to export to only three hosts, then we duplicate this entry three times.
// It's true that this is not an efficient use of storage space, but it maps
// logically to a future data structure where data is encrypted to the public
// key of that specific host where we wouldn't be able to de-duplicate anyways.
type ResExport struct {
Kind string
Name string
Host string // to/for this host
Data string // encoded res data
}
// ResDelete represents the uniqueness key for stored resources. As a result,
// this triple is a useful map key in various locations.
type ResDelete struct {
Kind string
Name string
Host string // to/for this host
}
// SchedulerWorld is an interface that has to do with distributed scheduling.