diff --git a/README.md b/README.md index 6026e67a7..996035b7c 100644 --- a/README.md +++ b/README.md @@ -230,6 +230,7 @@ Documentation for Snap will be kept in this repository for now with an emphasis * [build and test](docs/BUILD_AND_TEST.md) * [REST API](docs/REST_API.md) * [tasks](docs/TASKS.md) +* [plugin life cycle](docs/PLUGIN_LIFECYCLE.md) * [plugin signing](docs/PLUGIN_SIGNING.md) * [tribe](docs/TRIBE.md) diff --git a/control/available_plugin.go b/control/available_plugin.go index f6e470655..440172bc0 100644 --- a/control/available_plugin.go +++ b/control/available_plugin.go @@ -54,6 +54,7 @@ const ( var ( ErrPoolNotFound = errors.New("plugin pool not found") ErrBadKey = errors.New("bad key") + ErrBadTaskId = errors.New("bad task id") ) // availablePlugin represents a plugin which is @@ -538,6 +539,8 @@ func (ap *availablePlugins) findLatestPool(pType, name string) (strategy.Pool, s } func (ap *availablePlugins) getOrCreatePool(key string) (strategy.Pool, error) { + ap.Lock() + defer ap.Unlock() var err error pool, ok := ap.table[key] if ok { diff --git a/control/available_plugin_test.go b/control/available_plugin_test.go index ac9b2b936..ee216fdb8 100644 --- a/control/available_plugin_test.go +++ b/control/available_plugin_test.go @@ -53,6 +53,7 @@ func TestAvailablePlugin(t *testing.T) { Convey("Stop()", t, func() { Convey("returns nil if plugin successfully stopped", func() { r := newRunner() + r.SetEmitter(new(MockEmitter)) a := plugin.Arg{ PluginLogPath: "/tmp/snap-test-plugin-stop.log", } diff --git a/control/control.go b/control/control.go index db02b8c77..4e5e0bd72 100644 --- a/control/control.go +++ b/control/control.go @@ -90,8 +90,22 @@ type pluginControl struct { grpcServer *grpc.Server closingChan chan bool wg sync.WaitGroup + + subscriptionGroups ManagesSubscriptionGroups +} + +type subscribedPlugin struct { + typeName string + name string + version int + config *cdata.ConfigDataNode } +func (s subscribedPlugin) TypeName() string { return s.typeName } +func (s subscribedPlugin) Name() string { return s.name } +func (s subscribedPlugin) Version() int { return s.version } +func (s subscribedPlugin) Config() *cdata.ConfigDataNode { return s.config } + type runsPlugins interface { Start() error Stop() []error @@ -118,7 +132,7 @@ type managesPlugins interface { type catalogsMetrics interface { Get(core.Namespace, int) (*metricType, error) GetQueriedNamespaces(core.Namespace) ([]core.Namespace, error) - MatchQuery(core.Namespace) ([]core.Namespace, error) + UpdateQueriedNamespaces(core.Namespace) Add(*metricType) AddLoadedMetricType(*loadedPlugin, core.Metric) error RmUnloadedPluginMetrics(lp *loadedPlugin) @@ -126,6 +140,7 @@ type catalogsMetrics interface { Fetch(core.Namespace) ([]*metricType, error) Item() (string, []*metricType) Next() bool + Keys() []string Subscribe([]string, int) error Unsubscribe([]string, int) error GetPlugin(core.Namespace, int) (*loadedPlugin, error) @@ -171,7 +186,6 @@ func New(cfg *Config) *pluginControl { c := &pluginControl{} c.Config = cfg // Initialize components - // // Event Manager c.eventManager = gomit.NewEventController() @@ -209,6 +223,12 @@ func New(cfg *Config) *pluginControl { c.pluginRunner.SetMetricCatalog(c.metricCatalog) c.pluginRunner.SetPluginManager(c.pluginManager) + // Pass runner events to control main module + c.eventManager.RegisterHandler(c.Name(), c) + + // Create subscription group - used for managing a group of subscriptions + c.subscriptionGroups = newSubscriptionGroups(c) + // Start stuff err := c.pluginRunner.Start() if err != nil { @@ -226,6 +246,34 @@ func New(cfg *Config) *pluginControl { return c } +func (p *pluginControl) HandleGomitEvent(e gomit.Event) { + switch v := e.Body.(type) { + case *control_event.LoadPluginEvent: + serrs := p.subscriptionGroups.Process() + if serrs != nil { + for _, err := range serrs { + controlLogger.WithFields(log.Fields{ + "_block": "LoadPluginEvent", + }).Error(err) + } + } + case *control_event.UnloadPluginEvent: + serrs := p.subscriptionGroups.Process() + if serrs != nil { + for _, err := range serrs { + controlLogger.WithFields(log.Fields{ + "_block": "UnloadPluginEvent", + }).Error(err) + } + } + default: + runnerLog.WithFields(log.Fields{ + "_block": "handle-events", + "event": v.Namespace(), + }).Info("Nothing to do for this event") + } +} + func (p *pluginControl) Name() string { return "control" } @@ -360,8 +408,8 @@ func (p *pluginControl) Start() error { if err != nil { select { case <-p.closingChan: - // If we called Stop() then there will be a value in p.closingChan, so - // we'll get here and we can exit without showing the error. + // If we called Stop() then there will be a value in p.closingChan, so + // we'll get here and we can exit without showing the error. default: controlLogger.Fatal(err) } @@ -596,19 +644,9 @@ func (p *pluginControl) SwapPlugins(in *core.RequestedPlugin, out core.Cataloged } // MatchQueryToNamespaces performs the process of matching the 'ns' with namespaces of all cataloged metrics -func (p *pluginControl) MatchQueryToNamespaces(ns core.Namespace) ([]core.Namespace, serror.SnapError) { +func (p *pluginControl) matchQueryToNamespaces(ns core.Namespace) ([]core.Namespace, serror.SnapError) { // carry out the matching process - nss, err := p.metricCatalog.MatchQuery(ns) - if err != nil { - return nil, serror.New(err) - } - return nss, nil -} - -// ExpandWildcards returns all matched metrics namespaces with given 'ns' -// as the results of matching query process which has been done -func (p *pluginControl) ExpandWildcards(ns core.Namespace) ([]core.Namespace, serror.SnapError) { - // retrieve queried namespaces + p.metricCatalog.UpdateQueriedNamespaces(ns) nss, err := p.metricCatalog.GetQueriedNamespaces(ns) if err != nil { return nil, serror.New(err) @@ -616,354 +654,127 @@ func (p *pluginControl) ExpandWildcards(ns core.Namespace) ([]core.Namespace, se return nss, nil } -func (p *pluginControl) ValidateDeps(mts []core.Metric, plugins []core.SubscribedPlugin) []serror.SnapError { - var serrs []serror.SnapError - for _, mt := range mts { - errs := p.validateMetricTypeSubscription(mt, mt.Config()) - if len(errs) > 0 { - serrs = append(serrs, errs...) - } - } - if len(serrs) > 0 { - return serrs - } - - //validate plugins - for _, plg := range plugins { - typ, err := core.ToPluginType(plg.TypeName()) - if err != nil { - return []serror.SnapError{serror.New(err)} - } - plg.Config().ReverseMerge(p.Config.Plugins.getPluginConfigDataNode(typ, plg.Name(), plg.Version())) - errs := p.validatePluginSubscription(plg) - if len(errs) > 0 { - serrs = append(serrs, errs...) - return serrs - } - } - - return serrs +func (p *pluginControl) ValidateDeps(requested []core.RequestedMetric, plugins []core.SubscribedPlugin, configTree *cdata.ConfigDataTree) []serror.SnapError { + return p.subscriptionGroups.ValidateDeps(requested, plugins, configTree) } -func (p *pluginControl) validatePluginSubscription(pl core.SubscribedPlugin) []serror.SnapError { - var serrs = []serror.SnapError{} - controlLogger.WithFields(log.Fields{ - "_block": "validate-plugin-subscription", - "plugin": fmt.Sprintf("%s:%d", pl.Name(), pl.Version()), - }).Info(fmt.Sprintf("validating dependencies for plugin %s:%d", pl.Name(), pl.Version())) - lp, err := p.pluginManager.get(fmt.Sprintf("%s:%s:%d", pl.TypeName(), pl.Name(), pl.Version())) - if err != nil { - se := serror.New(fmt.Errorf("Plugin not found: type(%s) name(%s) version(%d)", pl.TypeName(), pl.Name(), pl.Version())) - se.SetFields(map[string]interface{}{ - "name": pl.Name(), - "version": pl.Version(), - "type": pl.TypeName(), - }) - serrs = append(serrs, se) - return serrs - } - - if lp.ConfigPolicy != nil { - ncd := lp.ConfigPolicy.Get([]string{""}) - _, errs := ncd.Process(pl.Config().Table()) - if errs != nil && errs.HasErrors() { - for _, e := range errs.Errors() { - se := serror.New(e) - se.SetFields(map[string]interface{}{"name": pl.Name(), "version": pl.Version()}) - serrs = append(serrs, se) - } - } - } - return serrs +// SubscribeDeps will subscribe to collectors, processors and publishers. The collectors are subscribed by mapping the provided +// array of core.RequestedMetrics to the corresponding plugins while processors and publishers provided in the array of core.Plugin +// will be subscribed directly. The ID provides a logical grouping of subscriptions. +func (p *pluginControl) SubscribeDeps(id string, requested []core.RequestedMetric, plugins []core.SubscribedPlugin, configTree *cdata.ConfigDataTree) (serrs []serror.SnapError) { + return p.subscriptionGroups.Add(id, requested, configTree, plugins) } -func (p *pluginControl) validateMetricTypeSubscription(mt core.RequestedMetric, cd *cdata.ConfigDataNode) []serror.SnapError { - var serrs []serror.SnapError - controlLogger.WithFields(log.Fields{ - "_block": "validate-metric-subscription", - "namespace": mt.Namespace(), - "version": mt.Version(), - }).Info("subscription called on metric") - - m, err := p.metricCatalog.Get(mt.Namespace(), mt.Version()) +// UnsubscribeDeps unsubscribes a group of dependencies provided the subscription group ID +func (p *pluginControl) UnsubscribeDeps(id string) []serror.SnapError { + // update view and unsubscribe to plugins + return p.subscriptionGroups.Remove(id) +} +func (p *pluginControl) verifyPlugin(lp *loadedPlugin) error { + b, err := ioutil.ReadFile(lp.Details.Path) if err != nil { - serrs = append(serrs, serror.New(err, map[string]interface{}{ - "name": mt.Namespace().String(), - "version": mt.Version(), - })) - return serrs - } - - // No metric found return error. - if m == nil { - serrs = append(serrs, serror.New(fmt.Errorf("no metric found cannot subscribe: (%s) version(%d)", mt.Namespace(), mt.Version()))) - return serrs - } - - m.config = cd - - typ, serr := core.ToPluginType(m.Plugin.TypeName()) - if serr != nil { - return []serror.SnapError{serror.New(err)} + return err } - - // merge global plugin config - if m.config != nil { - m.config.ReverseMerge(p.Config.Plugins.getPluginConfigDataNode(typ, m.Plugin.Name(), m.Plugin.Version())) - } else { - m.config = p.Config.Plugins.getPluginConfigDataNode(typ, m.Plugin.Name(), m.Plugin.Version()) + cs := sha256.Sum256(b) + if lp.Details.CheckSum != cs { + return fmt.Errorf(fmt.Sprintf("Current plugin checksum (%x) does not match checksum when plugin was first loaded (%x).", cs, lp.Details.CheckSum)) } - - // When a metric is added to the MetricCatalog, the policy of rules defined by the plugin is added to the metric's policy. - // If no rules are defined for a metric, we set the metric's policy to an empty ConfigPolicyNode. - // Checking m.policy for nil will not work, we need to check if rules are nil. - if m.policy.HasRules() { - if m.Config() == nil { - serrs = append(serrs, serror.New(fmt.Errorf("Policy defined for metric, (%s) version (%d), but no config defined in manifest", mt.Namespace(), mt.Version()))) - return serrs - } - ncdTable, errs := m.policy.Process(m.Config().Table()) - if errs != nil && errs.HasErrors() { - for _, e := range errs.Errors() { - serrs = append(serrs, serror.New(e)) - } - return serrs - } - m.config = cdata.FromTable(*ncdTable) + if lp.Details.Signed { + return p.signingManager.ValidateSignature(p.keyringFiles, lp.Details.Path, lp.Details.Signature) } - - return serrs -} - -type gatheredPlugin struct { - plugin core.Plugin - subscriptionType strategy.SubscriptionType + return nil } -func (p *pluginControl) gatherCollectors(mts []core.Metric) ([]gatheredPlugin, []serror.SnapError) { - var ( - plugins []gatheredPlugin - serrs []serror.SnapError - ) - // here we resolve and retrieve plugins for each metric type. - // if the incoming metric type version is < 1, we treat that as - // latest as with plugins. The following two loops create a set - // of plugins with proper versions needed to discern the subscription - // types. - colPlugins := make(map[string]gatheredPlugin) - for _, mt := range mts { - // If the version provided is <1 we will get the latest - // plugin for the given metric. - m, err := p.metricCatalog.Get(mt.Namespace(), mt.Version()) +func (p *pluginControl) getMetricsAndCollectors(requested []core.RequestedMetric, configTree *cdata.ConfigDataTree) ([]core.Metric, []core.SubscribedPlugin, []serror.SnapError) { + newMetrics := []core.Metric{} + newPlugins := []core.SubscribedPlugin{} + var serrs []serror.SnapError + for _, r := range requested { + // get expanded namespaces from requested metrics + newNss, err := p.matchQueryToNamespaces(r.Namespace()) if err != nil { - serrs = append(serrs, serror.New(err, map[string]interface{}{ - "name": mt.Namespace().String(), - "version": mt.Version(), - })) + log.WithFields(log.Fields{ + "_block": "control", + "action": "expanding-requested-metrics", + "query": r.Namespace(), + }).Error("error matching requested namespace with metric catalog") + serrs = append(serrs, err) continue } - subType := strategy.BoundSubscriptionType - if mt.Version() < 1 { - subType = strategy.UnboundSubscriptionType - } - colPlugins[fmt.Sprintf("%s:%d", m.Plugin.Key(), subType)] = gatheredPlugin{ - plugin: m.Plugin, - subscriptionType: subType, - } - } - if len(serrs) > 0 { - return plugins, serrs - } - - for _, lp := range colPlugins { - plugins = append(plugins, lp) - } - if len(plugins) == 0 { - serrs = append(serrs, serror.New(errors.New("No plugins found"))) - return nil, serrs - } - return plugins, nil -} - -func (p *pluginControl) SubscribeDeps(taskID string, mts []core.Metric, plugins []core.Plugin) []serror.SnapError { - var serrs []serror.SnapError - if len(mts) != 0 { - collectors, errs := p.gatherCollectors(mts) - if len(errs) > 0 { - serrs = append(serrs, errs...) - } - for _, gc := range collectors { - pool, err := p.pluginRunner.AvailablePlugins().getOrCreatePool(fmt.Sprintf("%s:%s:%d", gc.plugin.TypeName(), gc.plugin.Name(), gc.plugin.Version())) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - pool.Subscribe(taskID, gc.subscriptionType) - if pool.Eligible() { - err = p.verifyPlugin(gc.plugin.(*loadedPlugin)) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - err = p.pluginRunner.runPlugin(gc.plugin.(*loadedPlugin).Details) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - } - serr := p.sendPluginSubscriptionEvent(taskID, gc.plugin) - if serr != nil { - serrs = append(serrs, serr) + if controlLogger.Level >= log.DebugLevel { + for _, n := range newNss { + controlLogger.WithFields(log.Fields{ + "_block": "control", + "ns": n.String(), + }).Debug("Expanded namespaces found") } } - } - for _, sub := range plugins { - // pools are created statically, not with keys like "publisher:foo:-1" - // here we check to see if the version of the incoming plugin is -1, and - // if it is, we look up the latest in loaded plugins, and use that key to - // create the pool. - if sub.Version() < 1 { - latest, err := p.pluginManager.get(fmt.Sprintf("%s:%s:%d", sub.TypeName(), sub.Name(), sub.Version())) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - pool, err := p.pluginRunner.AvailablePlugins().getOrCreatePool(latest.Key()) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - pool.Subscribe(taskID, strategy.UnboundSubscriptionType) - if pool.Eligible() { - err = p.verifyPlugin(latest) + + if len(newNss) > 0 { + for _, ns := range newNss { + // Get metric types from metric catalog + m, err := p.metricCatalog.Get(ns, r.Version()) if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs + log.WithFields(log.Fields{ + "_block": "control", + "action": "expanding requested metrics", + "namespace": ns.String(), + "version": r.Version(), + }).Error("error retreiving metric given a namespace and version.") + serrs = append(serrs, serror.New(fmt.Errorf("error retreiving metric %s:%d", ns.String(), r.Version()))) + continue } - err = p.pluginRunner.runPlugin(latest.Details) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs + // in case config tree doesn't have any configuration for current namespace + // it's needed to initialize config, otherwise it will stay nil and panic later on + config := configTree.Get(ns.Strings()) + if config == nil { + config = cdata.NewNode() } - } - } else { - pool, err := p.pluginRunner.AvailablePlugins().getOrCreatePool(fmt.Sprintf("%s:%s:%d", sub.TypeName(), sub.Name(), sub.Version())) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs - } - pool.Subscribe(taskID, strategy.BoundSubscriptionType) - if pool.Eligible() { - pl, err := p.pluginManager.get(fmt.Sprintf("%s:%s:%d", sub.TypeName(), sub.Name(), sub.Version())) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs + newMetrics = append(newMetrics, &metric{ + namespace: m.Namespace(), + version: m.Version(), + config: config, + }) + + config = configTree.Get([]string{""}) + if config == nil { + config = cdata.NewNode() } - err = p.verifyPlugin(pl) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs + plugin := subscribedPlugin{ + name: m.Plugin.Name(), + typeName: m.Plugin.TypeName(), + version: m.Plugin.Version(), + config: config, } - err = p.pluginRunner.runPlugin(pl.Details) - if err != nil { - serrs = append(serrs, serror.New(err)) - return serrs + + if !containsPlugin(newPlugins, plugin) { + newPlugins = append(newPlugins, plugin) } } } - serr := p.sendPluginSubscriptionEvent(taskID, sub) - if serr != nil { - serrs = append(serrs, serr) - } } - return serrs -} -func (p *pluginControl) verifyPlugin(lp *loadedPlugin) error { - b, err := ioutil.ReadFile(lp.Details.Path) - if err != nil { - return err - } - cs := sha256.Sum256(b) - if lp.Details.CheckSum != cs { - return fmt.Errorf(fmt.Sprintf("Current plugin checksum (%x) does not match checksum when plugin was first loaded (%x).", cs, lp.Details.CheckSum)) - } - if lp.Details.Signed { - return p.signingManager.ValidateSignature(p.keyringFiles, lp.Details.Path, lp.Details.Signature) - } - return nil -} - -func (p *pluginControl) sendPluginSubscriptionEvent(taskID string, pl core.Plugin) serror.SnapError { - pt, err := core.ToPluginType(pl.TypeName()) - if err != nil { - return serror.New(err) - } - e := &control_event.PluginSubscriptionEvent{ - TaskId: taskID, - PluginType: int(pt), - PluginName: pl.Name(), - PluginVersion: pl.Version(), - SubscriptionType: int(strategy.UnboundSubscriptionType), - } - if pl.Version() > 0 { - e.SubscriptionType = int(strategy.BoundSubscriptionType) - } - if _, err := p.eventManager.Emit(e); err != nil { - return serror.New(err) - } - return nil -} - -func (p *pluginControl) UnsubscribeDeps(taskID string, mts []core.Metric, plugins []core.Plugin) []serror.SnapError { - var serrs []serror.SnapError - // If no metrics to unsubscribe then skip this section. Avoids errors when - // workflow is distributed and each node may not have metrics. - if len(mts) > 0 { - collectors, errs := p.gatherCollectors(mts) - if len(errs) > 0 { - serrs = append(serrs, errs...) - } - for _, gc := range collectors { - plugins = append(plugins, gc.plugin) + if controlLogger.Level >= log.DebugLevel { + for _, m := range newMetrics { + log.WithFields(log.Fields{ + "_block": "control", + "action": "gather", + "metric": fmt.Sprintf("%s:%d", m.Namespace().String(), m.Version()), + }).Debug("gathered metrics from workflow request") } - } - for _, sub := range plugins { - pool, err := p.pluginRunner.AvailablePlugins().getPool(fmt.Sprintf("%s:%s:%d", sub.TypeName(), sub.Name(), sub.Version())) - if err != nil { - serrs = append(serrs, err) - return serrs - } - if pool != nil { - pool.Unsubscribe(taskID) - } - serr := p.sendPluginUnsubscriptionEvent(taskID, sub) - if serr != nil { - serrs = append(serrs, serr) + for _, p := range newPlugins { + log.WithFields(log.Fields{ + "_block": "control", + "action": "gather", + "metric": fmt.Sprintf("%s:%s:%d", p.TypeName(), p.Name(), p.Version()), + }).Debug("gathered plugins from workflow request") } } - return serrs -} - -func (p *pluginControl) sendPluginUnsubscriptionEvent(taskID string, pl core.Plugin) serror.SnapError { - pt, err := core.ToPluginType(pl.TypeName()) - if err != nil { - return serror.New(err) - } - e := &control_event.PluginUnsubscriptionEvent{ - TaskId: taskID, - PluginType: int(pt), - PluginName: pl.Name(), - PluginVersion: pl.Version(), - } - if _, err := p.eventManager.Emit(e); err != nil { - return serror.New(err) - } - return nil + return newMetrics, newPlugins, serrs } // SetMonitorOptions exposes monitors options @@ -1062,13 +873,31 @@ func (p *pluginControl) MetricExists(mns core.Namespace, ver int) bool { // CollectMetrics is a blocking call to collector plugins returning a collection // of metrics and errors. If an error is encountered no metrics will be // returned. -func (p *pluginControl) CollectMetrics(metricTypes []core.Metric, deadline time.Time, taskID string, allTags map[string]map[string]string) (metrics []core.Metric, errs []error) { +func (p *pluginControl) CollectMetrics(id string, allTags map[string]map[string]string) (metrics []core.Metric, errs []error) { // If control is not started we don't want tasks to be able to // go through a workflow. if !p.Started { return nil, []error{ErrControllerNotStarted} } + // Subscription groups are processed anytime a plugin is loaded/unloaded. + results, serrs, err := p.subscriptionGroups.Get(id) + if err != nil { + controlLogger.WithFields(log.Fields{ + "_block": "CollectorMetrics", + "subscription-group-id": id, + }).Error(err) + errs = append(errs, err) + return + } + // If We received errors when the requested metrics were last processed + // against the metric catalog we need to return them to the caller. + if serrs != nil { + for _, e := range serrs { + errs = append(errs, e) + } + } + for ns, nsTags := range allTags { for k, v := range nsTags { log.WithFields(log.Fields{ @@ -1082,7 +911,7 @@ func (p *pluginControl) CollectMetrics(metricTypes []core.Metric, deadline time. } } - pluginToMetricMap, err := groupMetricTypesByPlugin(p.metricCatalog, metricTypes) + pluginToMetricMap, err := groupMetricTypesByPlugin(p.metricCatalog, results) if err != nil { errs = append(errs, err) return @@ -1104,7 +933,7 @@ func (p *pluginControl) CollectMetrics(metricTypes []core.Metric, deadline time. wg.Add(1) go func(pluginKey string, mt []core.Metric) { - mts, err := p.pluginRunner.AvailablePlugins().collectMetrics(pluginKey, mt, taskID) + mts, err := p.pluginRunner.AvailablePlugins().collectMetrics(pluginKey, mt, id) if err != nil { cError <- err } else { @@ -1277,3 +1106,14 @@ func groupMetricTypesByPlugin(cat catalogsMetrics, mts []core.Metric) (map[strin } return pmts, nil } + +func containsPlugin(slice []core.SubscribedPlugin, lookup subscribedPlugin) bool { + for _, plugin := range slice { + if plugin.Name() == lookup.Name() && + plugin.Version() == lookup.Version() && + plugin.TypeName() == lookup.TypeName() { + return true + } + } + return false +} diff --git a/control/control_grpc_server.go b/control/control_grpc_server.go index ef0bb4ec7..5ffe7e943 100644 --- a/control/control_grpc_server.go +++ b/control/control_grpc_server.go @@ -20,10 +20,8 @@ limitations under the License. package control import ( - "time" - "github.com/intelsdi-x/snap/core" - "github.com/intelsdi-x/snap/core/serror" + "github.com/intelsdi-x/snap/core/cdata" "github.com/intelsdi-x/snap/grpc/common" "github.com/intelsdi-x/snap/grpc/controlproxy/rpc" "golang.org/x/net/context" @@ -71,8 +69,6 @@ func (pc *ControlGRPCServer) ProcessMetrics(ctx context.Context, r *rpc.PubProcM } func (pc *ControlGRPCServer) CollectMetrics(ctx context.Context, r *rpc.CollectMetricsRequest) (*rpc.CollectMetricsResponse, error) { - metrics := common.ToCoreMetrics(r.Metrics) - deadline := time.Unix(r.Deadline.Sec, r.Deadline.Nsec) var AllTags map[string]map[string]string for k, v := range r.AllTags { AllTags[k] = make(map[string]string) @@ -80,7 +76,7 @@ func (pc *ControlGRPCServer) CollectMetrics(ctx context.Context, r *rpc.CollectM AllTags[k][entry.Key] = entry.Value } } - mts, errs := pc.control.CollectMetrics(metrics, deadline, r.TaskID, AllTags) + mts, errs := pc.control.CollectMetrics(r.TaskID, AllTags) var reply *rpc.CollectMetricsResponse if mts == nil { reply = &rpc.CollectMetricsResponse{ @@ -95,49 +91,25 @@ func (pc *ControlGRPCServer) CollectMetrics(ctx context.Context, r *rpc.CollectM return reply, nil } -func (pc *ControlGRPCServer) ExpandWildcards(ctx context.Context, r *rpc.ExpandWildcardsRequest) (*rpc.ExpandWildcardsReply, error) { - nss, serr := pc.control.ExpandWildcards(common.ToCoreNamespace(r.Namespace)) - reply := &rpc.ExpandWildcardsReply{} - if nss != nil { - reply.NSS = convertNSS(nss) - } - if serr != nil { - reply.Error = common.NewErrors([]serror.SnapError{serr})[0] - } - return reply, nil -} - func (pc *ControlGRPCServer) ValidateDeps(ctx context.Context, r *rpc.ValidateDepsRequest) (*rpc.ValidateDepsReply, error) { - metrics := common.ToCoreMetrics(r.Metrics) + metrics := common.ToRequestedMetrics(r.Metrics) plugins := common.ToSubPlugins(r.Plugins) - serrors := pc.control.ValidateDeps(metrics, plugins) + configTree := cdata.NewTree() + serrors := pc.control.ValidateDeps(metrics, plugins, configTree) return &rpc.ValidateDepsReply{Errors: common.NewErrors(serrors)}, nil } func (pc *ControlGRPCServer) SubscribeDeps(ctx context.Context, r *rpc.SubscribeDepsRequest) (*rpc.SubscribeDepsReply, error) { - metrics := common.ToCoreMetrics(r.Metrics) - plugins := common.MsgToCorePlugins(r.Plugins) - serrors := pc.control.SubscribeDeps(r.TaskId, metrics, plugins) - return &rpc.SubscribeDepsReply{Errors: common.NewErrors(serrors)}, nil -} - -func (pc *ControlGRPCServer) UnsubscribeDeps(ctx context.Context, r *rpc.SubscribeDepsRequest) (*rpc.SubscribeDepsReply, error) { - metrics := common.ToCoreMetrics(r.Metrics) - plugins := common.MsgToCorePlugins(r.Plugins) - serrors := pc.control.UnsubscribeDeps(r.TaskId, metrics, plugins) + plugins := common.ToSubPlugins(r.Plugins) + configTree := cdata.NewTree() + requested := common.MetricToRequested(r.Requested) + serrors := pc.control.SubscribeDeps(r.TaskId, requested, plugins, configTree) return &rpc.SubscribeDepsReply{Errors: common.NewErrors(serrors)}, nil } -func (pc *ControlGRPCServer) MatchQueryToNamespaces(ctx context.Context, r *rpc.ExpandWildcardsRequest) (*rpc.ExpandWildcardsReply, error) { - nss, serr := pc.control.MatchQueryToNamespaces(common.ToCoreNamespace(r.Namespace)) - reply := &rpc.ExpandWildcardsReply{} - if nss != nil { - reply.NSS = convertNSS(nss) - } - if serr != nil { - reply.Error = common.NewErrors([]serror.SnapError{serr})[0] - } - return reply, nil +func (pc *ControlGRPCServer) UnsubscribeDeps(ctx context.Context, r *rpc.UnsubscribeDepsRequest) (*rpc.UnsubscribeDepsReply, error) { + serrors := pc.control.UnsubscribeDeps(r.TaskId) + return &rpc.UnsubscribeDepsReply{Errors: common.NewErrors(serrors)}, nil } func (pc *ControlGRPCServer) GetAutodiscoverPaths(ctx context.Context, _ *common.Empty) (*rpc.GetAutodiscoverPathsReply, error) { diff --git a/control/control_grpc_server_test.go b/control/control_grpc_server_test.go index 750029815..6a48cf35c 100644 --- a/control/control_grpc_server_test.go +++ b/control/control_grpc_server_test.go @@ -27,15 +27,15 @@ import ( "net" "path" "testing" - "time" + "github.com/intelsdi-x/gomit" "golang.org/x/net/context" log "github.com/Sirupsen/logrus" "github.com/intelsdi-x/snap/control/fixtures" "github.com/intelsdi-x/snap/control/plugin" - "github.com/intelsdi-x/snap/control/strategy" "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/control_event" "github.com/intelsdi-x/snap/core/ctypes" "github.com/intelsdi-x/snap/grpc/common" "github.com/intelsdi-x/snap/grpc/controlproxy" @@ -44,6 +44,29 @@ import ( . "github.com/smartystreets/goconvey/convey" ) +type listenPluginLoadEvent struct { + plugin *mockPluginEvent + done chan struct{} +} + +func newFoo() *listenPluginLoadEvent { + return &listenPluginLoadEvent{ + done: make(chan struct{}), + } +} + +func (l *listenPluginLoadEvent) HandleGomitEvent(e gomit.Event) { + switch v := e.Body.(type) { + case *control_event.LoadPluginEvent: + l.done <- struct{}{} + default: + controlLogger.WithFields(log.Fields{ + "event:": v.Namespace(), + "_block": "HandleGomit", + }).Info("Unhandled Event") + } +} + // This test is meant to cover the grpc implementation of the subset of control // features that scheduler uses. It is not intended to test the control features // themselves, only that we are correctly passing data over grpc and correctly @@ -56,6 +79,9 @@ func TestGRPCServerScheduler(t *testing.T) { c := New(cfg) err := c.Start() + lpe := newFoo() + c.eventManager.RegisterHandler("Control.PluginLoaded", lpe) + Convey("Starting control_proxy server/client", t, func() { Convey("So err should be nil", func() { So(err, ShouldBeNil) @@ -69,18 +95,46 @@ func TestGRPCServerScheduler(t *testing.T) { if err != nil { log.Fatal(err) } - c.Load(mock) + _, serr := c.Load(mock) + Convey("Loading mock collector", t, func() { + Convey("should not error", func() { + So(serr, ShouldBeNil) + }) + }) + <-lpe.done passthru, err := core.NewRequestedPlugin(path.Join(fixtures.SnapPath, "plugin", "snap-plugin-processor-passthru")) if err != nil { log.Fatal(err) } - c.Load(passthru) + catalogedPassthru, serr := c.Load(passthru) + Convey("Loading passthru processor", t, func() { + Convey("should not error", func() { + So(serr, ShouldBeNil) + }) + }) + subscribedPassThruPlugin := subscribedPlugin{ + name: catalogedPassthru.Name(), + version: catalogedPassthru.Version(), + typeName: catalogedPassthru.TypeName(), + } + <-lpe.done filepub, err := core.NewRequestedPlugin(path.Join(fixtures.SnapPath, "plugin", "snap-plugin-publisher-mock-file")) if err != nil { log.Fatal(err) } - c.Load(filepub) + catalogedFile, serr := c.Load(filepub) + Convey("Loading file publisher", t, func() { + Convey("should not error", func() { + So(serr, ShouldBeNil) + }) + }) + subscribedFilePlugin := subscribedPlugin{ + name: catalogedFile.Name(), + version: catalogedFile.Version(), + typeName: catalogedFile.TypeName(), + } + <-lpe.done conn, err := rpcutil.GetClientConnection(c.Config.ListenAddr, c.Config.ListenPort) Convey("Creating an rpc connection", t, func() { @@ -151,9 +205,9 @@ func TestGRPCServerScheduler(t *testing.T) { Convey("SubscribeDeps", t, func() { Convey("Should Error with invalid inputs", func() { req := &rpc.SubscribeDepsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.InvalidMetric}), - Plugins: common.ToCorePluginsMsg([]core.Plugin{}), - TaskId: "my-snowflake-id", + Requested: []*common.Metric{&common.Metric{Namespace: common.ToNamespace(fixtures.InvalidMetric.Namespace()), Version: int64(fixtures.InvalidMetric.Version())}}, + Plugins: common.ToSubPluginsMsg([]core.SubscribedPlugin{subscribedFilePlugin, subscribedPassThruPlugin}), + TaskId: "my-snowflake-id", } reply, err := client.SubscribeDeps(context.Background(), req) // we don't expect rpc errors @@ -163,9 +217,9 @@ func TestGRPCServerScheduler(t *testing.T) { }) Convey("Should not error with valid inputs", func() { req := &rpc.SubscribeDepsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.ValidMetric}), - Plugins: common.ToCorePluginsMsg([]core.Plugin{}), - TaskId: "my-snowflake-id", + Requested: []*common.Metric{&common.Metric{Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), Version: int64(fixtures.ValidMetric.Version())}}, + Plugins: common.ToSubPluginsMsg([]core.SubscribedPlugin{}), + TaskId: "my-snowflake-valid", } reply, err := client.SubscribeDeps(context.Background(), req) // we don't expect rpc errors @@ -173,121 +227,34 @@ func TestGRPCServerScheduler(t *testing.T) { So(len(reply.Errors), ShouldEqual, 0) }) }) - // unsubscribedeps -- valid/invalid - Convey("UnsubscribeDeps", t, func() { - Convey("Should Error with invalid inputs", func() { - req := &rpc.SubscribeDepsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.InvalidMetric}), - Plugins: common.ToCorePluginsMsg([]core.Plugin{}), - TaskId: "my-snowflake-id", - } - reply, err := client.UnsubscribeDeps(context.Background(), req) - // we don't expect rpc errors - So(err, ShouldBeNil) - So(len(reply.Errors), ShouldNotEqual, 0) - So(reply.Errors[0].ErrorString, ShouldResemble, "Metric not found: /this/is/invalid") - }) - Convey("Should not error with valid inputs", func() { - req := &rpc.SubscribeDepsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.ValidMetric}), - Plugins: common.ToCorePluginsMsg([]core.Plugin{}), - TaskId: "my-snowflake-id", - } - reply, err := client.UnsubscribeDeps(context.Background(), req) - // we don't expect rpc errors - So(err, ShouldBeNil) - So(len(reply.Errors), ShouldEqual, 0) - }) - }) - //matchquerytonamespaces -- valid/invalid - Convey("MatchingQueryToNamespaces", t, func() { - Convey("Should error with invalid inputs", func() { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(fixtures.InvalidMetric.Namespace()), - } - reply, err := client.MatchQueryToNamespaces(context.Background(), req) - // we don't expect rpc.errors - So(err, ShouldBeNil) - So(reply.Error, ShouldNotBeNil) - So(reply.Error.ErrorString, ShouldResemble, "Metric not found: /this/is/invalid") - }) - Convey("Should not error with invalid inputs", func() { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), - } - reply, err := client.MatchQueryToNamespaces(context.Background(), req) - // we don't expect rpc.errors - So(err, ShouldBeNil) - So(reply.Error, ShouldBeNil) - }) - }) - //expandwildcards -- valid/invalid - Convey("ExpandWildcards", t, func() { - Convey("Should error with invalid inputs", func() { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(fixtures.InvalidMetric.Namespace()), - } - reply, err := client.ExpandWildcards(context.Background(), req) - // we don't expect rpc errors - So(err, ShouldBeNil) - So(reply.Error, ShouldNotBeNil) - So(reply.Error.ErrorString, ShouldResemble, "Metric not found: /this/is/invalid") - }) - Convey("Should not error with valid inputs", func() { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), - } - reply, err := client.ExpandWildcards(context.Background(), req) - // we don't expect rpc errors - So(err, ShouldBeNil) - So(reply.Error, ShouldBeNil) - }) - }) - // start plugin pools/provide task info so we can do collect/process/publishMetrics - // errors here indicate problems outside the scope of this test. - plugins := []string{"collector:mock:1", "processor:passthru:1", "publisher:mock-file:3"} - lps := make([]*loadedPlugin, len(plugins)) - pools := make([]strategy.Pool, len(plugins)) - for i, v := range plugins { - lps[i], err = c.pluginManager.get(v) - if err != nil { - log.Fatal(err) - } - pools[i], err = c.pluginRunner.AvailablePlugins().getOrCreatePool(v) - if err != nil { - log.Fatal(err) - } - pools[i].Subscribe("my-snowflake-id", strategy.BoundSubscriptionType) - err = c.pluginRunner.runPlugin(lps[i].Details) - if err != nil { - log.Fatal(err) - } - } //our returned metrics var mts []core.Metric //collect Convey("CollectMetrics", t, func() { - Convey("Should error with invalid inputs", func() { + req := &rpc.SubscribeDepsRequest{ + Requested: []*common.Metric{&common.Metric{Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), Version: int64(fixtures.ValidMetric.Version())}}, + Plugins: common.ToSubPluginsMsg([]core.SubscribedPlugin{ + subscribedPassThruPlugin, + subscribedFilePlugin, + }, + ), + TaskId: "my-snowflake-id", + } + _, err := client.SubscribeDeps(context.Background(), req) + So(err, ShouldBeNil) + + Convey("should error with invalid inputs", func() { req := &rpc.CollectMetricsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.InvalidMetric}), - Deadline: &common.Time{ - Sec: int64(time.Now().Unix()), - Nsec: int64(time.Now().Nanosecond()), - }, - TaskID: "my-snowflake-id", + TaskID: "my-fake-snowflake-id", } reply, err := client.CollectMetrics(context.Background(), req) So(err, ShouldBeNil) So(len(reply.Errors), ShouldNotEqual, 0) }) + Convey("should not error with valid inputs", func() { req := &rpc.CollectMetricsRequest{ - Metrics: common.NewMetrics([]core.Metric{fixtures.ValidMetric}), - Deadline: &common.Time{ - Sec: int64(time.Now().Unix()), - Nsec: int64(time.Now().Nanosecond()), - }, - TaskID: "my-snowflake-id", + TaskID: "my-snowflake-valid", } reply, err := client.CollectMetrics(context.Background(), req) So(err, ShouldBeNil) @@ -301,13 +268,32 @@ func TestGRPCServerScheduler(t *testing.T) { var content []byte //process Convey("ProcessMetrics", t, func() { - Convey("Should error with invalid inputs", func() { - req := controlproxy.GetPubProcReq("snap.gob", []byte{}, "bad name", 1, map[string]ctypes.ConfigValue{}, "my-snowflake-id") + req := &rpc.SubscribeDepsRequest{ + Requested: []*common.Metric{&common.Metric{Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), Version: int64(fixtures.ValidMetric.Version())}}, + Plugins: common.ToSubPluginsMsg([]core.SubscribedPlugin{ + subscribedPassThruPlugin, + subscribedFilePlugin, + }, + ), + TaskId: "my-snowflake-id", + } + _, err := client.SubscribeDeps(context.Background(), req) + So(err, ShouldBeNil) + Convey("should error with invalid inputs", func() { + var buf bytes.Buffer + enc := gob.NewEncoder(&buf) + metrics := make([]plugin.MetricType, len(mts)) + for i, m := range mts { + mt := plugin.NewMetricType(m.Namespace(), m.Timestamp(), m.Tags(), m.Unit(), m.Data()) + metrics[i] = *mt + } + enc.Encode(metrics) + req := controlproxy.GetPubProcReq("snap.gob", buf.Bytes(), "passthru-invalid", 1, map[string]ctypes.ConfigValue{}, "my-snowflake-id") reply, err := client.ProcessMetrics(context.Background(), req) // we don't expect rpc errors So(err, ShouldBeNil) So(len(reply.Errors), ShouldNotEqual, 0) - So(reply.Errors[0], ShouldResemble, "bad key") + // content to pass to publisher }) Convey("should not error with valid inputs", func() { var buf bytes.Buffer @@ -325,20 +311,32 @@ func TestGRPCServerScheduler(t *testing.T) { So(len(reply.Errors), ShouldEqual, 0) // content to pass to publisher content = reply.Content - }) }) //publishmetrics Convey("PublishMetrics", t, func() { - Convey("Should error with invalid inputs", func() { - req := controlproxy.GetPubProcReq("snap.gob", []byte{}, "bad name", 1, map[string]ctypes.ConfigValue{}, "my-snowflake-id") + req := &rpc.SubscribeDepsRequest{ + Requested: []*common.Metric{&common.Metric{Namespace: common.ToNamespace(fixtures.ValidMetric.Namespace()), Version: int64(fixtures.ValidMetric.Version())}}, + Plugins: common.ToSubPluginsMsg([]core.SubscribedPlugin{ + subscribedPassThruPlugin, + subscribedFilePlugin, + }, + ), + TaskId: "my-snowflake-id", + } + _, err := client.SubscribeDeps(context.Background(), req) + So(err, ShouldBeNil) + + Convey("should error with invalid inputs", func() { + config := make(map[string]ctypes.ConfigValue) + config["file"] = ctypes.ConfigValueStr{Value: "/tmp/grpcservertest.snap"} + req := controlproxy.GetPubProcReq("snap.gob", content, "mock-file-invalid", 3, config, "my-snowflake-id") reply, err := client.PublishMetrics(context.Background(), req) // we don't expect rpc errors So(err, ShouldBeNil) So(len(reply.Errors), ShouldNotEqual, 0) - - So(reply.Errors[0], ShouldResemble, "bad key") }) + // Publish only returns no errors on success Convey("should not error with valid inputs", func() { config := make(map[string]ctypes.ConfigValue) diff --git a/control/control_test.go b/control/control_test.go index e0963ba94..b419c3df7 100644 --- a/control/control_test.go +++ b/control/control_test.go @@ -271,14 +271,18 @@ type mockPluginEvent struct { } type listenToPluginEvent struct { - plugin *mockPluginEvent - done chan struct{} + plugin *mockPluginEvent + done chan struct{} + max chan struct{} + restarted chan struct{} } func newListenToPluginEvent() *listenToPluginEvent { return &listenToPluginEvent{ - done: make(chan struct{}), - plugin: &mockPluginEvent{}, + done: make(chan struct{}), + restarted: make(chan struct{}), + max: make(chan struct{}), + plugin: &mockPluginEvent{}, } } @@ -286,13 +290,16 @@ func (l *listenToPluginEvent) HandleGomitEvent(e gomit.Event) { switch v := e.Body.(type) { case *control_event.RestartedAvailablePluginEvent: l.plugin.EventNamespace = v.Namespace() - l.done <- struct{}{} + l.restarted <- struct{}{} case *control_event.MaxPluginRestartsExceededEvent: l.plugin.EventNamespace = v.Namespace() - l.done <- struct{}{} + l.max <- struct{}{} case *control_event.DeadAvailablePluginEvent: l.plugin.EventNamespace = v.Namespace() l.done <- struct{}{} + case *control_event.HealthCheckFailedEvent: + l.plugin.EventNamespace = v.Namespace() + l.done <- struct{}{} case *control_event.LoadPluginEvent: l.plugin.LoadedPluginName = v.Name l.plugin.LoadedPluginVersion = v.Version @@ -313,9 +320,6 @@ func (l *listenToPluginEvent) HandleGomitEvent(e gomit.Event) { l.plugin.PluginType = v.PluginType l.plugin.EventNamespace = v.Namespace() l.done <- struct{}{} - case *control_event.MovePluginSubscriptionEvent: - l.plugin.EventNamespace = v.Namespace() - l.done <- struct{}{} case *control_event.PluginSubscriptionEvent: l.plugin.EventNamespace = v.Namespace() l.done <- struct{}{} @@ -688,6 +692,7 @@ func (m *mc) Unsubscribe(ns []string, ver int) error { func (m *mc) Add(*metricType) {} func (m *mc) Table() map[string][]*metricType { return map[string][]*metricType{} } func (m *mc) Item() (string, []*metricType) { return "", []*metricType{} } +func (m *mc) Keys() []string { return []string{} } func (m *mc) Next() bool { m.e = 1 @@ -707,7 +712,10 @@ func (m *mc) GetQueriedNamespaces(ns core.Namespace) ([]core.Namespace, error) { return []core.Namespace{ns}, nil } -func (m *mc) MatchQuery(ns core.Namespace) ([]core.Namespace, error) { +func (m *mc) UpdateQueriedNamespaces(ns core.Namespace) { +} + +func (m *mc) MatchNamespaces(ns core.Namespace) ([]core.Namespace, error) { return []core.Namespace{ns}, nil } @@ -791,13 +799,15 @@ func TestMetricConfig(t *testing.T) { } Convey("So metric should not be valid without config", func() { - errs := c.validateMetricTypeSubscription(m1, cd) + errs := c.subscriptionGroups.validateMetric(m1) So(errs, ShouldNotBeNil) }) cd.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) Convey("So metric should be valid with config", func() { - errs := c.validateMetricTypeSubscription(m1, cd) + m1.Cfg = cdata.NewNode() + m1.Cfg.AddItem("password", ctypes.ConfigValueStr{Value: "password"}) + errs := c.subscriptionGroups.validateMetric(m1) So(errs, ShouldBeNil) }) @@ -805,7 +815,7 @@ func TestMetricConfig(t *testing.T) { m := fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "bad"), } - errs := c.validateMetricTypeSubscription(m, cd) + errs := c.subscriptionGroups.validateMetric(m) So(errs, ShouldNotBeNil) }) @@ -822,13 +832,12 @@ func TestMetricConfig(t *testing.T) { _, err := load(c, fixtures.JSONRPCPluginPath) So(err, ShouldBeNil) <-lpe.done - cd := cdata.NewNode() m1 := fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "foo"), } Convey("So metric should be valid with config", func() { - errs := c.validateMetricTypeSubscription(m1, cd) + errs := c.subscriptionGroups.validateMetric(m1) So(errs, ShouldBeNil) }) Convey("So mock should have name: bob config from defaults", func() { @@ -841,22 +850,23 @@ func TestMetricConfig(t *testing.T) { Convey("nil config provided by task", t, func() { config := getTestConfig() - config.Plugins.All.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) c := New(config) c.Start() lpe := newListenToPluginEvent() c.eventManager.RegisterHandler("Control.PluginLoaded", lpe) _, err := load(c, fixtures.JSONRPCPluginPath) So(err, ShouldBeNil) - <-lpe.done - var cd *cdata.ConfigDataNode + + cfg := cdata.NewNode() + cfg.AddItem("password", ctypes.ConfigValueStr{Value: "password"}) m1 := fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "foo"), + Cfg: cfg, } Convey("So metric should be valid with config", func() { - errs := c.validateMetricTypeSubscription(m1, cd) + errs := c.subscriptionGroups.validateMetric(m1) So(errs, ShouldBeNil) }) c.Stop() @@ -872,12 +882,11 @@ func TestMetricConfig(t *testing.T) { _, err := load(c, fixtures.JSONRPCPluginPath) So(err, ShouldBeNil) <-lpe.done - cd := cdata.NewNode() m1 := fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "foo"), Ver: 1, } - errs := c.validateMetricTypeSubscription(m1, cd) + errs := c.subscriptionGroups.validateMetric(m1) Convey("So metric should be valid with config", func() { So(errs, ShouldBeNil) }) @@ -905,6 +914,12 @@ func TestRoutingCachingStrategy(t *testing.T) { Cfg: cdata.NewNode(), } <-lpe.done + + cdt := cdata.NewTree() + node := cdata.NewNode() + node.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) + cdt.Add([]string{"intel", "mock"}, node) + Convey("Start the plugins", func() { lp, err := c.pluginManager.get("collector:mock:2") So(err, ShouldBeNil) @@ -920,9 +935,11 @@ func TestRoutingCachingStrategy(t *testing.T) { uuid.New(), } for _, id := range tasks { - pool.Subscribe(id, strategy.BoundSubscriptionType) + pool.Subscribe(id) err = c.pluginRunner.runPlugin(lp.Details) So(err, ShouldBeNil) + serr := c.subscriptionGroups.Add(id, []core.RequestedMetric{metric}, cdt, []core.SubscribedPlugin{}) + So(serr, ShouldBeNil) } // The cache ttl should be 500ms. The system default is 500ms, but the plugin exposed 100ms which is less than the system default. ttl, err := pool.CacheTTL(tasks[0]) @@ -934,7 +951,7 @@ func TestRoutingCachingStrategy(t *testing.T) { Convey("Collect metrics", func() { taskID := tasks[rand.Intn(len(tasks))] for i := 0; i < 10; i++ { - _, errs := c.CollectMetrics([]core.Metric{metric}, time.Now().Add(time.Second*1), taskID, nil) + _, errs := c.CollectMetrics(taskID, nil) So(errs, ShouldBeEmpty) } Convey("Check cache stats", func() { @@ -967,6 +984,14 @@ func TestRoutingCachingStrategy(t *testing.T) { So(metric.Namespace().String(), ShouldResemble, "/intel/mock/foo") So(err, ShouldBeNil) <-lpe.done + + cdt := cdata.NewTree() + node := cdata.NewNode() + node.AddItem("user", ctypes.ConfigValueStr{Value: "jane"}) + node.AddItem("test", ctypes.ConfigValueBool{Value: true}) + node.AddItem("password", ctypes.ConfigValueStr{Value: "doe"}) + cdt.Add([]string{"intel", "mock"}, node) + Convey("Start the plugins", func() { lp, err := c.pluginManager.get("collector:mock:1") So(err, ShouldBeNil) @@ -983,9 +1008,11 @@ func TestRoutingCachingStrategy(t *testing.T) { uuid.New(), } for _, id := range tasks { - pool.Subscribe(id, strategy.BoundSubscriptionType) + pool.Subscribe(id) err = c.pluginRunner.runPlugin(lp.Details) So(err, ShouldBeNil) + serrs := c.subscriptionGroups.Add(id, []core.RequestedMetric{metric}, cdt, []core.SubscribedPlugin{}) + So(serrs, ShouldBeNil) } // The cache ttl should be 100ms which is what the plugin exposed (no system default was provided) ttl, err := pool.CacheTTL(tasks[0]) @@ -997,7 +1024,7 @@ func TestRoutingCachingStrategy(t *testing.T) { Convey("Collect metrics", func() { taskID := tasks[rand.Intn(len(tasks))] for i := 0; i < 10; i++ { - cr, errs := c.CollectMetrics([]core.Metric{metric}, time.Now().Add(time.Second*1), taskID, nil) + cr, errs := c.CollectMetrics(taskID, nil) So(errs, ShouldBeEmpty) for i := range cr { So(cr[i].Data(), ShouldContainSubstring, "The mock collected data!") @@ -1044,7 +1071,6 @@ func TestCollectDynamicMetrics(t *testing.T) { t.FailNow() } <-lpe.done - cd := cdata.NewNode() metrics, err := c.metricCatalog.Fetch(core.NewNamespace()) So(err, ShouldBeNil) So(len(metrics), ShouldEqual, 6) @@ -1053,12 +1079,10 @@ func TestCollectDynamicMetrics(t *testing.T) { So(err, ShouldBeNil) So(m, ShouldNotBeNil) - jsonm, err := c.metricCatalog.Get(core.NewNamespace("intel", "mock", "*", "baz"), 1) - So(err, ShouldBeNil) - So(jsonm, ShouldNotBeNil) - - errs := c.validateMetricTypeSubscription(m, cd) + errs := c.subscriptionGroups.validateMetric(m) So(errs, ShouldBeNil) + cdt := cdata.NewTree() + Convey("collects metrics from plugin using native client", func() { lp, err := c.pluginManager.get("collector:mock:2") So(err, ShouldBeNil) @@ -1067,27 +1091,37 @@ func TestCollectDynamicMetrics(t *testing.T) { So(errp, ShouldBeNil) So(pool, ShouldNotBeNil) taskID := uuid.New() + ttl, err := pool.CacheTTL(taskID) So(err, ShouldResemble, strategy.ErrPoolEmpty) So(ttl, ShouldEqual, 0) So(pool.Count(), ShouldEqual, 0) So(pool.SubscriptionCount(), ShouldEqual, 0) - pool.Subscribe(taskID, strategy.UnboundSubscriptionType) - err = c.pluginRunner.runPlugin(lp.Details) - So(pool.Count(), ShouldEqual, 1) - So(pool.SubscriptionCount(), ShouldEqual, 1) - So(err, ShouldBeNil) + + serrs := c.SubscribeDeps(taskID, []core.RequestedMetric{m}, + []core.SubscribedPlugin{subscribedPlugin{ + typeName: "collector", + name: "mock", + version: 2, + config: cdata.NewNode(), + }}, cdt) + So(serrs, ShouldBeNil) + ttl, err = pool.CacheTTL(taskID) So(err, ShouldBeNil) + So(ttl, ShouldEqual, time.Second) + So(pool.Count(), ShouldEqual, 1) + So(pool.SubscriptionCount(), ShouldEqual, 1) + // The minimum TTL advertised by the plugin is 100ms therefore the TTL for th // pool should be the global cache expiration So(ttl, ShouldEqual, strategy.GlobalCacheExpiration) - mts, errs := c.CollectMetrics([]core.Metric{m}, time.Now().Add(time.Second*1), taskID, nil) + mts, errs := c.CollectMetrics(taskID, nil) hits, err := pool.CacheHits(m.namespace.String(), 2, taskID) So(err, ShouldBeNil) So(hits, ShouldEqual, 0) So(errs, ShouldBeNil) So(len(mts), ShouldEqual, 10) - mts, errs = c.CollectMetrics([]core.Metric{m}, time.Now().Add(time.Second*1), taskID, nil) + mts, errs = c.CollectMetrics(taskID, nil) hits, err = pool.CacheHits(m.namespace.String(), 2, taskID) So(err, ShouldBeNil) @@ -1100,54 +1134,6 @@ func TestCollectDynamicMetrics(t *testing.T) { pool.SelectAndKill(taskID, "unsubscription event") So(pool.Count(), ShouldEqual, 0) So(pool.SubscriptionCount(), ShouldEqual, 0) - Convey("collects metrics from plugin using httpjson client", func() { - lp, err := c.pluginManager.get("collector:mock:1") - So(err, ShouldBeNil) - So(lp, ShouldNotBeNil) - pool, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") - So(errp, ShouldBeNil) - So(pool, ShouldNotBeNil) - ttl, err := pool.CacheTTL(taskID) - So(err, ShouldResemble, strategy.ErrPoolEmpty) - So(ttl, ShouldEqual, 0) - So(pool.Count(), ShouldEqual, 0) - So(pool.SubscriptionCount(), ShouldEqual, 0) - pool.Subscribe("1", strategy.UnboundSubscriptionType) - err = c.pluginRunner.runPlugin(lp.Details) - So(pool.Count(), ShouldEqual, 1) - So(pool.SubscriptionCount(), ShouldEqual, 1) - So(err, ShouldBeNil) - ttl, err = pool.CacheTTL(taskID) - So(err, ShouldBeNil) - So(ttl, ShouldEqual, 1100*time.Millisecond) - mts, errs := c.CollectMetrics([]core.Metric{jsonm}, time.Now().Add(time.Second*1), uuid.New(), nil) - hits, err := pool.CacheHits(jsonm.namespace.String(), jsonm.version, taskID) - So(pool.SubscriptionCount(), ShouldEqual, 1) - So(pool.Strategy, ShouldNotBeNil) - So(len(mts), ShouldBeGreaterThan, 0) - So(err, ShouldBeNil) - So(hits, ShouldEqual, 0) - So(errs, ShouldBeNil) - So(len(mts), ShouldEqual, 10) - mts, errs = c.CollectMetrics([]core.Metric{jsonm}, time.Now().Add(time.Second*1), uuid.New(), nil) - hits, err = pool.CacheHits(m.namespace.String(), 1, taskID) - So(err, ShouldBeNil) - - // todo resolve problem with caching for dynamic metrics - // So(hits, ShouldEqual, 1) - - So(errs, ShouldBeNil) - So(len(mts), ShouldEqual, 10) - - // todo resolve problem with caching for dynamic metrics - // So(pool.AllCacheHits(), ShouldEqual, 1) - // So(pool.AllCacheMisses(), ShouldEqual, 1) - - pool.Unsubscribe("1") - pool.SelectAndKill("1", "unsubscription event") - So(pool.Count(), ShouldEqual, 0) - So(pool.SubscriptionCount(), ShouldEqual, 0) - }) }) c.Stop() time.Sleep(100 * time.Millisecond) @@ -1169,17 +1155,28 @@ func TestFailedPlugin(t *testing.T) { <-lpe.done _, err := c.MetricCatalog() So(err, ShouldBeNil) - // metrics to collect cfg := cdata.NewNode() cfg.AddItem("panic", ctypes.ConfigValueBool{Value: true}) - m := []core.Metric{ + mets := []core.Metric{ fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "foo"), Cfg: cfg, }, } + r := []core.RequestedMetric{} + for _, m := range mets { + r = append(r, m) + } + + cps := []core.SubscribedPlugin{fixtures.NewMockPlugin(core.CollectorPluginType, "mock", 2)} + cdt := cdata.NewTree() + cdt.Add([]string{"intel", "mock"}, cfg) + taskID := "taskID" + serrs := c.SubscribeDeps(taskID, r, cps, cdt) + So(serrs, ShouldBeNil) + // retrieve loaded plugin lp, err := c.pluginManager.get("collector:mock:2") So(err, ShouldBeNil) @@ -1188,31 +1185,26 @@ func TestFailedPlugin(t *testing.T) { Convey("create a pool, add subscriptions and start plugins", func() { pool, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:2") So(errp, ShouldBeNil) - pool.Subscribe("1", strategy.UnboundSubscriptionType) - err = c.pluginRunner.runPlugin(lp.Details) - So(err, ShouldBeNil) - Convey("collect metrics against a plugin that will panic", func() { - So(len(pool.Plugins()), ShouldEqual, 1) + So(pool.Count(), ShouldEqual, 1) var err []error var cr []core.Metric eventMap := map[string]int{} for i := 0; i < MaxPluginRestartCount+1; i++ { - cr, err = c.CollectMetrics(m, time.Now().Add(time.Second*1), uuid.New(), nil) + cr, err = c.CollectMetrics(taskID, nil) So(err, ShouldNotBeNil) So(cr, ShouldBeNil) <-lpe.done - eventMap[lpe.plugin.EventNamespace]++ if i < MaxPluginRestartCount { - <-lpe.done + <-lpe.restarted eventMap[lpe.plugin.EventNamespace]++ So(pool.RestartCount(), ShouldEqual, i+1) So(lpe.plugin.EventNamespace, ShouldEqual, control_event.AvailablePluginRestarted) } } - <-lpe.done + <-lpe.max So(lpe.plugin.EventNamespace, ShouldEqual, control_event.PluginRestartsExceeded) So(eventMap[control_event.AvailablePluginRestarted], ShouldEqual, MaxPluginRestartCount) So(len(pool.Plugins()), ShouldEqual, 0) @@ -1252,7 +1244,7 @@ func TestCollectMetrics(t *testing.T) { cd := cdata.NewNode() cd.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) - m := []core.Metric{} + //m := []core.Metric{} m1 := fixtures.MockMetricType{ Namespace_: core.NewNamespace("intel", "mock", "foo"), Cfg: cd, @@ -1270,19 +1262,29 @@ func TestCollectMetrics(t *testing.T) { lp, err := c.pluginManager.get("collector:mock:1") So(err, ShouldBeNil) So(lp, ShouldNotBeNil) + + r := []core.RequestedMetric{} + for _, m := range []fixtures.MockMetricType{m1, m2, m3} { + r = append(r, m) + } + + cdt := cdata.NewTree() + cdt.Add([]string{"intel", "mock"}, cd) + taskHit := "hitting" + taskNonHit := "not-hitting" + Convey("create a pool, add subscriptions and start plugins", func() { + serrs := c.SubscribeDeps(taskHit, r, []core.SubscribedPlugin{subscribedPlugin{typeName: "collector", name: "mock", version: 1}}, cdt) + So(serrs, ShouldBeNil) + serrs = c.SubscribeDeps(taskNonHit, r, []core.SubscribedPlugin{subscribedPlugin{typeName: "collector", name: "mock", version: 1}}, cdt) + So(serrs, ShouldBeNil) + pool, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") So(errp, ShouldBeNil) - pool.Subscribe("1", strategy.UnboundSubscriptionType) - err = c.pluginRunner.runPlugin(lp.Details) - So(err, ShouldBeNil) - pool.Subscribe("2", strategy.UnboundSubscriptionType) - err = c.pluginRunner.runPlugin(lp.Details) - So(err, ShouldBeNil) - m = append(m, m1, m2, m3) + Convey("collect metrics", func() { for x := 0; x < 4; x++ { - cr, err := c.CollectMetrics(m, time.Now().Add(time.Second*1), uuid.New(), nil) + cr, err := c.CollectMetrics(taskHit, nil) So(err, ShouldBeNil) for i := range cr { So(cr[i].Data(), ShouldContainSubstring, "The mock collected data!") @@ -1304,164 +1306,6 @@ func TestCollectMetrics(t *testing.T) { }) }) }) - - // Not sure what this was supposed to test, because it's actually testing nothing - SkipConvey("Pool", t, func() { - // adjust HB timeouts for test - plugin.PingTimeoutLimit = 1 - plugin.PingTimeoutDurationDefault = time.Second * 1 - // Create controller - c := New(getTestConfig()) - c.pluginRunner.(*runner).monitor.duration = time.Millisecond * 100 - c.Start() - load(c, fixtures.PluginPath) - m := []core.Metric{} - c.CollectMetrics(m, time.Now().Add(time.Second*60), uuid.New(), nil) - c.Stop() - time.Sleep(100 * time.Millisecond) - }) -} - -func TestExpandWildcards(t *testing.T) { - Convey("pluginControl.ExpandWildcards()", t, func() { - // adjust HB timeouts for test - plugin.PingTimeoutLimit = 1 - plugin.PingTimeoutDurationDefault = time.Second * 1 - - // Create controller - config := getTestConfig() - config.Plugins.All.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) - c := New(config) - c.pluginRunner.(*runner).monitor.duration = time.Millisecond * 100 - c.Start() - lpe := newListenToPluginEvent() - c.eventManager.RegisterHandler("Control.PluginLoaded", lpe) - - // Add a global plugin config - c.Config.Plugins.Collector.Plugins["mock"] = newPluginConfigItem(optAddPluginConfigItem("test", ctypes.ConfigValueBool{Value: true})) - - // Load plugin - _, e := load(c, fixtures.JSONRPCPluginPath) - So(e, ShouldBeNil) - <-lpe.done - mts, err := c.MetricCatalog() - So(err, ShouldBeNil) - So(len(mts), ShouldEqual, 4) - Convey("expand metric with an asterisk", func() { - ns := core.NewNamespace("intel", "mock", "*") - c.MatchQueryToNamespaces(ns) - nss, err := c.ExpandWildcards(ns) - So(err, ShouldBeNil) - // "intel/mock/*" should be expanded to all available mock metrics - So(len(nss), ShouldEqual, len(mts)) - So(nss, ShouldResemble, []core.Namespace{ - core.NewNamespace("intel", "mock", "test"), - core.NewNamespace("intel", "mock", "foo"), - core.NewNamespace("intel", "mock", "bar"), - core.NewNamespace("intel", "mock", "*", "baz"), - }) - }) - Convey("expand metric with a tuple", func() { - ns := core.NewNamespace("intel", "mock", "(test|foo|bad)") - c.MatchQueryToNamespaces(ns) - nss, err := c.ExpandWildcards(ns) - So(err, ShouldBeNil) - // '/intel/mock/bad' does not exist in metric catalog and shouldn't be returned - So(len(nss), ShouldEqual, 2) - So(nss, ShouldResemble, []core.Namespace{ - core.NewNamespace("intel", "mock", "test"), - core.NewNamespace("intel", "mock", "foo"), - }) - }) - Convey("expanding for dynamic metrics", func() { - // if asterisk is acceptable by plugin in this location, leave that - ns := core.NewNamespace("intel", "mock", "*", "baz") - c.MatchQueryToNamespaces(ns) - nss, err := c.ExpandWildcards(ns) - So(err, ShouldBeNil) - So(len(nss), ShouldEqual, 1) - So(nss, ShouldResemble, []core.Namespace{ns}) - }) - Convey("expanding for invalid metric name", func() { - // if asterisk is acceptable by plugin in this location, leave that - ns := core.NewNamespace("intel", "mock", "invalid", "metric") - c.MatchQueryToNamespaces(ns) - nss, err := c.ExpandWildcards(ns) - So(err, ShouldNotBeNil) - So(nss, ShouldBeEmpty) - So(err.Error(), ShouldContainSubstring, "Metric not found:") - }) - - c.Stop() - }) -} - -func TestGatherCollectors(t *testing.T) { - Convey("pluginControl.gatherCollectors()", t, func() { - // adjust HB timeouts for test - plugin.PingTimeoutLimit = 1 - plugin.PingTimeoutDurationDefault = time.Second * 1 - - // Create controller - config := getTestConfig() - config.Plugins.All.AddItem("password", ctypes.ConfigValueStr{Value: "testval"}) - c := New(config) - c.pluginRunner.(*runner).monitor.duration = time.Millisecond * 100 - c.Start() - lpe := newListenToPluginEvent() - c.eventManager.RegisterHandler("Control.PluginLoaded", lpe) - - // Add a global plugin config - c.Config.Plugins.Collector.Plugins["mock"] = newPluginConfigItem(optAddPluginConfigItem("test", ctypes.ConfigValueBool{Value: true})) - - // Load plugin - _, e := load(c, fixtures.JSONRPCPluginPath) - So(e, ShouldBeNil) - <-lpe.done - - mts, err := c.MetricCatalog() - ns := core.NewNamespace("intel", "mock", "foo") - So(err, ShouldBeNil) - So(len(mts), ShouldEqual, 4) - Convey("it gathers the latest version", func() { - m := []core.Metric{ - fixtures.MockMetricType{ - Namespace_: ns, - }, - } - plgs, errs := c.gatherCollectors(m) - So(errs, ShouldBeNil) - So(plgs, ShouldNotBeEmpty) - So(plgs[0].plugin.Version(), ShouldEqual, 1) - }) - Convey("it gathers the queried version of plugin", func() { - Convey("the version is available", func() { - v := 1 - m := []core.Metric{ - fixtures.MockMetricType{ - Namespace_: ns, - Ver: v, - }, - } - plgs, errs := c.gatherCollectors(m) - So(errs, ShouldBeNil) - So(plgs, ShouldNotBeEmpty) - So(plgs[0].plugin.Version(), ShouldEqual, v) - }) - Convey("the version is not available", func() { - m := []core.Metric{ - fixtures.MockMetricType{ - Namespace_: ns, - Ver: 30, - }, - } - plgs, errs := c.gatherCollectors(m) - So(errs, ShouldNotBeNil) - So(plgs, ShouldBeEmpty) - }) - }) - c.Stop() - }) } func TestPublishMetrics(t *testing.T) { @@ -1491,11 +1335,8 @@ func TestPublishMetrics(t *testing.T) { Convey("Subscribe to file publisher with good config", func() { n := cdata.NewNode() c.Config.Plugins.Publisher.Plugins[lp.Name()] = newPluginConfigItem(optAddPluginConfigItem("file", ctypes.ConfigValueStr{Value: "/tmp/snap-TestPublishMetrics.out"})) - pool, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("publisher:mock-file:3") - So(errp, ShouldBeNil) - pool.Subscribe("1", strategy.UnboundSubscriptionType) - err := c.pluginRunner.runPlugin(lp.Details) - So(err, ShouldBeNil) + serrs := c.SubscribeDeps("1", []core.RequestedMetric{}, []core.SubscribedPlugin{subscribedPlugin{typeName: "publisher", name: "mock-file", version: 3}}, cdata.NewTree()) + So(serrs, ShouldBeNil) time.Sleep(2500 * time.Millisecond) Convey("Publish to file", func() { @@ -1506,7 +1347,7 @@ func TestPublishMetrics(t *testing.T) { enc := gob.NewEncoder(&buf) enc.Encode(metrics) contentType := plugin.SnapGOBContentType - errs := c.PublishMetrics(contentType, buf.Bytes(), "mock-file", 3, n.Table(), uuid.New()) + errs := c.PublishMetrics(contentType, buf.Bytes(), "mock-file", 3, n.Table(), "1") So(errs, ShouldBeNil) ap := c.AvailablePlugins() So(ap, ShouldNotBeEmpty) @@ -1544,11 +1385,8 @@ func TestProcessMetrics(t *testing.T) { Convey("Subscribe to passthru processor with good config", func() { n := cdata.NewNode() - pool, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("processor:passthru:1") - So(errp, ShouldBeNil) - pool.Subscribe("1", strategy.UnboundSubscriptionType) - err := c.pluginRunner.runPlugin(lp.Details) - So(err, ShouldBeNil) + serrs := c.SubscribeDeps("1", []core.RequestedMetric{}, []core.SubscribedPlugin{subscribedPlugin{typeName: "processor", name: "passthru", version: 1}}, cdata.NewTree()) + So(serrs, ShouldBeNil) time.Sleep(2500 * time.Millisecond) Convey("process metrics", func() { @@ -1559,7 +1397,7 @@ func TestProcessMetrics(t *testing.T) { enc := gob.NewEncoder(&buf) enc.Encode(metrics) contentType := plugin.SnapGOBContentType - _, ct, errs := c.ProcessMetrics(contentType, buf.Bytes(), "passthru", 1, n.Table(), uuid.New()) + _, ct, errs := c.ProcessMetrics(contentType, buf.Bytes(), "passthru", 1, n.Table(), "1") So(errs, ShouldBeEmpty) mts := []plugin.MetricType{} dec := gob.NewDecoder(bytes.NewBuffer(ct)) @@ -1582,23 +1420,37 @@ func TestProcessMetrics(t *testing.T) { }) } -type listenToPluginSubscriptionEvent struct { - plugin *mockPluginEvent - done chan struct{} +type listenToPluginEvents struct { + plugin *mockPluginEvent + load chan struct{} + sub chan struct{} + unsub chan struct{} + started chan struct{} } -func newListenToPluginSubscriptionEvent() *listenToPluginSubscriptionEvent { - return &listenToPluginSubscriptionEvent{ - done: make(chan struct{}), - plugin: &mockPluginEvent{}, +func newListenToPluginEvents() *listenToPluginEvents { + return &listenToPluginEvents{ + load: make(chan struct{}), + unsub: make(chan struct{}), + sub: make(chan struct{}), + started: make(chan struct{}), + plugin: &mockPluginEvent{}, } } -func (l *listenToPluginSubscriptionEvent) HandleGomitEvent(e gomit.Event) { +func (l *listenToPluginEvents) HandleGomitEvent(e gomit.Event) { switch v := e.Body.(type) { - case *control_event.MovePluginSubscriptionEvent: + case *control_event.PluginSubscriptionEvent: l.plugin.EventNamespace = v.Namespace() - l.done <- struct{}{} + l.sub <- struct{}{} + case *control_event.PluginUnsubscriptionEvent: + l.plugin.EventNamespace = v.Namespace() + l.unsub <- struct{}{} + case *control_event.LoadPluginEvent: + l.plugin.EventNamespace = v.Namespace() + l.load <- struct{}{} + case *control_event.StartPluginEvent: + l.started <- struct{}{} default: controlLogger.WithFields(log.Fields{ "event:": v.Namespace(), @@ -1606,13 +1458,15 @@ func (l *listenToPluginSubscriptionEvent) HandleGomitEvent(e gomit.Event) { }).Info("Unhandled Event") } } + func TestMetricSubscriptionToNewVersion(t *testing.T) { Convey("Given a metric that is being collected at v1", t, func() { c := New(getTestConfig()) - lpe := newListenToPluginSubscriptionEvent() + lpe := newListenToPluginEvents() c.eventManager.RegisterHandler("TestMetricSubscriptionToNewVersion", lpe) c.Start() _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock1")) + <-lpe.load So(err, ShouldBeNil) So(len(c.pluginManager.all()), ShouldEqual, 1) lp, err2 := c.pluginManager.get("collector:mock:1") @@ -1625,10 +1479,16 @@ func TestMetricSubscriptionToNewVersion(t *testing.T) { Ver: 0, } So(metric.Version(), ShouldEqual, 0) - serr := c.SubscribeDeps("testTaskID", []core.Metric{metric}, []core.Plugin{}) + ct := cdata.NewTree() + n := cdata.NewNode() + n.AddItem("pass", ctypes.ConfigValueBool{true}) + ct.Add([]string{""}, n) + serr := c.SubscribeDeps("testTaskID", []core.RequestedMetric{metric}, []core.SubscribedPlugin{}, ct) + <-lpe.sub // wait for subscription event + <-lpe.started So(serr, ShouldBeNil) // collect metrics as a sanity check that everything is setup correctly - mts, errs := c.CollectMetrics([]core.Metric{metric}, time.Now(), "testTaskID", nil) + mts, errs := c.CollectMetrics("testTaskID", nil) So(errs, ShouldBeNil) So(len(mts), ShouldEqual, 1) // ensure the data coming back is from v1. V1's data is type string @@ -1640,17 +1500,13 @@ func TestMetricSubscriptionToNewVersion(t *testing.T) { So(err, ShouldBeNil) select { // Wait on subscriptionMovedEvent - case <-lpe.done: - case <-time.After(10 * time.Second): - fmt.Println("timeout waiting for move subscription event") + case <-lpe.sub: + case <-time.After(3 * time.Second): + fmt.Println("timeout waiting for subscription event") So(false, ShouldEqual, true) } - // Check for subscription movement. - // Give some time for subscription to be moved. - var pool1 strategy.Pool - var errp error - pool1, errp = c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") + pool1, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") So(errp, ShouldBeNil) So(pool1.SubscriptionCount(), ShouldEqual, 0) @@ -1658,7 +1514,7 @@ func TestMetricSubscriptionToNewVersion(t *testing.T) { So(errp, ShouldBeNil) So(pool2.SubscriptionCount(), ShouldEqual, 1) - mts, errs = c.CollectMetrics([]core.Metric{metric}, time.Now(), "testTaskID", nil) + mts, errs = c.CollectMetrics("testTaskID", nil) So(len(mts), ShouldEqual, 1) // ensure the data coming back is from v2, V2's data is type int @@ -1672,25 +1528,26 @@ func TestMetricSubscriptionToNewVersion(t *testing.T) { func TestMetricSubscriptionToOlderVersion(t *testing.T) { Convey("Given a metric that is being collected at v2", t, func() { c := New(getTestConfig()) - lpe := newListenToPluginSubscriptionEvent() + lpe := newListenToPluginEvents() c.eventManager.RegisterHandler("TestMetricSubscriptionToOlderVersion", lpe) c.Start() _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock2")) + <-lpe.load So(err, ShouldBeNil) So(len(c.pluginManager.all()), ShouldEqual, 1) lp, err2 := c.pluginManager.get("collector:mock:2") So(err2, ShouldBeNil) So(lp.Name(), ShouldResemble, "mock") - //Subscribe deps to create pools. - metric := fixtures.MockMetricType{ - Namespace_: core.NewNamespace("intel", "mock", "foo"), - Cfg: cdata.NewNode(), - Ver: 0, - } - serr := c.SubscribeDeps("testTaskID", []core.Metric{metric}, []core.Plugin{}) + requestedMetric := fixtures.NewMockRequestedMetric( + core.NewNamespace("intel", "mock", "bar"), + 0, + ) + serr := c.SubscribeDeps("testTaskID", []core.RequestedMetric{requestedMetric}, []core.SubscribedPlugin{}, cdata.NewTree()) + <-lpe.sub // wait for subscription event + <-lpe.started So(serr, ShouldBeNil) // collect metrics as a sanity check that everything is setup correctly - mts, errs := c.CollectMetrics([]core.Metric{metric}, time.Now(), "testTaskID", nil) + mts, errs := c.CollectMetrics("testTaskID", nil) So(errs, ShouldBeNil) So(len(mts), ShouldEqual, 1) // ensure the data coming back is from v2. V2's data is type int @@ -1702,16 +1559,23 @@ func TestMetricSubscriptionToOlderVersion(t *testing.T) { Convey("Loading v1 of that plugin and unloading v2 should move subscriptions to older version", func() { // Load version snap-plugin-collector-mock2 _, err = load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock1")) + <-lpe.load So(err, ShouldBeNil) - _, err = c.Unload(mockv2) + unloadedPlugin, err := c.Unload(mockv2) So(err, ShouldBeNil) + So(unloadedPlugin, ShouldNotBeNil) + <-lpe.unsub + _, subscriptionErros, serr := c.subscriptionGroups.Get("testTaskID") + So(subscriptionErros, ShouldBeNil) + So(serr, ShouldBeNil) + select { - // Wait on subscriptionMovedEvent - case <-lpe.done: - case <-time.After(10 * time.Second): - fmt.Println("Timeout waiting for move subscription event") + case <-lpe.sub: + case <-time.After(3 * time.Second): + fmt.Println("timeout waiting for subscription event") So(false, ShouldEqual, true) } + // Check for subscription movement. // Give some time for subscription to be moved. var pool1 strategy.Pool @@ -1725,15 +1589,258 @@ func TestMetricSubscriptionToOlderVersion(t *testing.T) { So(errp, ShouldBeNil) So(pool2.SubscriptionCount(), ShouldEqual, 1) - mts, errs = c.CollectMetrics([]core.Metric{metric}, time.Now(), "testTaskID", nil) + mts, errs := c.CollectMetrics("testTaskID", nil) So(errs, ShouldBeEmpty) So(len(mts), ShouldEqual, 1) // ensure the data coming back is from v1, V1's data is type string - _, ok = mts[0].Data().(string) + _, ok := mts[0].Data().(string) So(ok, ShouldEqual, true) }) c.Stop() }) } + +func TestDynamicMetricSubscriptionLoad(t *testing.T) { + Convey("Given a dynamic metric that is being collected", t, func() { + log.SetLevel(log.DebugLevel) + c := New(getTestConfig()) + lpe := newListenToPluginEvents() + c.eventManager.RegisterHandler("TestDynamicMetricSubscriptionLoad", lpe) + c.Start() + _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + So(len(c.pluginManager.all()), ShouldEqual, 1) + lp, err2 := c.pluginManager.get("collector:mock:1") + So(err2, ShouldBeNil) + So(lp.Name(), ShouldResemble, "mock") + //Subscribe deps to create pools. + metric := fixtures.MockMetricType{ + Namespace_: core.NewNamespace("intel").AddDynamicElement("*", "dynamic request"), + } + ct := cdata.NewTree() + n := cdata.NewNode() + n.AddItem("pass", ctypes.ConfigValueBool{true}) + ct.Add([]string{""}, n) + serr := c.SubscribeDeps("testTaskID", []core.RequestedMetric{metric}, []core.SubscribedPlugin{}, ct) + <-lpe.load // wait for load event + <-lpe.sub // wait for subscription event + So(serr, ShouldBeNil) + // collect metrics as a sanity check that everything is setup correctly + mts1, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts1), ShouldBeGreaterThan, 1) + // ensure the data coming back is from v1. V1's data is type string + _, ok := mts1[0].Data().(string) + So(ok, ShouldEqual, true) + Convey("Loading another plugin should add subscriptions", func() { + // Load version snap-plugin-collector-anothermock + _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-anothermock1")) + So(err, ShouldBeNil) + <-lpe.load // wait for load event + <-lpe.sub // wait for subscription event + + pool1, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") + So(errp, ShouldBeNil) + So(pool1.SubscriptionCount(), ShouldEqual, 1) + + pool2, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:anothermock:1") + So(errp, ShouldBeNil) + So(pool2.SubscriptionCount(), ShouldEqual, 1) + + mts2, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts2), ShouldBeGreaterThan, len(mts1)) + + }) + c.Stop() + }) +} + +func TestDynamicMetricSubscriptionUnload(t *testing.T) { + Convey("Given a dynamic metric that is being collected", t, func() { + c := New(getTestConfig()) + lpe := newListenToPluginEvents() + c.eventManager.RegisterHandler("TestDynamicMetricSubscriptionUnload", lpe) + c.Start() + _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + _, err = load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-anothermock1")) + So(err, ShouldBeNil) + So(len(c.pluginManager.all()), ShouldEqual, 2) + lpMock, err2 := c.pluginManager.get("collector:mock:1") + So(err2, ShouldBeNil) + So(lpMock.Name(), ShouldResemble, "mock") + lpAMock, err3 := c.pluginManager.get("collector:anothermock:1") + So(err3, ShouldBeNil) + So(lpAMock.Name(), ShouldResemble, "anothermock") + + //Subscribe deps to create pools. + metric := fixtures.MockMetricType{ + Namespace_: core.NewNamespace("intel").AddDynamicElement("*", "dynamic request"), + } + ct := cdata.NewTree() + n := cdata.NewNode() + n.AddItem("pass", ctypes.ConfigValueBool{true}) + ct.Add([]string{""}, n) + serr := c.SubscribeDeps("testTaskID", []core.RequestedMetric{metric}, []core.SubscribedPlugin{}, ct) + <-lpe.sub + subsCount := 0 + L: + for { + select { + case <-lpe.load: + subsCount += 1 + case <-time.After(6 * time.Second): + fmt.Println("timeout waiting for subscription event") + So(false, ShouldEqual, true) + default: + if subsCount == 2 { + break L + } + } + } + So(serr, ShouldBeNil) + // collect metrics as a sanity check that everything is setup correctly + mts1, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts1), ShouldBeGreaterThan, 1) + Convey("Unloading mock plugin should remove its subscriptions", func() { + pool1, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") + So(errp, ShouldBeNil) + So(pool1.SubscriptionCount(), ShouldEqual, 1) + _, err = c.Unload(lpMock) + So(err, ShouldBeNil) + <-lpe.unsub + <-lpe.sub + So(pool1.SubscriptionCount(), ShouldEqual, 0) + pool2, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:anothermock:1") + So(errp, ShouldBeNil) + So(pool2.SubscriptionCount(), ShouldEqual, 1) + mts2, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts2), ShouldBeLessThan, len(mts1)) + + // ensure the data coming back is from another mock (values over 9000) + val, ok := mts2[0].Data().(int) + So(ok, ShouldEqual, true) + So(val, ShouldBeGreaterThan, 9000) + }) + c.Stop() + }) +} + +func TestDynamicMetricSubscriptionLoadLessMetrics(t *testing.T) { + Convey("Given a dynamic metric that is being collected", t, func() { + log.SetLevel(log.DebugLevel) + c := New(getTestConfig()) + testLessMetrics := cdata.NewNode() + testLessMetrics.AddItem("test-less", ctypes.ConfigValueBool{true}) + c.Config.Plugins.Collector.All = testLessMetrics + + lpe := newListenToPluginEvents() + c.eventManager.RegisterHandler("TestDynamicMetricSubscriptionLoadLessMetrics", lpe) + c.Start() + _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + So(len(c.pluginManager.all()), ShouldEqual, 1) + lp, err2 := c.pluginManager.get("collector:mock:1") + So(err2, ShouldBeNil) + So(lp.Name(), ShouldResemble, "mock") + //Subscribe deps to create pools. + metric := fixtures.MockMetricType{ + Namespace_: core.NewNamespace("intel").AddDynamicElement("*", "dynamic request"), + } + ct := cdata.NewTree() + n := cdata.NewNode() + n.AddItem("pass", ctypes.ConfigValueBool{true}) + n.AddItem("test-less", ctypes.ConfigValueBool{true}) + ct.Add([]string{""}, n) + serr := c.SubscribeDeps("testTaskID", []core.RequestedMetric{metric}, []core.SubscribedPlugin{}, ct) + <-lpe.load // wait for load event + <-lpe.sub // wait for subscription event + So(serr, ShouldBeNil) + lpMock, err2 := c.pluginManager.get("collector:mock:1") + So(err2, ShouldBeNil) + So(lpMock.Name(), ShouldResemble, "mock") + // collect metrics as a sanity check that everything is setup correctly + mts1, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts1), ShouldBeGreaterThan, 1) + Convey("metrics are collected from mock1", func() { + for _, m := range mts1 { + if strings.Contains(m.Namespace().String(), "host") { + val, ok := m.Data().(int64) + So(ok, ShouldEqual, true) + So(val, ShouldBeLessThan, 100) + } else { + _, ok := m.Data().(string) + So(ok, ShouldEqual, true) + } + } + }) + // ensure the data coming back is from v1. V1's data is type string + _, ok := mts1[0].Data().(string) + So(ok, ShouldEqual, true) + Convey("Loading higher plugin version with less metrics", func() { + // Load version snap-plugin-collector-mock2 + _, err := load(c, path.Join(fixtures.SnapPath, "plugin", "snap-plugin-collector-mock2")) + So(err, ShouldBeNil) + <-lpe.load // wait for load event + <-lpe.sub // wait for subscription event + + pool1, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") + So(errp, ShouldBeNil) + So(pool1.SubscriptionCount(), ShouldEqual, 1) + + pool2, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:2") + So(errp, ShouldBeNil) + So(pool2.SubscriptionCount(), ShouldEqual, 1) + + mts2, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts2), ShouldEqual, len(mts1)) + + Convey("metrics are collected from mock1 and mock2", func() { + // ensure the data coming back is from mock 1 and mock 2 + for _, m := range mts2 { + if strings.Contains(m.Namespace().String(), "host") || strings.Contains(m.Namespace().String(), "bar") { + val, ok := m.Data().(int) + So(ok, ShouldEqual, true) + So(val, ShouldBeGreaterThan, 1000) + } else { + _, ok := m.Data().(string) + So(ok, ShouldEqual, true) + } + } + }) + + Convey("Unloading lower plugin version", func() { + _, err = c.Unload(lpMock) + So(err, ShouldBeNil) + <-lpe.unsub + + pool1, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:1") + So(errp, ShouldBeNil) + So(pool1.SubscriptionCount(), ShouldEqual, 0) + + pool2, errp := c.pluginRunner.AvailablePlugins().getOrCreatePool("collector:mock:2") + So(errp, ShouldBeNil) + So(pool2.SubscriptionCount(), ShouldEqual, 1) + + mts3, errs := c.CollectMetrics("testTaskID", nil) + So(errs, ShouldBeNil) + So(len(mts3), ShouldBeLessThan, len(mts2)) + + // ensure the data coming back is from mock 2 (values over 1000) + for _, m := range mts3 { + val, ok := m.Data().(int) + So(ok, ShouldEqual, true) + So(val, ShouldBeGreaterThan, 1000) + } + }) + }) + c.Stop() + }) +} diff --git a/control/fixtures/fixtures.go b/control/fixtures/fixtures.go index 9b6bca978..12aff7931 100644 --- a/control/fixtures/fixtures.go +++ b/control/fixtures/fixtures.go @@ -115,15 +115,40 @@ func (m *mockMetric) Data() interface{} { return m.data } +func NewMockPlugin(plgType core.PluginType, name string, version int) MockPlugin { + return MockPlugin{ + pluginType: plgType, + name: name, + ver: version, + } +} + // mocks a plugin -type mockPlugin struct { +type MockPlugin struct { pluginType core.PluginType name string ver int config *cdata.ConfigDataNode } -func (m mockPlugin) Name() string { return m.name } -func (m mockPlugin) TypeName() string { return m.pluginType.String() } -func (m mockPlugin) Version() int { return m.ver } -func (m mockPlugin) Config() *cdata.ConfigDataNode { return m.config } +func (m MockPlugin) Name() string { return m.name } +func (m MockPlugin) TypeName() string { return m.pluginType.String() } +func (m MockPlugin) Version() int { return m.ver } +func (m MockPlugin) Config() *cdata.ConfigDataNode { return m.config } + +type MockRequestedMetric struct { + namespace core.Namespace + version int +} + +func NewMockRequestedMetric(ns core.Namespace, ver int) MockRequestedMetric { + return MockRequestedMetric{namespace: ns, version: ver} +} + +func (m MockRequestedMetric) Version() int { + return m.version +} + +func (m MockRequestedMetric) Namespace() core.Namespace { + return m.namespace +} diff --git a/control/metrics.go b/control/metrics.go index d69956cae..fa3dbe578 100644 --- a/control/metrics.go +++ b/control/metrics.go @@ -131,6 +131,31 @@ type metricType struct { unit string } +type metric struct { + namespace core.Namespace + version int + config *cdata.ConfigDataNode +} + +func (m *metric) Namespace() core.Namespace { + return m.namespace +} + +func (m *metric) Config() *cdata.ConfigDataNode { + return m.config +} + +func (m *metric) Version() int { + return m.version +} + +func (m *metric) Data() interface{} { return nil } +func (m *metric) Description() string { return "" } +func (m *metric) Unit() string { return "" } +func (m *metric) Tags() map[string]string { return nil } +func (m *metric) LastAdvertisedTime() time.Time { return time.Unix(0, 0) } +func (m *metric) Timestamp() time.Time { return time.Unix(0, 0) } + type processesConfigData interface { Process(map[string]ctypes.ConfigValue) (*map[string]ctypes.ConfigValue, *cpolicy.ProcessingErrors) HasRules() bool @@ -260,9 +285,8 @@ func (mc *metricCatalog) GetQueriedNamespaces(ns core.Namespace) ([]core.Namespa return mc.matchedNamespaces(wkey) } -// MatchQuery matches given 'ns' which could contain an asterisk or a tuple and add them to matching map under key 'ns' -// The matched metrics namespaces are also returned (as a []core.Namespace) -func (mc *metricCatalog) MatchQuery(ns core.Namespace) ([]core.Namespace, error) { +// UpdateQueriedNamespaces matches given 'ns' which could contain an asterisk or a tuple and add them to matching map under key 'ns' +func (mc *metricCatalog) UpdateQueriedNamespaces(ns core.Namespace) { mc.mutex.Lock() defer mc.mutex.Unlock() @@ -271,8 +295,6 @@ func (mc *metricCatalog) MatchQuery(ns core.Namespace) ([]core.Namespace, error) // adding matched namespaces to map mc.addItemToMatchingMap(wkey) - - return mc.matchedNamespaces(wkey) } func convertKeysToNamespaces(keys []string) []core.Namespace { @@ -290,6 +312,16 @@ func convertKeysToNamespaces(keys []string) []core.Namespace { // addItemToMatchingMap adds `wkey` to matching map (or updates if `wkey` exists) with corresponding cataloged keys as a content; // if this 'wkey' does not match to any cataloged keys, it will be removed from matching map func (mc *metricCatalog) addItemToMatchingMap(wkey string) { + matchedKeys := mc.matchKeys(wkey) + if len(matchedKeys) == 0 { + mc.removeItemFromMatchingMap(wkey) + } else { + mc.mKeys[wkey] = matchedKeys + } +} + +// matchKeys returns all keys matching with provided key +func (mc *metricCatalog) matchKeys(wkey string) []string { matchedKeys := []string{} // wkey contains `.` which should not be interpreted as regexp tokens, but as a single character @@ -306,11 +338,7 @@ func (mc *metricCatalog) addItemToMatchingMap(wkey string) { } matchedKeys = appendIfMissing(matchedKeys, key) } - if len(matchedKeys) == 0 { - mc.removeItemFromMatchingMap(wkey) - } else { - mc.mKeys[wkey] = matchedKeys - } + return matchedKeys } // removeItemFromMatchingMap removes `wkey` from matching map @@ -414,6 +442,14 @@ func (mc *metricCatalog) RmUnloadedPluginMetrics(lp *loadedPlugin) { mc.mutex.Lock() defer mc.mutex.Unlock() mc.tree.DeleteByPlugin(lp) + + // Update metric catalog keys + mc.keys = []string{} + mts := mc.tree.gatherMetricTypes() + for _, m := range mts { + mc.keys = append(mc.keys, m.Namespace().Key()) + } + // update the contents of matching map (mKeys) mc.updateMatchingMap() } diff --git a/control/metrics_test.go b/control/metrics_test.go index f643f7501..7fbab296f 100644 --- a/control/metrics_test.go +++ b/control/metrics_test.go @@ -82,7 +82,7 @@ func TestMetricType(t *testing.T) { } func TestMetricMatching(t *testing.T) { - Convey("metricCatalog.MatchQuery()", t, func() { + Convey("metricCatalog.GetQueriedNamespaces()", t, func() { Convey("verify query support for static metrics", func() { mc := newMetricCatalog() ns := []core.Namespace{ @@ -111,8 +111,10 @@ func TestMetricMatching(t *testing.T) { for _, v := range mt { mc.Add(v) } + Convey("match /mock/foo/*", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "foo", "*")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "foo", "*")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "foo", "*")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 2) So(nss, ShouldResemble, []core.Namespace{ @@ -122,7 +124,8 @@ func TestMetricMatching(t *testing.T) { }) Convey("match /mock/test/*", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "test", "*")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "test", "*")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "test", "*")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 4) So(nss, ShouldResemble, []core.Namespace{ @@ -133,7 +136,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/*/bar", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "*", "bar")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "*", "bar")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "*", "bar")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 2) So(nss, ShouldResemble, []core.Namespace{ @@ -142,13 +146,15 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/*", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "*")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "*")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "*")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, len(ns)) So(nss, ShouldResemble, ns) }) Convey("match /mock/(foo|asdf)/baz", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "(foo|asdf)", "baz")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "(foo|asdf)", "baz")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "(foo|asdf)", "baz")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 2) So(nss, ShouldResemble, []core.Namespace{ @@ -157,7 +163,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/test/(1|2|3)", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "test", "(1|2|3)")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "test", "(1|2|3)")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "test", "(1|2|3)")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 3) So(nss, ShouldResemble, []core.Namespace{ @@ -167,7 +174,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("invalid matching", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "not", "exist", "metric")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "not", "exist", "metric")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "not", "exist", "metric")) So(err, ShouldNotBeNil) So(nss, ShouldBeEmpty) So(err.Error(), ShouldContainSubstring, "Metric not found:") @@ -205,13 +213,15 @@ func TestMetricMatching(t *testing.T) { So(len(mc.Keys()), ShouldEqual, len(ns)) Convey("match /mock/cgroups/*", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "cgroups", "*")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, len(ns)) So(nss, ShouldResemble, ns) }) Convey("match /mock/cgroups/*/bar", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "cgroups", "*", "bar")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "bar")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "bar")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 1) So(nss, ShouldResemble, []core.Namespace{ @@ -219,7 +229,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/cgroups/*/(bar|baz)", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "cgroups", "*", "(bar|baz)")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "(bar|baz)")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "(bar|baz)")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 2) So(nss, ShouldResemble, []core.Namespace{ @@ -228,7 +239,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/cgroups/*/test/*", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "cgroups", "*", "test", "*")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "test", "*")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "test", "*")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 4) So(nss, ShouldResemble, []core.Namespace{ @@ -239,7 +251,8 @@ func TestMetricMatching(t *testing.T) { }) }) Convey("match /mock/cgroups/*/test/(1|2|3)", func() { - nss, err := mc.MatchQuery(core.NewNamespace("mock", "cgroups", "*", "test", "(1|2|3)")) + mc.UpdateQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "test", "(1|2|3)")) + nss, err := mc.GetQueriedNamespaces(core.NewNamespace("mock", "cgroups", "*", "test", "(1|2|3)")) So(err, ShouldBeNil) So(len(nss), ShouldEqual, 3) So(nss, ShouldResemble, []core.Namespace{ diff --git a/control/plugin_manager.go b/control/plugin_manager.go index 3b4af6ed5..743564014 100644 --- a/control/plugin_manager.go +++ b/control/plugin_manager.go @@ -496,7 +496,6 @@ func (p *pluginManager) LoadPlugin(details *pluginDetails, emitter gomit.Emitter // UnloadPlugin unloads a plugin from the LoadedPlugins table func (p *pluginManager) UnloadPlugin(pl core.Plugin) (*loadedPlugin, serror.SnapError) { - plugin, err := p.loadedPlugins.get(fmt.Sprintf("%s:%s:%d", pl.TypeName(), pl.Name(), pl.Version())) if err != nil { se := serror.New(ErrPluginNotFound, map[string]interface{}{ @@ -506,6 +505,10 @@ func (p *pluginManager) UnloadPlugin(pl core.Plugin) (*loadedPlugin, serror.Snap }) return nil, se } + pmLogger.WithFields(log.Fields{ + "_block": "unload-plugin", + "path": filepath.Base(plugin.Details.Exec), + }).Info("plugin unload called") if plugin.State != LoadedState { se := serror.New(ErrPluginNotInLoadedState, map[string]interface{}{ diff --git a/control/runner.go b/control/runner.go index d22f2e10d..9fa5f5a79 100644 --- a/control/runner.go +++ b/control/runner.go @@ -24,14 +24,12 @@ import ( "fmt" "os" "path" - "strings" "time" log "github.com/Sirupsen/logrus" "github.com/intelsdi-x/gomit" "github.com/intelsdi-x/snap/control/plugin" - "github.com/intelsdi-x/snap/control/strategy" "github.com/intelsdi-x/snap/core" "github.com/intelsdi-x/snap/core/control_event" "github.com/intelsdi-x/snap/pkg/aci" @@ -190,14 +188,22 @@ func (r *runner) startPlugin(p executablePlugin) (*availablePlugin, error) { if err != nil { return nil, err } - r.availablePlugins.insert(ap) + runnerLog.WithFields(log.Fields{ "_block": "start-plugin", "available-plugin": ap.String(), "available-plugin-type": ap.TypeName(), }).Info("available plugin started") + defer r.emitter.Emit(&control_event.StartPluginEvent{ + Name: ap.Name(), + Version: ap.Version(), + Type: int(ap.Type()), + Key: ap.key, + Id: ap.ID(), + }) + return ap, nil } @@ -285,205 +291,6 @@ func (r *runner) HandleGomitEvent(e gomit.Event) { if err != nil { return } - case *control_event.UnloadPluginEvent: - // On plugin unload, first find the key and pool info for the plugin being unloaded - // so that we can determine what, if any, subscriptions need to be moved to another - // version of that plugin (if one is available). - r.availablePlugins.RLock() - var pool strategy.Pool - var k string - for key, p := range r.availablePlugins.table { - tnv := strings.Split(key, ":") - if core.PluginType(v.Type).String() == tnv[0] && v.Name == tnv[1] && v.Version == p.Version() { - pool = p - k = key - break - } - } - - r.availablePlugins.RUnlock() - if pool == nil { - // if we didn't find the named plugin in one of our pools, then simply return - return - } - // Otherwise, we need to roll any subscriptions that may have existed for the plugin being - // unloaded to a new version (if we can find one). First, check for the highest version of the - // plugin being unloaded that is still available (if there is one) - plugin, err := r.pluginManager.get(fmt.Sprintf("%s:%s:%d", core.PluginType(v.Type).String(), v.Name, -1)) - if err != nil { - if err == ErrPluginNotFound { - // if we got to here via a "PluginNotFoundError", then we can't find - // another version of this plugin, so we need to kill all instances - // of the plugin from this pool - pool.KillAll("Last version of plugin unloaded; no additional versions to roll over to") - } - return - } - // if we found another version, then move the subscriptions that are not currently bound to a - // plugin version to our pool - newPool, err := r.availablePlugins.getOrCreatePool(plugin.Key()) - if err != nil { - return - } - subs := pool.MoveSubscriptions(newPool) - // Start new plugins in newPool if needed - if newPool.Eligible() { - e := r.restartPlugin(plugin.Key()) - if e != nil { - runnerLog.WithFields(log.Fields{ - "_block": "handle-events", - }).Error(e.Error()) - return - } - } - // Finally, remove the unloaded plugin from available plugins - r.availablePlugins.Lock() - delete(r.availablePlugins.table, k) - r.availablePlugins.Unlock() - pool.RLock() - defer pool.RUnlock() - if len(subs) != 0 { - runnerLog.WithFields(log.Fields{ - "_block": "subscribe-pool", - "event": v.Namespace(), - "plugin-name": v.Name, - "plugin-version": v.Version, - "plugin-type": core.PluginType(v.Type).String(), - }).Info("pool with subscriptions to move found") - for _, sub := range subs { - r.emitter.Emit(&control_event.PluginSubscriptionEvent{ - PluginName: v.Name, - PluginVersion: v.Version, - TaskId: sub.TaskID, - PluginType: v.Type, - SubscriptionType: int(strategy.UnboundSubscriptionType), - }) - r.emitter.Emit(&control_event.PluginUnsubscriptionEvent{ - PluginName: v.Name, - PluginVersion: pool.Version(), - TaskId: sub.TaskID, - PluginType: v.Type, - }) - r.emitter.Emit(&control_event.MovePluginSubscriptionEvent{ - PluginName: v.Name, - PreviousVersion: pool.Version(), - NewVersion: v.Version, - TaskId: sub.TaskID, - PluginType: v.Type, - }) - } - } - case *control_event.LoadPluginEvent: - // On loaded plugin event all subscriptions that are not bound to a specific version - // need to moved to the loaded version if it's version is greater than the currently - // available plugin. - var pool strategy.Pool - //k := fmt.Sprintf("%v:%v:%v", core.PluginType(v.Type).String(), v.Name, -1) - //pool, _ = r.availablePlugins.getPool(k) - r.availablePlugins.RLock() - currentHighestVersion := -1 - for key, p := range r.availablePlugins.pools() { - // tuple of type name and version - // type @ index 0, name @ index 1, version @ index 2 - tnv := strings.Split(key, ":") - // make sure we don't panic and crash the service if a junk key is retrieved - if len(tnv) != 3 { - runnerLog.WithFields(log.Fields{ - "_block": "subscribe-pool", - "event": v.Namespace(), - "plugin-name": v.Name, - "plugin-version": v.Version, - "plugin-type": core.PluginType(v.Type).String(), - "plugin-signed": v.Signed, - }).Info("pool has bad key ", key) - continue - } - // attempt to find a pool whose type and name are the same, and whose version is - // less than newly loaded plugin. - if core.PluginType(v.Type).String() == tnv[0] && v.Name == tnv[1] && v.Version > p.Version() { - // See if the pool version is higher than the current highest. - // We only want to move subscriptions from the currentHighest - // because that is where subscriptions that are bound to the - // latest version will be. - if p.Version() > currentHighestVersion { - pool = p - currentHighestVersion = p.Version() - } - } - } - r.availablePlugins.RUnlock() - - // now check to see if anything was put where pool points. - // if not, there are no older pools whose subscriptions need to be - // moved. - if pool == nil { - runnerLog.WithFields(log.Fields{ - "_block": "subscribe-pool", - "event": v.Namespace(), - "plugin-name": v.Name, - "plugin-version": v.Version, - "plugin-type": core.PluginType(v.Type).String(), - }).Info("No previous pool found for loaded plugin") - return - } - plugin, err := r.pluginManager.get(fmt.Sprintf("%s:%s:%d", core.PluginType(v.Type).String(), v.Name, v.Version)) - if err != nil { - return - } - newPool, err := r.availablePlugins.getOrCreatePool(plugin.Key()) - if err != nil { - return - } - - // Move subscriptions to the new, higher versioned pool - subs := pool.MoveSubscriptions(newPool) - if newPool.Eligible() { - e := r.restartPlugin(plugin.Key()) - if e != nil { - runnerLog.WithFields(log.Fields{ - "_block": "handle-events", - }).Error(e.Error()) - return - } - runnerLog.WithFields(log.Fields{ - "_block": "pool eligible", - }).Info("starting plugin") - } - - pool.RLock() - defer pool.RUnlock() - - if len(subs) != 0 { - runnerLog.WithFields(log.Fields{ - "_block": "subscribe-pool", - "event": v.Namespace(), - "plugin-name": v.Name, - "plugin-version": v.Version, - "plugin-type": core.PluginType(v.Type).String(), - }).Info("pool with subscriptions to move found") - for _, sub := range subs { - r.emitter.Emit(&control_event.PluginSubscriptionEvent{ - PluginName: v.Name, - PluginVersion: v.Version, - TaskId: sub.TaskID, - PluginType: v.Type, - SubscriptionType: int(strategy.UnboundSubscriptionType), - }) - r.emitter.Emit(&control_event.PluginUnsubscriptionEvent{ - PluginName: v.Name, - PluginVersion: pool.Version(), - TaskId: sub.TaskID, - PluginType: v.Type, - }) - r.emitter.Emit(&control_event.MovePluginSubscriptionEvent{ - PluginName: v.Name, - PreviousVersion: pool.Version(), - NewVersion: v.Version, - TaskId: sub.TaskID, - PluginType: v.Type, - }) - } - } default: runnerLog.WithFields(log.Fields{ "_block": "handle-events", diff --git a/control/runner_test.go b/control/runner_test.go index 1b42bcc22..5fb3aa751 100644 --- a/control/runner_test.go +++ b/control/runner_test.go @@ -493,6 +493,7 @@ func TestRunnerPluginRunning(t *testing.T) { Convey("stopPlugin", func() { Convey("should return an AvailablePlugin in a Running state", func() { r := newRunner() + r.SetEmitter(new(MockEmitter)) a := plugin.Arg{ PluginLogPath: "/tmp/snap-test-plugin-stop.log", } diff --git a/control/strategy/pool.go b/control/strategy/pool.go index 66d88efed..9ddc63f3c 100644 --- a/control/strategy/pool.go +++ b/control/strategy/pool.go @@ -38,15 +38,6 @@ import ( "github.com/intelsdi-x/snap/core/serror" ) -type SubscriptionType int - -const ( - // this subscription is bound to an explicit version - BoundSubscriptionType SubscriptionType = iota - // this subscription is akin to "latest" and must be moved if a newer version is loaded. - UnboundSubscriptionType -) - var ( // This defines the maximum running instances of a loaded plugin. // It is initialized at runtime via the cli. @@ -65,14 +56,13 @@ type Pool interface { Eligible() bool Insert(a AvailablePlugin) error Kill(id uint32, reason string) - MoveSubscriptions(to Pool) []subscription Plugins() MapAvailablePlugin RLock() RUnlock() SelectAndKill(taskID, reason string) SelectAP(taskID string, configID map[string]ctypes.ConfigValue) (AvailablePlugin, serror.SnapError) Strategy() RoutingAndCaching - Subscribe(taskID string, subType SubscriptionType) + Subscribe(taskID string) SubscriptionCount() int Unsubscribe(taskID string) Version() int @@ -96,7 +86,6 @@ type AvailablePlugin interface { } type subscription struct { - SubType SubscriptionType Version int TaskID string } @@ -204,7 +193,6 @@ func (p *pool) Insert(a AvailablePlugin) error { a.SetID(p.generatePID()) p.plugins[a.ID()] = a - return nil } @@ -244,7 +232,7 @@ func (p *pool) applyPluginMeta(a AvailablePlugin) error { // subscribe adds a subscription to the pool. // Using subscribe is idempotent. -func (p *pool) Subscribe(taskID string, subType SubscriptionType) { +func (p *pool) Subscribe(taskID string) { p.Lock() defer p.Unlock() @@ -253,7 +241,6 @@ func (p *pool) Subscribe(taskID string, subType SubscriptionType) { // to retrieve it for the subscription. p.subs[taskID] = &subscription{ TaskID: taskID, - SubType: subType, Version: p.version, } } @@ -325,6 +312,7 @@ func (p *pool) SelectAndKill(id, reason string) { "taskID": id, "reason": reason, }).Error(err) + return } if err := rp.Stop(reason); err != nil { log.WithFields(log.Fields{ @@ -415,28 +403,6 @@ func (p *pool) generatePID() uint32 { return p.pidCounter } -// MoveSubscriptions moves subscriptions to another pool -func (p *pool) MoveSubscriptions(to Pool) []subscription { - var subs []subscription - // If attempting to move between the same pool - // bail to prevent deadlock. - if to.(*pool) == p { - return []subscription{} - } - p.Lock() - defer p.Unlock() - - for task, sub := range p.subs { - // ensure that this sub was not bound to this pool specifically before moving - if sub.SubType == UnboundSubscriptionType { - subs = append(subs, *sub) - to.Subscribe(task, UnboundSubscriptionType) - delete(p.subs, task) - } - } - return subs -} - // CacheTTL returns the cacheTTL for the pool func (p *pool) CacheTTL(taskID string) (time.Duration, error) { if len(p.plugins) == 0 { diff --git a/control/strategy/pool_test.go b/control/strategy/pool_test.go index 1be70fbb2..32369fd93 100644 --- a/control/strategy/pool_test.go +++ b/control/strategy/pool_test.go @@ -194,7 +194,7 @@ func TestPoolEligibility(t *testing.T) { pool, _ := NewPool(plg.String(), plg) for j := 0; j < tc.Subscriptions; j++ { - pool.Subscribe(strconv.Itoa(j), BoundSubscriptionType) + pool.Subscribe(strconv.Itoa(j)) } Convey(fmt.Sprintf( diff --git a/control/subscription_group_medium_test.go b/control/subscription_group_medium_test.go new file mode 100644 index 000000000..a0f9b32e3 --- /dev/null +++ b/control/subscription_group_medium_test.go @@ -0,0 +1,671 @@ +// +build medium + +/* +http://www.apache.org/licenses/LICENSE-2.0.txt + + +Copyright 2015 Intel Corporation + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package control + +import ( + "net" + "os" + "path" + "testing" + + "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/cdata" + "github.com/intelsdi-x/snap/core/control_event" + "github.com/intelsdi-x/snap/core/serror" + + log "github.com/Sirupsen/logrus" + "github.com/intelsdi-x/gomit" + . "github.com/smartystreets/goconvey/convey" +) + +func TestComparePlugins(t *testing.T) { + Convey("Given new collector:plg:2 and old collector:plg:1", t, func() { + plg2 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 2, + config: cdata.NewNode(), + } + + plg1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 1, + config: cdata.NewNode(), + } + Convey("When comparing new and old plugins", func() { + adds, removes := comparePlugins([]core.SubscribedPlugin{plg2}, []core.SubscribedPlugin{plg1}) + + Convey("Plugins to add and plugins to remove have proper elements", func() { + So(len(adds), ShouldEqual, 1) + So(len(removes), ShouldEqual, 1) + So(subscribedPluginsContain(adds, plg2), ShouldBeTrue) + So(subscribedPluginsContain(adds, plg1), ShouldBeFalse) + So(subscribedPluginsContain(removes, plg2), ShouldBeFalse) + So(subscribedPluginsContain(removes, plg1), ShouldBeTrue) + }) + }) + }) + + Convey("Given new collector:plg:2 and collector:plg:1 and old collector:plg:1", t, func() { + plg1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 2, + config: cdata.NewNode(), + } + + plg2 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 1, + config: cdata.NewNode(), + } + Convey("When comparing new and old plugins", func() { + adds, removes := comparePlugins([]core.SubscribedPlugin{plg1, plg2}, []core.SubscribedPlugin{plg1}) + + Convey("Plugins to add and plugins to remove have proper elements", func() { + So(len(adds), ShouldEqual, 1) + So(len(removes), ShouldEqual, 0) + So(subscribedPluginsContain(adds, plg2), ShouldBeTrue) + So(subscribedPluginsContain(adds, plg1), ShouldBeFalse) + }) + }) + }) + + Convey("Given new collector:plg:1 and old collector:plg:2 and collector:plg:1", t, func() { + plg1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 2, + config: cdata.NewNode(), + } + + plg2 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "plg", + version: 1, + config: cdata.NewNode(), + } + Convey("When comparing new and old plugins", func() { + adds, removes := comparePlugins([]core.SubscribedPlugin{plg1}, []core.SubscribedPlugin{plg1, plg2}) + + Convey("Plugins to add and plugins to remove have proper elements", func() { + So(len(adds), ShouldEqual, 0) + So(len(removes), ShouldEqual, 1) + So(subscribedPluginsContain(removes, plg2), ShouldBeTrue) + So(subscribedPluginsContain(removes, plg1), ShouldBeFalse) + }) + }) + }) +} + +func TestSubscriptionGroups_ProcessStaticNegative(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_Process", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel", "mock", "foo")} + subsPlugin := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{subsPlugin}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldEqual, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, subsPlugin), ShouldBeTrue) + + Convey("loading another mock", func() { + anotherMock := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "anothermock", + version: 1, + config: cdata.NewNode(), + } + + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-anothermock1")) + So(err, ShouldBeNil) + <-lpe.load + serrs := sg.Process() + So(len(serrs), ShouldEqual, 0) + + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldEqual, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, subsPlugin), ShouldBeTrue) + So(subscribedPluginsContain(group.plugins, anotherMock), ShouldBeFalse) + }) + }) + }) +} + +func TestSubscriptionGroups_ProcessStaticPositive(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_Process", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel", "mock", "foo")} + mock1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{mock1}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldEqual, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeTrue) + + Convey("loading another mock", func() { + mock2 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 2, + config: cdata.NewNode(), + } + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock2")) + So(err, ShouldBeNil) + <-lpe.load + serrs := sg.Process() + So(len(serrs), ShouldEqual, 0) + + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldEqual, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeFalse) + So(subscribedPluginsContain(group.plugins, mock2), ShouldBeTrue) + }) + }) + }) +} + +func TestSubscriptionGroups_ProcessDynamicPositive(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_Process", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel").AddDynamicElement("wild", "wild description")} + mock1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{mock1}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldBeGreaterThan, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeTrue) + mts1 := len(group.metrics) + + Convey("loading another mock", func() { + anotherMock1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "anothermock", + version: 1, + config: cdata.NewNode(), + } + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-anothermock1")) + So(err, ShouldBeNil) + <-lpe.load + serrs := sg.Process() + So(len(serrs), ShouldEqual, 0) + + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 2) + So(len(group.metrics), ShouldBeGreaterThan, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeTrue) + So(subscribedPluginsContain(group.plugins, anotherMock1), ShouldBeTrue) + mts2 := len(group.metrics) + So(mts2, ShouldBeGreaterThan, mts1) + }) + }) + }) +} + +func TestSubscriptionGroups_ProcessDynamicNegative(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_Process", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel", "mock").AddDynamicElement("wild", "wild description")} + mock1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{mock1}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldBeGreaterThan, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeTrue) + mts1 := len(group.metrics) + + Convey("loading another mock", func() { + anotherMock1 := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "anothermock", + version: 1, + config: cdata.NewNode(), + } + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-anothermock1")) + So(err, ShouldBeNil) + <-lpe.load + serrs := sg.Process() + So(len(serrs), ShouldEqual, 0) + + So(len(sg.subscriptionMap), ShouldEqual, 1) + group, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(group, ShouldNotBeNil) + So(len(group.plugins), ShouldEqual, 1) + So(len(group.metrics), ShouldBeGreaterThan, 1) + So(len(group.requestedMetrics), ShouldEqual, 1) + So(subscribedPluginsContain(group.plugins, mock1), ShouldBeTrue) + So(subscribedPluginsContain(group.plugins, anotherMock1), ShouldBeFalse) + mts2 := len(group.metrics) + So(mts1, ShouldEqual, mts2) + }) + }) + }) +} + +func TestSubscriptionGroups_AddRemoveStatic(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_AddRemove", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel", "mock", "foo")} + subsPlugin := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{subsPlugin}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + val, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(val, ShouldNotBeNil) + + serrs := sg.Remove("task-id") + <-lpe.unsub + So(len(serrs), ShouldEqual, 0) + So(len(sg.subscriptionMap), ShouldEqual, 0) + }) + }) +} + +func TestSubscriptionGroups_AddRemoveDynamic(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_AddRemove", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with wildcards", func() { + requested := mockRequestedMetric{ + namespace: core.NewNamespace("intel", "mock").AddDynamicElement("wild", "wild description"), + version: -1, + } + subsPlugin := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{subsPlugin}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + val, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(val, ShouldNotBeNil) + + serrs := sg.Remove("task-id") + <-lpe.unsub + So(len(serrs), ShouldEqual, 0) + So(len(sg.subscriptionMap), ShouldEqual, 0) + }) + }) +} + +func TestSubscriptionGroups_GetStatic(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_AddRemove", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel", "mock", "foo")} + subsPlugin := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{subsPlugin}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + val, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(val, ShouldNotBeNil) + + mts, serrs, err := sg.Get("task-id") + So(len(serrs), ShouldEqual, 0) + So(err, ShouldBeNil) + So(len(mts), ShouldEqual, 1) + + mts, serrs, err = sg.Get("task-fake-id") + So(len(serrs), ShouldEqual, 0) + So(err, ShouldNotBeNil) + So(err, ShouldResemble, ErrSubscriptionGroupDoesNotExist) + So(len(mts), ShouldEqual, 0) + }) + }) +} + +func TestSubscriptionGroups_GetDynamic(t *testing.T) { + log.SetLevel(log.DebugLevel) + c := New(getTestSGConfig()) + + lpe := newLstnToPluginEvents() + c.eventManager.RegisterHandler("TestSubscriptionGroups_AddRemove", lpe) + c.Start() + + Convey("Loading a mock collector plugn", t, func() { + _, err := loadPlg(c, path.Join(os.Getenv("SNAP_PATH"), "plugin", "snap-plugin-collector-mock1")) + So(err, ShouldBeNil) + <-lpe.load + + Convey("Subscription group created for requested metric with no wildcards", func() { + requested := mockRequestedMetric{namespace: core.NewNamespace("intel").AddDynamicElement("wild", "wild description")} + subsPlugin := mockSubscribedPlugin{ + typeName: core.CollectorPluginType, + name: "mock", + version: 1, + config: cdata.NewNode(), + } + + sg := newSubscriptionGroups(c) + So(sg, ShouldNotBeNil) + sg.Add("task-id", []core.RequestedMetric{requested}, cdata.NewTree(), []core.SubscribedPlugin{subsPlugin}) + <-lpe.sub + So(len(sg.subscriptionMap), ShouldEqual, 1) + val, ok := sg.subscriptionMap["task-id"] + So(ok, ShouldBeTrue) + So(val, ShouldNotBeNil) + + mts, serrs, err := sg.Get("task-id") + So(len(serrs), ShouldEqual, 0) + So(err, ShouldBeNil) + So(len(mts), ShouldBeGreaterThan, 1) + + mts, serrs, err = sg.Get("task-fake-id") + So(len(serrs), ShouldEqual, 0) + So(err, ShouldNotBeNil) + So(err, ShouldResemble, ErrSubscriptionGroupDoesNotExist) + So(len(mts), ShouldEqual, 0) + }) + }) +} + +type lstnToPluginEvents struct { + load chan struct{} + sub chan struct{} + unsub chan struct{} + started chan struct{} +} + +func newLstnToPluginEvents() *lstnToPluginEvents { + return &lstnToPluginEvents{ + load: make(chan struct{}), + unsub: make(chan struct{}), + sub: make(chan struct{}), + started: make(chan struct{}), + } +} + +func (l *lstnToPluginEvents) HandleGomitEvent(e gomit.Event) { + switch v := e.Body.(type) { + case *control_event.PluginSubscriptionEvent: + l.sub <- struct{}{} + case *control_event.PluginUnsubscriptionEvent: + l.unsub <- struct{}{} + case *control_event.LoadPluginEvent: + l.load <- struct{}{} + case *control_event.StartPluginEvent: + l.started <- struct{}{} + default: + controlLogger.WithFields(log.Fields{ + "event:": v.Namespace(), + "_block": "HandleGomit", + }).Info("Unhandled Event") + } +} + +func getTestSGConfig() *Config { + config := GetDefaultConfig() + config.ListenPort = getTestSGPort() + return config +} + +func getTestSGPort() int { + count := 0 + for count < 1000 { + ln, err := net.Listen("tcp", "127.0.0.1:0") + if err == nil { + p := ln.Addr().(*net.TCPAddr).Port + ln.Close() + return p + } + count++ + } + panic("Could not find an available port") +} + +func loadPlg(c *pluginControl, paths ...string) (core.CatalogedPlugin, serror.SnapError) { + // This is a Travis optimized loading of plugins. From time to time, tests will error in Travis + // due to a timeout when waiting for a response from a plugin. We are going to attempt loading a plugin + // 3 times before letting the error through. Hopefully this cuts down on the number of Travis failures + var e serror.SnapError + var p core.CatalogedPlugin + rp, err := core.NewRequestedPlugin(paths[0]) + if err != nil { + return nil, serror.New(err) + } + if len(paths) > 1 { + rp.SetSignature([]byte{00, 00, 00}) + } + for i := 0; i < 3; i++ { + p, e = c.Load(rp) + if e == nil { + break + } + if e != nil && i == 2 { + return nil, e + + } + } + return p, nil +} + +type mockSubscribedPlugin struct { + typeName core.PluginType + name string + version int + config *cdata.ConfigDataNode +} + +func (msp mockSubscribedPlugin) TypeName() string { + return msp.typeName.String() +} + +func (msp mockSubscribedPlugin) Name() string { + return msp.name +} + +func (msp mockSubscribedPlugin) Version() int { + return msp.version +} + +func (msp mockSubscribedPlugin) Config() *cdata.ConfigDataNode { + return msp.config +} + +type mockRequestedMetric struct { + namespace core.Namespace + version int +} + +func (mrm mockRequestedMetric) Namespace() core.Namespace { + return mrm.namespace +} + +func (mrm mockRequestedMetric) Version() int { + return mrm.version +} + +func subscribedPluginsContain(list []core.SubscribedPlugin, lookup core.SubscribedPlugin) bool { + for _, plugin := range list { + if plugin.TypeName() == lookup.TypeName() && plugin.Name() == lookup.Name() && plugin.Version() == lookup.Version() { + return true + } + } + return false +} diff --git a/core/control_event/control_event.go b/core/control_event/control_event.go index da1843da6..60a8835b9 100644 --- a/core/control_event/control_event.go +++ b/core/control_event/control_event.go @@ -23,6 +23,7 @@ const ( AvailablePluginDead = "Control.AvailablePluginDead" AvailablePluginRestarted = "Control.RestartedAvailablePlugin" PluginRestartsExceeded = "Control.PluginRestartsExceeded" + PluginStarted = "Control.PluginStarted" PluginLoaded = "Control.PluginLoaded" PluginUnloaded = "Control.PluginUnloaded" PluginsSwapped = "Control.PluginsSwapped" @@ -36,6 +37,18 @@ const ( MoveSubscription = "Control.PluginSubscriptionMoved" ) +type StartPluginEvent struct { + Name string + Version int + Type int + Key string + Id uint32 +} + +func (e StartPluginEvent) Namespace() string { + return PluginStarted +} + type LoadPluginEvent struct { Name string Version int @@ -107,11 +120,10 @@ func (s SwapPluginsEvent) Namespace() string { } type PluginSubscriptionEvent struct { - PluginName string - PluginVersion int - PluginType int - SubscriptionType int - TaskId string + PluginName string + PluginVersion int + PluginType int + TaskId string } func (ps PluginSubscriptionEvent) Namespace() string { @@ -138,15 +150,3 @@ type HealthCheckFailedEvent struct { func (hfe HealthCheckFailedEvent) Namespace() string { return HealthCheckFailed } - -type MovePluginSubscriptionEvent struct { - TaskId string - PluginName string - PreviousVersion int - NewVersion int - PluginType int -} - -func (mse MovePluginSubscriptionEvent) Namespace() string { - return MoveSubscription -} diff --git a/grpc/common/common.go b/grpc/common/common.go index 2ba08915a..fbc44cce8 100644 --- a/grpc/common/common.go +++ b/grpc/common/common.go @@ -163,6 +163,18 @@ func ToCoreMetric(mt *Metric) core.Metric { return ret } +func MetricToRequested(mts []*Metric) []core.RequestedMetric { + ret := make([]core.RequestedMetric, len(mts)) + for i, mt := range mts { + met := &metric{ + namespace: ToCoreNamespace(mt.Namespace), + version: int(mt.Version), + } + ret[i] = met + } + return ret +} + // Convert common.Namespace protobuf message to core.Namespace func ToCoreNamespace(n []*NamespaceElement) core.Namespace { var namespace core.Namespace @@ -186,6 +198,37 @@ func ToCoreMetrics(mts []*Metric) []core.Metric { return metrics } +// Convert slice of common.Metric to []core.RequestedMetric +func ToRequestedMetrics(mts []*Metric) []core.RequestedMetric { + metrics := make([]core.RequestedMetric, len(mts)) + for i, mt := range mts { + metrics[i] = ToCoreMetric(mt) + } + return metrics +} + +func RequestedToMetric(requested []core.RequestedMetric) []*Metric { + reqMets := make([]*Metric, len(requested)) + for i, r := range requested { + rm := &Metric{ + Namespace: ToNamespace(r.Namespace()), + Version: int64(r.Version()), + Config: &ConfigMap{}, + Tags: map[string]string{}, + Timestamp: &Time{ + Sec: time.Now().Unix(), + Nsec: int64(time.Now().Nanosecond()), + }, + LastAdvertisedTime: &Time{ + Sec: time.Now().Unix(), + Nsec: int64(time.Now().Nanosecond()), + }, + } + reqMets[i] = rm + } + return reqMets +} + // implements core.SubscribedPlugin type SubPlugin struct { typeName string diff --git a/grpc/controlproxy/controlproxy.go b/grpc/controlproxy/controlproxy.go index 53795b05d..408c614f9 100644 --- a/grpc/controlproxy/controlproxy.go +++ b/grpc/controlproxy/controlproxy.go @@ -25,6 +25,7 @@ import ( "golang.org/x/net/context" "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/cdata" "github.com/intelsdi-x/snap/core/ctypes" "github.com/intelsdi-x/snap/core/serror" "github.com/intelsdi-x/snap/grpc/common" @@ -56,20 +57,6 @@ func New(addr string, port int) (ControlProxy, error) { return ControlProxy{Client: c}, nil } -func (c ControlProxy) ExpandWildcards(namespace core.Namespace) ([]core.Namespace, serror.SnapError) { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(namespace), - } - reply, err := c.Client.ExpandWildcards(getContext(), req) - if err != nil { - return nil, serror.New(err) - } - if reply.Error != nil { - return nil, common.ToSnapError(reply.Error) - } - nss := toNSS(reply.NSS) - return nss, nil -} func (c ControlProxy) PublishMetrics(contentType string, content []byte, pluginName string, pluginVersion int, config map[string]ctypes.ConfigValue, taskID string) []error { req := GetPubProcReq(contentType, content, pluginName, pluginVersion, config, taskID) reply, err := c.Client.PublishMetrics(getContext(), req) @@ -96,7 +83,7 @@ func (c ControlProxy) ProcessMetrics(contentType string, content []byte, pluginN return reply.ContentType, reply.Content, errs } -func (c ControlProxy) CollectMetrics(mts []core.Metric, deadline time.Time, taskID string, AllTags map[string]map[string]string) ([]core.Metric, []error) { +func (c ControlProxy) CollectMetrics(taskID string, AllTags map[string]map[string]string) ([]core.Metric, []error) { var allTags map[string]*rpc.Map for k, v := range AllTags { tags := &rpc.Map{} @@ -110,11 +97,6 @@ func (c ControlProxy) CollectMetrics(mts []core.Metric, deadline time.Time, task allTags[k] = tags } req := &rpc.CollectMetricsRequest{ - Metrics: common.NewMetrics(mts), - Deadline: &common.Time{ - Sec: deadline.Unix(), - Nsec: int64(deadline.Nanosecond()), - }, TaskID: taskID, AllTags: allTags, } @@ -149,9 +131,9 @@ func (c ControlProxy) GetPluginContentTypes(n string, t core.PluginType, v int) return reply.AcceptedTypes, reply.ReturnedTypes, nil } -func (c ControlProxy) ValidateDeps(mts []core.Metric, plugins []core.SubscribedPlugin) []serror.SnapError { +func (c ControlProxy) ValidateDeps(mts []core.RequestedMetric, plugins []core.SubscribedPlugin, ctree *cdata.ConfigDataTree) []serror.SnapError { req := &rpc.ValidateDepsRequest{ - Metrics: common.NewMetrics(mts), + Metrics: common.RequestedToMetric(mts), Plugins: common.ToSubPluginsMsg(plugins), } reply, err := c.Client.ValidateDeps(getContext(), req) @@ -162,8 +144,8 @@ func (c ControlProxy) ValidateDeps(mts []core.Metric, plugins []core.SubscribedP return serrs } -func (c ControlProxy) SubscribeDeps(taskID string, mts []core.Metric, plugins []core.Plugin) []serror.SnapError { - req := depsRequest(taskID, mts, plugins) +func (c ControlProxy) SubscribeDeps(taskID string, requested []core.RequestedMetric, plugins []core.SubscribedPlugin, configTree *cdata.ConfigDataTree) []serror.SnapError { + req := depsRequest(taskID, requested, plugins, configTree) reply, err := c.Client.SubscribeDeps(getContext(), req) if err != nil { return []serror.SnapError{serror.New(err)} @@ -172,8 +154,8 @@ func (c ControlProxy) SubscribeDeps(taskID string, mts []core.Metric, plugins [] return serrs } -func (c ControlProxy) UnsubscribeDeps(taskID string, mts []core.Metric, plugins []core.Plugin) []serror.SnapError { - req := depsRequest(taskID, mts, plugins) +func (c ControlProxy) UnsubscribeDeps(taskID string) []serror.SnapError { + req := &rpc.UnsubscribeDepsRequest{TaskId: taskID} reply, err := c.Client.UnsubscribeDeps(getContext(), req) if err != nil { return []serror.SnapError{serror.New(err)} @@ -182,21 +164,6 @@ func (c ControlProxy) UnsubscribeDeps(taskID string, mts []core.Metric, plugins return serrs } -func (c ControlProxy) MatchQueryToNamespaces(namespace core.Namespace) ([]core.Namespace, serror.SnapError) { - req := &rpc.ExpandWildcardsRequest{ - Namespace: common.ToNamespace(namespace), - } - reply, err := c.Client.MatchQueryToNamespaces(getContext(), req) - if err != nil { - return nil, serror.New(err) - } - if reply.Error != nil { - return nil, common.ToSnapError(reply.Error) - } - nss := toNSS(reply.NSS) - return nss, nil -} - func (c ControlProxy) GetAutodiscoverPaths() []string { req := &common.Empty{} reply, err := c.Client.GetAutodiscoverPaths(getContext(), req) @@ -220,11 +187,11 @@ func getPluginType(t core.PluginType) int32 { return val } -func depsRequest(taskID string, mts []core.Metric, plugins []core.Plugin) *rpc.SubscribeDepsRequest { +func depsRequest(taskID string, requested []core.RequestedMetric, plugins []core.SubscribedPlugin, configTree *cdata.ConfigDataTree) *rpc.SubscribeDepsRequest { req := &rpc.SubscribeDepsRequest{ - Metrics: common.NewMetrics(mts), - Plugins: common.ToCorePluginsMsg(plugins), - TaskId: taskID, + Requested: common.RequestedToMetric(requested), + Plugins: common.ToSubPluginsMsg(plugins), + TaskId: taskID, } return req } diff --git a/grpc/controlproxy/controlproxy_medium_test.go b/grpc/controlproxy/controlproxy_medium_test.go index 76f17e9a9..7bf555b38 100644 --- a/grpc/controlproxy/controlproxy_medium_test.go +++ b/grpc/controlproxy/controlproxy_medium_test.go @@ -31,6 +31,7 @@ import ( "github.com/intelsdi-x/snap/grpc/common" "github.com/intelsdi-x/snap/grpc/controlproxy/rpc" + "github.com/intelsdi-x/snap/core/cdata" . "github.com/smartystreets/goconvey/convey" "golang.org/x/net/context" "google.golang.org/grpc" @@ -49,7 +50,7 @@ type mockClient struct { ContentTypeReply *rpc.GetPluginContentTypesReply ValidateReply *rpc.ValidateDepsReply SubscribeReply *rpc.SubscribeDepsReply - UnsubscribeReply *rpc.SubscribeDepsReply + UnsubscribeReply *rpc.UnsubscribeDepsReply MatchReply *rpc.ExpandWildcardsReply AutoDiscoReply *rpc.GetAutodiscoverPathsReply } @@ -67,12 +68,6 @@ func (mc mockClient) GetPluginContentTypes(ctx context.Context, in *rpc.GetPlugi } return mc.ContentTypeReply, nil } -func (mc mockClient) ExpandWildcards(ctx context.Context, in *rpc.ExpandWildcardsRequest, opts ...grpc.CallOption) (*rpc.ExpandWildcardsReply, error) { - if mc.RpcErr { - return nil, rpcErr - } - return mc.ExpandReply, nil -} func (mc mockClient) CollectMetrics(ctx context.Context, in *rpc.CollectMetricsRequest, opts ...grpc.CallOption) (*rpc.CollectMetricsResponse, error) { if mc.RpcErr { return nil, rpcErr @@ -103,77 +98,12 @@ func (mc mockClient) SubscribeDeps(ctx context.Context, in *rpc.SubscribeDepsReq } return mc.SubscribeReply, nil } -func (mc mockClient) UnsubscribeDeps(ctx context.Context, in *rpc.SubscribeDepsRequest, opts ...grpc.CallOption) (*rpc.SubscribeDepsReply, error) { +func (mc mockClient) UnsubscribeDeps(ctx context.Context, in *rpc.UnsubscribeDepsRequest, opts ...grpc.CallOption) (*rpc.UnsubscribeDepsReply, error) { if mc.RpcErr { return nil, rpcErr } return mc.UnsubscribeReply, nil } -func (mc mockClient) MatchQueryToNamespaces(ctx context.Context, in *rpc.ExpandWildcardsRequest, opts ...grpc.CallOption) (*rpc.ExpandWildcardsReply, error) { - if mc.RpcErr { - return nil, rpcErr - } - return mc.MatchReply, nil -} - -func TestExpandWildcards(t *testing.T) { - Convey("RPC client errors", t, func() { - input := core.NewNamespace("testing", "this") - - proxy := ControlProxy{Client: mockClient{RpcErr: true}} - ns, err := proxy.ExpandWildcards(input) - - Convey("So the error should be passed through", func() { - So(err.Error(), ShouldResemble, rpcErr.Error()) - }) - Convey("So The namespace ShouldBeNil", func() { - So(ns, ShouldBeNil) - }) - }) - - Convey("call to Control.ExpandWildcards returns error", t, func() { - input := core.NewNamespace("testing", "this") - reply := &rpc.ExpandWildcardsReply{ - Error: &common.SnapError{ - ErrorFields: map[string]string{}, - ErrorString: "Error from control", - }, - } - - proxy := ControlProxy{Client: mockClient{ExpandReply: reply}} - ns, err := proxy.ExpandWildcards(input) - - Convey("So the err should be: "+reply.Error.ErrorString, func() { - So(err.Error(), ShouldResemble, common.ToSnapError(reply.Error).Error()) - }) - Convey("So Namespaces should be nil", func() { - So(ns, ShouldBeNil) - }) - }) - - Convey("Successful call", t, func() { - input := core.NewNamespace("testing", "this") - a := core.NewNamespace("testing", "this") - b := core.NewNamespace("stuff", "more") - proto_a := &rpc.ArrString{S: common.ToNamespace(a)} - proto_b := &rpc.ArrString{S: common.ToNamespace(b)} - reply := &rpc.ExpandWildcardsReply{ - Error: nil, - NSS: []*rpc.ArrString{proto_a, proto_b}, - } - - proxy := ControlProxy{Client: mockClient{ExpandReply: reply}} - ns, err := proxy.ExpandWildcards(input) - - Convey("so the err Should be nil", func() { - So(err, ShouldBeNil) - }) - Convey("So namespaces should resemble:"+a.String()+","+b.String(), func() { - So(ns, ShouldResemble, []core.Namespace{a, b}) - }) - - }) -} func TestPublishMetrics(t *testing.T) { Convey("RPC client errors", t, func() { @@ -270,7 +200,7 @@ func TestProcessMetrics(t *testing.T) { func TestCollectMetrics(t *testing.T) { Convey("RPC client errors", t, func() { proxy := ControlProxy{Client: mockClient{RpcErr: true}} - _, errs := proxy.CollectMetrics([]core.Metric{}, time.Now(), "", map[string]map[string]string{}) + _, errs := proxy.CollectMetrics("", map[string]map[string]string{}) Convey("So the error should be passed through", func() { So(errs[0].Error(), ShouldResemble, rpcErr.Error()) @@ -284,7 +214,7 @@ func TestCollectMetrics(t *testing.T) { } proxy := ControlProxy{Client: mockClient{CollectReply: reply}} - _, errs := proxy.CollectMetrics([]core.Metric{}, time.Now(), "", map[string]map[string]string{}) + _, errs := proxy.CollectMetrics("", map[string]map[string]string{}) Convey("So len of errs should be 1", func() { So(len(errs), ShouldEqual, 1) @@ -308,7 +238,7 @@ func TestCollectMetrics(t *testing.T) { } proxy := ControlProxy{Client: mockClient{CollectReply: reply}} - mts, errs := proxy.CollectMetrics([]core.Metric{}, time.Now(), "", map[string]map[string]string{}) + mts, errs := proxy.CollectMetrics("", map[string]map[string]string{}) Convey("So len of errs should be 0", func() { So(len(errs), ShouldEqual, 0) @@ -380,7 +310,7 @@ func TestGetPluginContentTypes(t *testing.T) { func TestValidateDeps(t *testing.T) { Convey("RPC client errors", t, func() { proxy := ControlProxy{Client: mockClient{RpcErr: true}} - errs := proxy.ValidateDeps([]core.Metric{}, []core.SubscribedPlugin{}) + errs := proxy.ValidateDeps([]core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) So(errs, ShouldNotBeNil) So(len(errs), ShouldBeGreaterThan, 0) Convey("So the error should be passed through", func() { @@ -394,7 +324,7 @@ func TestValidateDeps(t *testing.T) { } proxy := ControlProxy{Client: mockClient{ValidateReply: reply}} - errs := proxy.ValidateDeps([]core.Metric{}, []core.SubscribedPlugin{}) + errs := proxy.ValidateDeps([]core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) So(errs, ShouldNotBeNil) So(len(errs), ShouldEqual, 1) Convey("So the error should resemble 'test'", func() { @@ -407,7 +337,7 @@ func TestValidateDeps(t *testing.T) { reply := &rpc.ValidateDepsReply{} proxy := ControlProxy{Client: mockClient{ValidateReply: reply}} - errs := proxy.ValidateDeps([]core.Metric{}, []core.SubscribedPlugin{}) + errs := proxy.ValidateDeps([]core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) Convey("So the there should be no errors", func() { So(len(errs), ShouldEqual, 0) }) @@ -418,7 +348,7 @@ func TestValidateDeps(t *testing.T) { func TestSubscribeDeps(t *testing.T) { Convey("RPC client errors", t, func() { proxy := ControlProxy{Client: mockClient{RpcErr: true}} - errs := proxy.SubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.SubscribeDeps("", []core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) So(errs, ShouldNotBeNil) So(len(errs), ShouldBeGreaterThan, 0) Convey("So the error should be passed through", func() { @@ -432,7 +362,7 @@ func TestSubscribeDeps(t *testing.T) { } proxy := ControlProxy{Client: mockClient{SubscribeReply: reply}} - errs := proxy.SubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.SubscribeDeps("", []core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) So(errs, ShouldNotBeNil) So(len(errs), ShouldEqual, 1) Convey("So the error should resemble 'test'", func() { @@ -445,7 +375,7 @@ func TestSubscribeDeps(t *testing.T) { reply := &rpc.SubscribeDepsReply{} proxy := ControlProxy{Client: mockClient{SubscribeReply: reply}} - errs := proxy.SubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.SubscribeDeps("", []core.RequestedMetric{}, []core.SubscribedPlugin{}, cdata.NewTree()) Convey("So the there should be no errors", func() { So(len(errs), ShouldEqual, 0) }) @@ -456,7 +386,7 @@ func TestSubscribeDeps(t *testing.T) { func TestUnsubscribeDeps(t *testing.T) { Convey("RPC client errors", t, func() { proxy := ControlProxy{Client: mockClient{RpcErr: true}} - errs := proxy.UnsubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.UnsubscribeDeps("") So(errs, ShouldNotBeNil) So(len(errs), ShouldBeGreaterThan, 0) Convey("So the error should be passed through", func() { @@ -465,12 +395,12 @@ func TestUnsubscribeDeps(t *testing.T) { }) Convey("Control.UnsubscribeDeps returns an error", t, func() { - reply := &rpc.SubscribeDepsReply{ + reply := &rpc.UnsubscribeDepsReply{ Errors: []*common.SnapError{&common.SnapError{ErrorFields: map[string]string{}, ErrorString: "test"}}, } proxy := ControlProxy{Client: mockClient{UnsubscribeReply: reply}} - errs := proxy.UnsubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.UnsubscribeDeps("") So(errs, ShouldNotBeNil) So(len(errs), ShouldEqual, 1) Convey("So the error should resemble 'test'", func() { @@ -480,10 +410,10 @@ func TestUnsubscribeDeps(t *testing.T) { }) Convey("Control.UnsubscribeDeps returns successfully", t, func() { - reply := &rpc.SubscribeDepsReply{} + reply := &rpc.UnsubscribeDepsReply{} proxy := ControlProxy{Client: mockClient{UnsubscribeReply: reply}} - errs := proxy.UnsubscribeDeps("", []core.Metric{}, []core.Plugin{}) + errs := proxy.UnsubscribeDeps("") Convey("So the there should be no errors", func() { So(len(errs), ShouldEqual, 0) }) @@ -491,66 +421,6 @@ func TestUnsubscribeDeps(t *testing.T) { }) } -func TestMatchQueryToNamespaces(t *testing.T) { - Convey("RPC client errors", t, func() { - input := core.NewNamespace("testing", "this") - - proxy := ControlProxy{Client: mockClient{RpcErr: true}} - ns, err := proxy.ExpandWildcards(input) - - Convey("So the error should be passed through", func() { - So(err.Error(), ShouldResemble, rpcErr.Error()) - }) - Convey("So The namespace ShouldBeNil", func() { - So(ns, ShouldBeNil) - }) - }) - - Convey("call to Control.MatchQueryToNamespaces returns error", t, func() { - input := core.NewNamespace("testing", "this") - reply := &rpc.ExpandWildcardsReply{ - Error: &common.SnapError{ - ErrorFields: map[string]string{}, - ErrorString: "Error from control", - }, - } - - proxy := ControlProxy{Client: mockClient{MatchReply: reply}} - ns, err := proxy.MatchQueryToNamespaces(input) - - Convey("So the err should be: "+reply.Error.ErrorString, func() { - So(err.Error(), ShouldResemble, common.ToSnapError(reply.Error).Error()) - }) - Convey("So Namespaces should be nil", func() { - So(ns, ShouldBeNil) - }) - }) - - Convey("Control.MatchQueryToNamespaces returns successfully", t, func() { - input := core.NewNamespace("testing", "this") - a := core.NewNamespace("testing", "this") - b := core.NewNamespace("stuff", "more") - proto_a := &rpc.ArrString{S: common.ToNamespace(a)} - proto_b := &rpc.ArrString{S: common.ToNamespace(b)} - reply := &rpc.ExpandWildcardsReply{ - Error: nil, - NSS: []*rpc.ArrString{proto_a, proto_b}, - } - - proxy := ControlProxy{Client: mockClient{MatchReply: reply}} - ns, err := proxy.MatchQueryToNamespaces(input) - - Convey("so the err Should be nil", func() { - So(err, ShouldBeNil) - }) - Convey("So namespaces should resemble:"+a.String()+","+b.String(), func() { - So(ns, ShouldResemble, []core.Namespace{a, b}) - }) - - }) - -} - func TestGetAutoDiscoverPaths(t *testing.T) { Convey("Able to call successfully", t, func() { reply := &rpc.GetAutodiscoverPathsReply{ diff --git a/grpc/controlproxy/rpc/control.pb.go b/grpc/controlproxy/rpc/control.pb.go index c52c82ac4..7dcd875c1 100644 --- a/grpc/controlproxy/rpc/control.pb.go +++ b/grpc/controlproxy/rpc/control.pb.go @@ -38,6 +38,8 @@ It has these top-level messages: ValidateDepsReply SubscribeDepsRequest SubscribeDepsReply + UnsubscribeDepsRequest + UnsubscribeDepsReply Map MapEntry CollectMetricsRequest @@ -182,9 +184,9 @@ func (m *ValidateDepsReply) GetErrors() []*common.SnapError { } type SubscribeDepsRequest struct { - Metrics []*common.Metric `protobuf:"bytes,1,rep,name=Metrics" json:"Metrics,omitempty"` - Plugins []*common.Plugin `protobuf:"bytes,2,rep,name=Plugins" json:"Plugins,omitempty"` - TaskId string `protobuf:"bytes,3,opt,name=TaskId" json:"TaskId,omitempty"` + Requested []*common.Metric `protobuf:"bytes,1,rep,name=Requested" json:"Requested,omitempty"` + Plugins []*common.SubscribedPlugin `protobuf:"bytes,2,rep,name=Plugins" json:"Plugins,omitempty"` + TaskId string `protobuf:"bytes,3,opt,name=TaskId" json:"TaskId,omitempty"` } func (m *SubscribeDepsRequest) Reset() { *m = SubscribeDepsRequest{} } @@ -192,14 +194,14 @@ func (m *SubscribeDepsRequest) String() string { return proto.Compact func (*SubscribeDepsRequest) ProtoMessage() {} func (*SubscribeDepsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } -func (m *SubscribeDepsRequest) GetMetrics() []*common.Metric { +func (m *SubscribeDepsRequest) GetRequested() []*common.Metric { if m != nil { - return m.Metrics + return m.Requested } return nil } -func (m *SubscribeDepsRequest) GetPlugins() []*common.Plugin { +func (m *SubscribeDepsRequest) GetPlugins() []*common.SubscribedPlugin { if m != nil { return m.Plugins } @@ -222,6 +224,31 @@ func (m *SubscribeDepsReply) GetErrors() []*common.SnapError { return nil } +type UnsubscribeDepsRequest struct { + TaskId string `protobuf:"bytes,1,opt,name=TaskId" json:"TaskId,omitempty"` +} + +func (m *UnsubscribeDepsRequest) Reset() { *m = UnsubscribeDepsRequest{} } +func (m *UnsubscribeDepsRequest) String() string { return proto.CompactTextString(m) } +func (*UnsubscribeDepsRequest) ProtoMessage() {} +func (*UnsubscribeDepsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } + +type UnsubscribeDepsReply struct { + Errors []*common.SnapError `protobuf:"bytes,1,rep,name=Errors" json:"Errors,omitempty"` +} + +func (m *UnsubscribeDepsReply) Reset() { *m = UnsubscribeDepsReply{} } +func (m *UnsubscribeDepsReply) String() string { return proto.CompactTextString(m) } +func (*UnsubscribeDepsReply) ProtoMessage() {} +func (*UnsubscribeDepsReply) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } + +func (m *UnsubscribeDepsReply) GetErrors() []*common.SnapError { + if m != nil { + return m.Errors + } + return nil +} + type Map struct { Entries []*MapEntry `protobuf:"bytes,1,rep,name=Entries" json:"Entries,omitempty"` } @@ -229,7 +256,7 @@ type Map struct { func (m *Map) Reset() { *m = Map{} } func (m *Map) String() string { return proto.CompactTextString(m) } func (*Map) ProtoMessage() {} -func (*Map) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } +func (*Map) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } func (m *Map) GetEntries() []*MapEntry { if m != nil { @@ -246,33 +273,17 @@ type MapEntry struct { func (m *MapEntry) Reset() { *m = MapEntry{} } func (m *MapEntry) String() string { return proto.CompactTextString(m) } func (*MapEntry) ProtoMessage() {} -func (*MapEntry) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } +func (*MapEntry) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } type CollectMetricsRequest struct { - TaskID string `protobuf:"bytes,1,opt,name=TaskID" json:"TaskID,omitempty"` - Metrics []*common.Metric `protobuf:"bytes,2,rep,name=Metrics" json:"Metrics,omitempty"` - Deadline *common.Time `protobuf:"bytes,3,opt,name=Deadline" json:"Deadline,omitempty"` - AllTags map[string]*Map `protobuf:"bytes,4,rep,name=AllTags" json:"AllTags,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + TaskID string `protobuf:"bytes,1,opt,name=TaskID" json:"TaskID,omitempty"` + AllTags map[string]*Map `protobuf:"bytes,2,rep,name=AllTags" json:"AllTags,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` } func (m *CollectMetricsRequest) Reset() { *m = CollectMetricsRequest{} } func (m *CollectMetricsRequest) String() string { return proto.CompactTextString(m) } func (*CollectMetricsRequest) ProtoMessage() {} -func (*CollectMetricsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } - -func (m *CollectMetricsRequest) GetMetrics() []*common.Metric { - if m != nil { - return m.Metrics - } - return nil -} - -func (m *CollectMetricsRequest) GetDeadline() *common.Time { - if m != nil { - return m.Deadline - } - return nil -} +func (*CollectMetricsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} } func (m *CollectMetricsRequest) GetAllTags() map[string]*Map { if m != nil { @@ -289,7 +300,7 @@ type CollectMetricsResponse struct { func (m *CollectMetricsResponse) Reset() { *m = CollectMetricsResponse{} } func (m *CollectMetricsResponse) String() string { return proto.CompactTextString(m) } func (*CollectMetricsResponse) ProtoMessage() {} -func (*CollectMetricsResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } +func (*CollectMetricsResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} } func (m *CollectMetricsResponse) GetMetrics() []*common.Metric { if m != nil { @@ -305,7 +316,7 @@ type ExpandWildcardsRequest struct { func (m *ExpandWildcardsRequest) Reset() { *m = ExpandWildcardsRequest{} } func (m *ExpandWildcardsRequest) String() string { return proto.CompactTextString(m) } func (*ExpandWildcardsRequest) ProtoMessage() {} -func (*ExpandWildcardsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} } +func (*ExpandWildcardsRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} } func (m *ExpandWildcardsRequest) GetNamespace() []*common.NamespaceElement { if m != nil { @@ -321,7 +332,7 @@ type ArrString struct { func (m *ArrString) Reset() { *m = ArrString{} } func (m *ArrString) String() string { return proto.CompactTextString(m) } func (*ArrString) ProtoMessage() {} -func (*ArrString) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} } +func (*ArrString) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} } func (m *ArrString) GetS() []*common.NamespaceElement { if m != nil { @@ -338,7 +349,7 @@ type ExpandWildcardsReply struct { func (m *ExpandWildcardsReply) Reset() { *m = ExpandWildcardsReply{} } func (m *ExpandWildcardsReply) String() string { return proto.CompactTextString(m) } func (*ExpandWildcardsReply) ProtoMessage() {} -func (*ExpandWildcardsReply) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} } +func (*ExpandWildcardsReply) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} } func (m *ExpandWildcardsReply) GetNSS() []*ArrString { if m != nil { @@ -361,7 +372,7 @@ type GetAutodiscoverPathsReply struct { func (m *GetAutodiscoverPathsReply) Reset() { *m = GetAutodiscoverPathsReply{} } func (m *GetAutodiscoverPathsReply) String() string { return proto.CompactTextString(m) } func (*GetAutodiscoverPathsReply) ProtoMessage() {} -func (*GetAutodiscoverPathsReply) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} } +func (*GetAutodiscoverPathsReply) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} } func init() { proto.RegisterType((*SerrorReply)(nil), "rpc.SerrorReply") @@ -374,6 +385,8 @@ func init() { proto.RegisterType((*ValidateDepsReply)(nil), "rpc.ValidateDepsReply") proto.RegisterType((*SubscribeDepsRequest)(nil), "rpc.SubscribeDepsRequest") proto.RegisterType((*SubscribeDepsReply)(nil), "rpc.SubscribeDepsReply") + proto.RegisterType((*UnsubscribeDepsRequest)(nil), "rpc.UnsubscribeDepsRequest") + proto.RegisterType((*UnsubscribeDepsReply)(nil), "rpc.UnsubscribeDepsReply") proto.RegisterType((*Map)(nil), "rpc.Map") proto.RegisterType((*MapEntry)(nil), "rpc.MapEntry") proto.RegisterType((*CollectMetricsRequest)(nil), "rpc.CollectMetricsRequest") @@ -397,14 +410,12 @@ const _ = grpc.SupportPackageIsVersion2 type MetricManagerClient interface { // managesMetrics from scheduler GetPluginContentTypes(ctx context.Context, in *GetPluginContentTypesRequest, opts ...grpc.CallOption) (*GetPluginContentTypesReply, error) - ExpandWildcards(ctx context.Context, in *ExpandWildcardsRequest, opts ...grpc.CallOption) (*ExpandWildcardsReply, error) CollectMetrics(ctx context.Context, in *CollectMetricsRequest, opts ...grpc.CallOption) (*CollectMetricsResponse, error) PublishMetrics(ctx context.Context, in *PubProcMetricsRequest, opts ...grpc.CallOption) (*ErrorReply, error) ProcessMetrics(ctx context.Context, in *PubProcMetricsRequest, opts ...grpc.CallOption) (*ProcessMetricsReply, error) ValidateDeps(ctx context.Context, in *ValidateDepsRequest, opts ...grpc.CallOption) (*ValidateDepsReply, error) SubscribeDeps(ctx context.Context, in *SubscribeDepsRequest, opts ...grpc.CallOption) (*SubscribeDepsReply, error) - UnsubscribeDeps(ctx context.Context, in *SubscribeDepsRequest, opts ...grpc.CallOption) (*SubscribeDepsReply, error) - MatchQueryToNamespaces(ctx context.Context, in *ExpandWildcardsRequest, opts ...grpc.CallOption) (*ExpandWildcardsReply, error) + UnsubscribeDeps(ctx context.Context, in *UnsubscribeDepsRequest, opts ...grpc.CallOption) (*UnsubscribeDepsReply, error) GetAutodiscoverPaths(ctx context.Context, in *common.Empty, opts ...grpc.CallOption) (*GetAutodiscoverPathsReply, error) } @@ -425,15 +436,6 @@ func (c *metricManagerClient) GetPluginContentTypes(ctx context.Context, in *Get return out, nil } -func (c *metricManagerClient) ExpandWildcards(ctx context.Context, in *ExpandWildcardsRequest, opts ...grpc.CallOption) (*ExpandWildcardsReply, error) { - out := new(ExpandWildcardsReply) - err := grpc.Invoke(ctx, "/rpc.MetricManager/ExpandWildcards", in, out, c.cc, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *metricManagerClient) CollectMetrics(ctx context.Context, in *CollectMetricsRequest, opts ...grpc.CallOption) (*CollectMetricsResponse, error) { out := new(CollectMetricsResponse) err := grpc.Invoke(ctx, "/rpc.MetricManager/CollectMetrics", in, out, c.cc, opts...) @@ -479,8 +481,8 @@ func (c *metricManagerClient) SubscribeDeps(ctx context.Context, in *SubscribeDe return out, nil } -func (c *metricManagerClient) UnsubscribeDeps(ctx context.Context, in *SubscribeDepsRequest, opts ...grpc.CallOption) (*SubscribeDepsReply, error) { - out := new(SubscribeDepsReply) +func (c *metricManagerClient) UnsubscribeDeps(ctx context.Context, in *UnsubscribeDepsRequest, opts ...grpc.CallOption) (*UnsubscribeDepsReply, error) { + out := new(UnsubscribeDepsReply) err := grpc.Invoke(ctx, "/rpc.MetricManager/UnsubscribeDeps", in, out, c.cc, opts...) if err != nil { return nil, err @@ -488,15 +490,6 @@ func (c *metricManagerClient) UnsubscribeDeps(ctx context.Context, in *Subscribe return out, nil } -func (c *metricManagerClient) MatchQueryToNamespaces(ctx context.Context, in *ExpandWildcardsRequest, opts ...grpc.CallOption) (*ExpandWildcardsReply, error) { - out := new(ExpandWildcardsReply) - err := grpc.Invoke(ctx, "/rpc.MetricManager/MatchQueryToNamespaces", in, out, c.cc, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *metricManagerClient) GetAutodiscoverPaths(ctx context.Context, in *common.Empty, opts ...grpc.CallOption) (*GetAutodiscoverPathsReply, error) { out := new(GetAutodiscoverPathsReply) err := grpc.Invoke(ctx, "/rpc.MetricManager/GetAutodiscoverPaths", in, out, c.cc, opts...) @@ -511,14 +504,12 @@ func (c *metricManagerClient) GetAutodiscoverPaths(ctx context.Context, in *comm type MetricManagerServer interface { // managesMetrics from scheduler GetPluginContentTypes(context.Context, *GetPluginContentTypesRequest) (*GetPluginContentTypesReply, error) - ExpandWildcards(context.Context, *ExpandWildcardsRequest) (*ExpandWildcardsReply, error) CollectMetrics(context.Context, *CollectMetricsRequest) (*CollectMetricsResponse, error) PublishMetrics(context.Context, *PubProcMetricsRequest) (*ErrorReply, error) ProcessMetrics(context.Context, *PubProcMetricsRequest) (*ProcessMetricsReply, error) ValidateDeps(context.Context, *ValidateDepsRequest) (*ValidateDepsReply, error) SubscribeDeps(context.Context, *SubscribeDepsRequest) (*SubscribeDepsReply, error) - UnsubscribeDeps(context.Context, *SubscribeDepsRequest) (*SubscribeDepsReply, error) - MatchQueryToNamespaces(context.Context, *ExpandWildcardsRequest) (*ExpandWildcardsReply, error) + UnsubscribeDeps(context.Context, *UnsubscribeDepsRequest) (*UnsubscribeDepsReply, error) GetAutodiscoverPaths(context.Context, *common.Empty) (*GetAutodiscoverPathsReply, error) } @@ -544,24 +535,6 @@ func _MetricManager_GetPluginContentTypes_Handler(srv interface{}, ctx context.C return interceptor(ctx, in, info, handler) } -func _MetricManager_ExpandWildcards_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ExpandWildcardsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MetricManagerServer).ExpandWildcards(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/rpc.MetricManager/ExpandWildcards", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MetricManagerServer).ExpandWildcards(ctx, req.(*ExpandWildcardsRequest)) - } - return interceptor(ctx, in, info, handler) -} - func _MetricManager_CollectMetrics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(CollectMetricsRequest) if err := dec(in); err != nil { @@ -653,7 +626,7 @@ func _MetricManager_SubscribeDeps_Handler(srv interface{}, ctx context.Context, } func _MetricManager_UnsubscribeDeps_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(SubscribeDepsRequest) + in := new(UnsubscribeDepsRequest) if err := dec(in); err != nil { return nil, err } @@ -665,25 +638,7 @@ func _MetricManager_UnsubscribeDeps_Handler(srv interface{}, ctx context.Context FullMethod: "/rpc.MetricManager/UnsubscribeDeps", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MetricManagerServer).UnsubscribeDeps(ctx, req.(*SubscribeDepsRequest)) - } - return interceptor(ctx, in, info, handler) -} - -func _MetricManager_MatchQueryToNamespaces_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ExpandWildcardsRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(MetricManagerServer).MatchQueryToNamespaces(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/rpc.MetricManager/MatchQueryToNamespaces", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(MetricManagerServer).MatchQueryToNamespaces(ctx, req.(*ExpandWildcardsRequest)) + return srv.(MetricManagerServer).UnsubscribeDeps(ctx, req.(*UnsubscribeDepsRequest)) } return interceptor(ctx, in, info, handler) } @@ -714,10 +669,6 @@ var _MetricManager_serviceDesc = grpc.ServiceDesc{ MethodName: "GetPluginContentTypes", Handler: _MetricManager_GetPluginContentTypes_Handler, }, - { - MethodName: "ExpandWildcards", - Handler: _MetricManager_ExpandWildcards_Handler, - }, { MethodName: "CollectMetrics", Handler: _MetricManager_CollectMetrics_Handler, @@ -742,10 +693,6 @@ var _MetricManager_serviceDesc = grpc.ServiceDesc{ MethodName: "UnsubscribeDeps", Handler: _MetricManager_UnsubscribeDeps_Handler, }, - { - MethodName: "MatchQueryToNamespaces", - Handler: _MetricManager_MatchQueryToNamespaces_Handler, - }, { MethodName: "GetAutodiscoverPaths", Handler: _MetricManager_GetAutodiscoverPaths_Handler, @@ -759,60 +706,58 @@ func init() { } var fileDescriptor0 = []byte{ - // 879 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xac, 0x56, 0xdb, 0x6e, 0xdb, 0x46, - 0x10, 0xb5, 0xc4, 0x48, 0xb2, 0x87, 0xba, 0x20, 0x6b, 0x5b, 0x61, 0xe8, 0xc0, 0x8e, 0x59, 0xf4, - 0x8e, 0x4a, 0x85, 0x02, 0xb4, 0x46, 0x1f, 0x5a, 0xa8, 0xb1, 0xe0, 0x06, 0x86, 0x02, 0x45, 0x52, - 0x52, 0xf4, 0xa1, 0x0f, 0x2b, 0x72, 0x2b, 0x11, 0xa1, 0x48, 0x76, 0x77, 0x19, 0x58, 0xef, 0xfd, - 0x8b, 0xfe, 0x5a, 0x3f, 0xa6, 0x7b, 0x21, 0xa9, 0x4b, 0x18, 0xdb, 0x45, 0xf3, 0x64, 0x73, 0x66, - 0xf6, 0xec, 0x99, 0xcb, 0x99, 0x15, 0xfc, 0x38, 0xf7, 0xf9, 0x22, 0x99, 0x75, 0xdc, 0x68, 0xd9, - 0xf5, 0x43, 0x4e, 0x02, 0xe6, 0xf9, 0xdf, 0xdc, 0x74, 0x59, 0x88, 0xe3, 0xee, 0x9c, 0xc6, 0x6e, - 0xd7, 0x8d, 0x42, 0x4e, 0xa3, 0x20, 0xa6, 0xd1, 0xcd, 0xaa, 0xbb, 0x61, 0xe8, 0x08, 0x0b, 0x8f, - 0x90, 0x21, 0x4c, 0xf6, 0xb3, 0xbb, 0x41, 0x96, 0xcb, 0x28, 0x4c, 0xff, 0xe8, 0x93, 0x4e, 0x03, - 0xcc, 0x09, 0xa1, 0x34, 0xa2, 0x63, 0x12, 0x07, 0x2b, 0xe7, 0xef, 0x12, 0x1c, 0x8f, 0x92, 0xd9, - 0x88, 0x46, 0xee, 0x90, 0x70, 0xea, 0xbb, 0x6c, 0x4c, 0xfe, 0x4c, 0x08, 0xe3, 0xe8, 0x10, 0xcc, - 0xe7, 0xe2, 0x4e, 0x12, 0xf2, 0xe9, 0x2a, 0x26, 0x56, 0xe9, 0x69, 0xe9, 0x8b, 0x03, 0xd4, 0x82, - 0x5a, 0x6a, 0xb4, 0xca, 0xc2, 0x50, 0x47, 0x08, 0x60, 0x14, 0x24, 0x73, 0x3f, 0x7c, 0x89, 0x97, - 0xc4, 0x32, 0x54, 0xd0, 0x31, 0x34, 0xb4, 0xed, 0x0d, 0xa1, 0xcc, 0x8f, 0x42, 0xeb, 0x81, 0x30, - 0x1b, 0xe8, 0x1c, 0xaa, 0xe2, 0xec, 0x1f, 0xfe, 0xdc, 0xaa, 0x88, 0x6f, 0xb3, 0xf7, 0xb0, 0x93, - 0x12, 0xd3, 0xd6, 0x21, 0x8e, 0x51, 0x13, 0xaa, 0x53, 0xcc, 0xde, 0xbe, 0xf0, 0xac, 0xaa, 0x44, - 0x72, 0x9e, 0x00, 0x0c, 0x72, 0xae, 0xd2, 0xab, 0xbe, 0x98, 0x20, 0x63, 0x08, 0xef, 0x35, 0x1c, - 0x4a, 0xde, 0x84, 0xb1, 0x9c, 0xba, 0x0c, 0xbb, 0x1f, 0xf1, 0x35, 0x98, 0xa1, 0xc0, 0x5e, 0xc1, - 0x93, 0x2b, 0xc2, 0x35, 0xef, 0x8d, 0xe3, 0x79, 0x39, 0xea, 0xf0, 0x40, 0xa5, 0xa8, 0xe1, 0xf2, - 0xb4, 0xd5, 0x15, 0x12, 0xb1, 0x22, 0xaf, 0xc8, 0x12, 0x96, 0x75, 0xa8, 0x38, 0xbf, 0x81, 0xfd, - 0x01, 0x48, 0x49, 0x53, 0x54, 0xa9, 0xef, 0xba, 0x24, 0xe6, 0xc4, 0x53, 0x56, 0x9d, 0x94, 0x34, - 0x8f, 0x09, 0x4f, 0x68, 0x98, 0x99, 0xcb, 0xca, 0xdc, 0x80, 0x8a, 0xa2, 0xab, 0x4b, 0xec, 0x60, - 0x38, 0x7c, 0x83, 0x03, 0xdf, 0xc3, 0x9c, 0x5c, 0x92, 0x38, 0x27, 0x79, 0x06, 0xb5, 0xb4, 0x14, - 0x0a, 0xcd, 0xec, 0x35, 0xb3, 0x1a, 0x6b, 0x33, 0xfa, 0x12, 0x6a, 0x9a, 0x8f, 0xc6, 0x35, 0x7b, - 0x56, 0x16, 0x30, 0x49, 0x66, 0xcc, 0xa5, 0xfe, 0x8c, 0x78, 0x3a, 0xc0, 0xf9, 0x0e, 0x1e, 0x6e, - 0x5f, 0x21, 0x49, 0x9f, 0x6f, 0xb5, 0x60, 0xa3, 0x87, 0x13, 0x31, 0x74, 0xca, 0xe3, 0x2c, 0xe0, - 0x28, 0xc7, 0xfa, 0x4f, 0xdc, 0xce, 0x76, 0xb9, 0xe5, 0x01, 0xda, 0xbc, 0x31, 0x1d, 0xba, 0x08, - 0xdf, 0x03, 0xda, 0xb9, 0xe9, 0x9e, 0x14, 0x3f, 0x05, 0x43, 0x4e, 0xdb, 0x29, 0xd4, 0x06, 0x42, - 0x54, 0x3e, 0xc9, 0x42, 0x1b, 0x1d, 0xa1, 0x9a, 0x8e, 0x70, 0x49, 0xf3, 0xca, 0xf9, 0x0c, 0xf6, - 0xb3, 0xff, 0x91, 0x09, 0xc6, 0x35, 0x59, 0xa5, 0xdd, 0x17, 0xcd, 0x10, 0xa5, 0x49, 0x74, 0xe3, - 0x0f, 0x9c, 0x7f, 0x84, 0x86, 0x9e, 0x47, 0x41, 0x40, 0x5c, 0xbe, 0xa3, 0xa1, 0x8c, 0xf1, 0x65, - 0x7a, 0x70, 0xa3, 0x06, 0xe5, 0xc2, 0x1a, 0x9c, 0xc2, 0xfe, 0x25, 0xc1, 0x5e, 0xe0, 0x87, 0x5a, - 0x4c, 0x66, 0xaf, 0x9e, 0x45, 0x4c, 0xfd, 0x25, 0x41, 0x17, 0x50, 0xeb, 0x07, 0xc1, 0x14, 0xcf, - 0x99, 0x10, 0x95, 0x04, 0xf8, 0x5c, 0x51, 0x2e, 0xbc, 0xbd, 0x93, 0x46, 0xaa, 0x04, 0xec, 0x0b, - 0xa8, 0x6f, 0x7e, 0xcb, 0x84, 0xde, 0xe6, 0x09, 0x3d, 0x82, 0xca, 0xbb, 0x3c, 0x21, 0xb3, 0xb7, - 0x9f, 0xd5, 0xe1, 0x87, 0xf2, 0x45, 0xc9, 0x79, 0x01, 0xed, 0x5d, 0x7c, 0x16, 0x47, 0x21, 0x23, - 0x77, 0xb7, 0x74, 0x2d, 0x32, 0x35, 0xc5, 0xce, 0x00, 0xda, 0x83, 0x9b, 0x18, 0x87, 0xde, 0xaf, - 0x7e, 0xe0, 0xb9, 0x98, 0x7a, 0x79, 0xa5, 0xbe, 0x86, 0x03, 0x29, 0x2f, 0x16, 0x63, 0x97, 0xa4, - 0x60, 0xf9, 0x68, 0xe6, 0x8e, 0x41, 0x40, 0x96, 0x42, 0x44, 0xce, 0xb7, 0x70, 0xd0, 0xa7, 0x74, - 0x22, 0xae, 0x08, 0xe7, 0xe8, 0x13, 0x28, 0x4d, 0xee, 0x3c, 0xf1, 0x1a, 0x8e, 0xde, 0xbb, 0x58, - 0x0e, 0xcb, 0x09, 0x18, 0x2f, 0x27, 0x93, 0x9c, 0xbd, 0x4c, 0x7b, 0x8d, 0xfc, 0x34, 0xd3, 0x5c, - 0x79, 0x7b, 0x5f, 0xad, 0x07, 0xe9, 0x2b, 0x78, 0x2c, 0x14, 0xde, 0x4f, 0x78, 0xe4, 0xf9, 0xcc, - 0x8d, 0xde, 0x11, 0x3a, 0xc2, 0x7c, 0x91, 0x62, 0x8b, 0x29, 0x51, 0x5f, 0x5a, 0xd8, 0xbd, 0xbf, - 0xaa, 0xd0, 0xd0, 0x65, 0x19, 0xe2, 0x10, 0xcf, 0x09, 0x45, 0xbf, 0xc3, 0x71, 0xe1, 0x7e, 0x40, - 0xe7, 0x8a, 0xc8, 0x6d, 0xeb, 0xc8, 0x3e, 0xbb, 0x2d, 0x44, 0x2e, 0xf6, 0x3d, 0x74, 0x0d, 0xad, - 0x9d, 0x9c, 0xd1, 0x89, 0x3a, 0x55, 0xdc, 0x02, 0xfb, 0x71, 0xb1, 0x33, 0x03, 0x6b, 0x6e, 0x0f, - 0x01, 0xb2, 0x3f, 0x3c, 0x79, 0xf6, 0x49, 0xa1, 0x4f, 0x4f, 0x8d, 0x00, 0xfb, 0x09, 0x9a, 0xe2, - 0xcd, 0x09, 0x7c, 0xb6, 0xd8, 0x06, 0x2b, 0x7c, 0x88, 0xec, 0x96, 0xe6, 0xb5, 0x7e, 0xb3, 0xf6, - 0xd0, 0x2f, 0x02, 0x60, 0x6b, 0xf3, 0xdf, 0x0a, 0x60, 0x69, 0xdf, 0xfb, 0x4f, 0x85, 0x40, 0xfa, - 0x19, 0xea, 0x9b, 0x5b, 0x0e, 0xe9, 0xd8, 0x82, 0xdd, 0x6a, 0xb7, 0x0b, 0x3c, 0x1a, 0x63, 0x00, - 0x8d, 0xad, 0x3d, 0x84, 0x74, 0x25, 0x8b, 0xb6, 0xa0, 0xfd, 0xa8, 0xc8, 0xa5, 0x61, 0xae, 0xa0, - 0xf5, 0x3a, 0x64, 0x1f, 0x01, 0x68, 0x0c, 0xed, 0x21, 0xe6, 0xee, 0xe2, 0x55, 0x42, 0xe8, 0x6a, - 0x1a, 0xe5, 0x62, 0xf8, 0x3f, 0xfd, 0xbf, 0x82, 0xa3, 0xa2, 0x49, 0x47, 0x8d, 0x4c, 0x14, 0x83, - 0x65, 0xcc, 0x57, 0xf6, 0x69, 0x36, 0x96, 0xc5, 0x9a, 0x70, 0xf6, 0x66, 0x55, 0xf5, 0x33, 0xe4, - 0xd9, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x88, 0x96, 0x25, 0x04, 0x02, 0x09, 0x00, 0x00, + // 840 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xa4, 0x56, 0xdb, 0x4e, 0xf3, 0x46, + 0x10, 0x26, 0x31, 0x49, 0xc8, 0xe4, 0x80, 0x58, 0x42, 0x30, 0x06, 0x01, 0x71, 0xd5, 0x96, 0xb6, + 0x6a, 0x52, 0x05, 0xa9, 0xa5, 0xbd, 0x68, 0x95, 0x42, 0x44, 0x11, 0x02, 0xa5, 0x04, 0xa8, 0x7a, + 0xd1, 0x0b, 0xc7, 0xde, 0x06, 0x0b, 0xc7, 0x76, 0x77, 0xd7, 0x88, 0x3c, 0x4b, 0x1f, 0xb0, 0xaf, + 0xd1, 0x3d, 0xd8, 0xce, 0x01, 0x73, 0x90, 0xfe, 0xab, 0x78, 0x67, 0x66, 0xbf, 0xf9, 0xe6, 0xb8, + 0x81, 0x9f, 0xc7, 0x2e, 0x7b, 0x88, 0x46, 0x6d, 0x3b, 0x98, 0x74, 0x5c, 0x9f, 0x61, 0x8f, 0x3a, + 0xee, 0xb7, 0xcf, 0x1d, 0xea, 0x5b, 0x61, 0x67, 0x4c, 0x42, 0xbb, 0x63, 0x07, 0x3e, 0x23, 0x81, + 0x17, 0x92, 0xe0, 0x79, 0xda, 0x99, 0x13, 0xb4, 0xb9, 0x84, 0x05, 0x48, 0xe3, 0x22, 0xe3, 0xf8, + 0x7d, 0x90, 0xc9, 0x24, 0xf0, 0xe3, 0x1f, 0x75, 0xd3, 0xac, 0x41, 0x65, 0x88, 0x09, 0x09, 0xc8, + 0x0d, 0x0e, 0xbd, 0xa9, 0xf9, 0x6f, 0x0e, 0xb6, 0x06, 0xd1, 0x68, 0x40, 0x02, 0xfb, 0x0a, 0x33, + 0xe2, 0xda, 0xf4, 0x06, 0xff, 0x13, 0x61, 0xca, 0xd0, 0x26, 0x54, 0x4e, 0xb9, 0x4f, 0xec, 0xb3, + 0xdb, 0x69, 0x88, 0xf5, 0xdc, 0x61, 0xee, 0xa8, 0x8c, 0xd6, 0xa1, 0x14, 0x0b, 0xf5, 0x3c, 0x17, + 0x54, 0x11, 0x02, 0x18, 0x78, 0xd1, 0xd8, 0xf5, 0xaf, 0xad, 0x09, 0xd6, 0x35, 0x69, 0xb4, 0x05, + 0x35, 0x25, 0xbb, 0xc7, 0x84, 0xba, 0x81, 0xaf, 0xaf, 0x72, 0xb1, 0x86, 0x5a, 0x50, 0xe4, 0x77, + 0xff, 0x76, 0xc7, 0x7a, 0x81, 0x9f, 0x2b, 0xdd, 0x8d, 0x76, 0x4c, 0x4c, 0x49, 0xaf, 0xac, 0x10, + 0xd5, 0xa1, 0x78, 0x6b, 0xd1, 0xc7, 0x0b, 0x47, 0x2f, 0x0a, 0x24, 0x73, 0x0f, 0xa0, 0x9f, 0x72, + 0x15, 0x5a, 0x79, 0xa2, 0x9c, 0x8c, 0xc6, 0xb5, 0x97, 0xb0, 0x29, 0x78, 0x63, 0x4a, 0x53, 0xea, + 0xc2, 0xec, 0x63, 0xc4, 0x67, 0x60, 0x9a, 0x04, 0xfb, 0x1d, 0xf6, 0xce, 0x31, 0x53, 0xbc, 0xe7, + 0xae, 0xa7, 0xe9, 0xa8, 0xc2, 0xaa, 0x0c, 0x51, 0xc1, 0xa5, 0x61, 0x4b, 0x17, 0x02, 0xb1, 0x20, + 0x5c, 0x24, 0x01, 0x8b, 0x3c, 0x14, 0xcc, 0x3f, 0xc1, 0x78, 0x05, 0x52, 0xd0, 0xe4, 0x59, 0xea, + 0xd9, 0x36, 0x0e, 0x19, 0x76, 0xa4, 0x54, 0x05, 0x25, 0xc4, 0x37, 0x98, 0x45, 0xc4, 0x4f, 0xc4, + 0x79, 0x29, 0xae, 0x41, 0x41, 0xd2, 0x55, 0x29, 0x36, 0x2d, 0xd8, 0xbc, 0xb7, 0x3c, 0xd7, 0xb1, + 0x18, 0x3e, 0xc3, 0x61, 0x4a, 0xf2, 0x00, 0x4a, 0x71, 0x2a, 0x24, 0x5a, 0xa5, 0x5b, 0x4f, 0x72, + 0xac, 0xc4, 0xe8, 0x2b, 0x28, 0x29, 0x3e, 0x0a, 0xb7, 0xd2, 0xd5, 0x13, 0x83, 0x61, 0x34, 0xa2, + 0x36, 0x71, 0x47, 0xd8, 0x51, 0x06, 0xe6, 0xf7, 0xb0, 0xb1, 0xe8, 0x42, 0x90, 0x6e, 0x2d, 0x94, + 0x60, 0xae, 0x86, 0x43, 0xde, 0x74, 0x52, 0x63, 0x32, 0x68, 0xa4, 0x58, 0xf3, 0xdc, 0x5a, 0x50, + 0x8e, 0x3f, 0xb1, 0xf3, 0xc9, 0xec, 0xe6, 0x3a, 0x45, 0x25, 0xe4, 0x07, 0x40, 0x4b, 0x5e, 0x3f, + 0x48, 0xf7, 0x08, 0x9a, 0x77, 0x3e, 0xcd, 0x22, 0x3c, 0x73, 0x21, 0x6b, 0x6e, 0xfe, 0x08, 0x8d, + 0x17, 0x96, 0x1f, 0x74, 0xf2, 0x39, 0x68, 0xa2, 0xbd, 0xf7, 0xa1, 0xd4, 0xe7, 0x53, 0xec, 0xe2, + 0xc4, 0xb4, 0xd6, 0xe6, 0x63, 0xda, 0xe6, 0x2a, 0x21, 0x9e, 0x9a, 0x5f, 0xc0, 0x5a, 0xf2, 0x8d, + 0x2a, 0xa0, 0x5d, 0xe2, 0x69, 0xdc, 0x6e, 0xbc, 0xfa, 0xbc, 0x16, 0x91, 0xea, 0xb4, 0xb2, 0x1c, + 0xda, 0xd3, 0xc0, 0xf3, 0xb0, 0xcd, 0x96, 0x86, 0x36, 0xe1, 0x7c, 0x16, 0x5f, 0x3c, 0x81, 0x52, + 0xcf, 0xf3, 0x6e, 0xad, 0x71, 0x92, 0xd1, 0x2f, 0xa5, 0xc7, 0xcc, 0xcb, 0xed, 0xd8, 0x52, 0xfa, + 0x37, 0x4e, 0xa0, 0x3a, 0x7f, 0x16, 0x7c, 0x1e, 0x53, 0x3e, 0xdb, 0x50, 0x78, 0x4a, 0xf9, 0x54, + 0xba, 0x6b, 0x49, 0x18, 0x3f, 0xe5, 0x4f, 0x72, 0xe6, 0x05, 0x34, 0x97, 0xf1, 0x69, 0x18, 0xf8, + 0x14, 0xbf, 0xdf, 0x9e, 0xb3, 0xa1, 0x94, 0x5d, 0x6f, 0xf6, 0xa1, 0xd9, 0x7f, 0x0e, 0x2d, 0xdf, + 0xf9, 0xc3, 0xf5, 0x1c, 0xdb, 0x22, 0x4e, 0x1a, 0xe8, 0x37, 0x50, 0x16, 0xe3, 0x48, 0x43, 0xcb, + 0xc6, 0x31, 0x58, 0xda, 0x2c, 0xa9, 0xa2, 0xef, 0xe1, 0x09, 0x1f, 0x3a, 0xf3, 0x3b, 0x28, 0xf7, + 0x08, 0x19, 0x72, 0x17, 0xfe, 0x18, 0x7d, 0x06, 0xb9, 0xe1, 0xbb, 0x37, 0xee, 0xa0, 0xf1, 0xc2, + 0xb1, 0xa8, 0xf5, 0x2e, 0x68, 0xd7, 0xc3, 0x61, 0xca, 0x5e, 0x84, 0x3d, 0x43, 0x3e, 0x4c, 0x66, + 0x34, 0xbf, 0xb8, 0xdf, 0x66, 0x7d, 0xf0, 0x35, 0xec, 0xf0, 0x8d, 0xd0, 0x8b, 0x58, 0xe0, 0xb8, + 0xd4, 0x0e, 0x9e, 0x30, 0x19, 0x58, 0xec, 0x21, 0xc6, 0xe6, 0x45, 0x96, 0x27, 0xb5, 0x08, 0xba, + 0xff, 0xad, 0x42, 0x4d, 0xa5, 0xe5, 0xca, 0xf2, 0xad, 0x31, 0x26, 0xe8, 0x2f, 0xd8, 0xca, 0xdc, + 0x27, 0xa8, 0x25, 0x89, 0xbc, 0xb5, 0xbe, 0x8c, 0x83, 0xb7, 0x4c, 0xc4, 0x43, 0xb0, 0x82, 0x2e, + 0xa1, 0xbe, 0x58, 0x37, 0x64, 0xbc, 0xde, 0x2c, 0xc6, 0x6e, 0xa6, 0x4e, 0x15, 0x9a, 0x83, 0xfd, + 0x02, 0x75, 0xfe, 0xac, 0x78, 0x2e, 0x7d, 0x58, 0x04, 0xcb, 0x7c, 0x6b, 0x8c, 0x75, 0xa9, 0x9b, + 0xad, 0x7a, 0x0e, 0xf0, 0x1b, 0x07, 0x58, 0x58, 0xee, 0x6f, 0x02, 0xe8, 0x4a, 0xf7, 0xf2, 0x35, + 0xe0, 0x48, 0xbf, 0x42, 0x75, 0x7e, 0x91, 0x21, 0x65, 0x9b, 0xb1, 0x3e, 0x8d, 0x66, 0x86, 0x46, + 0x61, 0xf4, 0xa1, 0xb6, 0xb0, 0x5e, 0xd0, 0x8e, 0x34, 0xcd, 0x5a, 0x74, 0xc6, 0x76, 0x96, 0x2a, + 0x49, 0xf1, 0xfa, 0xd2, 0x0a, 0x41, 0x2a, 0x8f, 0xd9, 0x2b, 0xc8, 0xd8, 0xc9, 0x56, 0x2a, 0xb0, + 0x73, 0x68, 0x64, 0x35, 0x13, 0xaa, 0x25, 0x7d, 0xd7, 0x9f, 0x84, 0x6c, 0x6a, 0xec, 0x27, 0x95, + 0xcf, 0x6e, 0x3b, 0x73, 0x65, 0x54, 0x94, 0xff, 0x0c, 0x8e, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, + 0x4e, 0xc0, 0x13, 0x3e, 0x95, 0x08, 0x00, 0x00, } diff --git a/grpc/controlproxy/rpc/control.proto b/grpc/controlproxy/rpc/control.proto index 285aac680..587d14a56 100644 --- a/grpc/controlproxy/rpc/control.proto +++ b/grpc/controlproxy/rpc/control.proto @@ -25,14 +25,12 @@ import "github.com/intelsdi-x/snap/grpc/common/common.proto"; service MetricManager { // managesMetrics from scheduler rpc GetPluginContentTypes(GetPluginContentTypesRequest) returns (GetPluginContentTypesReply) {} - rpc ExpandWildcards(ExpandWildcardsRequest) returns (ExpandWildcardsReply) {} rpc CollectMetrics(CollectMetricsRequest) returns (CollectMetricsResponse) {} rpc PublishMetrics(PubProcMetricsRequest) returns (ErrorReply) {} rpc ProcessMetrics(PubProcMetricsRequest) returns (ProcessMetricsReply) {} rpc ValidateDeps(ValidateDepsRequest) returns (ValidateDepsReply) {} rpc SubscribeDeps(SubscribeDepsRequest) returns (SubscribeDepsReply) {} - rpc UnsubscribeDeps(SubscribeDepsRequest) returns (SubscribeDepsReply) {} - rpc MatchQueryToNamespaces(ExpandWildcardsRequest) returns (ExpandWildcardsReply) {} + rpc UnsubscribeDeps(UnsubscribeDepsRequest) returns (UnsubscribeDepsReply) {} rpc GetAutodiscoverPaths(common.Empty) returns (GetAutodiscoverPathsReply) {} } @@ -78,17 +76,26 @@ message ValidateDepsRequest { message ValidateDepsReply { repeated common.SnapError Errors = 1; -} +} message SubscribeDepsRequest { - repeated common.Metric Metrics = 1; - repeated common.Plugin Plugins = 2; + repeated common.Metric Requested = 1; + repeated common.SubscribedPlugin Plugins = 2; string TaskId = 3; } message SubscribeDepsReply { repeated common.SnapError Errors = 1; } + +message UnsubscribeDepsRequest { + string TaskId = 1; +} + +message UnsubscribeDepsReply { + repeated common.SnapError Errors = 1; +} + message Map { repeated MapEntry Entries = 1; } @@ -100,9 +107,7 @@ message MapEntry { message CollectMetricsRequest { string TaskID = 1; - repeated common.Metric Metrics = 2; - common.Time Deadline = 3; - map AllTags = 4; + map AllTags = 2; } message CollectMetricsResponse { diff --git a/mgmt/rest/client/client_func_test.go b/mgmt/rest/client/client_func_test.go index 1c6920dca..df42c168f 100644 --- a/mgmt/rest/client/client_func_test.go +++ b/mgmt/rest/client/client_func_test.go @@ -45,11 +45,12 @@ var ( // Change to set the REST API logging to debug LOG_LEVEL = log.FatalLevel - SNAP_PATH = os.Getenv("SNAP_PATH") - MOCK_PLUGIN_PATH1 = []string{SNAP_PATH + "/plugin/snap-plugin-collector-mock1"} - MOCK_PLUGIN_PATH2 = []string{SNAP_PATH + "/plugin/snap-plugin-collector-mock2"} - FILE_PLUGIN_PATH = []string{SNAP_PATH + "/plugin/snap-plugin-publisher-mock-file"} - DIRECTORY_PATH = []string{SNAP_PATH + "/plugin/"} + SNAP_PATH = os.Getenv("SNAP_PATH") + MOCK_PLUGIN_PATH1 = []string{SNAP_PATH + "/plugin/snap-plugin-collector-mock1"} + MOCK_PLUGIN_PATH2 = []string{SNAP_PATH + "/plugin/snap-plugin-collector-mock2"} + ANOTHERMOCK_PLUGIN_PATH = []string{SNAP_PATH + "/plugin/snap-plugin-collector-anothermock1"} + FILE_PLUGIN_PATH = []string{SNAP_PATH + "/plugin/snap-plugin-publisher-mock-file"} + DIRECTORY_PATH = []string{SNAP_PATH + "/plugin/"} NextPort = 45000 diff --git a/mgmt/rest/tribe_test.go b/mgmt/rest/tribe_test.go index fc33d097f..c11c74ac5 100644 --- a/mgmt/rest/tribe_test.go +++ b/mgmt/rest/tribe_test.go @@ -201,7 +201,7 @@ func TestTribeTaskAgreements(t *testing.T) { So(timedOut, ShouldEqual, false) Convey("Plugins and a task are uploaded", func() { - resp := uploadPlugin(MOCK_PLUGIN_PATH1, mgtPorts[0]) + resp := uploadPlugin(MOCK_PLUGIN_PATH2, mgtPorts[0]) So(resp.Meta.Code, ShouldEqual, 201) So(resp.Meta.Type, ShouldEqual, rbody.PluginsLoadedType) resp = getPluginList(mgtPorts[0]) @@ -212,19 +212,14 @@ func TestTribeTaskAgreements(t *testing.T) { resp = getAgreement(mgtPorts[0], aName) So(resp.Meta.Code, ShouldEqual, 200) So(len(resp.Body.(*rbody.TribeGetAgreement).Agreement.PluginAgreement.Plugins), ShouldEqual, 1) - resp = createTask("3.json", "task1", "1s", true, mgtPorts[0]) - So(resp.Meta.Code, ShouldEqual, 201) - So(resp.Meta.Type, ShouldEqual, rbody.AddScheduledTaskType) - So(resp.Body, ShouldHaveSameTypeAs, new(rbody.AddScheduledTask)) - taskID := resp.Body.(*rbody.AddScheduledTask).ID - Convey("The cluster agrees on tasks", func(c C) { + Convey("The cluster agrees on plugins", func(c C) { var wg sync.WaitGroup timedOut := false for _, i := range mgtPorts { timer := time.After(15 * time.Second) wg.Add(1) - go func(port int, name string) { + go func(port int) { defer wg.Done() for { select { @@ -232,66 +227,66 @@ func TestTribeTaskAgreements(t *testing.T) { timedOut = true return default: - resp := getAgreement(port, name) + resp := getPluginList(port) if resp.Meta.Code == 200 { - c.So(resp.Body.(*rbody.TribeGetAgreement), ShouldHaveSameTypeAs, new(rbody.TribeGetAgreement)) - if len(resp.Body.(*rbody.TribeGetAgreement).Agreement.TaskAgreement.Tasks) == 1 { + c.So(resp.Body.(*rbody.PluginList), ShouldHaveSameTypeAs, new(rbody.PluginList)) + if len(resp.Body.(*rbody.PluginList).LoadedPlugins) == 1 { return } } time.Sleep(200 * time.Millisecond) } } - }(i, aName) + }(i) } wg.Wait() So(timedOut, ShouldEqual, false) - Convey("The task is started", func() { - resp := startTask(taskID, mgtPorts[0]) - So(resp.Meta.Code, ShouldEqual, 200) - So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskStartedType) - Convey("The task is started on all members of the tribe", func(c C) { - var wg sync.WaitGroup - timedOut := false - for i := 0; i < numOfNodes; i++ { - timer := time.After(15 * time.Second) - wg.Add(1) - go func(port int) { - defer wg.Done() - for { - select { - case <-timer: - timedOut = true - return - default: - resp := getTask(taskID, port) - if resp.Meta.Code == 200 { - if resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskSpinning.String() || resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskFiring.String() { - return - } - log.Debugf("port %v has task in state %v", port, resp.Body.(*rbody.ScheduledTaskReturned).State) - } else { - log.Debugf("node %v error getting task", port) + + resp = createTask("3.json", "task1", "1s", true, mgtPorts[0]) + So(resp.Meta.Code, ShouldEqual, 201) + So(resp.Meta.Type, ShouldEqual, rbody.AddScheduledTaskType) + So(resp.Body, ShouldHaveSameTypeAs, new(rbody.AddScheduledTask)) + taskID := resp.Body.(*rbody.AddScheduledTask).ID + + Convey("The cluster agrees on tasks", func(c C) { + var wg sync.WaitGroup + timedOut := false + for _, i := range mgtPorts { + timer := time.After(15 * time.Second) + wg.Add(1) + go func(port int, name string) { + defer wg.Done() + for { + select { + case <-timer: + timedOut = true + return + default: + resp := getAgreement(port, name) + if resp.Meta.Code == 200 { + c.So(resp.Body.(*rbody.TribeGetAgreement), ShouldHaveSameTypeAs, new(rbody.TribeGetAgreement)) + if len(resp.Body.(*rbody.TribeGetAgreement).Agreement.TaskAgreement.Tasks) == 1 { + return } - time.Sleep(400 * time.Millisecond) } + time.Sleep(200 * time.Millisecond) } - }(mgtPorts[i]) - } - wg.Wait() - So(timedOut, ShouldEqual, false) - Convey("A new node joins the agreement", func() { - mgtPort, _, _ := startTribes(1, fmt.Sprintf("127.0.0.1:%d", tribePort)) - j := joinAgreement(mgtPort[0], fmt.Sprintf("member-%d", mgtPort[0]), aName) - mgtPorts = append(mgtPorts, mgtPort[0]) - So(j.Meta.Code, ShouldEqual, 200) - So(j.Body, ShouldHaveSameTypeAs, new(rbody.TribeJoinAgreement)) + } + }(i, aName) + } + wg.Wait() + So(timedOut, ShouldEqual, false) + Convey("The task is started", func() { + resp := startTask(taskID, mgtPorts[0]) + So(resp.Meta.Code, ShouldEqual, 200) + So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskStartedType) + Convey("The task is started on all members of the tribe", func(c C) { var wg sync.WaitGroup timedOut := false - for _, i := range mgtPort { + for i := 0; i < numOfNodes; i++ { timer := time.After(15 * time.Second) wg.Add(1) - go func(port int, name string) { + go func(port int) { defer wg.Done() for { select { @@ -311,22 +306,22 @@ func TestTribeTaskAgreements(t *testing.T) { time.Sleep(400 * time.Millisecond) } } - }(i, fmt.Sprintf("member-%d", i)) + }(mgtPorts[i]) } wg.Wait() - time.Sleep(1 * time.Second) So(timedOut, ShouldEqual, false) - Convey("The task is stopped", func() { - resp := stopTask(taskID, mgtPorts[0]) - So(resp.Meta.Code, ShouldEqual, 200) - So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskStoppedType) - So(resp.Body, ShouldHaveSameTypeAs, new(rbody.ScheduledTaskStopped)) + Convey("A new node joins the agreement", func() { + mgtPort, _, _ := startTribes(1, fmt.Sprintf("127.0.0.1:%d", tribePort)) + j := joinAgreement(mgtPort[0], fmt.Sprintf("member-%d", mgtPort[0]), aName) + mgtPorts = append(mgtPorts, mgtPort[0]) + So(j.Meta.Code, ShouldEqual, 200) + So(j.Body, ShouldHaveSameTypeAs, new(rbody.TribeJoinAgreement)) var wg sync.WaitGroup timedOut := false - for i := 0; i < numOfNodes; i++ { + for _, i := range mgtPort { timer := time.After(15 * time.Second) wg.Add(1) - go func(port int) { + go func(port int, name string) { defer wg.Done() for { select { @@ -336,27 +331,26 @@ func TestTribeTaskAgreements(t *testing.T) { default: resp := getTask(taskID, port) if resp.Meta.Code == 200 { - if resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskStopped.String() { + if resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskSpinning.String() || resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskFiring.String() { return } + log.Debugf("port %v has task in state %v", port, resp.Body.(*rbody.ScheduledTaskReturned).State) + } else { + log.Debugf("node %v error getting task", port) } time.Sleep(400 * time.Millisecond) } } - }(mgtPorts[i]) + }(i, fmt.Sprintf("member-%d", i)) } wg.Wait() + time.Sleep(1 * time.Second) So(timedOut, ShouldEqual, false) - Convey("The task is removed", func() { - for _, port := range mgtPorts { - resp := getTask(taskID, port) - So(resp.Meta.Code, ShouldEqual, 200) - So(resp.Body.(*rbody.ScheduledTaskReturned).State, ShouldResemble, core.TaskStopped.String()) - } - resp := removeTask(taskID, mgtPorts[0]) + Convey("The task is stopped", func() { + resp := stopTask(taskID, mgtPorts[0]) So(resp.Meta.Code, ShouldEqual, 200) - So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskRemovedType) - So(resp.Body, ShouldHaveSameTypeAs, new(rbody.ScheduledTaskRemoved)) + So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskStoppedType) + So(resp.Body, ShouldHaveSameTypeAs, new(rbody.ScheduledTaskStopped)) var wg sync.WaitGroup timedOut := false for i := 0; i < numOfNodes; i++ { @@ -371,8 +365,10 @@ func TestTribeTaskAgreements(t *testing.T) { return default: resp := getTask(taskID, port) - if resp.Meta.Code == 404 { - return + if resp.Meta.Code == 200 { + if resp.Body.(*rbody.ScheduledTaskReturned).State == core.TaskStopped.String() { + return + } } time.Sleep(400 * time.Millisecond) } @@ -381,11 +377,16 @@ func TestTribeTaskAgreements(t *testing.T) { } wg.Wait() So(timedOut, ShouldEqual, false) - Convey("The plugins are unloaded", func(c C) { - resp := unloadPlugin(mgtPorts[0], pluginToUnload.Type, pluginToUnload.Name, pluginToUnload.Version) + Convey("The task is removed", func() { + for _, port := range mgtPorts { + resp := getTask(taskID, port) + So(resp.Meta.Code, ShouldEqual, 200) + So(resp.Body.(*rbody.ScheduledTaskReturned).State, ShouldResemble, core.TaskStopped.String()) + } + resp := removeTask(taskID, mgtPorts[0]) So(resp.Meta.Code, ShouldEqual, 200) - So(resp.Meta.Type, ShouldEqual, rbody.PluginUnloadedType) - So(resp.Body, ShouldHaveSameTypeAs, new(rbody.PluginUnloaded)) + So(resp.Meta.Type, ShouldEqual, rbody.ScheduledTaskRemovedType) + So(resp.Body, ShouldHaveSameTypeAs, new(rbody.ScheduledTaskRemoved)) var wg sync.WaitGroup timedOut := false for i := 0; i < numOfNodes; i++ { @@ -399,9 +400,8 @@ func TestTribeTaskAgreements(t *testing.T) { timedOut = true return default: - resp = getPluginList(port) - c.So(resp.Meta.Code, ShouldEqual, 200) - if len(resp.Body.(*rbody.PluginList).LoadedPlugins) == 0 { + resp := getTask(taskID, port) + if resp.Meta.Code == 404 { return } time.Sleep(400 * time.Millisecond) @@ -411,6 +411,37 @@ func TestTribeTaskAgreements(t *testing.T) { } wg.Wait() So(timedOut, ShouldEqual, false) + Convey("The plugins are unloaded", func(c C) { + resp := unloadPlugin(mgtPorts[0], pluginToUnload.Type, pluginToUnload.Name, pluginToUnload.Version) + So(resp.Meta.Code, ShouldEqual, 200) + So(resp.Meta.Type, ShouldEqual, rbody.PluginUnloadedType) + So(resp.Body, ShouldHaveSameTypeAs, new(rbody.PluginUnloaded)) + var wg sync.WaitGroup + timedOut := false + for i := 0; i < numOfNodes; i++ { + timer := time.After(15 * time.Second) + wg.Add(1) + go func(port int) { + defer wg.Done() + for { + select { + case <-timer: + timedOut = true + return + default: + resp = getPluginList(port) + c.So(resp.Meta.Code, ShouldEqual, 200) + if len(resp.Body.(*rbody.PluginList).LoadedPlugins) == 0 { + return + } + time.Sleep(400 * time.Millisecond) + } + } + }(mgtPorts[i]) + } + wg.Wait() + So(timedOut, ShouldEqual, false) + }) }) }) }) @@ -480,7 +511,7 @@ func TestTribePluginAgreements(t *testing.T) { So(timedOut, ShouldEqual, false) Convey("A plugin is uploaded", func() { - resp := uploadPlugin(MOCK_PLUGIN_PATH1, mgtPorts[0]) + resp := uploadPlugin(MOCK_PLUGIN_PATH2, mgtPorts[0]) So(resp.Meta.Code, ShouldEqual, 201) So(resp.Meta.Type, ShouldEqual, rbody.PluginsLoadedType) lpName = resp.Body.(*rbody.PluginsLoaded).LoadedPlugins[0].Name @@ -712,7 +743,10 @@ func startTribes(count int, seed string) ([]int, int, *listenToSeedEvents) { t.EventManager.RegisterHandler("tribe.tests", lpe) } - c := control.New(control.GetDefaultConfig()) + cfg := control.GetDefaultConfig() + // get an available port to avoid conflicts (we aren't testing remote workflows here) + cfg.ListenPort = getAvailablePort() + c := control.New(cfg) c.RegisterEventHandler("tribe", t) c.Start() s := scheduler.New(scheduler.GetDefaultConfig()) @@ -751,7 +785,7 @@ func startTribes(count int, seed string) ([]int, int, *listenToSeedEvents) { return mgtPorts, tribePort, lpe } -var nextPort uint64 = 55234 +var nextPort uint64 = 51234 func getAvailablePort() int { atomic.AddUint64(&nextPort, 1) diff --git a/mgmt/rest/wmap_sample/3.json b/mgmt/rest/wmap_sample/3.json index edb2f87d8..683f7d4ac 100644 --- a/mgmt/rest/wmap_sample/3.json +++ b/mgmt/rest/wmap_sample/3.json @@ -1,7 +1,7 @@ { "collect": { "metrics": { - "/intel/mock/foo": {"version": 1} + "/intel/mock/foo": {"version": 2} }, "config": { "/intel/mock/foo": { diff --git a/mgmt/tribe/delegate.go b/mgmt/tribe/delegate.go index 644565622..406d4f38c 100644 --- a/mgmt/tribe/delegate.go +++ b/mgmt/tribe/delegate.go @@ -117,7 +117,9 @@ func (t *delegate) NotifyMsg(buf []byte) { if err := decodeMessage(buf[1:], msg); err != nil { panic(err) } + t.tribe.mutex.RLock() queryResp, ok := t.tribe.taskStateResponses[msg.UUID] + t.tribe.mutex.RUnlock() if !ok { logger.WithFields(log.Fields{ "_block": "delegate-notify-msg", diff --git a/mgmt/tribe/tribe_test.go b/mgmt/tribe/tribe_test.go index b4ee6d841..819fff42b 100644 --- a/mgmt/tribe/tribe_test.go +++ b/mgmt/tribe/tribe_test.go @@ -1139,7 +1139,7 @@ func getTribes(numOfTribes int, seedTribe *tribe) []*tribe { return tribes } -var nextPort uint64 = 55234 +var nextPort uint64 = 61234 func getAvailablePort() int { atomic.AddUint64(&nextPort, 1) diff --git a/mgmt/tribe/worker/worker.go b/mgmt/tribe/worker/worker.go index 6419af5e8..ddb606528 100644 --- a/mgmt/tribe/worker/worker.go +++ b/mgmt/tribe/worker/worker.go @@ -404,6 +404,23 @@ func (w worker) createTask(taskID string, startOnCreate bool) { logger.WithField("err", taskResult.Err.Error()).Debug("error getting task") continue } + // this block addresses the condition when we are creating and starting + // a task and the task is created but fails to start (deps were not yet met) + if startOnCreate { + if _, err := w.taskManager.GetTask(taskID); err == nil { + logger.Debug("starting task") + if errs := w.taskManager.StartTaskTribe(taskID); errs != nil { + fields := log.Fields{} + for idx, e := range errs { + fields[fmt.Sprintf("err-%d", idx)] = e.Error() + } + logger.WithFields(fields).Error("error starting task") + continue + } + done = true + break + } + } logger.Debug("creating task") opt := core.SetTaskID(taskID) _, errs := w.taskManager.CreateTaskTribe( diff --git a/plugin/collector/snap-plugin-collector-anothermock1/README.md b/plugin/collector/snap-plugin-collector-anothermock1/README.md new file mode 100644 index 000000000..570e422ab --- /dev/null +++ b/plugin/collector/snap-plugin-collector-anothermock1/README.md @@ -0,0 +1,58 @@ + + +## snap Collector Plugin Structure +--- +Mock plugins are for testing purposes and not meant as examples. + +#### Plugin binary + +./main.go + +#### Collector Implementation + +./collector/collector.go + +#### JSON RPC examples (using curl) + +If calling a GO based plugin you will want to ensure that the plugin is started in JSON RPC mode. This is done by setting the plugins meta data field RPCType to plugin.JSONRPC. + +You can start a plugin manually for testing by increasing the ping timeout duration. The timeout will be reset each time you call into the plugin. + +``` +./snap-collector-anothermock1 '{"NoDaemon": false, "PingTimeoutDuration": 1000000000000}' +``` + +###### GetConfigPolicy + +``` +curl -d '{"method": "Collector.GetConfigPolicy", "params": [], "id": 1}' http://127.0.0.1: | python -m "json.tool" +``` + +###### GetMetricTypes + +``` +curl -d '{"method": "Collector.GetMetricTypes", "params": [], "id": 1}' http://127.0.0.1: +``` + +###### CollectMetrics + +``` +curl -X POST -H "Content-Type: application/json" -d '{"method": "Collector.CollectMetrics", "params": [[{"namespace": ["intel","anothermock", "bar"]},{"namespace": ["intel","anothermock","foo"], "config": {"table": {"password": {"Value": "asdf"}}}}]], "id": 1}' http://127.0.0.1: | python -m "json.tool" +``` diff --git a/plugin/collector/snap-plugin-collector-anothermock1/anothermock/anothermock.go b/plugin/collector/snap-plugin-collector-anothermock1/anothermock/anothermock.go new file mode 100644 index 000000000..adc082bbf --- /dev/null +++ b/plugin/collector/snap-plugin-collector-anothermock1/anothermock/anothermock.go @@ -0,0 +1,145 @@ +/* +http://www.apache.org/licenses/LICENSE-2.0.txt + + +Copyright 2016 Intel Corporation + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package anothermock + +import ( + "fmt" + "log" + "math/rand" + "time" + + "github.com/intelsdi-x/snap/control/plugin" + "github.com/intelsdi-x/snap/control/plugin/cpolicy" + "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/ctypes" +) + +const ( + // Name of plugin + Name = "anothermock" + // Version of plugin + Version = 1 + // Type of plugin + Type = plugin.CollectorPluginType +) + +// Mock collector implementation used for testing +type AnotherMock struct { +} + +// CollectMetrics collects metrics for testing +func (f *AnotherMock) CollectMetrics(mts []plugin.MetricType) ([]plugin.MetricType, error) { + for _, p := range mts { + log.Printf("collecting %+v\n", p) + } + + rand.Seed(time.Now().UTC().UnixNano()) + metrics := []plugin.MetricType{} + for i := range mts { + if c, ok := mts[i].Config().Table()["panic"]; ok && c.(ctypes.ConfigValueBool).Value { + panic("Oops!") + } + if mts[i].Namespace()[2].Value == "*" { + for j := 0; j < 10; j++ { + ns := make([]core.NamespaceElement, len(mts[i].Namespace())) + copy(ns, mts[i].Namespace()) + ns[2].Value = fmt.Sprintf("host%d", j) + data := 9000 + randInt(65, 90) + mt := plugin.MetricType{ + Data_: data, + Namespace_: ns, + Timestamp_: time.Now(), + Version_: mts[i].Version(), + Unit_: mts[i].Unit(), + } + metrics = append(metrics, mt) + } + } else { + data := 9000 + randInt(65, 90) + mts[i].Data_ = data + mts[i].Timestamp_ = time.Now() + metrics = append(metrics, mts[i]) + } + } + return metrics, nil +} + +//GetMetricTypes returns metric types for testing +func (f *AnotherMock) GetMetricTypes(cfg plugin.ConfigType) ([]plugin.MetricType, error) { + mts := []plugin.MetricType{} + if _, ok := cfg.Table()["test-fail"]; ok { + return mts, fmt.Errorf("testing") + } + if _, ok := cfg.Table()["test"]; ok { + mts = append(mts, plugin.MetricType{ + Namespace_: core.NewNamespace("intel", "anothermock", "test"), + Description_: "anothermock description", + Unit_: "anothermock unit", + }) + } + mts = append(mts, plugin.MetricType{ + Namespace_: core.NewNamespace("intel", "anothermock", "foo"), + Description_: "anothermock description", + Unit_: "anothermock unit", + }) + mts = append(mts, plugin.MetricType{ + Namespace_: core.NewNamespace("intel", "anothermock", "bar"), + Description_: "anothermock description", + Unit_: "anothermock unit", + }) + mts = append(mts, plugin.MetricType{ + Namespace_: core.NewNamespace("intel", "anothermock"). + AddDynamicElement("host", "name of the host"). + AddStaticElement("baz"), + Description_: "anothermock description", + Unit_: "anothermock unit", + }) + return mts, nil +} + +//GetConfigPolicy returns a ConfigPolicy for testing +func (f *AnotherMock) GetConfigPolicy() (*cpolicy.ConfigPolicy, error) { + c := cpolicy.New() + rule, _ := cpolicy.NewStringRule("name", false, "bob") + rule2, _ := cpolicy.NewStringRule("password", true) + p := cpolicy.NewPolicyNode() + p.Add(rule) + p.Add(rule2) + c.Add([]string{"intel", "anothermock", "foo"}, p) + return c, nil +} + +//Meta returns meta data for testing +func Meta() *plugin.PluginMeta { + return plugin.NewPluginMeta( + Name, + Version, + Type, + []string{plugin.SnapGOBContentType}, + []string{plugin.SnapGOBContentType}, + plugin.CacheTTL(100*time.Millisecond), + plugin.RoutingStrategy(plugin.StickyRouting), + ) +} + +//Random number generator +func randInt(min int, max int) int { + return min + rand.Intn(max-min) +} diff --git a/plugin/collector/snap-plugin-collector-anothermock1/main.go b/plugin/collector/snap-plugin-collector-anothermock1/main.go new file mode 100644 index 000000000..6e1b9f746 --- /dev/null +++ b/plugin/collector/snap-plugin-collector-anothermock1/main.go @@ -0,0 +1,42 @@ +/* +http://www.apache.org/licenses/LICENSE-2.0.txt + + +Copyright 2016 Intel Corporation + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package main + +import ( + "os" + + // Import the snap plugin library + "github.com/intelsdi-x/snap/control/plugin" + // Import our collector plugin implementation + "github.com/intelsdi-x/snap/plugin/collector/snap-plugin-collector-anothermock1/anothermock" +) + +func main() { + // Provided: + // the definition of the plugin metadata + // the implementation satisfying plugin.CollectorPlugin + + // Define metadata about Plugin + meta := anothermock.Meta() + // meta.RPCType = plugin.JSONRPC + + // Start a collector + plugin.Start(meta, new(anothermock.AnotherMock), os.Args[1]) +} diff --git a/plugin/collector/snap-plugin-collector-anothermock1/main_small_test.go b/plugin/collector/snap-plugin-collector-anothermock1/main_small_test.go new file mode 100644 index 000000000..1385d72a9 --- /dev/null +++ b/plugin/collector/snap-plugin-collector-anothermock1/main_small_test.go @@ -0,0 +1,36 @@ +// +build small + +/* +http://www.apache.org/licenses/LICENSE-2.0.txt + + +Copyright 2016 Intel Corporation + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package main + +import ( + "os" + "testing" + + . "github.com/smartystreets/goconvey/convey" +) + +func TestMain(t *testing.T) { + Convey("ensure plugin loads and responds", t, func() { + os.Args = []string{"", "{\"NoDaemon\": true}"} + So(func() { main() }, ShouldNotPanic) + }) +} diff --git a/plugin/collector/snap-plugin-collector-anothermock1/main_test.go b/plugin/collector/snap-plugin-collector-anothermock1/main_test.go new file mode 100644 index 000000000..a8c9fb65c --- /dev/null +++ b/plugin/collector/snap-plugin-collector-anothermock1/main_test.go @@ -0,0 +1,60 @@ +// +build legacy + +/* +http://www.apache.org/licenses/LICENSE-2.0.txt + + +Copyright 2015 Intel Corporation + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package main + +import ( + "os" + "path" + "testing" + + "github.com/intelsdi-x/snap/control" + "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/plugin/helper" + . "github.com/smartystreets/goconvey/convey" +) + +var ( + PluginName = "snap-plugin-collector-anothermock1" + PluginType = "collector" + SnapPath = os.Getenv("SNAP_PATH") + PluginPath = path.Join(SnapPath, "plugin", PluginName) +) + +func TestMockPluginLoad(t *testing.T) { + // These tests only work if SNAP_PATH is known. + // It is the responsibility of the testing framework to + // build the plugins first into the build dir. + Convey("make sure plugin has been built", t, func() { + err := helper.CheckPluginBuilt(SnapPath, PluginName) + So(err, ShouldBeNil) + + Convey("ensure plugin loads and responds", func() { + c := control.New(control.GetDefaultConfig()) + c.Start() + rp, _ := core.NewRequestedPlugin(PluginPath) + _, err := c.Load(rp) + + So(err, ShouldBeNil) + }) + + }) +} diff --git a/plugin/collector/snap-plugin-collector-mock2/mock/mock.go b/plugin/collector/snap-plugin-collector-mock2/mock/mock.go index 3a7d4ad88..5a763344c 100644 --- a/plugin/collector/snap-plugin-collector-mock2/mock/mock.go +++ b/plugin/collector/snap-plugin-collector-mock2/mock/mock.go @@ -61,7 +61,7 @@ func (f *Mock) CollectMetrics(mts []plugin.MetricType) ([]plugin.MetricType, err ns := make([]core.NamespaceElement, len(mts[i].Namespace())) copy(ns, mts[i].Namespace()) ns[2].Value = fmt.Sprintf("host%d", j) - data := randInt(65, 90) + data := randInt(65, 90) + 1000 mt := plugin.MetricType{ Data_: data, Namespace_: ns, @@ -72,7 +72,7 @@ func (f *Mock) CollectMetrics(mts []plugin.MetricType) ([]plugin.MetricType, err metrics = append(metrics, mt) } } else { - data := randInt(65, 90) + data := randInt(65, 90) + 1000 mts[i].Data_ = data mts[i].Timestamp_ = time.Now() metrics = append(metrics, mts[i]) @@ -94,11 +94,13 @@ func (f *Mock) GetMetricTypes(cfg plugin.ConfigType) ([]plugin.MetricType, error Unit_: "mock unit", }) } - mts = append(mts, plugin.MetricType{ - Namespace_: core.NewNamespace("intel", "mock", "foo"), - Description_: "mock description", - Unit_: "mock unit", - }) + if _, ok := cfg.Table()["test-less"]; !ok { + mts = append(mts, plugin.MetricType{ + Namespace_: core.NewNamespace("intel", "mock", "foo"), + Description_: "mock description", + Unit_: "mock unit", + }) + } mts = append(mts, plugin.MetricType{ Namespace_: core.NewNamespace("intel", "mock", "bar"), Description_: "mock description", diff --git a/scheduler/distributed_task_test.go b/scheduler/distributed_task_test.go index 96e2d4c67..cbe6ada3e 100644 --- a/scheduler/distributed_task_test.go +++ b/scheduler/distributed_task_test.go @@ -32,6 +32,7 @@ import ( "github.com/intelsdi-x/gomit" "github.com/intelsdi-x/snap/control" "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/cdata" "github.com/intelsdi-x/snap/core/scheduler_event" "github.com/intelsdi-x/snap/core/serror" "github.com/intelsdi-x/snap/grpc/controlproxy" @@ -339,7 +340,7 @@ type subscriptionManager struct { UnsubscribeCallCount int } -func (m *subscriptionManager) SubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *subscriptionManager) SubscribeDeps(taskID string, reqs []core.RequestedMetric, cps []core.SubscribedPlugin, cdt *cdata.ConfigDataTree) []serror.SnapError { if m.Fail { return []serror.SnapError{serror.New(errors.New("error"))} } @@ -347,7 +348,7 @@ func (m *subscriptionManager) SubscribeDeps(taskID string, mts []core.Metric, pr return nil } -func (m *subscriptionManager) UnsubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *subscriptionManager) UnsubscribeDeps(taskID string) []serror.SnapError { m.UnsubscribeCallCount += 1 return nil } diff --git a/scheduler/job.go b/scheduler/job.go index ce35a11cc..3de632c1e 100644 --- a/scheduler/job.go +++ b/scheduler/job.go @@ -239,31 +239,7 @@ func (c *collectorJob) Run() { } } - metrics := []core.Metric{} - for _, rmt := range c.metricTypes { - nss, err := c.collector.ExpandWildcards(rmt.Namespace()) - if err != nil { - // use metric directly from the workflow - nss = []core.Namespace{rmt.Namespace()} - } - - for _, ns := range nss { - config := c.configDataTree.Get(ns.Strings()) - - if config == nil { - config = cdata.NewNode() - } - - metric := &metric{ - namespace: ns, - version: rmt.Version(), - config: config, - } - metrics = append(metrics, metric) - } - } - - ret, errs := c.collector.CollectMetrics(metrics, c.Deadline(), c.TaskID(), c.tags) + ret, errs := c.collector.CollectMetrics(c.TaskID(), c.tags) log.WithFields(log.Fields{ "_module": "scheduler-job", diff --git a/scheduler/job_test.go b/scheduler/job_test.go index f5ea6871a..da0dbaf59 100644 --- a/scheduler/job_test.go +++ b/scheduler/job_test.go @@ -36,7 +36,7 @@ import ( type mockCollector struct{} -func (m *mockCollector) CollectMetrics([]core.Metric, time.Time, string, map[string]map[string]string) ([]core.Metric, []error) { +func (m *mockCollector) CollectMetrics(string, map[string]map[string]string) ([]core.Metric, []error) { return nil, nil } diff --git a/scheduler/scheduler.go b/scheduler/scheduler.go index 2de6e2652..ee15be5c0 100644 --- a/scheduler/scheduler.go +++ b/scheduler/scheduler.go @@ -27,7 +27,6 @@ import ( "path" "path/filepath" "strings" - "time" log "github.com/Sirupsen/logrus" @@ -36,6 +35,7 @@ import ( "github.com/intelsdi-x/gomit" "github.com/intelsdi-x/snap/core" + "github.com/intelsdi-x/snap/core/cdata" "github.com/intelsdi-x/snap/core/ctypes" "github.com/intelsdi-x/snap/core/scheduler_event" "github.com/intelsdi-x/snap/core/serror" @@ -73,6 +73,15 @@ const ( schedulerStarted ) +type depGroupMap map[string]struct { + requestedMetrics []core.RequestedMetric + subscribedPlugins []core.SubscribedPlugin +} + +func newDepGroup() depGroupMap { + return depGroupMap{} +} + // ManagesMetric is implemented by control // On startup a scheduler will be created and passed a reference to control type managesMetrics interface { @@ -81,10 +90,9 @@ type managesMetrics interface { processesMetrics managesPluginContentTypes GetAutodiscoverPaths() []string - ValidateDeps([]core.Metric, []core.SubscribedPlugin) []serror.SnapError - SubscribeDeps(string, []core.Metric, []core.Plugin) []serror.SnapError - UnsubscribeDeps(string, []core.Metric, []core.Plugin) []serror.SnapError - MatchQueryToNamespaces(core.Namespace) ([]core.Namespace, serror.SnapError) + ValidateDeps([]core.RequestedMetric, []core.SubscribedPlugin, *cdata.ConfigDataTree) []serror.SnapError + SubscribeDeps(string, []core.RequestedMetric, []core.SubscribedPlugin, *cdata.ConfigDataTree) []serror.SnapError + UnsubscribeDeps(string) []serror.SnapError } // ManagesPluginContentTypes is an interface to a plugin manager that can tell us what content accept and returns are supported. @@ -93,8 +101,7 @@ type managesPluginContentTypes interface { } type collectsMetrics interface { - ExpandWildcards(core.Namespace) ([]core.Namespace, serror.SnapError) - CollectMetrics([]core.Metric, time.Time, string, map[string]map[string]string) ([]core.Metric, []error) + CollectMetrics(string, map[string]map[string]string) ([]core.Metric, []error) } type publishesMetrics interface { @@ -285,8 +292,9 @@ func (s *scheduler) CreateTaskTribe(sch schedule.Schedule, wfMap *wmap.WorkflowM func (s *scheduler) createTask(sch schedule.Schedule, wfMap *wmap.WorkflowMap, startOnCreate bool, source string, opts ...core.TaskOption) (core.Task, core.TaskErrors) { logger := schedulerLogger.WithFields(log.Fields{ - "_block": "create-task", - "source": source, + "_block": "create-task", + "source": source, + "start-on-create": startOnCreate, }) // Create a container for task errors te := &taskErrors{ @@ -329,14 +337,16 @@ func (s *scheduler) createTask(sch schedule.Schedule, wfMap *wmap.WorkflowMap, s // Group dependencies by the node they live on // and validate them. - depGroupMap := s.gatherMetricsAndPlugins(wf) - for k, val := range depGroupMap { + depGroups := getWorkflowPlugins(wf.processNodes, wf.publishNodes, wf.metrics) + for k, group := range depGroups { manager, err := task.RemoteManagers.Get(k) if err != nil { te.errs = append(te.errs, serror.New(err)) return nil, te } - errs := manager.ValidateDeps(val.Metrics, val.Plugins) + var errs []serror.SnapError + errs = manager.ValidateDeps(group.requestedMetrics, group.subscribedPlugins, wf.configTree) + if len(errs) > 0 { te.errs = append(te.errs, errs...) return nil, te @@ -413,6 +423,7 @@ func (s *scheduler) removeTask(id, source string) error { TaskID: t.id, Source: source, } + defer s.eventManager.Emit(event) return s.tasks.remove(t) } @@ -484,30 +495,30 @@ func (s *scheduler) startTask(id, source string) []serror.SnapError { serror.New(ErrTaskAlreadyRunning), } } + // Group dependencies by the node they live on // and subscribe to them. - depGroupMap := s.gatherMetricsAndPlugins(t.workflow) + depGroups := getWorkflowPlugins(t.workflow.processNodes, t.workflow.publishNodes, t.workflow.metrics) var subbedDeps []string - for k := range depGroupMap { + for k := range depGroups { var errs []serror.SnapError - cps := returnCorePlugin(depGroupMap[k].Plugins) + // cps := returnCorePlugin(depGroups[k].subscribedPlugins) mgr, err := t.RemoteManagers.Get(k) if err != nil { errs = append(errs, serror.New(err)) } else { - errs = mgr.SubscribeDeps(t.ID(), depGroupMap[k].Metrics, cps) + errs = mgr.SubscribeDeps(t.ID(), depGroups[k].requestedMetrics, depGroups[k].subscribedPlugins, t.workflow.configTree) } // If there are errors with subscribing any deps, go through and unsubscribe all other // deps that may have already been subscribed then return the errors. if len(errs) > 0 { for _, key := range subbedDeps { - cps := returnCorePlugin(depGroupMap[key].Plugins) - mts := depGroupMap[key].Metrics mgr, err := t.RemoteManagers.Get(key) if err != nil { errs = append(errs, serror.New(err)) } else { - uerrs := mgr.UnsubscribeDeps(t.ID(), mts, cps) + // sending empty mts to unsubscribe to indicate task should not start + uerrs := mgr.UnsubscribeDeps(t.ID()) errs = append(errs, uerrs...) } } @@ -573,38 +584,39 @@ func (s *scheduler) stopTask(id, source string) []serror.SnapError { serror.New(ErrTaskDisabledNotStoppable), } default: - // Group depndencies by the host they live on and - // unsubscirbe them since task is stopping. - depGroupMap := s.gatherMetricsAndPlugins(t.workflow) - + // Group dependencies by the host they live on and + // unsubscribe them since task is stopping. + depGroups := getWorkflowPlugins(t.workflow.processNodes, t.workflow.publishNodes, t.workflow.metrics) var errs []serror.SnapError - for k := range depGroupMap { + for k := range depGroups { mgr, err := t.RemoteManagers.Get(k) if err != nil { errs = append(errs, serror.New(err)) } else { - uerrs := mgr.UnsubscribeDeps(t.ID(), depGroupMap[k].Metrics, returnCorePlugin(depGroupMap[k].Plugins)) + uerrs := mgr.UnsubscribeDeps(t.ID()) if len(uerrs) > 0 { errs = append(errs, uerrs...) } } - } - if len(errs) > 0 { - return errs - } + if len(errs) > 0 { + return errs + } + + event := &scheduler_event.TaskStoppedEvent{ + TaskID: t.ID(), + Source: source, + } + defer s.eventManager.Emit(event) + t.Stop() + logger.WithFields(log.Fields{ + "task-id": t.ID(), + "task-state": t.State(), + }).Info("task stopped") - event := &scheduler_event.TaskStoppedEvent{ - TaskID: t.ID(), - Source: source, } - defer s.eventManager.Emit(event) - t.Stop() - logger.WithFields(log.Fields{ - "task-id": t.ID(), - "task-state": t.State(), - }).Info("task stopped") - return nil } + + return nil } //EnableTask changes state from disabled to stopped @@ -782,12 +794,11 @@ func (s *scheduler) HandleGomitEvent(e gomit.Event) { }).Debug("event received") // We need to unsubscribe from deps when a task goes disabled task, _ := s.getTask(v.TaskID) - depGroupMap := s.gatherMetricsAndPlugins(task.workflow) - for k := range depGroupMap { - cps := returnCorePlugin(depGroupMap[k].Plugins) + depGroups := getWorkflowPlugins(task.workflow.processNodes, task.workflow.publishNodes, task.workflow.metrics) + for k := range depGroups { mgr, err := task.RemoteManagers.Get(k) if err == nil { - mgr.UnsubscribeDeps(task.ID(), depGroupMap[k].Metrics, cps) + mgr.UnsubscribeDeps(task.ID()) } } s.taskWatcherColl.handleTaskDisabled(v.TaskID, v.Why) @@ -808,57 +819,39 @@ func (s *scheduler) getTask(id string) (*task, error) { return task, nil } -type depGroup struct { - Metrics []core.Metric - Plugins []core.SubscribedPlugin -} - -func (s *scheduler) gatherMetricsAndPlugins(wf *schedulerWorkflow) map[string]depGroup { - var mts []core.Metric - depGroupMap := make(map[string]depGroup) - for _, m := range wf.metrics { - nss, err := s.metricManager.MatchQueryToNamespaces(m.Namespace()) - if err != nil { - // use metric directly from the workflow - nss = []core.Namespace{m.Namespace()} - } - - for _, ns := range nss { - mts = append(mts, &metric{ - namespace: ns, - version: m.Version(), - config: wf.configTree.Get(ns.Strings()), - }) - } - } +func getWorkflowPlugins(prnodes []*processNode, pbnodes []*publishNode, requestedMetrics []core.RequestedMetric) depGroupMap { + depGroup := depGroupMap{} // Add metrics to depGroup map under local host(signified by empty string) // for now since remote collection not supported - depGroupMap[""] = depGroup{Metrics: mts} - s.walkWorkflow(wf.processNodes, wf.publishNodes, depGroupMap) - - return depGroupMap + depGroup[""] = struct { + requestedMetrics []core.RequestedMetric + subscribedPlugins []core.SubscribedPlugin + }{requestedMetrics: requestedMetrics, + subscribedPlugins: nil} + return walkWorkflowForDeps(prnodes, pbnodes, requestedMetrics, depGroup) } -func (s *scheduler) walkWorkflow(prnodes []*processNode, pbnodes []*publishNode, depGroupMap map[string]depGroup) { +func walkWorkflowForDeps(prnodes []*processNode, pbnodes []*publishNode, requestedMetrics []core.RequestedMetric, depGroup depGroupMap) depGroupMap { for _, pr := range prnodes { - if _, ok := depGroupMap[pr.Target]; ok { - dg := depGroupMap[pr.Target] - dg.Plugins = append(dg.Plugins, pr) - depGroupMap[pr.Target] = dg + processors := depGroup[pr.Target] + if _, ok := depGroup[pr.Target]; ok { + processors.subscribedPlugins = append(processors.subscribedPlugins, pr) } else { - depGroupMap[pr.Target] = depGroup{Plugins: []core.SubscribedPlugin{pr}} + processors.subscribedPlugins = []core.SubscribedPlugin{pr} } - s.walkWorkflow(pr.ProcessNodes, pr.PublishNodes, depGroupMap) + depGroup[pr.Target] = processors + walkWorkflowForDeps(pr.ProcessNodes, pr.PublishNodes, requestedMetrics, depGroup) } for _, pb := range pbnodes { - if _, ok := depGroupMap[pb.Target]; ok { - dg := depGroupMap[pb.Target] - dg.Plugins = append(dg.Plugins, pb) - depGroupMap[pb.Target] = dg + publishers := depGroup[pb.Target] + if _, ok := depGroup[pb.Target]; ok { + publishers.subscribedPlugins = append(publishers.subscribedPlugins, pb) } else { - depGroupMap[pb.Target] = depGroup{Plugins: []core.SubscribedPlugin{pb}} + publishers.subscribedPlugins = []core.SubscribedPlugin{pb} } + depGroup[pb.Target] = publishers } + return depGroup } func returnCorePlugin(plugins []core.SubscribedPlugin) []core.Plugin { diff --git a/scheduler/scheduler_medium_test.go b/scheduler/scheduler_medium_test.go index 98413de51..2be0bcfed 100644 --- a/scheduler/scheduler_medium_test.go +++ b/scheduler/scheduler_medium_test.go @@ -82,7 +82,7 @@ func (m *mockMetricManager) GetPluginContentTypes(n string, t core.PluginType, v return m.acceptedContentTypes[key], m.returnedContentTypes[key], nil } -func (m *mockMetricManager) CollectMetrics([]core.Metric, time.Time, string, map[string]map[string]string) ([]core.Metric, []error) { +func (m *mockMetricManager) CollectMetrics(string, map[string]map[string]string) ([]core.Metric, []error) { return nil, nil } @@ -94,7 +94,7 @@ func (m *mockMetricManager) ProcessMetrics(contentType string, content []byte, p return "", nil, nil } -func (m *mockMetricManager) ValidateDeps(mts []core.Metric, prs []core.SubscribedPlugin) []serror.SnapError { +func (m *mockMetricManager) ValidateDeps(mts []core.RequestedMetric, prs []core.SubscribedPlugin, ctree *cdata.ConfigDataTree) []serror.SnapError { if m.failValidatingMetrics { return []serror.SnapError{ serror.New(errors.New("metric validation error")), @@ -102,24 +102,16 @@ func (m *mockMetricManager) ValidateDeps(mts []core.Metric, prs []core.Subscribe } return nil } -func (m *mockMetricManager) SubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *mockMetricManager) SubscribeDeps(taskID string, reqs []core.RequestedMetric, prs []core.SubscribedPlugin, ctree *cdata.ConfigDataTree) []serror.SnapError { return []serror.SnapError{ serror.New(errors.New("metric validation error")), } } -func (m *mockMetricManager) UnsubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *mockMetricManager) UnsubscribeDeps(taskID string) []serror.SnapError { return nil } -func (m *mockMetricManager) MatchQueryToNamespaces(core.Namespace) ([]core.Namespace, serror.SnapError) { - return nil, nil -} - -func (m *mockMetricManager) ExpandWildcards(core.Namespace) ([]core.Namespace, serror.SnapError) { - return nil, nil -} - func (m *mockMetricManager) SetAutodiscoverPaths(paths []string) { m.autodiscoverPaths = paths } diff --git a/scheduler/scheduler_test.go b/scheduler/scheduler_test.go index cb8901e83..591078cce 100644 --- a/scheduler/scheduler_test.go +++ b/scheduler/scheduler_test.go @@ -30,8 +30,10 @@ import ( log "github.com/Sirupsen/logrus" . "github.com/smartystreets/goconvey/convey" + "github.com/intelsdi-x/gomit" "github.com/intelsdi-x/snap/core" "github.com/intelsdi-x/snap/core/cdata" + "github.com/intelsdi-x/snap/core/control_event" "github.com/intelsdi-x/snap/core/ctypes" "github.com/intelsdi-x/snap/core/serror" "github.com/intelsdi-x/snap/pkg/schedule" @@ -82,7 +84,7 @@ func (m *mockMetricManager) GetPluginContentTypes(n string, t core.PluginType, v return m.acceptedContentTypes[key], m.returnedContentTypes[key], nil } -func (m *mockMetricManager) CollectMetrics([]core.Metric, time.Time, string, map[string]map[string]string) ([]core.Metric, []error) { +func (m *mockMetricManager) CollectMetrics(string, map[string]map[string]string) ([]core.Metric, []error) { return nil, nil } @@ -93,8 +95,7 @@ func (m *mockMetricManager) PublishMetrics(contentType string, content []byte, p func (m *mockMetricManager) ProcessMetrics(contentType string, content []byte, pluginName string, pluginVersion int, config map[string]ctypes.ConfigValue, taskID string) (string, []byte, []error) { return "", nil, nil } - -func (m *mockMetricManager) ValidateDeps(mts []core.Metric, prs []core.SubscribedPlugin) []serror.SnapError { +func (m *mockMetricManager) ValidateDeps(mts []core.RequestedMetric, prs []core.SubscribedPlugin, cdt *cdata.ConfigDataTree) []serror.SnapError { if m.failValidatingMetrics { return []serror.SnapError{ serror.New(errors.New("metric validation error")), @@ -102,24 +103,16 @@ func (m *mockMetricManager) ValidateDeps(mts []core.Metric, prs []core.Subscribe } return nil } -func (m *mockMetricManager) SubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *mockMetricManager) SubscribeDeps(taskID string, req []core.RequestedMetric, prs []core.SubscribedPlugin, cft *cdata.ConfigDataTree) []serror.SnapError { return []serror.SnapError{ serror.New(errors.New("metric validation error")), } } -func (m *mockMetricManager) UnsubscribeDeps(taskID string, mts []core.Metric, prs []core.Plugin) []serror.SnapError { +func (m *mockMetricManager) UnsubscribeDeps(taskID string) []serror.SnapError { return nil } -func (m *mockMetricManager) MatchQueryToNamespaces(core.Namespace) ([]core.Namespace, serror.SnapError) { - return nil, nil -} - -func (m *mockMetricManager) ExpandWildcards(core.Namespace) ([]core.Namespace, serror.SnapError) { - return nil, nil -} - func (m *mockMetricManager) SetAutodiscoverPaths(paths []string) { m.autodiscoverPaths = paths } @@ -174,6 +167,27 @@ func (m mockScheduleResponse) missedIntervals() uint { return 0 } +type listenToPluginEvent struct { + pluginLoaded chan struct{} + pluginStarted chan struct{} +} + +func newListenToPluginEvent() *listenToPluginEvent { + return &listenToPluginEvent{ + pluginLoaded: make(chan struct{}), + pluginStarted: make(chan struct{}), + } +} + +func (l *listenToPluginEvent) HandleGomitEvent(e gomit.Event) { + switch e.Body.(type) { + case *control_event.LoadPluginEvent: + l.pluginLoaded <- struct{}{} + case *control_event.StartPluginEvent: + l.pluginStarted <- struct{}{} + } +} + func TestScheduler(t *testing.T) { log.SetLevel(log.FatalLevel) Convey("NewTask", t, func() { diff --git a/scheduler/workflow_test.go b/scheduler/workflow_test.go index 9f609da70..2b67403b6 100644 --- a/scheduler/workflow_test.go +++ b/scheduler/workflow_test.go @@ -265,7 +265,7 @@ type Mock1 struct { queue map[string]int } -func (m *Mock1) CollectMetrics([]core.Metric, time.Time, string, map[string]map[string]string) ([]core.Metric, []error) { +func (m *Mock1) CollectMetrics(string, map[string]map[string]string) ([]core.Metric, []error) { return nil, nil }