mirror of
https://github.com/akvorado/akvorado.git
synced 2025-12-11 22:14:02 +01:00
outlet/metadata: synchronous fetching of metadata
As we are not constrained by time that much in the outlet, we can simplify the fetching of metadata by doing it synchronously. We still keep the breaker design to avoid continously polling a source that is not responsive, so we still can loose some data if we are not able to poll metadata. We also keep the background cache refresh. We also introduce a grace time of 1 minute to avoid loosing data during start. For the static provider, we wait for the remote data sources to be ready. For the gNMI provider, there are target windows of availability during which the cached data can be polled. The SNMP provider is loosing its ability to coalesce requests.
This commit is contained in:
@@ -226,7 +226,9 @@ func OutletConfigurationUnmarshallerHook() mapstructure.DecodeHookFunc {
|
||||
continue
|
||||
}
|
||||
if helpers.MapStructureMatchName(k.String(), "PollerCoalesce") {
|
||||
metadataValue["MaxBatchRequests"] = snmpMap.MapIndex(snmpKeys[i]).Interface()
|
||||
continue
|
||||
}
|
||||
if helpers.MapStructureMatchName(k.String(), "Workers") {
|
||||
continue
|
||||
}
|
||||
metadataConfig := reflect.TypeOf(metadata.Configuration{})
|
||||
|
||||
@@ -4,7 +4,7 @@ paths:
|
||||
- kafka:9092
|
||||
inlet.0.flow.inputs.0.decoder: netflow
|
||||
inlet.0.flow.inputs.1.decoder: sflow
|
||||
outlet.0.metadata.workers: 10
|
||||
outlet.0.metadata.providers.0.type: snmp
|
||||
inlet.0.kafka.brokers:
|
||||
- kafka:9092
|
||||
outlet.0.kafka.brokers:
|
||||
|
||||
@@ -1,12 +1,12 @@
|
||||
---
|
||||
paths:
|
||||
outlet.0.metadata:
|
||||
workers: 10
|
||||
maxbatchrequests: 20
|
||||
cacheduration: 30m0s
|
||||
cacherefresh: 30m0s
|
||||
cachecheckinterval: 2m0s
|
||||
cachepersistfile: ""
|
||||
initialdelay: 1m0s
|
||||
querytimeout: 5s
|
||||
providers:
|
||||
- type: snmp
|
||||
pollerretries: 3
|
||||
|
||||
@@ -161,10 +161,7 @@ func (c *Component[T]) Start() error {
|
||||
customBackoff := backoff.NewExponentialBackOff()
|
||||
customBackoff.MaxElapsedTime = 0
|
||||
customBackoff.MaxInterval = source.Interval
|
||||
customBackoff.InitialInterval = source.Interval / 10
|
||||
if customBackoff.InitialInterval > time.Second {
|
||||
customBackoff.InitialInterval = time.Second
|
||||
}
|
||||
customBackoff.InitialInterval = min(time.Second, source.Interval/10)
|
||||
return backoff.NewTicker(customBackoff)
|
||||
}
|
||||
newRegularTicker := func() *time.Ticker {
|
||||
|
||||
@@ -1,6 +1,5 @@
|
||||
---
|
||||
metadata:
|
||||
workers: 10
|
||||
provider:
|
||||
type: snmp
|
||||
credentials:
|
||||
|
||||
@@ -253,8 +253,9 @@ following keys are accepted:
|
||||
about to expire or need an update
|
||||
- `cache-persist-file` tells where to store cached data on shutdown and
|
||||
read them back on startup
|
||||
- `workers` tell how many workers to spawn to fetch metadata.
|
||||
- `max-batch-requests` define how many requests can be batched together
|
||||
- `query-timeout` tells how long to wait for a provider to answer a query
|
||||
- `initial-delay` tells how long to wait after starting before applying the
|
||||
standard query timeout
|
||||
- `providers` defines the provider configurations
|
||||
|
||||
As flows missing any interface information are discarded, persisting the cache
|
||||
|
||||
@@ -136,12 +136,10 @@ the name `clickhouse` or on `localhost`.
|
||||
## SNMP
|
||||
|
||||
SNMP polling is accomplished with [GoSNMP](https://github.com/gosnmp/gosnmp).
|
||||
The cache layer is tailored specifically for our needs. Cached information
|
||||
can expire if not accessed or refreshed periodically.
|
||||
Some coalescing of the requests is done when they are queued.
|
||||
This adds some code complexity, maybe it was not worth it.
|
||||
If an exporter fails to answer too frequently, a backoff will be triggered
|
||||
for a minute to ensure it does not eat up all the workers' resources.
|
||||
The cache layer is tailored specifically for our needs. Cached information can
|
||||
expire if not accessed or refreshed periodically. If an exporter fails to answer
|
||||
too frequently, a backoff will be triggered for a minute to ensure it does not
|
||||
eat up all the workers' resources.
|
||||
|
||||
Testing is done by another implementation of an [SNMP
|
||||
agent](https://github.com/slayercat/GoSNMPServer).
|
||||
|
||||
@@ -164,6 +164,9 @@ services:
|
||||
- ./clickhouse/keeper.xml:/etc/clickhouse-keeper/keeper_config.d/keeper.xml
|
||||
|
||||
srlinux:
|
||||
# The SR Linux container does not like to be restarted. If you get:
|
||||
# Error: Peer netns reference is invalid.
|
||||
# Be sure to put it down before starting it.
|
||||
extends:
|
||||
file: versions.yml
|
||||
service: srlinux
|
||||
|
||||
@@ -34,8 +34,8 @@ func (w *worker) enrichFlow(exporterIP netip.Addr, exporterStr string) (skip boo
|
||||
c := w.c
|
||||
|
||||
if flow.InIf != 0 {
|
||||
answer, ok := c.d.Metadata.Lookup(t, exporterIP, uint(flow.InIf))
|
||||
if !ok {
|
||||
answer := c.d.Metadata.Lookup(t, exporterIP, uint(flow.InIf))
|
||||
if !answer.Found {
|
||||
c.metrics.flowsErrors.WithLabelValues(exporterStr, "SNMP cache miss").Inc()
|
||||
skip = true
|
||||
} else {
|
||||
@@ -57,8 +57,8 @@ func (w *worker) enrichFlow(exporterIP netip.Addr, exporterStr string) (skip boo
|
||||
}
|
||||
|
||||
if flow.OutIf != 0 {
|
||||
answer, ok := c.d.Metadata.Lookup(t, exporterIP, uint(flow.OutIf))
|
||||
if !ok {
|
||||
answer := c.d.Metadata.Lookup(t, exporterIP, uint(flow.OutIf))
|
||||
if !answer.Found {
|
||||
// Only register a cache miss if we don't have one.
|
||||
// TODO: maybe we could do one SNMP query for both interfaces.
|
||||
if !skip {
|
||||
|
||||
@@ -645,9 +645,6 @@ ClassifyProviderRegex(Interface.Description, "^Transit: ([^ ]+)", "$1")`,
|
||||
t.Fatalf("proto.Marshal() error: %v", err)
|
||||
}
|
||||
|
||||
// Test twice to check cache behavior
|
||||
incoming <- data
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
incoming <- data
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
@@ -666,10 +663,9 @@ ClassifyProviderRegex(Interface.Description, "^Transit: ([^ ]+)", "$1")`,
|
||||
}
|
||||
gotMetrics := r.GetMetrics("akvorado_outlet_core_", "-processing_", "flows_", "received_", "forwarded_")
|
||||
expectedMetrics := map[string]string{
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.142"}`: "1",
|
||||
`flows_http_clients`: "0",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "2",
|
||||
`received_raw_flows_total`: "2",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "1",
|
||||
`received_raw_flows_total`: "1",
|
||||
}
|
||||
if tc.OutputFlow != nil {
|
||||
expectedMetrics[`forwarded_flows_total{exporter="192.0.2.142"}`] = "1"
|
||||
|
||||
@@ -140,33 +140,26 @@ func TestCore(t *testing.T) {
|
||||
clickhouseMessages = clickhouseMessages[:0]
|
||||
clickhouseMessagesMutex.Unlock()
|
||||
|
||||
// Inject several messages with a cache miss from the SNMP component.
|
||||
// Inject several messages
|
||||
injectFlow(flowMessage("192.0.2.142", 434, 677))
|
||||
injectFlow(flowMessage("192.0.2.143", 434, 677))
|
||||
injectFlow(flowMessage("192.0.2.143", 437, 677))
|
||||
injectFlow(flowMessage("192.0.2.143", 434, 679))
|
||||
injectFlow(flowMessage("192.0.2.143", 437, 679))
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
|
||||
gotMetrics := r.GetMetrics("akvorado_outlet_core_", "-flows_processing_")
|
||||
expectedMetrics := map[string]string{
|
||||
`classifier_exporter_cache_items_total`: "0",
|
||||
`classifier_interface_cache_items_total`: "0",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.142"}`: "1",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.143"}`: "3",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "1",
|
||||
`received_flows_total{exporter="192.0.2.143"}`: "3",
|
||||
`received_raw_flows_total`: "4",
|
||||
`received_flows_total{exporter="192.0.2.143"}`: "1",
|
||||
`forwarded_flows_total{exporter="192.0.2.142"}`: "1",
|
||||
`forwarded_flows_total{exporter="192.0.2.143"}`: "1",
|
||||
`received_raw_flows_total`: "2",
|
||||
`flows_http_clients`: "0",
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
// Inject again the messages, this time, we will get a cache hit!
|
||||
injectFlow(flowMessage("192.0.2.142", 434, 677))
|
||||
injectFlow(flowMessage("192.0.2.143", 437, 679))
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
|
||||
// Should have 2 more flows in clickhouseMessages now
|
||||
clickhouseMessagesMutex.Lock()
|
||||
clickhouseMessagesLen := len(clickhouseMessages)
|
||||
@@ -175,24 +168,6 @@ func TestCore(t *testing.T) {
|
||||
t.Fatalf("Expected at least 2 flows in clickhouseMessages, got %d", clickhouseMessagesLen)
|
||||
}
|
||||
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
gotMetrics = r.GetMetrics("akvorado_outlet_core_", "classifier_", "-flows_processing_", "flows_", "received_", "forwarded_")
|
||||
expectedMetrics = map[string]string{
|
||||
`classifier_exporter_cache_items_total`: "0",
|
||||
`classifier_interface_cache_items_total`: "0",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.142"}`: "1",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.143"}`: "3",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "2",
|
||||
`received_flows_total{exporter="192.0.2.143"}`: "4",
|
||||
`received_raw_flows_total`: "6",
|
||||
`forwarded_flows_total{exporter="192.0.2.142"}`: "1",
|
||||
`forwarded_flows_total{exporter="192.0.2.143"}`: "1",
|
||||
`flows_http_clients`: "0",
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
// Now, check we get the message we expect
|
||||
clickhouseMessagesMutex.Lock()
|
||||
clickhouseMessages = clickhouseMessages[:0]
|
||||
@@ -221,15 +196,13 @@ func TestCore(t *testing.T) {
|
||||
expectedMetrics = map[string]string{
|
||||
`classifier_exporter_cache_items_total`: "0",
|
||||
`classifier_interface_cache_items_total`: "0",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.142"}`: "1",
|
||||
`flows_errors_total{error="SNMP cache miss",exporter="192.0.2.143"}`: "3",
|
||||
`flows_errors_total{error="sampling rate missing",exporter="192.0.2.142"}`: "1",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "4",
|
||||
`received_flows_total{exporter="192.0.2.143"}`: "4",
|
||||
`received_flows_total{exporter="192.0.2.142"}`: "3",
|
||||
`received_flows_total{exporter="192.0.2.143"}`: "1",
|
||||
`forwarded_flows_total{exporter="192.0.2.142"}`: "2",
|
||||
`forwarded_flows_total{exporter="192.0.2.143"}`: "1",
|
||||
`flows_http_clients`: "0",
|
||||
`received_raw_flows_total`: "8",
|
||||
`received_raw_flows_total`: "4",
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
|
||||
@@ -27,10 +27,11 @@ type Configuration struct {
|
||||
// Provider defines the configuration of the providers to use
|
||||
Providers []ProviderConfiguration
|
||||
|
||||
// Workers define the number of workers used to poll metadata
|
||||
Workers int `validate:"min=1"`
|
||||
// MaxBatchRequests define how many requests to pass to a worker at once if possible
|
||||
MaxBatchRequests int `validate:"min=0"`
|
||||
// QueryTimeout defines how long to wait for a provider to answer.
|
||||
QueryTimeout time.Duration `validate:"min=100ms,max=1m"`
|
||||
// InitialDelay defines how long to wait at start (when receiving the first
|
||||
// packets) before applying the query timeout
|
||||
InitialDelay time.Duration `validate:"min=1s,max=1h"`
|
||||
}
|
||||
|
||||
// DefaultConfiguration represents the default configuration for the metadata provider.
|
||||
@@ -39,9 +40,8 @@ func DefaultConfiguration() Configuration {
|
||||
CacheDuration: 30 * time.Minute,
|
||||
CacheRefresh: time.Hour,
|
||||
CacheCheckInterval: 2 * time.Minute,
|
||||
CachePersistFile: "",
|
||||
Workers: 1,
|
||||
MaxBatchRequests: 10,
|
||||
QueryTimeout: 5 * time.Second,
|
||||
InitialDelay: time.Minute,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -52,7 +52,7 @@ type ProviderConfiguration struct {
|
||||
}
|
||||
|
||||
// MarshalYAML undoes ConfigurationUnmarshallerHook().
|
||||
func (pc ProviderConfiguration) MarshalYAML() (interface{}, error) {
|
||||
func (pc ProviderConfiguration) MarshalYAML() (any, error) {
|
||||
return helpers.ParametrizedConfigurationMarshalYAML(pc, providers)
|
||||
}
|
||||
|
||||
@@ -67,4 +67,5 @@ func init() {
|
||||
helpers.RenameKeyUnmarshallerHook(Configuration{}, "Provider", "Providers"))
|
||||
helpers.RegisterMapstructureUnmarshallerHook(
|
||||
helpers.ParametrizedConfigurationUnmarshallerHook(ProviderConfiguration{}, providers))
|
||||
helpers.RegisterMapstructureDeprecatedFields[Configuration]("Workers", "MaxBatchRequests")
|
||||
}
|
||||
|
||||
@@ -22,7 +22,8 @@ import (
|
||||
// exporterState is the state of an exporter.
|
||||
type exporterState struct {
|
||||
Name string
|
||||
Ready bool
|
||||
ready bool // ready for the first time
|
||||
Ready chan bool // not polling, data ready
|
||||
Interfaces map[uint]provider.Interface
|
||||
}
|
||||
|
||||
@@ -243,49 +244,70 @@ retryDetect:
|
||||
l.Debug().Msg("polling")
|
||||
start := time.Now()
|
||||
subscribeResp, err := tg.SubscribeOnce(ctx, subscribeReq)
|
||||
p.metrics.times.WithLabelValues(exporterStr).Observe(time.Now().Sub(start).Seconds())
|
||||
p.metrics.times.WithLabelValues(exporterStr).Observe(time.Since(start).Seconds())
|
||||
if err == nil {
|
||||
events := subscribeResponsesToEvents(subscribeResp)
|
||||
p.metrics.paths.WithLabelValues(exporterStr).Set(float64(len(events)))
|
||||
p.stateLock.Lock()
|
||||
state.update(events, model)
|
||||
state.Ready = true
|
||||
state.ready = true
|
||||
p.stateLock.Unlock()
|
||||
l.Debug().Msg("state updated")
|
||||
p.metrics.ready.WithLabelValues(exporterStr).Set(1)
|
||||
p.metrics.updates.WithLabelValues(exporterStr).Inc()
|
||||
|
||||
// On success, wait a bit before next refresh interval
|
||||
// In the following window, we consider ourselves ready and unlock
|
||||
// waiting clients to check for data.
|
||||
|
||||
// On success, wait a bit before next refresh interval and ignore
|
||||
// any refresh requests.
|
||||
next := time.NewTimer(p.config.MinimalRefreshInterval)
|
||||
outerWaitRefreshTimer:
|
||||
for {
|
||||
select {
|
||||
case state.Ready <- true:
|
||||
case <-ctx.Done():
|
||||
next.Stop()
|
||||
return
|
||||
case <-next.C:
|
||||
}
|
||||
// Drain any message in refresh queue (we ignore them)
|
||||
select {
|
||||
case <-p.refresh:
|
||||
default:
|
||||
case <-next.C:
|
||||
break outerWaitRefreshTimer
|
||||
}
|
||||
}
|
||||
// Wait for a new message in refresh queue
|
||||
l.Debug().Msg("wait for refresh request")
|
||||
outerWaitRefresh:
|
||||
for {
|
||||
select {
|
||||
case state.Ready <- true:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-p.refresh:
|
||||
break outerWaitRefresh
|
||||
}
|
||||
}
|
||||
// Reset retry timer and do the next fresh
|
||||
retryFetchBackoff.Reset()
|
||||
} else {
|
||||
// On error, retry a bit later
|
||||
// On error, retry a bit later. While retrying, if we have an
|
||||
// initial state, consider ourselves ready.
|
||||
l.Err(err).Msg("cannot poll")
|
||||
p.metrics.errors.WithLabelValues(exporterStr, "cannot poll").Inc()
|
||||
next := time.NewTimer(retryFetchBackoff.NextBackOff())
|
||||
var readyChan chan bool
|
||||
if state.ready {
|
||||
readyChan = state.Ready
|
||||
}
|
||||
outerWaitRetryTimer:
|
||||
for {
|
||||
select {
|
||||
case readyChan <- true:
|
||||
case <-ctx.Done():
|
||||
next.Stop()
|
||||
return
|
||||
case <-next.C:
|
||||
break outerWaitRetryTimer
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,19 +19,16 @@ type Provider struct {
|
||||
config *Configuration
|
||||
metrics metrics
|
||||
|
||||
put func(provider.Update)
|
||||
refresh chan bool
|
||||
|
||||
state map[netip.Addr]*exporterState
|
||||
stateLock sync.Mutex
|
||||
refresh chan bool
|
||||
}
|
||||
|
||||
// New creates a new gNMI provider from configuration
|
||||
func (configuration Configuration) New(r *reporter.Reporter, put func(provider.Update)) (provider.Provider, error) {
|
||||
func (configuration Configuration) New(r *reporter.Reporter) (provider.Provider, error) {
|
||||
p := Provider{
|
||||
r: r,
|
||||
config: &configuration,
|
||||
put: put,
|
||||
state: map[netip.Addr]*exporterState{},
|
||||
refresh: make(chan bool),
|
||||
}
|
||||
@@ -40,42 +37,51 @@ func (configuration Configuration) New(r *reporter.Reporter, put func(provider.U
|
||||
}
|
||||
|
||||
// Query queries exporter to get information through gNMI.
|
||||
func (p *Provider) Query(ctx context.Context, q *provider.BatchQuery) error {
|
||||
func (p *Provider) Query(ctx context.Context, q provider.Query) (provider.Answer, error) {
|
||||
p.stateLock.Lock()
|
||||
defer p.stateLock.Unlock()
|
||||
state, ok := p.state[q.ExporterIP]
|
||||
// If we don't have a collector for the provided IP, starts one. We should
|
||||
// be sure we don't have several collectors for the same exporter, hence the
|
||||
// write lock for everything.
|
||||
if !ok {
|
||||
state := exporterState{}
|
||||
p.state[q.ExporterIP] = &state
|
||||
go p.startCollector(ctx, q.ExporterIP, &state)
|
||||
state = &exporterState{
|
||||
Ready: make(chan bool),
|
||||
}
|
||||
p.state[q.ExporterIP] = state
|
||||
p.metrics.collectorCount.Inc()
|
||||
return nil
|
||||
go p.startCollector(ctx, q.ExporterIP, state)
|
||||
}
|
||||
// If the collector exists and already provided some data, populate the
|
||||
// cache.
|
||||
if state.Ready {
|
||||
for _, ifindex := range q.IfIndexes {
|
||||
p.put(provider.Update{
|
||||
Query: provider.Query{
|
||||
ExporterIP: q.ExporterIP,
|
||||
IfIndex: ifindex,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
Exporter: provider.Exporter{
|
||||
Name: state.Name,
|
||||
},
|
||||
Interface: state.Interfaces[ifindex],
|
||||
},
|
||||
})
|
||||
}
|
||||
// Also trigger a refresh
|
||||
|
||||
// Trigger a refresh
|
||||
select {
|
||||
case p.refresh <- true:
|
||||
default:
|
||||
}
|
||||
|
||||
// Wait for the collector to be ready.
|
||||
select {
|
||||
case <-state.Ready:
|
||||
// Most common case, keep the lock
|
||||
default:
|
||||
// Not ready, release the lock until ready
|
||||
p.stateLock.Unlock()
|
||||
select {
|
||||
case <-state.Ready:
|
||||
p.stateLock.Lock()
|
||||
case <-ctx.Done():
|
||||
p.metrics.errors.WithLabelValues(q.ExporterIP.Unmap().String(), "not ready").Inc()
|
||||
return provider.Answer{}, ctx.Err()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
defer p.stateLock.Unlock()
|
||||
|
||||
// Return the result from the state
|
||||
iface, ok := state.Interfaces[q.IfIndex]
|
||||
if !ok {
|
||||
return provider.Answer{}, nil
|
||||
}
|
||||
return provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: state.Name,
|
||||
},
|
||||
Interface: iface,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -722,18 +722,16 @@ commit now
|
||||
configP.Ports = helpers.MustNewSubnetMap(map[string]uint16{
|
||||
"::/0": netip.MustParseAddrPort(srLinuxGNMI).Port(),
|
||||
})
|
||||
put := func(update provider.Update) {
|
||||
got = append(got, fmt.Sprintf("%s %s %d %s %s %d",
|
||||
update.ExporterIP.Unmap().String(), update.Exporter.Name,
|
||||
update.IfIndex, update.Interface.Name, update.Interface.Description, update.Interface.Speed))
|
||||
formatUpdate := func(exporter netip.Addr, iface string, answer provider.Answer) string {
|
||||
return fmt.Sprintf("%s %v %s %s %s %s %d",
|
||||
exporter.Unmap().String(), answer.Found, answer.Exporter.Name,
|
||||
iface, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed)
|
||||
}
|
||||
r := reporter.NewMock(t)
|
||||
p, err := configP.New(r, put)
|
||||
p, err := configP.New(r)
|
||||
if err != nil {
|
||||
t.Fatalf("New() error:\n%+v", err)
|
||||
}
|
||||
// Let's trigger a request now
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{641}})
|
||||
|
||||
// We need the indexes
|
||||
subscribeReq, err := api.NewSubscribeRequest(
|
||||
@@ -760,23 +758,27 @@ commit now
|
||||
t.Logf("indexes: %v", indexes)
|
||||
|
||||
// Wait a bit
|
||||
time.Sleep(500 * time.Millisecond)
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{indexes["name=ethernet-1/1"]}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{indexes["name=ethernet-1/2"]}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo,
|
||||
IfIndexes: []uint{indexes["name=lag1"], indexes["name=ethernet-1/3"]}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{5}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo,
|
||||
IfIndexes: []uint{indexes["name=ethernet-1/4,index=1"]}})
|
||||
answer, _ := p.Query(context.Background(), provider.Query{ExporterIP: lo, IfIndex: indexes["name=ethernet-1/1"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/1", answer))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: lo, IfIndex: indexes["name=ethernet-1/2"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/2", answer))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: lo, IfIndex: indexes["name=lag1"]})
|
||||
got = append(got, formatUpdate(lo, "lag1", answer))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: lo, IfIndex: indexes["name=ethernet-1/3"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/3", answer))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: lo, IfIndex: 5})
|
||||
got = append(got, formatUpdate(lo, "idx5", answer))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: lo,
|
||||
IfIndex: indexes["name=ethernet-1/4,index=1"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/4,index=1", answer))
|
||||
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
if diff := helpers.Diff(got, []string{
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/1 1st interface 100000", indexes["name=ethernet-1/1"]),
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/2 2nd interface 100000", indexes["name=ethernet-1/2"]),
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d lag1 lag interface 0", indexes["name=lag1"]),
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/3 3rd interface 100000", indexes["name=ethernet-1/3"]),
|
||||
"127.0.0.1 srlinux 5 0",
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/4.1 4th interface 100000", indexes["name=ethernet-1/4,index=1"]),
|
||||
"127.0.0.1 true srlinux ethernet-1/1 ethernet-1/1 1st interface 100000",
|
||||
"127.0.0.1 true srlinux ethernet-1/2 ethernet-1/2 2nd interface 100000",
|
||||
"127.0.0.1 true srlinux lag1 lag1 lag interface 0",
|
||||
"127.0.0.1 true srlinux ethernet-1/3 ethernet-1/3 3rd interface 100000",
|
||||
"127.0.0.1 false idx5 0",
|
||||
"127.0.0.1 true srlinux ethernet-1/4,index=1 ethernet-1/4.1 4th interface 100000",
|
||||
}); diff != "" {
|
||||
t.Fatalf("Query() (-got, +want):\n%s", diff)
|
||||
}
|
||||
@@ -812,20 +814,19 @@ commit now
|
||||
if resp.Failed != nil {
|
||||
t.Fatalf("SendConfig() error:\n%+v", resp.Failed)
|
||||
}
|
||||
time.Sleep(500 * time.Millisecond) // We should exceed the second now and next request will trigger a refresh
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{indexes["name=ethernet-1/1"]}})
|
||||
time.Sleep(300 * time.Millisecond) // Do it again to get the fresh value
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo, IfIndexes: []uint{indexes["name=ethernet-1/1"]}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: lo,
|
||||
IfIndexes: []uint{indexes["name=ethernet-1/4,index=1"]}})
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
time.Sleep(time.Second) // We should exceed the second now and next request will trigger a refresh
|
||||
t.Log("start queries")
|
||||
answer, _ = p.Query(context.Background(),
|
||||
provider.Query{ExporterIP: lo, IfIndex: indexes["name=ethernet-1/1"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/1", answer))
|
||||
answer, _ = p.Query(context.Background(),
|
||||
provider.Query{ExporterIP: lo, IfIndex: indexes["name=ethernet-1/4,index=1"]})
|
||||
got = append(got, formatUpdate(lo, "ethernet-1/4,index=1", answer))
|
||||
if diff := helpers.Diff(got, []string{
|
||||
// Previous value
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/1 1st interface 100000", indexes["name=ethernet-1/1"]),
|
||||
// Fresh value
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d ethernet-1/1 1st interface new 100000", indexes["name=ethernet-1/1"]),
|
||||
"127.0.0.1 true srlinux ethernet-1/1 ethernet-1/1 1st interface new 100000",
|
||||
// Removed value
|
||||
fmt.Sprintf("127.0.0.1 srlinux %d 0", indexes["name=ethernet-1/4,index=1"]),
|
||||
"127.0.0.1 false ethernet-1/4,index=1 0",
|
||||
}); diff != "" {
|
||||
t.Fatalf("Query() (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
@@ -13,9 +13,11 @@ import (
|
||||
"akvorado/common/schema"
|
||||
)
|
||||
|
||||
// ErrSkipProvider is the error returned on lookup for providers unwilling to
|
||||
// handle a request.
|
||||
var ErrSkipProvider = errors.New("provider skips query")
|
||||
var (
|
||||
// ErrSkipProvider is the error returned on lookup for providers unwilling to
|
||||
// handle a request.
|
||||
ErrSkipProvider = errors.New("provider skips query")
|
||||
)
|
||||
|
||||
// Interface contains the information about an interface.
|
||||
type Interface struct {
|
||||
@@ -49,32 +51,24 @@ type Query struct {
|
||||
IfIndex uint
|
||||
}
|
||||
|
||||
// BatchQuery is a batched query.
|
||||
type BatchQuery struct {
|
||||
ExporterIP netip.Addr
|
||||
IfIndexes []uint
|
||||
}
|
||||
|
||||
// Answer is the answer received from a provider.
|
||||
type Answer struct {
|
||||
Found bool
|
||||
Exporter Exporter
|
||||
Interface Interface
|
||||
}
|
||||
|
||||
// Update is an update received from a provider.
|
||||
type Update struct {
|
||||
Query
|
||||
Answer
|
||||
}
|
||||
|
||||
// Provider is the interface a provider should implement.
|
||||
type Provider interface {
|
||||
// Query asks the provider to query metadata for several requests.
|
||||
Query(ctx context.Context, query *BatchQuery) error
|
||||
// Query asks the provider to query metadata and return the result. The
|
||||
// error can be ErrSkipProvider to skip to the next provider or another
|
||||
// transient error. If the result is not found, Answer.Found is set to
|
||||
// False.
|
||||
Query(ctx context.Context, query Query) (Answer, error)
|
||||
}
|
||||
|
||||
// Configuration defines an interface to configure a provider.
|
||||
type Configuration interface {
|
||||
// New instantiates a new provider from its configuration.
|
||||
New(r *reporter.Reporter, put func(Update)) (Provider, error)
|
||||
New(r *reporter.Reporter) (Provider, error)
|
||||
}
|
||||
|
||||
@@ -20,7 +20,7 @@ import (
|
||||
type Configuration struct {
|
||||
// PollerRetries tell how many time a poller should retry before giving up
|
||||
PollerRetries int `validate:"min=0"`
|
||||
// PollerTimeout tell how much time a poller should wait for an answer
|
||||
// PollerTimeout tell how much time a poller should wait for an answer before retrying
|
||||
PollerTimeout time.Duration `validate:"min=100ms"`
|
||||
|
||||
// Credentials is a mapping from exporter IPs to credentials
|
||||
|
||||
@@ -17,34 +17,9 @@ import (
|
||||
"akvorado/outlet/metadata/provider"
|
||||
)
|
||||
|
||||
// Poll polls the SNMP provider for the requested interface indexes.
|
||||
func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port uint16, ifIndexes []uint, put func(provider.Update)) error {
|
||||
// Check if already have a request running
|
||||
// Poll polls the SNMP provider for the requested interface index.
|
||||
func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port uint16, ifIndex uint) (provider.Answer, error) {
|
||||
exporterStr := exporter.Unmap().String()
|
||||
filteredIfIndexes := make([]uint, 0, len(ifIndexes))
|
||||
keys := make([]string, 0, len(ifIndexes))
|
||||
p.pendingRequestsLock.Lock()
|
||||
for _, ifIndex := range ifIndexes {
|
||||
key := fmt.Sprintf("%s@%d", exporterStr, ifIndex)
|
||||
_, ok := p.pendingRequests[key]
|
||||
if !ok {
|
||||
p.pendingRequests[key] = struct{}{}
|
||||
filteredIfIndexes = append(filteredIfIndexes, ifIndex)
|
||||
keys = append(keys, key)
|
||||
}
|
||||
}
|
||||
p.pendingRequestsLock.Unlock()
|
||||
if len(filteredIfIndexes) == 0 {
|
||||
return nil
|
||||
}
|
||||
ifIndexes = filteredIfIndexes
|
||||
defer func() {
|
||||
p.pendingRequestsLock.Lock()
|
||||
for _, key := range keys {
|
||||
delete(p.pendingRequests, key)
|
||||
}
|
||||
p.pendingRequestsLock.Unlock()
|
||||
}()
|
||||
|
||||
// Instantiate an SNMP state
|
||||
g := &gosnmp.GoSNMP{
|
||||
@@ -99,16 +74,13 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
p.metrics.errors.WithLabelValues(exporterStr, "connect").Inc()
|
||||
p.errLogger.Err(err).Str("exporter", exporterStr).Msg("unable to connect")
|
||||
}
|
||||
requests := []string{"1.3.6.1.2.1.1.5.0"}
|
||||
for _, ifIndex := range ifIndexes {
|
||||
moreRequests := []string{
|
||||
requests := []string{
|
||||
"1.3.6.1.2.1.1.5.0",
|
||||
fmt.Sprintf("1.3.6.1.2.1.2.2.1.2.%d", ifIndex), // ifDescr
|
||||
fmt.Sprintf("1.3.6.1.2.1.31.1.1.1.1.%d", ifIndex), // ifName
|
||||
fmt.Sprintf("1.3.6.1.2.1.31.1.1.1.18.%d", ifIndex), // ifAlias
|
||||
fmt.Sprintf("1.3.6.1.2.1.31.1.1.1.15.%d", ifIndex), // ifSpeed
|
||||
}
|
||||
requests = append(requests, moreRequests...)
|
||||
}
|
||||
var results []gosnmp.SnmpPDU
|
||||
success := false
|
||||
|
||||
@@ -128,17 +100,17 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
g.Community = community
|
||||
currentResult, err := g.Get(requests)
|
||||
if errors.Is(err, context.Canceled) {
|
||||
return nil
|
||||
return provider.Answer{}, err
|
||||
}
|
||||
if err != nil && canError {
|
||||
return logError(err)
|
||||
return provider.Answer{}, logError(err)
|
||||
}
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
if currentResult.Error != gosnmp.NoError && currentResult.ErrorIndex == 0 && canError {
|
||||
// There is some error affecting the whole request
|
||||
return logError(fmt.Errorf("SNMP error %s(%d)", currentResult.Error, currentResult.Error))
|
||||
return provider.Answer{}, logError(fmt.Errorf("SNMP error %s(%d)", currentResult.Error, currentResult.Error))
|
||||
}
|
||||
success = true
|
||||
if results == nil {
|
||||
@@ -158,7 +130,7 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
if len(results) != len(requests) {
|
||||
logError(fmt.Errorf("SNMP mismatch on variable lengths"))
|
||||
}
|
||||
p.metrics.times.WithLabelValues(exporterStr).Observe(time.Now().Sub(start).Seconds())
|
||||
p.metrics.times.WithLabelValues(exporterStr).Observe(time.Since(start).Seconds())
|
||||
|
||||
processStr := func(idx int, what string) (string, bool) {
|
||||
switch results[idx].Type {
|
||||
@@ -186,22 +158,18 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
}
|
||||
sysNameVal, ok := processStr(0, "sysname")
|
||||
if !ok {
|
||||
return errors.New("unable to get sysName")
|
||||
return provider.Answer{}, errors.New("unable to get sysName")
|
||||
}
|
||||
for idx := 1; idx < len(requests)-3; idx += 4 {
|
||||
|
||||
var (
|
||||
name, description string
|
||||
speed uint
|
||||
)
|
||||
ifIndex := ifIndexes[(idx-1)/4]
|
||||
ok := true
|
||||
// We do not process results when index is 0 (this can happen for local
|
||||
// traffic, we only care for exporter name).
|
||||
if ifIndex > 0 {
|
||||
ifDescrVal, okDescr := processStr(idx, "ifdescr")
|
||||
ifNameVal, okName := processStr(idx+1, "ifname")
|
||||
ifAliasVal, okAlias := processStr(idx+2, "ifalias")
|
||||
ifSpeedVal, okSpeed := processUint(idx+3, "ifspeed")
|
||||
ok = true
|
||||
ifDescrVal, okDescr := processStr(1, "ifdescr")
|
||||
ifNameVal, okName := processStr(2, "ifname")
|
||||
ifAliasVal, okAlias := processStr(3, "ifalias")
|
||||
ifSpeedVal, okSpeed := processUint(4, "ifspeed")
|
||||
|
||||
// Many equipments are using ifDescr for the interface name and
|
||||
// ifAlias for the description, which is counter-intuitive. We want
|
||||
@@ -226,20 +194,10 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
// Speed is mandatory
|
||||
ok = ok && okSpeed
|
||||
speed = ifSpeedVal
|
||||
}
|
||||
if ok {
|
||||
p.metrics.successes.WithLabelValues(exporterStr).Inc()
|
||||
} else {
|
||||
name = ""
|
||||
description = ""
|
||||
speed = 0
|
||||
}
|
||||
put(provider.Update{
|
||||
Query: provider.Query{
|
||||
ExporterIP: exporter,
|
||||
IfIndex: ifIndex,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
return provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: sysNameVal,
|
||||
},
|
||||
@@ -248,11 +206,9 @@ func (p *Provider) Poll(ctx context.Context, exporter, agent netip.Addr, port ui
|
||||
Description: description,
|
||||
Speed: speed,
|
||||
},
|
||||
},
|
||||
})
|
||||
}, nil
|
||||
}
|
||||
|
||||
return nil
|
||||
return provider.Answer{}, nil
|
||||
}
|
||||
|
||||
type goSNMPLogger struct {
|
||||
|
||||
@@ -264,41 +264,52 @@ func TestPoller(t *testing.T) {
|
||||
config.Ports = helpers.MustNewSubnetMap(map[string]uint16{
|
||||
"::/0": uint16(port),
|
||||
})
|
||||
put := func(update provider.Update) {
|
||||
got = append(got, fmt.Sprintf("%s %s %d %s %s %d",
|
||||
update.ExporterIP.Unmap().String(), update.Exporter.Name,
|
||||
update.IfIndex, update.Interface.Name, update.Interface.Description, update.Interface.Speed))
|
||||
}
|
||||
p, err := config.New(r, put)
|
||||
p, err := config.New(r)
|
||||
if err != nil {
|
||||
t.Fatalf("New() error:\n%+v", err)
|
||||
}
|
||||
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: tc.ExporterIP, IfIndexes: []uint{641}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: tc.ExporterIP, IfIndexes: []uint{642}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: tc.ExporterIP, IfIndexes: []uint{643, 644, 645}})
|
||||
p.Query(context.Background(), &provider.BatchQuery{ExporterIP: tc.ExporterIP, IfIndexes: []uint{0}})
|
||||
// Collect results from all queries
|
||||
answer, _ := p.Query(context.Background(), provider.Query{ExporterIP: tc.ExporterIP, IfIndex: 641})
|
||||
got = append(got, fmt.Sprintf("%v %s %s %d %s %s %d",
|
||||
answer.Found, tc.ExporterIP.Unmap().String(), answer.Exporter.Name,
|
||||
641, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: tc.ExporterIP, IfIndex: 642})
|
||||
got = append(got, fmt.Sprintf("%v %s %s %d %s %s %d",
|
||||
answer.Found, tc.ExporterIP.Unmap().String(), answer.Exporter.Name,
|
||||
642, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: tc.ExporterIP, IfIndex: 643})
|
||||
got = append(got, fmt.Sprintf("%v %s %s %d %s %s %d",
|
||||
answer.Found, tc.ExporterIP.Unmap().String(), answer.Exporter.Name,
|
||||
643, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: tc.ExporterIP, IfIndex: 644})
|
||||
got = append(got, fmt.Sprintf("%v %s %s %d %s %s %d",
|
||||
answer.Found, tc.ExporterIP.Unmap().String(), answer.Exporter.Name,
|
||||
644, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed))
|
||||
answer, _ = p.Query(context.Background(), provider.Query{ExporterIP: tc.ExporterIP, IfIndex: 645})
|
||||
got = append(got, fmt.Sprintf("%v %s %s %d %s %s %d",
|
||||
answer.Found, tc.ExporterIP.Unmap().String(), answer.Exporter.Name,
|
||||
645, answer.Interface.Name, answer.Interface.Description, answer.Interface.Speed))
|
||||
|
||||
exporterStr := tc.ExporterIP.Unmap().String()
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
if diff := helpers.Diff(got, []string{
|
||||
fmt.Sprintf(`%s exporter62 641 Gi0/0/0/0 Transit 10000`, exporterStr),
|
||||
fmt.Sprintf(`%s exporter62 642 Gi0/0/0/1 Peering 20000`, exporterStr),
|
||||
fmt.Sprintf(`%s exporter62 643 Gi0/0/0/2 10000`, exporterStr), // no ifAlias
|
||||
fmt.Sprintf(`%s exporter62 644 0`, exporterStr), // negative cache
|
||||
fmt.Sprintf(`%s exporter62 645 Gi0/0/0/5 Correct description 1000`, exporterStr), // negative cache
|
||||
fmt.Sprintf(`%s exporter62 0 0`, exporterStr),
|
||||
fmt.Sprintf(`true %s exporter62 641 Gi0/0/0/0 Transit 10000`, exporterStr),
|
||||
fmt.Sprintf(`true %s exporter62 642 Gi0/0/0/1 Peering 20000`, exporterStr),
|
||||
fmt.Sprintf(`true %s exporter62 643 Gi0/0/0/2 10000`, exporterStr), // no ifAlias
|
||||
fmt.Sprintf(`false %s 644 0`, exporterStr),
|
||||
fmt.Sprintf(`true %s exporter62 645 Gi0/0/0/5 Correct description 1000`, exporterStr),
|
||||
}); diff != "" {
|
||||
t.Fatalf("Poll() (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
gotMetrics := r.GetMetrics("akvorado_outlet_metadata_provider_snmp_poller_", "error_", "pending_", "success_")
|
||||
gotMetrics := r.GetMetrics("akvorado_outlet_metadata_provider_snmp_poller_", "error_", "success_")
|
||||
expectedMetrics := map[string]string{
|
||||
fmt.Sprintf(`error_requests_total{error="ifalias missing",exporter="%s"}`, exporterStr): "2", // 643+644
|
||||
fmt.Sprintf(`error_requests_total{error="ifdescr missing",exporter="%s"}`, exporterStr): "1", // 644
|
||||
fmt.Sprintf(`error_requests_total{error="ifname missing",exporter="%s"}`, exporterStr): "1", // 644
|
||||
fmt.Sprintf(`error_requests_total{error="ifspeed missing",exporter="%s"}`, exporterStr): "1", // 644
|
||||
`pending_requests`: "0",
|
||||
fmt.Sprintf(`success_requests_total{exporter="%s"}`, exporterStr): "5", // 641+642+643+645+0
|
||||
fmt.Sprintf(`success_requests_total{exporter="%s"}`, exporterStr): "4", // 641+642+643+645
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
|
||||
@@ -8,7 +8,6 @@ package snmp
|
||||
import (
|
||||
"context"
|
||||
"net/netip"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"akvorado/common/reporter"
|
||||
@@ -19,15 +18,9 @@ import (
|
||||
type Provider struct {
|
||||
r *reporter.Reporter
|
||||
config *Configuration
|
||||
|
||||
pendingRequests map[string]struct{}
|
||||
pendingRequestsLock sync.Mutex
|
||||
errLogger reporter.Logger
|
||||
|
||||
put func(provider.Update)
|
||||
|
||||
metrics struct {
|
||||
pendingRequests reporter.GaugeFunc
|
||||
successes *reporter.CounterVec
|
||||
errors *reporter.CounterVec
|
||||
retries *reporter.CounterVec
|
||||
@@ -36,7 +29,7 @@ type Provider struct {
|
||||
}
|
||||
|
||||
// New creates a new SNMP provider from configuration
|
||||
func (configuration Configuration) New(r *reporter.Reporter, put func(provider.Update)) (provider.Provider, error) {
|
||||
func (configuration Configuration) New(r *reporter.Reporter) (provider.Provider, error) {
|
||||
for exporterIP, agentIP := range configuration.Agents {
|
||||
if exporterIP.Is4() || agentIP.Is4() {
|
||||
delete(configuration.Agents, exporterIP)
|
||||
@@ -49,22 +42,9 @@ func (configuration Configuration) New(r *reporter.Reporter, put func(provider.U
|
||||
p := Provider{
|
||||
r: r,
|
||||
config: &configuration,
|
||||
|
||||
pendingRequests: make(map[string]struct{}),
|
||||
errLogger: r.Sample(reporter.BurstSampler(10*time.Second, 3)),
|
||||
|
||||
put: put,
|
||||
}
|
||||
|
||||
p.metrics.pendingRequests = r.GaugeFunc(
|
||||
reporter.GaugeOpts{
|
||||
Name: "poller_pending_requests",
|
||||
Help: "Number of pending requests in pollers.",
|
||||
}, func() float64 {
|
||||
p.pendingRequestsLock.Lock()
|
||||
defer p.pendingRequestsLock.Unlock()
|
||||
return float64(len(p.pendingRequests))
|
||||
})
|
||||
p.metrics.successes = r.CounterVec(
|
||||
reporter.CounterOpts{
|
||||
Name: "poller_success_requests_total",
|
||||
@@ -91,12 +71,11 @@ func (configuration Configuration) New(r *reporter.Reporter, put func(provider.U
|
||||
}
|
||||
|
||||
// Query queries exporter to get information through SNMP.
|
||||
func (p *Provider) Query(ctx context.Context, query *provider.BatchQuery) error {
|
||||
// Avoid querying too much exporters with errors
|
||||
func (p *Provider) Query(ctx context.Context, query provider.Query) (provider.Answer, error) {
|
||||
agentIP, ok := p.config.Agents[query.ExporterIP]
|
||||
if !ok {
|
||||
agentIP = query.ExporterIP
|
||||
}
|
||||
agentPort := p.config.Ports.LookupOrDefault(query.ExporterIP, 161)
|
||||
return p.Poll(ctx, query.ExporterIP, agentIP, agentPort, query.IfIndexes, p.put)
|
||||
return p.Poll(ctx, query.ExporterIP, agentIP, agentPort, query.IfIndex)
|
||||
}
|
||||
|
||||
@@ -6,6 +6,8 @@
|
||||
package static
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"akvorado/common/helpers"
|
||||
"akvorado/common/remotedatasourcefetcher"
|
||||
"akvorado/common/reporter"
|
||||
@@ -20,64 +22,73 @@ import (
|
||||
// Provider represents the static provider.
|
||||
type Provider struct {
|
||||
r *reporter.Reporter
|
||||
|
||||
exporterSourcesFetcher *remotedatasourcefetcher.Component[exporterInfo]
|
||||
exportersMap map[string][]exporterInfo
|
||||
exporters atomic.Pointer[helpers.SubnetMap[ExporterConfiguration]]
|
||||
exportersLock sync.Mutex
|
||||
put func(provider.Update)
|
||||
|
||||
errLogger reporter.Logger
|
||||
|
||||
metrics struct {
|
||||
notReady reporter.Counter
|
||||
}
|
||||
}
|
||||
|
||||
// New creates a new static provider from configuration
|
||||
func (configuration Configuration) New(r *reporter.Reporter, put func(provider.Update)) (provider.Provider, error) {
|
||||
func (configuration Configuration) New(r *reporter.Reporter) (provider.Provider, error) {
|
||||
p := &Provider{
|
||||
r: r,
|
||||
exportersMap: map[string][]exporterInfo{},
|
||||
put: put,
|
||||
errLogger: r.Sample(reporter.BurstSampler(time.Minute, 3)),
|
||||
}
|
||||
p.exporters.Store(configuration.Exporters)
|
||||
p.initStaticExporters()
|
||||
|
||||
var err error
|
||||
p.exporterSourcesFetcher, err = remotedatasourcefetcher.New[exporterInfo](r, p.UpdateRemoteDataSource, "metadata", configuration.ExporterSources)
|
||||
p.exporterSourcesFetcher, err = remotedatasourcefetcher.New[exporterInfo](r,
|
||||
p.UpdateRemoteDataSource, "metadata", configuration.ExporterSources)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unable to initialize remote data source fetcher component: %w", err)
|
||||
}
|
||||
if err := p.exporterSourcesFetcher.Start(); err != nil {
|
||||
return nil, fmt.Errorf("unable to start network sources fetcher component: %w", err)
|
||||
}
|
||||
|
||||
p.metrics.notReady = r.Counter(
|
||||
reporter.CounterOpts{
|
||||
Name: "not_ready_total",
|
||||
Help: "Number of queries failing because the remote data sources are not ready",
|
||||
})
|
||||
|
||||
return p, nil
|
||||
}
|
||||
|
||||
// Query queries static configuration.
|
||||
func (p *Provider) Query(_ context.Context, query *provider.BatchQuery) error {
|
||||
func (p *Provider) Query(ctx context.Context, query provider.Query) (provider.Answer, error) {
|
||||
// We wait for all data sources to be ready
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
p.metrics.notReady.Inc()
|
||||
p.errLogger.Warn().Msg("remote datasources are not ready")
|
||||
return provider.Answer{}, ctx.Err()
|
||||
case <-p.exporterSourcesFetcher.DataSourcesReady:
|
||||
}
|
||||
exporter, ok := p.exporters.Load().Lookup(query.ExporterIP)
|
||||
if !ok {
|
||||
return provider.ErrSkipProvider
|
||||
return provider.Answer{}, provider.ErrSkipProvider
|
||||
}
|
||||
var skippedIfIndexes uint
|
||||
for _, ifIndex := range query.IfIndexes {
|
||||
iface, ok := exporter.IfIndexes[ifIndex]
|
||||
|
||||
iface, ok := exporter.IfIndexes[query.IfIndex]
|
||||
if !ok {
|
||||
if exporter.SkipMissingInterfaces {
|
||||
query.IfIndexes[skippedIfIndexes] = ifIndex
|
||||
skippedIfIndexes++
|
||||
continue
|
||||
return provider.Answer{}, provider.ErrSkipProvider
|
||||
}
|
||||
iface = exporter.Default
|
||||
}
|
||||
p.put(provider.Update{
|
||||
Query: provider.Query{
|
||||
ExporterIP: query.ExporterIP,
|
||||
IfIndex: ifIndex,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
return provider.Answer{
|
||||
Found: true,
|
||||
Exporter: exporter.Exporter,
|
||||
Interface: iface,
|
||||
},
|
||||
})
|
||||
}
|
||||
if skippedIfIndexes > 0 {
|
||||
query.IfIndexes = query.IfIndexes[:skippedIfIndexes]
|
||||
return provider.ErrSkipProvider
|
||||
}
|
||||
return nil
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -95,48 +95,69 @@ func TestStaticProvider(t *testing.T) {
|
||||
}),
|
||||
}
|
||||
|
||||
var got []provider.Update
|
||||
var got []provider.Answer
|
||||
r := reporter.NewMock(t)
|
||||
p, _ := config.New(r, func(update provider.Update) {
|
||||
got = append(got, update)
|
||||
})
|
||||
p, _ := config.New(r)
|
||||
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndexes: []uint{9, 10, 11},
|
||||
})
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndexes: []uint{9, 10, 11},
|
||||
})
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:3::10"),
|
||||
IfIndexes: []uint{10},
|
||||
})
|
||||
query := provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:4::10"),
|
||||
IfIndexes: []uint{9, 10, 11},
|
||||
}
|
||||
err := p.Query(context.Background(), &query)
|
||||
|
||||
expected := []provider.Update{
|
||||
{
|
||||
Query: provider.Query{
|
||||
answer, _ := p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 9,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
})
|
||||
got = append(got, answer)
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 10,
|
||||
})
|
||||
got = append(got, answer)
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 11,
|
||||
})
|
||||
got = append(got, answer)
|
||||
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 9,
|
||||
})
|
||||
got = append(got, answer)
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 10,
|
||||
})
|
||||
got = append(got, answer)
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 11,
|
||||
})
|
||||
got = append(got, answer)
|
||||
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:3::10"),
|
||||
IfIndex: 10,
|
||||
})
|
||||
got = append(got, answer)
|
||||
|
||||
var err error
|
||||
answer, _ = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:4::10"),
|
||||
IfIndex: 10,
|
||||
})
|
||||
got = append(got, answer)
|
||||
answer, err = p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:4::10"),
|
||||
IfIndex: 11,
|
||||
})
|
||||
got = append(got, answer)
|
||||
|
||||
expected := []provider.Answer{
|
||||
{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "nodefault",
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 10,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "nodefault",
|
||||
},
|
||||
@@ -146,13 +167,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 11,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "nodefault",
|
||||
},
|
||||
@@ -162,13 +178,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 9,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "default",
|
||||
},
|
||||
@@ -177,14 +188,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Description: "Default interface",
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 10,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
}, {
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "default",
|
||||
},
|
||||
@@ -193,14 +198,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Description: "10th interface",
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 11,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
}, {
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "default",
|
||||
},
|
||||
@@ -209,14 +208,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Description: "Default interface",
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:3::10"),
|
||||
IfIndex: 10,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
}, {
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "default with metadata",
|
||||
Region: "eu",
|
||||
@@ -233,14 +226,8 @@ func TestStaticProvider(t *testing.T) {
|
||||
Connectivity: "transit",
|
||||
Boundary: schema.InterfaceBoundaryExternal,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Query: provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:4::10"),
|
||||
IfIndex: 10,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
}, {
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "nodefault skip",
|
||||
},
|
||||
@@ -253,15 +240,12 @@ func TestStaticProvider(t *testing.T) {
|
||||
Boundary: schema.InterfaceBoundaryExternal,
|
||||
},
|
||||
},
|
||||
},
|
||||
{}, // Skip
|
||||
}
|
||||
|
||||
if diff := helpers.Diff(got, expected); diff != "" {
|
||||
t.Fatalf("static provider (-got, +want):\n%s", diff)
|
||||
}
|
||||
if diff := helpers.Diff(query.IfIndexes, []uint{9, 11}); diff != "" {
|
||||
t.Fatalf("static provider (-got, +want):\n%s", diff)
|
||||
}
|
||||
if diff := helpers.Diff(err, provider.ErrSkipProvider); diff != "" {
|
||||
t.Fatalf("static provider (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
@@ -5,6 +5,7 @@ package static
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
@@ -38,7 +39,6 @@ func TestInitStaticExporters(t *testing.T) {
|
||||
p := &Provider{
|
||||
r: r,
|
||||
exportersMap: map[string][]exporterInfo{},
|
||||
put: func(_ provider.Update) {},
|
||||
}
|
||||
p.exporters.Store(conf.Exporters)
|
||||
|
||||
@@ -70,6 +70,7 @@ func TestInitStaticExporters(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestRemoteExporterSources(t *testing.T) {
|
||||
|
||||
// Mux to answer requests
|
||||
ready := make(chan bool)
|
||||
mux := http.NewServeMux()
|
||||
@@ -186,42 +187,24 @@ func TestRemoteExporterSources(t *testing.T) {
|
||||
},
|
||||
},
|
||||
}
|
||||
var got []provider.Update
|
||||
var expected []provider.Update
|
||||
p, _ := config.New(r, func(update provider.Update) {
|
||||
got = append(got, update)
|
||||
})
|
||||
p, _ := config.New(r)
|
||||
|
||||
// Query when json is not ready yet, only static configured data available
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndexes: []uint{9},
|
||||
})
|
||||
|
||||
// Unknown Exporter at this moment
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndexes: []uint{1},
|
||||
})
|
||||
|
||||
expected = append(expected, provider.Update{
|
||||
Query: provider.Query{
|
||||
// Query when json is not ready yet, we should get a timeout
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond)
|
||||
defer cancel()
|
||||
answer1, err := p.Query(ctx, provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:1::10"),
|
||||
IfIndex: 9,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
Exporter: provider.Exporter{
|
||||
Name: "nodefault",
|
||||
},
|
||||
},
|
||||
})
|
||||
|
||||
if diff := helpers.Diff(got, expected); diff != "" {
|
||||
t.Fatalf("static provider - before remote source load (-got, +want):\n%s", diff)
|
||||
if err == nil {
|
||||
t.Fatalf("Query() should have been in error:\n%+v", answer1)
|
||||
}
|
||||
if !errors.Is(err, context.DeadlineExceeded) {
|
||||
t.Fatalf("Query() error:\n%+v", err)
|
||||
}
|
||||
|
||||
close(ready)
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
gotMetrics := r.GetMetrics("akvorado_common_remotedatasourcefetcher_data_")
|
||||
expectedMetrics := map[string]string{
|
||||
@@ -232,17 +215,13 @@ func TestRemoteExporterSources(t *testing.T) {
|
||||
}
|
||||
|
||||
// We now should be able to resolve our new exporter from remote source
|
||||
p.Query(context.Background(), &provider.BatchQuery{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndexes: []uint{1},
|
||||
})
|
||||
|
||||
expected = append(expected, provider.Update{
|
||||
Query: provider.Query{
|
||||
got, _ := p.Query(context.Background(), provider.Query{
|
||||
ExporterIP: netip.MustParseAddr("2001:db8:2::10"),
|
||||
IfIndex: 1,
|
||||
},
|
||||
Answer: provider.Answer{
|
||||
})
|
||||
|
||||
expected := provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "exporter1",
|
||||
},
|
||||
@@ -251,10 +230,9 @@ func TestRemoteExporterSources(t *testing.T) {
|
||||
Description: "foo:desc1",
|
||||
Speed: 1000,
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
if diff := helpers.Diff(got, expected); diff != "" {
|
||||
t.Fatalf("static provider - after remote source load(-got, +want):\n%s", diff)
|
||||
t.Fatalf("static provider (-got, +want):\n%s", diff)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,15 +7,16 @@
|
||||
package metadata
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/netip"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/benbjohnson/clock"
|
||||
"github.com/eapache/go-resiliency/breaker"
|
||||
"golang.org/x/sync/singleflight"
|
||||
"gopkg.in/tomb.v2"
|
||||
|
||||
"akvorado/common/daemon"
|
||||
@@ -31,22 +32,20 @@ type Component struct {
|
||||
config Configuration
|
||||
|
||||
sc *metadataCache
|
||||
sf singleflight.Group
|
||||
|
||||
healthyWorkers chan reporter.ChannelHealthcheckFunc
|
||||
providerChannel chan provider.BatchQuery
|
||||
dispatcherChannel chan provider.Query
|
||||
dispatcherBChannel chan (<-chan bool) // block channel for testing
|
||||
providerBreakersLock sync.Mutex
|
||||
providerBreakerLoggers map[netip.Addr]reporter.Logger
|
||||
providerBreakers map[netip.Addr]*breaker.Breaker
|
||||
providers []provider.Provider
|
||||
initialDeadline time.Time
|
||||
|
||||
metrics struct {
|
||||
cacheRefreshRuns reporter.Counter
|
||||
cacheRefresh reporter.Counter
|
||||
providerBusyCount *reporter.CounterVec
|
||||
providerBreakerOpenCount *reporter.CounterVec
|
||||
providerBatchedCount reporter.Counter
|
||||
providerRequests reporter.Counter
|
||||
providerErrors reporter.Counter
|
||||
}
|
||||
}
|
||||
|
||||
@@ -56,6 +55,11 @@ type Dependencies struct {
|
||||
Clock clock.Clock
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrQueryTimeout is the error returned when a query timeout.
|
||||
ErrQueryTimeout = errors.New("provider query timeout")
|
||||
)
|
||||
|
||||
// New creates a new metadata component.
|
||||
func New(r *reporter.Reporter, configuration Configuration, dependencies Dependencies) (*Component, error) {
|
||||
if configuration.CacheRefresh > 0 && configuration.CacheRefresh < configuration.CacheDuration {
|
||||
@@ -75,9 +79,6 @@ func New(r *reporter.Reporter, configuration Configuration, dependencies Depende
|
||||
config: configuration,
|
||||
sc: sc,
|
||||
|
||||
providerChannel: make(chan provider.BatchQuery),
|
||||
dispatcherChannel: make(chan provider.Query, 100*configuration.Workers),
|
||||
dispatcherBChannel: make(chan (<-chan bool)),
|
||||
providerBreakers: make(map[netip.Addr]*breaker.Breaker),
|
||||
providerBreakerLoggers: make(map[netip.Addr]reporter.Logger),
|
||||
providers: make([]provider.Provider, 0, 1),
|
||||
@@ -86,9 +87,7 @@ func New(r *reporter.Reporter, configuration Configuration, dependencies Depende
|
||||
|
||||
// Initialize providers
|
||||
for _, p := range c.config.Providers {
|
||||
selectedProvider, err := p.Config.New(r, func(update provider.Update) {
|
||||
c.sc.Put(c.d.Clock.Now(), update.Query, update.Answer)
|
||||
})
|
||||
selectedProvider, err := p.Config.New(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -102,33 +101,32 @@ func New(r *reporter.Reporter, configuration Configuration, dependencies Depende
|
||||
})
|
||||
c.metrics.cacheRefresh = r.Counter(
|
||||
reporter.CounterOpts{
|
||||
Name: "cache_refreshs_total",
|
||||
Name: "cache_refreshes_total",
|
||||
Help: "Number of entries refreshed in cache.",
|
||||
})
|
||||
c.metrics.providerBusyCount = r.CounterVec(
|
||||
reporter.CounterOpts{
|
||||
Name: "provider_dropped_requests_total",
|
||||
Help: "Providers where too busy and dropped requests.",
|
||||
},
|
||||
[]string{"exporter"})
|
||||
c.metrics.providerBreakerOpenCount = r.CounterVec(
|
||||
reporter.CounterOpts{
|
||||
Name: "provider_breaker_opens_total",
|
||||
Help: "Provider breaker was opened due to too many errors.",
|
||||
},
|
||||
[]string{"exporter"})
|
||||
c.metrics.providerBatchedCount = r.Counter(
|
||||
c.metrics.providerRequests = r.Counter(
|
||||
reporter.CounterOpts{
|
||||
Name: "provider_batched_requests_total",
|
||||
Help: "Several requests were batched into one.",
|
||||
},
|
||||
)
|
||||
Name: "provider_requests_total",
|
||||
Help: "Number of provider requests.",
|
||||
})
|
||||
c.metrics.providerErrors = r.Counter(
|
||||
reporter.CounterOpts{
|
||||
Name: "provider_errors_total",
|
||||
Help: "Number of provider errors.",
|
||||
})
|
||||
return &c, nil
|
||||
}
|
||||
|
||||
// Start starts the metadata component.
|
||||
func (c *Component) Start() error {
|
||||
c.r.Info().Msg("starting metadata component")
|
||||
c.initialDeadline = time.Now().Add(c.config.InitialDelay)
|
||||
|
||||
// Load cache
|
||||
if c.config.CachePersistFile != "" {
|
||||
@@ -160,61 +158,12 @@ func (c *Component) Start() error {
|
||||
}
|
||||
})
|
||||
|
||||
// Goroutine to fetch incoming requests and dispatch them to workers
|
||||
healthyDispatcher := make(chan reporter.ChannelHealthcheckFunc)
|
||||
c.r.RegisterHealthcheck("metadata/dispatcher", reporter.ChannelHealthcheck(c.t.Context(nil), healthyDispatcher))
|
||||
c.t.Go(func() error {
|
||||
dying := c.t.Dying()
|
||||
for {
|
||||
select {
|
||||
case <-dying:
|
||||
c.r.Debug().Msg("stopping metadata dispatcher")
|
||||
return nil
|
||||
case cb, ok := <-healthyDispatcher:
|
||||
if ok {
|
||||
cb(reporter.HealthcheckOK, "ok")
|
||||
}
|
||||
case ch := <-c.dispatcherBChannel:
|
||||
// This is to test batching
|
||||
<-ch
|
||||
case request := <-c.dispatcherChannel:
|
||||
c.dispatchIncomingRequest(request)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
// Goroutines to poll exporters
|
||||
c.healthyWorkers = make(chan reporter.ChannelHealthcheckFunc)
|
||||
c.r.RegisterHealthcheck("metadata/worker", reporter.ChannelHealthcheck(c.t.Context(nil), c.healthyWorkers))
|
||||
for i := range c.config.Workers {
|
||||
workerIDStr := strconv.Itoa(i)
|
||||
c.t.Go(func() error {
|
||||
c.r.Debug().Str("worker", workerIDStr).Msg("starting metadata provider")
|
||||
dying := c.t.Dying()
|
||||
for {
|
||||
select {
|
||||
case <-dying:
|
||||
c.r.Debug().Str("worker", workerIDStr).Msg("stopping metadata provider")
|
||||
return nil
|
||||
case cb, ok := <-c.healthyWorkers:
|
||||
if ok {
|
||||
cb(reporter.HealthcheckOK, fmt.Sprintf("worker %s ok", workerIDStr))
|
||||
}
|
||||
case request := <-c.providerChannel:
|
||||
c.providerIncomingRequest(request)
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop stops the metadata component
|
||||
func (c *Component) Stop() error {
|
||||
defer func() {
|
||||
close(c.dispatcherChannel)
|
||||
close(c.providerChannel)
|
||||
close(c.healthyWorkers)
|
||||
if c.config.CachePersistFile != "" {
|
||||
if err := c.sc.Save(c.config.CachePersistFile); err != nil {
|
||||
c.r.Err(err).Msg("cannot save cache")
|
||||
@@ -227,102 +176,102 @@ func (c *Component) Stop() error {
|
||||
return c.t.Wait()
|
||||
}
|
||||
|
||||
// Lookup for interface information for the provided exporter and ifIndex.
|
||||
// If the information is not in the cache, it will be polled, but
|
||||
// won't be returned immediately.
|
||||
func (c *Component) Lookup(t time.Time, exporterIP netip.Addr, ifIndex uint) (provider.Answer, bool) {
|
||||
// Lookup for interface information for the provided exporter and ifIndex. If
|
||||
// the information is not in the cache, it will be polled from the provider. The
|
||||
// returned result has a field Found to tell if the lookup is successful or not.
|
||||
func (c *Component) Lookup(t time.Time, exporterIP netip.Addr, ifIndex uint) provider.Answer {
|
||||
query := provider.Query{ExporterIP: exporterIP, IfIndex: ifIndex}
|
||||
answer, ok := c.sc.Lookup(t, query)
|
||||
if !ok {
|
||||
select {
|
||||
case c.dispatcherChannel <- query:
|
||||
default:
|
||||
c.metrics.providerBusyCount.WithLabelValues(exporterIP.Unmap().String()).Inc()
|
||||
|
||||
// Check cache first
|
||||
if answer, ok := c.sc.Lookup(t, query); ok {
|
||||
return answer
|
||||
}
|
||||
|
||||
// Use singleflight to prevent duplicate queries
|
||||
key := fmt.Sprintf("%s-%d", exporterIP, ifIndex)
|
||||
result, err, _ := c.sf.Do(key, func() (any, error) {
|
||||
return c.queryProviders(query)
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return provider.Answer{}
|
||||
}
|
||||
return answer, ok
|
||||
|
||||
return result.(provider.Answer)
|
||||
}
|
||||
|
||||
// dispatchIncomingRequest dispatches an incoming request to workers. It may
|
||||
// handle more than the provided request if it can.
|
||||
func (c *Component) dispatchIncomingRequest(request provider.Query) {
|
||||
requestsMap := map[netip.Addr][]uint{
|
||||
request.ExporterIP: {request.IfIndex},
|
||||
}
|
||||
dying := c.t.Dying()
|
||||
for c.config.MaxBatchRequests > 0 {
|
||||
select {
|
||||
case request := <-c.dispatcherChannel:
|
||||
indexes, ok := requestsMap[request.ExporterIP]
|
||||
if !ok {
|
||||
indexes = []uint{request.IfIndex}
|
||||
} else {
|
||||
indexes = append(indexes, request.IfIndex)
|
||||
}
|
||||
requestsMap[request.ExporterIP] = indexes
|
||||
// We don't want to exceed the configured limit but also there is no
|
||||
// point of batching requests of too many exporters.
|
||||
if len(indexes) < c.config.MaxBatchRequests && len(requestsMap) < 4 {
|
||||
continue
|
||||
}
|
||||
case <-dying:
|
||||
return
|
||||
default:
|
||||
// No more requests in queue
|
||||
}
|
||||
break
|
||||
}
|
||||
for exporterIP, ifIndexes := range requestsMap {
|
||||
if len(ifIndexes) > 1 {
|
||||
c.metrics.providerBatchedCount.Add(float64(len(ifIndexes)))
|
||||
}
|
||||
select {
|
||||
case <-dying:
|
||||
return
|
||||
case c.providerChannel <- provider.BatchQuery{ExporterIP: exporterIP, IfIndexes: ifIndexes}:
|
||||
}
|
||||
}
|
||||
}
|
||||
// queryProviders queries all providers. It returns the answer for the specific
|
||||
// query and cache it.
|
||||
func (c *Component) queryProviders(query provider.Query) (provider.Answer, error) {
|
||||
c.metrics.providerRequests.Inc()
|
||||
|
||||
// providerIncomingRequest handles an incoming request to the provider. It
|
||||
// uses a breaker to avoid pushing working on non-responsive exporters.
|
||||
func (c *Component) providerIncomingRequest(request provider.BatchQuery) {
|
||||
// Avoid querying too much exporters with errors
|
||||
// Check if provider breaker is open
|
||||
c.providerBreakersLock.Lock()
|
||||
providerBreaker, ok := c.providerBreakers[request.ExporterIP]
|
||||
providerBreaker, ok := c.providerBreakers[query.ExporterIP]
|
||||
if !ok {
|
||||
providerBreaker = breaker.New(20, 1, time.Minute)
|
||||
c.providerBreakers[request.ExporterIP] = providerBreaker
|
||||
c.providerBreakers[query.ExporterIP] = providerBreaker
|
||||
}
|
||||
c.providerBreakersLock.Unlock()
|
||||
|
||||
if err := providerBreaker.Run(func() error {
|
||||
ctx := c.t.Context(nil)
|
||||
var result provider.Answer
|
||||
err := providerBreaker.Run(func() error {
|
||||
deadline := time.Now().Add(c.config.QueryTimeout)
|
||||
if deadline.Before(c.initialDeadline) {
|
||||
deadline = c.initialDeadline
|
||||
}
|
||||
ctx, cancel := context.WithDeadlineCause(
|
||||
c.t.Context(nil),
|
||||
deadline,
|
||||
ErrQueryTimeout)
|
||||
defer cancel()
|
||||
|
||||
now := c.d.Clock.Now()
|
||||
for _, p := range c.providers {
|
||||
// Query providers in the order they are defined and stop on the
|
||||
// first provider accepting to handle the query.
|
||||
if err := p.Query(ctx, &request); err != nil && err != provider.ErrSkipProvider {
|
||||
return err
|
||||
} else if err == provider.ErrSkipProvider {
|
||||
answer, err := p.Query(ctx, query)
|
||||
if err == provider.ErrSkipProvider {
|
||||
// Next provider
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
c.sc.Put(now, query, answer)
|
||||
result = answer
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
}); err == breaker.ErrBreakerOpen {
|
||||
c.metrics.providerBreakerOpenCount.WithLabelValues(request.ExporterIP.Unmap().String()).Inc()
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
c.metrics.providerErrors.Inc()
|
||||
if err == breaker.ErrBreakerOpen {
|
||||
c.metrics.providerBreakerOpenCount.WithLabelValues(query.ExporterIP.Unmap().String()).Inc()
|
||||
c.providerBreakersLock.Lock()
|
||||
l, ok := c.providerBreakerLoggers[request.ExporterIP]
|
||||
l, ok := c.providerBreakerLoggers[query.ExporterIP]
|
||||
if !ok {
|
||||
l = c.r.Sample(reporter.BurstSampler(time.Minute, 1)).
|
||||
With().
|
||||
Str("exporter", request.ExporterIP.Unmap().String()).
|
||||
Str("exporter", query.ExporterIP.Unmap().String()).
|
||||
Logger()
|
||||
c.providerBreakerLoggers[request.ExporterIP] = l
|
||||
c.providerBreakerLoggers[query.ExporterIP] = l
|
||||
}
|
||||
l.Warn().Msg("provider breaker open")
|
||||
c.providerBreakersLock.Unlock()
|
||||
}
|
||||
return provider.Answer{}, err
|
||||
}
|
||||
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// refreshCacheEntry refreshes a single cache entry.
|
||||
func (c *Component) refreshCacheEntry(exporterIP netip.Addr, ifIndex uint) {
|
||||
query := provider.Query{
|
||||
ExporterIP: exporterIP,
|
||||
IfIndex: ifIndex,
|
||||
}
|
||||
c.queryProviders(query)
|
||||
}
|
||||
|
||||
// expireCache handles cache expiration and refresh.
|
||||
@@ -335,15 +284,8 @@ func (c *Component) expireCache() {
|
||||
toRefresh := c.sc.NeedUpdates(c.d.Clock.Now().Add(-c.config.CacheRefresh))
|
||||
for exporter, ifaces := range toRefresh {
|
||||
for _, ifIndex := range ifaces {
|
||||
select {
|
||||
case c.dispatcherChannel <- provider.Query{
|
||||
ExporterIP: exporter,
|
||||
IfIndex: ifIndex,
|
||||
}:
|
||||
go c.refreshCacheEntry(exporter, ifIndex)
|
||||
count++
|
||||
default:
|
||||
c.metrics.providerBusyCount.WithLabelValues(exporter.Unmap().String()).Inc()
|
||||
}
|
||||
}
|
||||
}
|
||||
c.r.Debug().Int("count", count).Msg("refreshed metadata cache")
|
||||
|
||||
@@ -16,6 +16,7 @@ import (
|
||||
"akvorado/common/daemon"
|
||||
"akvorado/common/helpers"
|
||||
"akvorado/common/reporter"
|
||||
"akvorado/common/schema"
|
||||
"akvorado/outlet/metadata/provider"
|
||||
"akvorado/outlet/metadata/provider/static"
|
||||
)
|
||||
@@ -23,7 +24,7 @@ import (
|
||||
func expectMockLookup(t *testing.T, c *Component, exporter string, ifIndex uint, expected provider.Answer) {
|
||||
t.Helper()
|
||||
ip := netip.AddrFrom16(netip.MustParseAddr(exporter).As16())
|
||||
got, _ := c.Lookup(time.Now(), ip, ifIndex)
|
||||
got := c.Lookup(time.Now(), ip, ifIndex)
|
||||
if diff := helpers.Diff(got, expected); diff != "" {
|
||||
t.Fatalf("Lookup() (-got, +want):\n%s", diff)
|
||||
}
|
||||
@@ -32,24 +33,53 @@ func expectMockLookup(t *testing.T, c *Component, exporter string, ifIndex uint,
|
||||
func TestLookup(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
c := NewMock(t, r, DefaultConfiguration(), Dependencies{Daemon: daemon.NewMock(t)})
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{})
|
||||
expectMockLookup(t, c, "127.0.0.1", 999, provider.Answer{})
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
},
|
||||
|
||||
Interface: provider.Interface{Name: "Gi0/0/765",
|
||||
Description: "Interface 765",
|
||||
Speed: 1000,
|
||||
},
|
||||
})
|
||||
expectMockLookup(t, c, "127.0.0.1", 999, provider.Answer{
|
||||
expectMockLookup(t, c, "127.0.0.1", 1010, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
Group: "metadata group",
|
||||
Region: "metadata region",
|
||||
Role: "metadata role",
|
||||
Site: "metadata site",
|
||||
Tenant: "metadata tenant",
|
||||
},
|
||||
Interface: provider.Interface{Name: "Gi0/0/1010",
|
||||
Description: "Interface 1010",
|
||||
Speed: 1000,
|
||||
},
|
||||
})
|
||||
expectMockLookup(t, c, "127.0.0.1", 2010, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
Group: "metadata group",
|
||||
Region: "metadata region",
|
||||
Role: "metadata role",
|
||||
Site: "metadata site",
|
||||
Tenant: "metadata tenant",
|
||||
},
|
||||
Interface: provider.Interface{Name: "Gi0/0/2010",
|
||||
Description: "Interface 2010",
|
||||
Speed: 1000,
|
||||
Boundary: schema.InterfaceBoundaryExternal,
|
||||
Connectivity: "metadata connectivity",
|
||||
Provider: "metadata provider",
|
||||
},
|
||||
})
|
||||
// With a simple lookup, this is not possible to distinguish between a
|
||||
// transient error or a fatal error. Only the caching subsystem knows.
|
||||
expectMockLookup(t, c, "127.0.0.1", 999, provider.Answer{})
|
||||
expectMockLookup(t, c, "127.0.0.1", 998, provider.Answer{})
|
||||
}
|
||||
|
||||
func TestComponentSaveLoad(t *testing.T) {
|
||||
@@ -59,10 +89,8 @@ func TestComponentSaveLoad(t *testing.T) {
|
||||
t.Run("save", func(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
c := NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t)})
|
||||
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{})
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
},
|
||||
@@ -76,8 +104,11 @@ func TestComponentSaveLoad(t *testing.T) {
|
||||
|
||||
t.Run("load", func(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
// Switch to the empty provider to check if we get answers from the cache.
|
||||
configuration.Providers = []ProviderConfiguration{{Config: emptyProviderConfiguration{}}}
|
||||
c := NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t)})
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
},
|
||||
@@ -96,14 +127,11 @@ func TestAutoRefresh(t *testing.T) {
|
||||
mockClock := clock.NewMock()
|
||||
c := NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t), Clock: mockClock})
|
||||
|
||||
// Fetch a value
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{})
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
},
|
||||
|
||||
Interface: provider.Interface{
|
||||
Name: "Gi0/0/765",
|
||||
Description: "Interface 765",
|
||||
@@ -119,10 +147,9 @@ func TestAutoRefresh(t *testing.T) {
|
||||
|
||||
// Go forward, we expect the entry to have been refreshed and be still present
|
||||
mockClock.Add(11 * time.Minute)
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
mockClock.Add(2 * time.Minute)
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
expectMockLookup(t, c, "127.0.0.1", 765, provider.Answer{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "127_0_0_1",
|
||||
},
|
||||
@@ -137,13 +164,13 @@ func TestAutoRefresh(t *testing.T) {
|
||||
for _, runs := range []string{"29", "30", "31"} { // 63/2
|
||||
expectedMetrics := map[string]string{
|
||||
`expired_entries_total`: "0",
|
||||
`hits_total`: "4",
|
||||
`misses_total`: "1",
|
||||
`misses_total`: "1", // First lookup misses
|
||||
`hits_total`: "3", // Subsequent ones hits
|
||||
`size_entries`: "1",
|
||||
`refresh_runs_total`: runs,
|
||||
`refreshs_total`: "1",
|
||||
`refreshes_total`: "1", // One refresh (after 1 hour)
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" && runs == "31" {
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" && runs == "19" {
|
||||
t.Fatalf("Metrics (-got, +want):\n%s", diff)
|
||||
} else if diff == "" {
|
||||
break
|
||||
@@ -184,22 +211,21 @@ func TestConfigCheck(t *testing.T) {
|
||||
})
|
||||
}
|
||||
|
||||
func TestStartStopWithMultipleWorkers(t *testing.T) {
|
||||
func TestStartStopSimple(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
configuration := DefaultConfiguration()
|
||||
configuration.Workers = 5
|
||||
NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t)})
|
||||
}
|
||||
|
||||
type errorProvider struct{}
|
||||
|
||||
func (ep errorProvider) Query(_ context.Context, _ *provider.BatchQuery) error {
|
||||
return errors.New("noooo")
|
||||
func (ep errorProvider) Query(_ context.Context, _ provider.Query) (provider.Answer, error) {
|
||||
return provider.Answer{}, errors.New("noooo")
|
||||
}
|
||||
|
||||
type errorProviderConfiguration struct{}
|
||||
|
||||
func (epc errorProviderConfiguration) New(_ *reporter.Reporter, _ func(provider.Update)) (provider.Provider, error) {
|
||||
func (epc errorProviderConfiguration) New(_ *reporter.Reporter) (provider.Provider, error) {
|
||||
return errorProvider{}, nil
|
||||
}
|
||||
|
||||
@@ -216,7 +242,6 @@ func TestProviderBreaker(t *testing.T) {
|
||||
t.Run(tc.Name, func(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
configuration := DefaultConfiguration()
|
||||
configuration.MaxBatchRequests = 0
|
||||
configuration.Providers = []ProviderConfiguration{{Config: tc.ProviderConfiguration}}
|
||||
c := NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t)})
|
||||
c.metrics.providerBreakerOpenCount.WithLabelValues("127.0.0.1").Add(0)
|
||||
@@ -240,72 +265,6 @@ func TestProviderBreaker(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
type batchProvider struct {
|
||||
config *batchProviderConfiguration
|
||||
}
|
||||
|
||||
func (bp *batchProvider) Query(_ context.Context, query *provider.BatchQuery) error {
|
||||
bp.config.received = append(bp.config.received, *query)
|
||||
return nil
|
||||
}
|
||||
|
||||
type batchProviderConfiguration struct {
|
||||
received []provider.BatchQuery
|
||||
}
|
||||
|
||||
func (bpc *batchProviderConfiguration) New(_ *reporter.Reporter, _ func(provider.Update)) (provider.Provider, error) {
|
||||
return &batchProvider{config: bpc}, nil
|
||||
}
|
||||
|
||||
func TestBatching(t *testing.T) {
|
||||
bcp := batchProviderConfiguration{
|
||||
received: []provider.BatchQuery{},
|
||||
}
|
||||
r := reporter.NewMock(t)
|
||||
t.Run("run", func(t *testing.T) {
|
||||
configuration := DefaultConfiguration()
|
||||
configuration.Providers = []ProviderConfiguration{{Config: &bcp}}
|
||||
c := NewMock(t, r, configuration, Dependencies{Daemon: daemon.NewMock(t)})
|
||||
|
||||
// Block dispatcher
|
||||
blocker := make(chan bool)
|
||||
c.dispatcherBChannel <- blocker
|
||||
|
||||
defer func() {
|
||||
// Unblock
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
close(blocker)
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
}()
|
||||
|
||||
// Queue requests
|
||||
c.Lookup(c.d.Clock.Now(), netip.MustParseAddr("::ffff:127.0.0.1"), 766)
|
||||
c.Lookup(c.d.Clock.Now(), netip.MustParseAddr("::ffff:127.0.0.1"), 767)
|
||||
c.Lookup(c.d.Clock.Now(), netip.MustParseAddr("::ffff:127.0.0.1"), 768)
|
||||
c.Lookup(c.d.Clock.Now(), netip.MustParseAddr("::ffff:127.0.0.1"), 769)
|
||||
})
|
||||
|
||||
t.Run("check", func(t *testing.T) {
|
||||
gotMetrics := r.GetMetrics("akvorado_outlet_metadata_provider_", "batched_requests_total")
|
||||
expectedMetrics := map[string]string{
|
||||
`batched_requests_total`: "4",
|
||||
}
|
||||
if diff := helpers.Diff(gotMetrics, expectedMetrics); diff != "" {
|
||||
t.Errorf("Metrics (-got, +want):\n%s", diff)
|
||||
}
|
||||
|
||||
expectedAccepted := []provider.BatchQuery{
|
||||
{
|
||||
ExporterIP: netip.MustParseAddr("::ffff:127.0.0.1"),
|
||||
IfIndexes: []uint{766, 767, 768, 769},
|
||||
},
|
||||
}
|
||||
if diff := helpers.Diff(bcp.received, expectedAccepted); diff != "" {
|
||||
t.Errorf("Accepted requests (-got, +want):\n%s", diff)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func TestMultipleProviders(t *testing.T) {
|
||||
r := reporter.NewMock(t)
|
||||
staticConfiguration1 := static.Configuration{
|
||||
@@ -359,11 +318,12 @@ func TestMultipleProviders(t *testing.T) {
|
||||
c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:1::1"), 10)
|
||||
c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:2::2"), 12)
|
||||
time.Sleep(30 * time.Millisecond)
|
||||
got1, _ := c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:1::1"), 10)
|
||||
got2, _ := c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:2::2"), 12)
|
||||
got1 := c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:1::1"), 10)
|
||||
got2 := c.Lookup(time.Now(), netip.MustParseAddr("2001:db8:2::2"), 12)
|
||||
got := []provider.Answer{got1, got2}
|
||||
expected := []provider.Answer{
|
||||
{
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "static1",
|
||||
},
|
||||
@@ -373,6 +333,7 @@ func TestMultipleProviders(t *testing.T) {
|
||||
Speed: 1000,
|
||||
},
|
||||
}, {
|
||||
Found: true,
|
||||
Exporter: provider.Exporter{
|
||||
Name: "static2",
|
||||
},
|
||||
|
||||
@@ -7,6 +7,7 @@ package metadata
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
@@ -18,23 +19,31 @@ import (
|
||||
)
|
||||
|
||||
// mockProvider represents a mock provider.
|
||||
type mockProvider struct {
|
||||
put func(provider.Update)
|
||||
}
|
||||
type mockProvider struct{}
|
||||
|
||||
// Query query the mock provider for a value.
|
||||
func (mp mockProvider) Query(_ context.Context, query *provider.BatchQuery) error {
|
||||
for _, ifIndex := range query.IfIndexes {
|
||||
// - ifIndex = 999 → not found
|
||||
// - ifIndex = 998 → transient error
|
||||
// - ifIndex = 1010 → with metadata for exporter
|
||||
// - ifIndex = 2010 → with metadata for exporter and interface
|
||||
func (mp mockProvider) Query(_ context.Context, query provider.Query) (provider.Answer, error) {
|
||||
ifIndex := query.IfIndex
|
||||
if ifIndex == 999 {
|
||||
return provider.Answer{}, nil
|
||||
}
|
||||
if ifIndex == 998 {
|
||||
return provider.Answer{}, errors.New("noooo")
|
||||
}
|
||||
|
||||
answer := provider.Answer{
|
||||
Exporter: provider.Exporter{
|
||||
Name: strings.ReplaceAll(query.ExporterIP.Unmap().String(), ".", "_"),
|
||||
},
|
||||
}
|
||||
if ifIndex != 999 {
|
||||
answer.Interface.Name = fmt.Sprintf("Gi0/0/%d", ifIndex)
|
||||
answer.Interface.Description = fmt.Sprintf("Interface %d", ifIndex)
|
||||
answer.Interface.Speed = 1000
|
||||
}
|
||||
|
||||
// in iface with metadata (overriden by out iface)
|
||||
if ifIndex == 1010 {
|
||||
answer.Exporter.Group = "metadata group"
|
||||
@@ -55,17 +64,17 @@ func (mp mockProvider) Query(_ context.Context, query *provider.BatchQuery) erro
|
||||
answer.Exporter.Site = "metadata site"
|
||||
answer.Exporter.Tenant = "metadata tenant"
|
||||
}
|
||||
mp.put(provider.Update{Query: provider.Query{ExporterIP: query.ExporterIP, IfIndex: ifIndex}, Answer: answer})
|
||||
}
|
||||
return nil
|
||||
|
||||
answer.Found = true
|
||||
return answer, nil
|
||||
}
|
||||
|
||||
// mockProviderConfiguration is the configuration for the mock provider.
|
||||
type mockProviderConfiguration struct{}
|
||||
|
||||
// New returns a new mock provider.
|
||||
func (mpc mockProviderConfiguration) New(_ *reporter.Reporter, put func(provider.Update)) (provider.Provider, error) {
|
||||
return mockProvider{put: put}, nil
|
||||
func (mpc mockProviderConfiguration) New(_ *reporter.Reporter) (provider.Provider, error) {
|
||||
return mockProvider{}, nil
|
||||
}
|
||||
|
||||
// NewMock creates a new metadata component building synthetic values. It is already started.
|
||||
@@ -81,3 +90,19 @@ func NewMock(t *testing.T, reporter *reporter.Reporter, configuration Configurat
|
||||
helpers.StartStop(t, c)
|
||||
return c
|
||||
}
|
||||
|
||||
// emptyProvider represents an empty mock provider.
|
||||
type emptyProvider struct{}
|
||||
|
||||
// Query returns always a not found status for the empty mock provider
|
||||
func (mp emptyProvider) Query(_ context.Context, _ provider.Query) (provider.Answer, error) {
|
||||
return provider.Answer{}, nil
|
||||
}
|
||||
|
||||
// emptyProviderConfiguration is the configuration for the empty provider.
|
||||
type emptyProviderConfiguration struct{}
|
||||
|
||||
// New returns a new empty provider.
|
||||
func (mpc emptyProviderConfiguration) New(_ *reporter.Reporter) (provider.Provider, error) {
|
||||
return emptyProvider{}, nil
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user