From d98076c7921b1c3e81854064968a690da21ec8bd Mon Sep 17 00:00:00 2001 From: Thomas Gruber Date: Fri, 25 Feb 2022 14:49:49 +0100 Subject: [PATCH 01/28] Merge current development version into main (#48) * DiskstatCollector: cast part_max_used metric to int * Add uint types to GangliaSink and LibgangliaSink * Use new sink instances to allow multiple of same sink type * Update sink README and SampleSink * Use new receiver instances to allow multiple of same receiver type * Fix metric scope in likwid configuration script * Mention likwid config script in LikwidCollector README * Refactor: Embed Init() into New() function * Refactor: Embed Init() into New() function * Fix: MetricReceiver uses uninitialized values, when initialization fails * Use Ganglia configuration (#44) * Copy all metric configurations from original Ganglia code * Use metric configurations from Ganglia for some metrics * Format value string also for known metrics * Numa-aware memstat collector (#45) * Add samples for collectors, sinks and receivers * Ping InfluxDB server after connecting to recognize faulty connections * Add sink for Prometheus monitoring system (#46) * Add sink for Prometheus monitoring system * Add prometheus sink to README * Add scraper for Prometheus clients (#47) Co-authored-by: Holger Obermaier Co-authored-by: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> --- collectors/diskstatMetric.go | 2 +- collectors/likwidMetric.md | 38 +++- collectors/memstatMetric.go | 176 ++++++++++++------ collectors/sampleMetric.go | 81 +++++++++ collectors/sampleTimerMetric.go | 122 +++++++++++++ receivers/README.md | 22 +-- receivers/metricReceiver.go | 9 +- receivers/natsReceiver.go | 65 ++++--- receivers/prometheusReceiver.go | 122 +++++++++++++ receivers/prometheusReceiver.md | 27 +++ receivers/receiveManager.go | 12 +- receivers/sampleReceiver.go | 77 ++++++++ scripts/likwid_perfgroup_to_cc_config.py | 2 +- sinks/README.md | 20 +- sinks/gangliaCommon.go | 218 +++++++++++++++++++++- sinks/gangliaSink.go | 156 +++++++--------- sinks/httpSink.go | 102 +++++------ sinks/influxAsyncSink.go | 87 +++++---- sinks/influxSink.go | 51 ++++-- sinks/libgangliaSink.go | 221 ++++++++++------------- sinks/metricSink.go | 3 - sinks/natsSink.go | 52 +++--- sinks/prometheusSink.go | 199 ++++++++++++++++++++ sinks/prometheusSink.md | 23 +++ sinks/sampleSink.go | 62 +++++++ sinks/sinkManager.go | 19 +- sinks/stdoutSink.go | 58 +++--- 27 files changed, 1509 insertions(+), 517 deletions(-) create mode 100644 collectors/sampleMetric.go create mode 100644 collectors/sampleTimerMetric.go create mode 100644 receivers/prometheusReceiver.go create mode 100644 receivers/prometheusReceiver.md create mode 100644 receivers/sampleReceiver.go create mode 100644 sinks/prometheusSink.go create mode 100644 sinks/prometheusSink.md create mode 100644 sinks/sampleSink.go diff --git a/collectors/diskstatMetric.go b/collectors/diskstatMetric.go index 819a1ab..16c70ba 100644 --- a/collectors/diskstatMetric.go +++ b/collectors/diskstatMetric.go @@ -102,7 +102,7 @@ func (m *DiskstatCollector) Read(interval time.Duration, output chan lp.CCMetric part_max_used = perc } } - y, err := lp.New("part_max_used", map[string]string{"type": "node"}, m.meta, map[string]interface{}{"value": part_max_used}, time.Now()) + y, err := lp.New("part_max_used", map[string]string{"type": "node"}, m.meta, map[string]interface{}{"value": int(part_max_used)}, time.Now()) if err == nil { y.AddMeta("unit", "percent") output <- y diff --git a/collectors/likwidMetric.md b/collectors/likwidMetric.md index 8b5dee2..3ef51f3 100644 --- a/collectors/likwidMetric.md +++ b/collectors/likwidMetric.md @@ -4,7 +4,7 @@ The `likwid` collector is probably the most complicated collector. The LIKWID library is included as static library with *direct* access mode. The *direct* access mode is suitable if the daemon is executed by a root user. The static library does not contain the performance groups, so all information needs to be provided in the configuration. The `likwid` configuration consists of two parts, the "eventsets" and "globalmetrics": -- An event set list itself has two parts, the "events" and a set of derivable "metrics". Each of the "events" is a counter:event pair in LIKWID's syntax. The "metrics" are a list of formulas to derive the metric value from the measurements of the "events". Each metric has a name, the formula, a scope and a publish flag. A counter names can be used like variables in the formulas, so `PMC0+PMC1` sums the measurements for the both events configured in the counters `PMC0` and `PMC1`. The scope tells the Collector whether it is a metric for each hardware thread (`cpu`) or each CPU socket (`socket`). The last one is the publishing flag. It tells the collector whether a metric should be sent to the router. +- An event set list itself has two parts, the "events" and a set of derivable "metrics". Each of the "events" is a counter:event pair in LIKWID's syntax. The "metrics" are a list of formulas to derive the metric value from the measurements of the "events". Each metric has a name, the formula, a scope and a publish flag. Counter names can be used like variables in the formulas, so `PMC0+PMC1` sums the measurements for the both events configured in the counters `PMC0` and `PMC1`. The scope tells the Collector whether it is a metric for each hardware thread (`cpu`) or each CPU socket (`socket`). The last one is the publishing flag. It tells the collector whether a metric should be sent to the router. - The global metrics are metrics which require data from all event set measurements to be derived. The inputs are the metrics in the event sets. Similar to the metrics in the event sets, the global metrics are defined by a name, a formula, a scope and a publish flag. See event set metrics for details. The only difference is that there is no access to the raw event measurements anymore but only to the metrics. So, the idea is to derive a metric in the "eventsets" section and reuse it in the "globalmetrics" part. If you need a metric only for deriving the global metrics, disable forwarding of the event set metrics. **Be aware** that the combination might be misleading because the "behavior" of a metric changes over time and the multiple measurements might count different computing phases. Additional options: @@ -26,6 +26,42 @@ As a guideline: - All `PWRx` counters have scope `socket`, except `"PWR1" : "RAPL_CORE_ENERGY"` has `cpu` scope - All `DFCx` counters have scope `socket` +### Help with the configuration + +The configuration for the `likwid` collector is quite complicated. Most users don't use LIKWID with the event:counter notation but rely on the performance groups defined by the LIKWID team for each architecture. In order to help with the `likwid` collector configuration, we included a script `scripts/likwid_perfgroup_to_cc_config.py` that creates the configuration of an `eventset` from a performance group (using a LIKWID installation in `$PATH`): +``` +$ likwid-perfctr -i +[...] +short name: ICX +[...] +$ likwid-perfctr -a +[...] +MEM_DP +MEM +FLOPS_SP +CLOCK +[...] +$ scripts/likwid_perfgroup_to_cc_config.py ICX MEM_DP +{ + "events": { + "FIXC0": "INSTR_RETIRED_ANY", + "..." : "..." + }, + "metrics" : [ + { + "calc": "time", + "name": "Runtime (RDTSC) [s]", + "publish": true, + "scope": "hwthread" + }, + { + "..." : "..." + } + ] +} +``` + +You can copy this JSON and add it to the `eventsets` list. If you specify multiple event sets, you can add globally derived metrics in the extra `global_metrics` section with the metric names as variables. ### Example configuration diff --git a/collectors/memstatMetric.go b/collectors/memstatMetric.go index b6ef855..3998537 100644 --- a/collectors/memstatMetric.go +++ b/collectors/memstatMetric.go @@ -1,35 +1,76 @@ package collectors import ( + "bufio" "encoding/json" "errors" "fmt" - "io/ioutil" - "log" + "os" + "path/filepath" + "regexp" "strconv" "strings" "time" + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" ) -const MEMSTATFILE = `/proc/meminfo` +const MEMSTATFILE = "/proc/meminfo" +const NUMA_MEMSTAT_BASE = "/sys/devices/system/node" type MemstatCollectorConfig struct { ExcludeMetrics []string `json:"exclude_metrics"` + NodeStats bool `json:"node_stats,omitempty"` + NumaStats bool `json:"numa_stats,omitempty"` +} + +type MemstatCollectorNode struct { + file string + tags map[string]string } type MemstatCollector struct { metricCollector - stats map[string]int64 - tags map[string]string - matches map[string]string - config MemstatCollectorConfig + stats map[string]int64 + tags map[string]string + matches map[string]string + config MemstatCollectorConfig + nodefiles map[int]MemstatCollectorNode +} + +func getStats(filename string) map[string]float64 { + stats := make(map[string]float64) + file, err := os.Open(filename) + if err != nil { + cclog.Error(err.Error()) + } + defer file.Close() + + scanner := bufio.NewScanner(file) + for scanner.Scan() { + line := scanner.Text() + linefields := strings.Fields(line) + if len(linefields) == 3 { + v, err := strconv.ParseFloat(linefields[1], 64) + if err == nil { + stats[strings.Trim(linefields[0], ":")] = v + } + } else if len(linefields) == 5 { + v, err := strconv.ParseFloat(linefields[3], 64) + if err == nil { + stats[strings.Trim(linefields[0], ":")] = v + } + } + } + return stats } func (m *MemstatCollector) Init(config json.RawMessage) error { var err error m.name = "MemstatCollector" + m.config.NodeStats = true + m.config.NumaStats = false if len(config) > 0 { err = json.Unmarshal(config, &m.config) if err != nil { @@ -40,7 +81,8 @@ func (m *MemstatCollector) Init(config json.RawMessage) error { m.stats = make(map[string]int64) m.matches = make(map[string]string) m.tags = map[string]string{"type": "node"} - matches := map[string]string{`MemTotal`: "mem_total", + matches := map[string]string{ + "MemTotal": "mem_total", "SwapTotal": "swap_total", "SReclaimable": "mem_sreclaimable", "Slab": "mem_slab", @@ -48,7 +90,9 @@ func (m *MemstatCollector) Init(config json.RawMessage) error { "Buffers": "mem_buffers", "Cached": "mem_cached", "MemAvailable": "mem_available", - "SwapFree": "swap_free"} + "SwapFree": "swap_free", + "MemShared": "mem_shared", + } for k, v := range matches { _, skip := stringArrayContains(m.config.ExcludeMetrics, k) if !skip { @@ -56,13 +100,44 @@ func (m *MemstatCollector) Init(config json.RawMessage) error { } } if len(m.matches) == 0 { - return errors.New("No metrics to collect") + return errors.New("no metrics to collect") } m.setup() - _, err = ioutil.ReadFile(string(MEMSTATFILE)) - if err == nil { - m.init = true + + if m.config.NodeStats { + if stats := getStats(MEMSTATFILE); len(stats) == 0 { + return fmt.Errorf("cannot read data from file %s", MEMSTATFILE) + } } + + if m.config.NumaStats { + globPattern := filepath.Join(NUMA_MEMSTAT_BASE, "node[0-9]*", "meminfo") + regex := regexp.MustCompile(filepath.Join(NUMA_MEMSTAT_BASE, "node(\\d+)", "meminfo")) + files, err := filepath.Glob(globPattern) + if err == nil { + m.nodefiles = make(map[int]MemstatCollectorNode) + for _, f := range files { + if stats := getStats(f); len(stats) == 0 { + return fmt.Errorf("cannot read data from file %s", f) + } + rematch := regex.FindStringSubmatch(f) + if len(rematch) == 2 { + id, err := strconv.Atoi(rematch[1]) + if err == nil { + f := MemstatCollectorNode{ + file: f, + tags: map[string]string{ + "type": "memoryDomain", + "type-id": fmt.Sprintf("%d", id), + }, + } + m.nodefiles[id] = f + } + } + } + } + } + m.init = true return err } @@ -71,56 +146,41 @@ func (m *MemstatCollector) Read(interval time.Duration, output chan lp.CCMetric) return } - buffer, err := ioutil.ReadFile(string(MEMSTATFILE)) - if err != nil { - log.Print(err) - return - } - - ll := strings.Split(string(buffer), "\n") - for _, line := range ll { - ls := strings.Split(line, `:`) - if len(ls) > 1 { - lv := strings.Fields(ls[1]) - m.stats[ls[0]], err = strconv.ParseInt(lv[0], 0, 64) + sendStats := func(stats map[string]float64, tags map[string]string) { + for match, name := range m.matches { + var value float64 = 0 + if v, ok := stats[match]; ok { + value = v + } + y, err := lp.New(name, tags, m.meta, map[string]interface{}{"value": value}, time.Now()) + if err == nil { + output <- y + } } - } - - if _, exists := m.stats[`MemTotal`]; !exists { - err = errors.New("Parse error") - log.Print(err) - return - } - - for match, name := range m.matches { - if _, exists := m.stats[match]; !exists { - err = fmt.Errorf("Parse error for %s : %s", match, name) - log.Print(err) - continue - } - y, err := lp.New(name, m.tags, m.meta, map[string]interface{}{"value": int(float64(m.stats[match]) * 1.0e-3)}, time.Now()) - if err == nil { - output <- y - } - } - - if _, free := m.stats[`MemFree`]; free { - if _, buffers := m.stats[`Buffers`]; buffers { - if _, cached := m.stats[`Cached`]; cached { - memUsed := m.stats[`MemTotal`] - (m.stats[`MemFree`] + m.stats[`Buffers`] + m.stats[`Cached`]) - _, skip := stringArrayContains(m.config.ExcludeMetrics, "mem_used") - y, err := lp.New("mem_used", m.tags, m.meta, map[string]interface{}{"value": int(float64(memUsed) * 1.0e-3)}, time.Now()) - if err == nil && !skip { - output <- y + if _, skip := stringArrayContains(m.config.ExcludeMetrics, "mem_used"); !skip { + if freeVal, free := stats["MemFree"]; free { + if bufVal, buffers := stats["Buffers"]; buffers { + if cacheVal, cached := stats["Cached"]; cached { + memUsed := stats["MemTotal"] - (freeVal + bufVal + cacheVal) + y, err := lp.New("mem_used", tags, m.meta, map[string]interface{}{"value": memUsed}, time.Now()) + if err == nil { + output <- y + } + } } } } } - if _, found := m.stats[`MemShared`]; found { - _, skip := stringArrayContains(m.config.ExcludeMetrics, "mem_shared") - y, err := lp.New("mem_shared", m.tags, m.meta, map[string]interface{}{"value": int(float64(m.stats[`MemShared`]) * 1.0e-3)}, time.Now()) - if err == nil && !skip { - output <- y + + if m.config.NodeStats { + nodestats := getStats(MEMSTATFILE) + sendStats(nodestats, m.tags) + } + + if m.config.NumaStats { + for _, nodeConf := range m.nodefiles { + stats := getStats(nodeConf.file) + sendStats(stats, nodeConf.tags) } } } diff --git a/collectors/sampleMetric.go b/collectors/sampleMetric.go new file mode 100644 index 0000000..cd3b4cc --- /dev/null +++ b/collectors/sampleMetric.go @@ -0,0 +1,81 @@ +package collectors + +import ( + "encoding/json" + "time" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +// These are the fields we read from the JSON configuration +type SampleCollectorConfig struct { + Interval string `json:"interval"` +} + +// This contains all variables we need during execution and the variables +// defined by metricCollector (name, init, ...) +type SampleCollector struct { + metricCollector + config SampleTimerCollectorConfig // the configuration structure + meta map[string]string // default meta information + tags map[string]string // default tags +} + +func (m *SampleCollector) Init(config json.RawMessage) error { + var err error = nil + // Always set the name early in Init() to use it in cclog.Component* functions + m.name = "InternalCollector" + // This is for later use, also call it early + m.setup() + // Define meta information sent with each metric + // (Can also be dynamic or this is the basic set with extension through AddMeta()) + m.meta = map[string]string{"source": m.name, "group": "SAMPLE"} + // Define tags sent with each metric + // The 'type' tag is always needed, it defines the granulatity of the metric + // node -> whole system + // socket -> CPU socket (requires socket ID as 'type-id' tag) + // cpu -> single CPU hardware thread (requires cpu ID as 'type-id' tag) + m.tags = map[string]string{"type": "node"} + // Read in the JSON configuration + if len(config) > 0 { + err = json.Unmarshal(config, &m.config) + if err != nil { + cclog.ComponentError(m.name, "Error reading config:", err.Error()) + return err + } + } + + // Set up everything that the collector requires during the Read() execution + // Check files required, test execution of some commands, create data structure + // for all topological entities (sockets, NUMA domains, ...) + // Return some useful error message in case of any failures + + // Set this flag only if everything is initialized properly, all required files exist, ... + m.init = true + return err +} + +func (m *SampleCollector) Read(interval time.Duration, output chan lp.CCMetric) { + // Create a sample metric + timestamp := time.Now() + + value := 1.0 + // If you want to measure something for a specific amout of time, use interval + // start := readState() + // time.Sleep(interval) + // stop := readState() + // value = (stop - start) / interval.Seconds() + + y, err := lp.New("sample_metric", m.tags, m.meta, map[string]interface{}{"value": value}, timestamp) + if err == nil { + // Send it to output channel + output <- y + } + +} + +func (m *SampleCollector) Close() { + // Unset flag + m.init = false +} diff --git a/collectors/sampleTimerMetric.go b/collectors/sampleTimerMetric.go new file mode 100644 index 0000000..aa6807e --- /dev/null +++ b/collectors/sampleTimerMetric.go @@ -0,0 +1,122 @@ +package collectors + +import ( + "encoding/json" + "sync" + "time" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +// These are the fields we read from the JSON configuration +type SampleTimerCollectorConfig struct { + Interval string `json:"interval"` +} + +// This contains all variables we need during execution and the variables +// defined by metricCollector (name, init, ...) +type SampleTimerCollector struct { + metricCollector + wg sync.WaitGroup // sync group for management + done chan bool // channel for management + meta map[string]string // default meta information + tags map[string]string // default tags + config SampleTimerCollectorConfig // the configuration structure + interval time.Duration // the interval parsed from configuration + ticker *time.Ticker // own timer + output chan lp.CCMetric // own internal output channel +} + +func (m *SampleTimerCollector) Init(name string, config json.RawMessage) error { + var err error = nil + // Always set the name early in Init() to use it in cclog.Component* functions + m.name = "SampleTimerCollector" + // This is for later use, also call it early + m.setup() + // Define meta information sent with each metric + // (Can also be dynamic or this is the basic set with extension through AddMeta()) + m.meta = map[string]string{"source": m.name, "group": "SAMPLE"} + // Define tags sent with each metric + // The 'type' tag is always needed, it defines the granulatity of the metric + // node -> whole system + // socket -> CPU socket (requires socket ID as 'type-id' tag) + // cpu -> single CPU hardware thread (requires cpu ID as 'type-id' tag) + m.tags = map[string]string{"type": "node"} + // Read in the JSON configuration + if len(config) > 0 { + err = json.Unmarshal(config, &m.config) + if err != nil { + cclog.ComponentError(m.name, "Error reading config:", err.Error()) + return err + } + } + // Parse the read interval duration + m.interval, err = time.ParseDuration(m.config.Interval) + if err != nil { + cclog.ComponentError(m.name, "Error parsing interval:", err.Error()) + return err + } + + // Storage for output channel + m.output = nil + // Mangement channel for the timer function. + m.done = make(chan bool) + // Create the own ticker + m.ticker = time.NewTicker(m.interval) + + // Start the timer loop with return functionality by sending 'true' to the done channel + m.wg.Add(1) + go func() { + select { + case <-m.done: + // Exit the timer loop + cclog.ComponentDebug(m.name, "Closing...") + m.wg.Done() + return + case timestamp := <-m.ticker.C: + // This is executed every timer tick but we have to wait until the first + // Read() to get the output channel + if m.output != nil { + m.ReadMetrics(timestamp) + } + } + }() + + // Set this flag only if everything is initialized properly, all required files exist, ... + m.init = true + return err +} + +// This function is called at each interval timer tick +func (m *SampleTimerCollector) ReadMetrics(timestamp time.Time) { + // Create a sample metric + + value := 1.0 + + // If you want to measure something for a specific amout of time, use interval + // start := readState() + // time.Sleep(interval) + // stop := readState() + // value = (stop - start) / interval.Seconds() + + y, err := lp.New("sample_metric", m.tags, m.meta, map[string]interface{}{"value": value}, timestamp) + if err == nil && m.output != nil { + // Send it to output channel if we have a valid channel + m.output <- y + } +} + +func (m *SampleTimerCollector) Read(interval time.Duration, output chan lp.CCMetric) { + // Capture output channel + m.output = output +} + +func (m *SampleTimerCollector) Close() { + // Send signal to the timer loop to stop it + m.done <- true + // Wait until the timer loop is done + m.wg.Wait() + // Unset flag + m.init = false +} diff --git a/receivers/README.md b/receivers/README.md index 24425f2..808dc74 100644 --- a/receivers/README.md +++ b/receivers/README.md @@ -7,17 +7,13 @@ This folder contains the ReceiveManager and receiver implementations for the cc- The configuration file for the receivers is a list of configurations. The `type` field in each specifies which receiver to initialize. ```json -[ - { - "type": "nats", - "address": "nats://my-url", - "port" : "4222", - "database": "testcluster" +{ + "myreceivername" : { + } -] +} ``` - ## Type `nats` ```json @@ -25,20 +21,20 @@ The configuration file for the receivers is a list of configurations. The `type` "type": "nats", "address": "", "port" : "", - "database": "" + "subject": "" } ``` The `nats` receiver subscribes to the topic `database` and listens on `address` and `port` for metrics in the InfluxDB line protocol. # Contributing own receivers -A receiver contains three functions and is derived from the type `Receiver` (in `metricReceiver.go`): -* `Init(config ReceiverConfig) error` +A receiver contains a few functions and is derived from the type `Receiver` (in `metricReceiver.go`): * `Start() error` * `Close()` * `Name() string` -* `SetSink(sink chan ccMetric.CCMetric)` +* `SetSink(sink chan lp.CCMetric)` +* `New(name string, config json.RawMessage)` The data structures should be set up in `Init()` like opening a file or server connection. The `Start()` function should either start a go routine or issue some other asynchronous mechanism for receiving metrics. The `Close()` function should tear down anything created in `Init()`. -Finally, the receiver needs to be registered in the `receiveManager.go`. There is a list of receivers called `AvailableReceivers` which is a map (`receiver_type_string` -> `pointer to Receiver interface`). Add a new entry with a descriptive name and the new receiver. +Finally, the receiver needs to be registered in the `receiveManager.go`. There is a list of receivers called `AvailableReceivers` which is a map (`receiver_type_string` -> `pointer to NewReceiver function`). Add a new entry with a descriptive name and the new receiver. diff --git a/receivers/metricReceiver.go b/receivers/metricReceiver.go index c712186..e133354 100644 --- a/receivers/metricReceiver.go +++ b/receivers/metricReceiver.go @@ -1,9 +1,6 @@ package receivers import ( - // "time" - "encoding/json" - lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" ) @@ -20,13 +17,11 @@ type ReceiverConfig struct { } type receiver struct { - typename string - name string - sink chan lp.CCMetric + name string + sink chan lp.CCMetric } type Receiver interface { - Init(name string, config json.RawMessage) error Start() Close() Name() string diff --git a/receivers/natsReceiver.go b/receivers/natsReceiver.go index dc96971..1a5f47b 100644 --- a/receivers/natsReceiver.go +++ b/receivers/natsReceiver.go @@ -32,39 +32,6 @@ var DefaultTime = func() time.Time { return time.Unix(42, 0) } -func (r *NatsReceiver) Init(name string, config json.RawMessage) error { - r.typename = "NatsReceiver" - r.name = name - r.config.Addr = nats.DefaultURL - r.config.Port = "4222" - if len(config) > 0 { - err := json.Unmarshal(config, &r.config) - if err != nil { - cclog.ComponentError(r.name, "Error reading config:", err.Error()) - return err - } - } - if len(r.config.Addr) == 0 || - len(r.config.Port) == 0 || - len(r.config.Subject) == 0 { - return errors.New("not all configuration variables set required by NatsReceiver") - } - r.meta = map[string]string{"source": r.name} - uri := fmt.Sprintf("%s:%s", r.config.Addr, r.config.Port) - cclog.ComponentDebug(r.name, "INIT", uri, "Subject", r.config.Subject) - nc, err := nats.Connect(uri) - if err == nil { - r.nc = nc - } else { - r.nc = nil - return err - } - r.handler = influx.NewMetricHandler() - r.parser = influx.NewParser(r.handler) - r.parser.SetTimeFunc(DefaultTime) - return err -} - func (r *NatsReceiver) Start() { cclog.ComponentDebug(r.name, "START") r.nc.Subscribe(r.config.Subject, r._NatsReceive) @@ -91,3 +58,35 @@ func (r *NatsReceiver) Close() { r.nc.Close() } } + +func NewNatsReceiver(name string, config json.RawMessage) (Receiver, error) { + r := new(NatsReceiver) + r.name = fmt.Sprintf("NatsReceiver(%s)", name) + r.config.Addr = nats.DefaultURL + r.config.Port = "4222" + if len(config) > 0 { + err := json.Unmarshal(config, &r.config) + if err != nil { + cclog.ComponentError(r.name, "Error reading config:", err.Error()) + return nil, err + } + } + if len(r.config.Addr) == 0 || + len(r.config.Port) == 0 || + len(r.config.Subject) == 0 { + return nil, errors.New("not all configuration variables set required by NatsReceiver") + } + r.meta = map[string]string{"source": r.name} + uri := fmt.Sprintf("%s:%s", r.config.Addr, r.config.Port) + cclog.ComponentDebug(r.name, "NewNatsReceiver", uri, "Subject", r.config.Subject) + if nc, err := nats.Connect(uri); err == nil { + r.nc = nc + } else { + r.nc = nil + return nil, err + } + r.handler = influx.NewMetricHandler() + r.parser = influx.NewParser(r.handler) + r.parser.SetTimeFunc(DefaultTime) + return r, nil +} diff --git a/receivers/prometheusReceiver.go b/receivers/prometheusReceiver.go new file mode 100644 index 0000000..c22969d --- /dev/null +++ b/receivers/prometheusReceiver.go @@ -0,0 +1,122 @@ +package receivers + +import ( + "bufio" + "encoding/json" + "errors" + "fmt" + "log" + "net/http" + "strconv" + "strings" + "sync" + "time" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +type PrometheusReceiverConfig struct { + defaultReceiverConfig + Addr string `json:"address"` + Port string `json:"port"` + Path string `json:"path"` + Interval string `json:"interval"` + SSL bool `json:"ssl"` +} + +type PrometheusReceiver struct { + receiver + meta map[string]string + config PrometheusReceiverConfig + interval time.Duration + done chan bool + wg sync.WaitGroup + ticker *time.Ticker + uri string +} + +func (r *PrometheusReceiver) Start() { + cclog.ComponentDebug(r.name, "START", r.uri) + r.wg.Add(1) + + r.ticker = time.NewTicker(r.interval) + go func() { + for { + select { + case <-r.done: + r.wg.Done() + return + case t := <-r.ticker.C: + resp, err := http.Get(r.uri) + if err != nil { + log.Fatal(err) + } + defer resp.Body.Close() + + scanner := bufio.NewScanner(resp.Body) + for scanner.Scan() { + line := scanner.Text() + if strings.HasPrefix(line, "#") { + continue + } + lineSplit := strings.Fields(line) + // separate metric name from tags (labels in Prometheus) + tags := map[string]string{} + name := lineSplit[0] + if sindex := strings.Index(name, "{"); sindex >= 0 { + eindex := strings.Index(name, "}") + for _, kv := range strings.Split(name[sindex+1:eindex], ",") { + eq := strings.Index(kv, "=") + tags[kv[0:eq]] = strings.Trim(kv[eq+1:], "\"") + } + name = lineSplit[0][0:sindex] + } + value, err := strconv.ParseFloat(lineSplit[1], 64) + if err == nil { + y, err := lp.New(name, tags, r.meta, map[string]interface{}{"value": value}, t) + if err == nil { + r.sink <- y + } + } + } + } + } + }() +} + +func (r *PrometheusReceiver) Close() { + cclog.ComponentDebug(r.name, "CLOSE") + r.done <- true + r.wg.Wait() +} + +func NewPrometheusReceiver(name string, config json.RawMessage) (Receiver, error) { + r := new(PrometheusReceiver) + r.name = fmt.Sprintf("PrometheusReceiver(%s)", name) + if len(config) > 0 { + err := json.Unmarshal(config, &r.config) + if err != nil { + cclog.ComponentError(r.name, "Error reading config:", err.Error()) + return nil, err + } + } + if len(r.config.Addr) == 0 || + len(r.config.Port) == 0 || + len(r.config.Interval) == 0 { + return nil, errors.New("not all configuration variables set required by PrometheusReceiver (address and port)") + } + if len(r.config.Interval) > 0 { + t, err := time.ParseDuration(r.config.Interval) + if err == nil { + r.interval = t + } + } + r.meta = map[string]string{"source": r.name} + proto := "http" + if r.config.SSL { + proto = "https" + } + r.uri = fmt.Sprintf("%s://%s:%s/%s", proto, r.config.Addr, r.config.Port, r.config.Path) + return r, nil +} diff --git a/receivers/prometheusReceiver.md b/receivers/prometheusReceiver.md new file mode 100644 index 0000000..5fe6e46 --- /dev/null +++ b/receivers/prometheusReceiver.md @@ -0,0 +1,27 @@ +## `prometheus` receiver + +The `prometheus` receiver can be used to scrape the metrics of a single `prometheus` client. It does **not** use any official Golang library but making simple HTTP get requests and parse the response. + +### Configuration structure + +```json +{ + "": { + "type": "prometheus", + "address" : "testpromhost", + "port" : "12345", + "port" : "/prometheus", + "interval": "5s", + "ssl" : true, + } +} +``` + +- `type`: makes the receiver a `prometheus` receiver +- `address`: Hostname or IP of the Prometheus agent +- `port`: Port of Prometheus agent +- `path`: Path to the Prometheus endpoint +- `interval`: Scrape the Prometheus endpoint in this interval (default '5s') +- `ssl`: Use SSL or not + +The receiver requests data from `http(s)://
:/`. \ No newline at end of file diff --git a/receivers/receiveManager.go b/receivers/receiveManager.go index 7141170..1c13026 100644 --- a/receivers/receiveManager.go +++ b/receivers/receiveManager.go @@ -9,8 +9,8 @@ import ( lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" ) -var AvailableReceivers = map[string]Receiver{ - "nats": &NatsReceiver{}, +var AvailableReceivers = map[string]func(name string, config json.RawMessage) (Receiver, error){ + "nats": NewNatsReceiver, } type receiveManager struct { @@ -30,11 +30,13 @@ type ReceiveManager interface { } func (rm *receiveManager) Init(wg *sync.WaitGroup, receiverConfigFile string) error { + // Initialize struct fields rm.inputs = make([]Receiver, 0) rm.output = nil rm.done = make(chan bool) rm.wg = wg rm.config = make([]json.RawMessage, 0) + configFile, err := os.Open(receiverConfigFile) if err != nil { cclog.ComponentError("ReceiveManager", err.Error()) @@ -51,6 +53,7 @@ func (rm *receiveManager) Init(wg *sync.WaitGroup, receiverConfigFile string) er for name, raw := range rawConfigs { rm.AddInput(name, raw) } + return nil } @@ -75,10 +78,9 @@ func (rm *receiveManager) AddInput(name string, rawConfig json.RawMessage) error cclog.ComponentError("ReceiveManager", "SKIP", config.Type, "unknown receiver:", err.Error()) return err } - r := AvailableReceivers[config.Type] - err = r.Init(name, rawConfig) + r, err := AvailableReceivers[config.Type](name, rawConfig) if err != nil { - cclog.ComponentError("ReceiveManager", "SKIP", r.Name(), "initialization failed:", err.Error()) + cclog.ComponentError("ReceiveManager", "SKIP", name, "initialization failed:", err.Error()) return err } rm.inputs = append(rm.inputs, r) diff --git a/receivers/sampleReceiver.go b/receivers/sampleReceiver.go new file mode 100644 index 0000000..e9edc90 --- /dev/null +++ b/receivers/sampleReceiver.go @@ -0,0 +1,77 @@ +package receivers + +import ( + "encoding/json" + "fmt" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" +) + +type SampleReceiverConfig struct { + Type string `json:"type"` + Addr string `json:"address"` + Port string `json:"port"` +} + +type SampleReceiver struct { + receiver + config SampleReceiverConfig + + // Storage for static information + meta map[string]string + // Use in case of own go routine + // done chan bool + // wg sync.WaitGroup +} + +func (r *SampleReceiver) Start() { + cclog.ComponentDebug(r.name, "START") + + // Start server process like http.ListenAndServe() + + // or use own go routine but always make sure it exits + // as soon as it gets the signal of the r.done channel + // r.wg.Add(1) + // go func() { + // for { + // select { + // case <-r.done: + // r.wg.Done() + // return + // } + // } + // r.wg.Done() + // }() +} + +func (r *SampleReceiver) Close() { + cclog.ComponentDebug(r.name, "CLOSE") + + // Close server like http.Shutdown() + + // in case of own go routine, send the signal and wait + // r.done <- true + // r.wg.Wait() +} + +func NewSampleReceiver(name string, config json.RawMessage) (Receiver, error) { + r := new(SampleReceiver) + r.name = fmt.Sprintf("HttpReceiver(%s)", name) + + // Set static information + r.meta = map[string]string{"source": r.name} + + // Read the sample receiver specific JSON config + if len(config) > 0 { + err := json.Unmarshal(config, &r.config) + if err != nil { + cclog.ComponentError(r.name, "Error reading config:", err.Error()) + return nil, err + } + } + + // Check that all required fields in the configuration are set + // Use 'if len(r.config.Option) > 0' for strings + + return r, nil +} diff --git a/scripts/likwid_perfgroup_to_cc_config.py b/scripts/likwid_perfgroup_to_cc_config.py index 52959ed..f1c3451 100755 --- a/scripts/likwid_perfgroup_to_cc_config.py +++ b/scripts/likwid_perfgroup_to_cc_config.py @@ -39,7 +39,7 @@ def group_to_json(groupfile): llist = re.split("\s+", line) calc = llist[-1] metric = " ".join(llist[:-1]) - scope = "hwthread" + scope = "cpu" if "BOX" in calc: scope = "socket" if "PWR" in calc: diff --git a/sinks/README.md b/sinks/README.md index 8ff3743..df532c9 100644 --- a/sinks/README.md +++ b/sinks/README.md @@ -6,9 +6,11 @@ This folder contains the SinkManager and sink implementations for the cc-metric- - [`stdout`](./stdoutSink.md): Print all metrics to `stdout`, `stderr` or a file - [`http`](./httpSink.md): Send metrics to an HTTP server as POST requests - [`influxdb`](./influxSink.md): Send metrics to an [InfluxDB](https://www.influxdata.com/products/influxdb/) database +- [`influxasync`](./influxAsyncSink.md): Send metrics to an [InfluxDB](https://www.influxdata.com/products/influxdb/) database with non-blocking write API - [`nats`](./natsSink.md): Publish metrics to the [NATS](https://nats.io/) network overlay system - [`ganglia`](./gangliaSink.md): Publish metrics in the [Ganglia Monitoring System](http://ganglia.info/) using the `gmetric` CLI tool - [`libganglia`](./libgangliaSink.md): Publish metrics in the [Ganglia Monitoring System](http://ganglia.info/) directly using `libganglia.so` +- [`prometeus`](./prometheusSink.md): Publish metrics for the [Prometheus Monitoring System](https://prometheus.io/) # Configuration @@ -34,11 +36,12 @@ The configuration file for the sinks is a list of configurations. The `type` fie # Contributing own sinks -A sink contains four functions and is derived from the type `sink`: -* `Init(config json.RawMessage) error` +A sink contains five functions and is derived from the type `sink`: +* `Init(name string, config json.RawMessage) error` * `Write(point CCMetric) error` * `Flush() error` * `Close()` +* `New(name string, config json.RawMessage) (Sink, error)` (calls the `Init()` function) The data structures should be set up in `Init()` like opening a file or server connection. The `Write()` function writes/sends the data. For non-blocking sinks, the `Flush()` method tells the sink to drain its internal buffers. The `Close()` function should tear down anything created in `Init()`. @@ -65,8 +68,8 @@ type SampleSink struct { } // Initialize the sink by giving it a name and reading in the config JSON -func (s *SampleSink) Init(config json.RawMessage) error { - s.name = "SampleSink" // Always specify a name here +func (s *SampleSink) Init(name string, config json.RawMessage) error { + s.name = fmt.Sprintf("SampleSink(%s)", name) // Always specify a name here // Read in the config JSON if len(config) > 0 { err := json.Unmarshal(config, &s.config) @@ -91,4 +94,13 @@ func (s *SampleSink) Flush() error { // Close sink: close network connection, close files, close libraries, ... func (s *SampleSink) Close() {} + + +// New function to create a new instance of the sink +func NewSampleSink(name string, config json.RawMessage) (Sink, error) { + s := new(SampleSink) + err := s.Init(name, config) + return s, err +} + ``` \ No newline at end of file diff --git a/sinks/gangliaCommon.go b/sinks/gangliaCommon.go index b939f16..b2a1b2c 100644 --- a/sinks/gangliaCommon.go +++ b/sinks/gangliaCommon.go @@ -1,6 +1,7 @@ package sinks import ( + "fmt" "strings" lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" @@ -23,11 +24,8 @@ func GangliaMetricName(point lp.CCMetric) string { return name } -func GangliaMetricRename(point lp.CCMetric) string { - name := point.Name() - if name == "mem_total" || name == "swap_total" { - return name - } else if name == "net_bytes_in" { +func GangliaMetricRename(name string) string { + if name == "net_bytes_in" { return "bytes_in" } else if name == "net_bytes_out" { return "bytes_out" @@ -48,3 +46,213 @@ func GangliaSlopeType(point lp.CCMetric) uint { } return 3 } + +const DEFAULT_GANGLIA_METRIC_TMAX = 300 +const DEFAULT_GANGLIA_METRIC_SLOPE = "both" + +type GangliaMetric struct { + Name string + Type string + Slope string + Tmax int + Unit string +} + +type GangliaMetricGroup struct { + Name string + Metrics []GangliaMetric +} + +var CommonGangliaMetrics = []GangliaMetricGroup{ + { + Name: "memory", + Metrics: []GangliaMetric{ + {"mem_total", "float", "zero", 1200, "KB"}, + {"swap_total", "float", "zero", 1200, "KB"}, + {"mem_free", "float", "both", 180, "KB"}, + {"mem_shared", "float", "both", 180, "KB"}, + {"mem_buffers", "float", "both", 180, "KB"}, + {"mem_cached", "float", "both", 180, "KB"}, + {"swap_free", "float", "both", 180, "KB"}, + {"mem_sreclaimable", "float", "both", 180, "KB"}, + {"mem_slab", "float", "both", 180, "KB"}, + }, + }, + { + Name: "cpu", + Metrics: []GangliaMetric{ + {"cpu_num", "uint32", "zero", 1200, "CPUs"}, + {"cpu_speed", "uint32", "zero", 1200, "MHz"}, + {"cpu_user", "float", "both", 90, "%"}, + {"cpu_nice", "float", "both", 90, "%"}, + {"cpu_system", "float", "both", 90, "%"}, + {"cpu_idle", "float", "both", 3800, "%"}, + {"cpu_aidle", "float", "both", 90, "%"}, + {"cpu_wio", "float", "both", 90, "%"}, + {"cpu_intr", "float", "both", 90, "%"}, + {"cpu_sintr", "float", "both", 90, "%"}, + {"cpu_steal", "float", "both", 90, "%"}, + {"cpu_guest", "float", "both", 90, "%"}, + {"cpu_gnice", "float", "both", 90, "%"}, + }, + }, + { + Name: "load", + Metrics: []GangliaMetric{ + {"load_one", "float", "both", 70, ""}, + {"load_five", "float", "both", 325, ""}, + {"load_fifteen", "float", "both", 950, ""}, + }, + }, + { + Name: "disk", + Metrics: []GangliaMetric{ + {"disk_total", "double", "both", 1200, "GB"}, + {"disk_free", "double", "both", 180, "GB"}, + {"part_max_used", "float", "both", 180, "%"}, + }, + }, + { + Name: "network", + Metrics: []GangliaMetric{ + {"bytes_out", "float", "both", 300, "bytes/sec"}, + {"bytes_in", "float", "both", 300, "bytes/sec"}, + {"pkts_in", "float", "both", 300, "packets/sec"}, + {"pkts_out", "float", "both", 300, "packets/sec"}, + }, + }, + { + Name: "process", + Metrics: []GangliaMetric{ + {"proc_run", "uint32", "both", 950, ""}, + {"proc_total", "uint32", "both", 950, ""}, + }, + }, + { + Name: "system", + Metrics: []GangliaMetric{ + {"boottime", "uint32", "zero", 1200, "s"}, + {"sys_clock", "uint32", "zero", 1200, "s"}, + {"machine_type", "string", "zero", 1200, ""}, + {"os_name", "string", "zero", 1200, ""}, + {"os_release", "string", "zero", 1200, ""}, + {"mtu", "uint32", "both", 1200, ""}, + }, + }, +} + +type GangliaMetricConfig struct { + Type string + Slope string + Tmax int + Unit string + Group string + Value string +} + +func GetCommonGangliaConfig(point lp.CCMetric) GangliaMetricConfig { + mname := GangliaMetricRename(point.Name()) + for _, group := range CommonGangliaMetrics { + for _, metric := range group.Metrics { + if metric.Name == mname { + valueStr := "" + value, ok := point.GetField("value") + if ok { + switch real := value.(type) { + case float64: + valueStr = fmt.Sprintf("%f", real) + case float32: + valueStr = fmt.Sprintf("%f", real) + case int64: + valueStr = fmt.Sprintf("%d", real) + case int32: + valueStr = fmt.Sprintf("%d", real) + case int: + valueStr = fmt.Sprintf("%d", real) + case uint64: + valueStr = fmt.Sprintf("%d", real) + case uint32: + valueStr = fmt.Sprintf("%d", real) + case uint: + valueStr = fmt.Sprintf("%d", real) + case string: + valueStr = real + default: + } + } + return GangliaMetricConfig{ + Group: group.Name, + Type: metric.Type, + Slope: metric.Slope, + Tmax: metric.Tmax, + Unit: metric.Unit, + Value: valueStr, + } + } + } + } + return GangliaMetricConfig{ + Group: "", + Type: "", + Slope: "", + Tmax: 0, + Unit: "", + Value: "", + } +} + +func GetGangliaConfig(point lp.CCMetric) GangliaMetricConfig { + group := "" + if g, ok := point.GetMeta("group"); ok { + group = g + } + unit := "" + if u, ok := point.GetMeta("unit"); ok { + unit = u + } + valueType := "double" + valueStr := "" + value, ok := point.GetField("value") + if ok { + switch real := value.(type) { + case float64: + valueStr = fmt.Sprintf("%f", real) + valueType = "double" + case float32: + valueStr = fmt.Sprintf("%f", real) + valueType = "float" + case int64: + valueStr = fmt.Sprintf("%d", real) + valueType = "int32" + case int32: + valueStr = fmt.Sprintf("%d", real) + valueType = "int32" + case int: + valueStr = fmt.Sprintf("%d", real) + valueType = "int32" + case uint64: + valueStr = fmt.Sprintf("%d", real) + valueType = "uint32" + case uint32: + valueStr = fmt.Sprintf("%d", real) + valueType = "uint32" + case uint: + valueStr = fmt.Sprintf("%d", real) + valueType = "uint32" + case string: + valueStr = real + valueType = "string" + default: + valueType = "invalid" + } + } + + return GangliaMetricConfig{ + Group: group, + Type: valueType, + Slope: DEFAULT_GANGLIA_METRIC_SLOPE, + Tmax: DEFAULT_GANGLIA_METRIC_TMAX, + Unit: unit, + Value: valueStr, + } +} diff --git a/sinks/gangliaSink.go b/sinks/gangliaSink.go index fa95f43..22096af 100644 --- a/sinks/gangliaSink.go +++ b/sinks/gangliaSink.go @@ -24,6 +24,7 @@ type GangliaSinkConfig struct { AddTagsAsDesc bool `json:"add_tags_as_desc,omitempty"` ClusterName string `json:"cluster_name,omitempty"` AddTypeToName bool `json:"add_type_to_name,omitempty"` + AddUnits bool `json:"add_units,omitempty"` } type GangliaSink struct { @@ -33,16 +34,73 @@ type GangliaSink struct { config GangliaSinkConfig } -func (s *GangliaSink) Init(config json.RawMessage) error { +func (s *GangliaSink) Write(point lp.CCMetric) error { var err error = nil - s.name = "GangliaSink" + //var tagsstr []string + var argstr []string + + // Get metric name + metricname := GangliaMetricRename(point.Name()) + + // Get metric config (type, value, ... in suitable format) + conf := GetCommonGangliaConfig(point) + if len(conf.Type) == 0 { + conf = GetGangliaConfig(point) + } + if len(conf.Type) == 0 { + return fmt.Errorf("metric %s has no 'value' field", metricname) + } + + if s.config.AddGangliaGroup { + argstr = append(argstr, fmt.Sprintf("--group=%s", conf.Group)) + } + if s.config.AddUnits && len(conf.Unit) > 0 { + argstr = append(argstr, fmt.Sprintf("--units=%s", conf.Unit)) + } + + if len(s.config.ClusterName) > 0 { + argstr = append(argstr, fmt.Sprintf("--cluster=%s", s.config.ClusterName)) + } + // if s.config.AddTagsAsDesc && len(tagsstr) > 0 { + // argstr = append(argstr, fmt.Sprintf("--desc=%q", strings.Join(tagsstr, ","))) + // } + if len(s.gmetric_config) > 0 { + argstr = append(argstr, fmt.Sprintf("--conf=%s", s.gmetric_config)) + } + if s.config.AddTypeToName { + argstr = append(argstr, fmt.Sprintf("--name=%s", GangliaMetricName(point))) + } else { + argstr = append(argstr, fmt.Sprintf("--name=%s", metricname)) + } + argstr = append(argstr, fmt.Sprintf("--slope=%s", conf.Slope)) + argstr = append(argstr, fmt.Sprintf("--value=%s", conf.Value)) + argstr = append(argstr, fmt.Sprintf("--type=%s", conf.Type)) + argstr = append(argstr, fmt.Sprintf("--tmax=%d", conf.Tmax)) + + cclog.ComponentDebug(s.name, s.gmetric_path, strings.Join(argstr, " ")) + command := exec.Command(s.gmetric_path, argstr...) + command.Wait() + _, err = command.Output() + return err +} + +func (s *GangliaSink) Flush() error { + return nil +} + +func (s *GangliaSink) Close() { +} + +func NewGangliaSink(name string, config json.RawMessage) (Sink, error) { + s := new(GangliaSink) + s.name = fmt.Sprintf("GangliaSink(%s)", name) s.config.AddTagsAsDesc = false s.config.AddGangliaGroup = false if len(config) > 0 { err := json.Unmarshal(config, &s.config) if err != nil { cclog.ComponentError(s.name, "Error reading config for", s.name, ":", err.Error()) - return err + return nil, err } } s.gmetric_path = "" @@ -60,98 +118,10 @@ func (s *GangliaSink) Init(config json.RawMessage) error { } } if len(s.gmetric_path) == 0 { - err = errors.New("cannot find executable 'gmetric'") + return nil, errors.New("cannot find executable 'gmetric'") } if len(s.config.GmetricConfig) > 0 { s.gmetric_config = s.config.GmetricConfig } - return err -} - -func (s *GangliaSink) Write(point lp.CCMetric) error { - var err error = nil - var tagsstr []string - var argstr []string - if s.config.AddGangliaGroup { - if point.HasTag("group") { - g, _ := point.GetTag("group") - argstr = append(argstr, fmt.Sprintf("--group=%s", g)) - } else if point.HasMeta("group") { - g, _ := point.GetMeta("group") - argstr = append(argstr, fmt.Sprintf("--group=%s", g)) - } - } - - for key, value := range point.Tags() { - switch key { - case "unit": - argstr = append(argstr, fmt.Sprintf("--units=%s", value)) - default: - tagsstr = append(tagsstr, fmt.Sprintf("%s=%s", key, value)) - } - } - if s.config.MetaAsTags { - for key, value := range point.Meta() { - switch key { - case "unit": - argstr = append(argstr, fmt.Sprintf("--units=%s", value)) - default: - tagsstr = append(tagsstr, fmt.Sprintf("%s=%s", key, value)) - } - } - } - if len(s.config.ClusterName) > 0 { - argstr = append(argstr, fmt.Sprintf("--cluster=%s", s.config.ClusterName)) - } - if s.config.AddTagsAsDesc && len(tagsstr) > 0 { - argstr = append(argstr, fmt.Sprintf("--desc=%q", strings.Join(tagsstr, ","))) - } - if len(s.gmetric_config) > 0 { - argstr = append(argstr, fmt.Sprintf("--conf=%s", s.gmetric_config)) - } - name := GangliaMetricRename(point) - if s.config.AddTypeToName { - argstr = append(argstr, fmt.Sprintf("--name=%s", GangliaMetricName(point))) - } else { - argstr = append(argstr, fmt.Sprintf("--name=%s", name)) - } - slope := GangliaSlopeType(point) - slopeStr := "both" - if slope == 0 { - slopeStr = "zero" - } - argstr = append(argstr, fmt.Sprintf("--slope=%s", slopeStr)) - - for k, v := range point.Fields() { - if k == "value" { - switch value := v.(type) { - case float64: - argstr = append(argstr, - fmt.Sprintf("--value=%v", value), "--type=double") - case float32: - argstr = append(argstr, - fmt.Sprintf("--value=%v", value), "--type=float") - case int: - argstr = append(argstr, - fmt.Sprintf("--value=%d", value), "--type=int32") - case int64: - argstr = append(argstr, - fmt.Sprintf("--value=%d", value), "--type=int32") - case string: - argstr = append(argstr, - fmt.Sprintf("--value=%q", value), "--type=string") - } - } - } - command := exec.Command(s.gmetric_path, argstr...) - command.Wait() - _, err = command.Output() - return err -} - -func (s *GangliaSink) Flush() error { - return nil -} - -func (s *GangliaSink) Close() { + return s, nil } diff --git a/sinks/httpSink.go b/sinks/httpSink.go index ce46bab..c2dd2ea 100644 --- a/sinks/httpSink.go +++ b/sinks/httpSink.go @@ -38,57 +38,6 @@ type HttpSink struct { flushDelay time.Duration } -func (s *HttpSink) Init(config json.RawMessage) error { - // Set default values - s.name = "HttpSink" - s.config.MaxIdleConns = 10 - s.config.IdleConnTimeout = "5s" - s.config.Timeout = "5s" - s.config.FlushDelay = "1s" - - // Read config - if len(config) > 0 { - err := json.Unmarshal(config, &s.config) - if err != nil { - return err - } - } - if len(s.config.URL) == 0 { - return errors.New("`url` config option is required for HTTP sink") - } - if s.config.MaxIdleConns > 0 { - s.maxIdleConns = s.config.MaxIdleConns - } - if len(s.config.IdleConnTimeout) > 0 { - t, err := time.ParseDuration(s.config.IdleConnTimeout) - if err == nil { - s.idleConnTimeout = t - } - } - if len(s.config.Timeout) > 0 { - t, err := time.ParseDuration(s.config.Timeout) - if err == nil { - s.timeout = t - } - } - if len(s.config.FlushDelay) > 0 { - t, err := time.ParseDuration(s.config.FlushDelay) - if err == nil { - s.flushDelay = t - } - } - tr := &http.Transport{ - MaxIdleConns: s.maxIdleConns, - IdleConnTimeout: s.idleConnTimeout, - } - s.client = &http.Client{Transport: tr, Timeout: s.timeout} - s.buffer = &bytes.Buffer{} - s.encoder = influx.NewEncoder(s.buffer) - s.encoder.SetPrecision(time.Second) - - return nil -} - func (s *HttpSink) Write(m lp.CCMetric) error { if s.buffer.Len() == 0 && s.flushDelay != 0 { // This is the first write since the last flush, start the flushTimer! @@ -169,3 +118,54 @@ func (s *HttpSink) Close() { } s.client.CloseIdleConnections() } + +func NewHttpSink(name string, config json.RawMessage) (Sink, error) { + s := new(HttpSink) + // Set default values + s.name = fmt.Sprintf("HttpSink(%s)", name) + s.config.MaxIdleConns = 10 + s.config.IdleConnTimeout = "5s" + s.config.Timeout = "5s" + s.config.FlushDelay = "1s" + + // Read config + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + return nil, err + } + } + if len(s.config.URL) == 0 { + return nil, errors.New("`url` config option is required for HTTP sink") + } + if s.config.MaxIdleConns > 0 { + s.maxIdleConns = s.config.MaxIdleConns + } + if len(s.config.IdleConnTimeout) > 0 { + t, err := time.ParseDuration(s.config.IdleConnTimeout) + if err == nil { + s.idleConnTimeout = t + } + } + if len(s.config.Timeout) > 0 { + t, err := time.ParseDuration(s.config.Timeout) + if err == nil { + s.timeout = t + } + } + if len(s.config.FlushDelay) > 0 { + t, err := time.ParseDuration(s.config.FlushDelay) + if err == nil { + s.flushDelay = t + } + } + tr := &http.Transport{ + MaxIdleConns: s.maxIdleConns, + IdleConnTimeout: s.idleConnTimeout, + } + s.client = &http.Client{Transport: tr, Timeout: s.timeout} + s.buffer = &bytes.Buffer{} + s.encoder = influx.NewEncoder(s.buffer) + s.encoder.SetPrecision(time.Second) + return s, nil +} diff --git a/sinks/influxAsyncSink.go b/sinks/influxAsyncSink.go index 20aa60c..7b38873 100644 --- a/sinks/influxAsyncSink.go +++ b/sinks/influxAsyncSink.go @@ -1,6 +1,7 @@ package sinks import ( + "context" "crypto/tls" "encoding/json" "errors" @@ -30,11 +31,10 @@ type InfluxAsyncSinkConfig struct { type InfluxAsyncSink struct { sink - client influxdb2.Client - writeApi influxdb2Api.WriteAPI - retPolicy string - errors <-chan error - config InfluxAsyncSinkConfig + client influxdb2.Client + writeApi influxdb2Api.WriteAPI + errors <-chan error + config InfluxAsyncSinkConfig } func (s *InfluxAsyncSink) connect() error { @@ -65,42 +65,16 @@ func (s *InfluxAsyncSink) connect() error { ) s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions) s.writeApi = s.client.WriteAPI(s.config.Organization, s.config.Database) + ok, err := s.client.Ping(context.Background()) + if err != nil { + return err + } + if !ok { + return fmt.Errorf("connection to %s not healthy", uri) + } return nil } -func (s *InfluxAsyncSink) Init(config json.RawMessage) error { - s.name = "InfluxSink" - - // Set default for maximum number of points sent to server in single request. - s.config.BatchSize = 100 - - if len(config) > 0 { - err := json.Unmarshal(config, &s.config) - if err != nil { - return err - } - } - if len(s.config.Host) == 0 || - len(s.config.Port) == 0 || - len(s.config.Database) == 0 || - len(s.config.Organization) == 0 || - len(s.config.Password) == 0 { - return errors.New("not all configuration variables set required by InfluxAsyncSink") - } - - // Connect to InfluxDB server - err := s.connect() - - // Start background: Read from error channel - s.errors = s.writeApi.Errors() - go func() { - for err := range s.errors { - cclog.ComponentError(s.name, err.Error()) - } - }() - return err -} - func (s *InfluxAsyncSink) Write(m lp.CCMetric) error { s.writeApi.WritePoint( m.ToPoint(s.config.MetaAsTags), @@ -118,3 +92,40 @@ func (s *InfluxAsyncSink) Close() { s.writeApi.Flush() s.client.Close() } + +func NewInfluxAsyncSink(name string, config json.RawMessage) (Sink, error) { + s := new(InfluxAsyncSink) + s.name = fmt.Sprintf("InfluxSink(%s)", name) + + // Set default for maximum number of points sent to server in single request. + s.config.BatchSize = 100 + + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + return nil, err + } + } + if len(s.config.Host) == 0 || + len(s.config.Port) == 0 || + len(s.config.Database) == 0 || + len(s.config.Organization) == 0 || + len(s.config.Password) == 0 { + return nil, errors.New("not all configuration variables set required by InfluxAsyncSink") + } + + // Connect to InfluxDB server + if err := s.connect(); err != nil { + return nil, fmt.Errorf("unable to connect: %v", err) + } + + // Start background: Read from error channel + s.errors = s.writeApi.Errors() + go func() { + for err := range s.errors { + cclog.ComponentError(s.name, err.Error()) + } + }() + + return s, nil +} diff --git a/sinks/influxSink.go b/sinks/influxSink.go index 99304c0..11859b2 100644 --- a/sinks/influxSink.go +++ b/sinks/influxSink.go @@ -54,29 +54,16 @@ func (s *InfluxSink) connect() error { ) s.client = influxdb2.NewClientWithOptions(uri, auth, clientOptions) s.writeApi = s.client.WriteAPIBlocking(s.config.Organization, s.config.Database) + ok, err := s.client.Ping(context.Background()) + if err != nil { + return err + } + if !ok { + return fmt.Errorf("connection to %s not healthy", uri) + } return nil } -func (s *InfluxSink) Init(config json.RawMessage) error { - s.name = "InfluxSink" - if len(config) > 0 { - err := json.Unmarshal(config, &s.config) - if err != nil { - return err - } - } - if len(s.config.Host) == 0 || - len(s.config.Port) == 0 || - len(s.config.Database) == 0 || - len(s.config.Organization) == 0 || - len(s.config.Password) == 0 { - return errors.New("not all configuration variables set required by InfluxSink") - } - - // Connect to InfluxDB server - return s.connect() -} - func (s *InfluxSink) Write(m lp.CCMetric) error { err := s.writeApi.WritePoint( @@ -94,3 +81,27 @@ func (s *InfluxSink) Close() { cclog.ComponentDebug(s.name, "Closing InfluxDB connection") s.client.Close() } + +func NewInfluxSink(name string, config json.RawMessage) (Sink, error) { + s := new(InfluxSink) + s.name = fmt.Sprintf("InfluxSink(%s)", name) + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + return nil, err + } + } + if len(s.config.Host) == 0 || + len(s.config.Port) == 0 || + len(s.config.Database) == 0 || + len(s.config.Organization) == 0 || + len(s.config.Password) == 0 { + return nil, errors.New("not all configuration variables set required by InfluxSink") + } + + // Connect to InfluxDB server + if err := s.connect(); err != nil { + return nil, fmt.Errorf("unable to connect: %v", err) + } + return s, nil +} diff --git a/sinks/libgangliaSink.go b/sinks/libgangliaSink.go index 051bd03..1fc7863 100644 --- a/sinks/libgangliaSink.go +++ b/sinks/libgangliaSink.go @@ -82,21 +82,21 @@ const ( GMOND_CONFIG_FILE = `/etc/ganglia/gmond.conf` ) -type LibgangliaSinkSpecialMetric struct { - MetricName string `json:"metric_name,omitempty"` - NewName string `json:"new_name,omitempty"` - Slope string `json:"slope,omitempty"` -} +// type LibgangliaSinkSpecialMetric struct { +// MetricName string `json:"metric_name,omitempty"` +// NewName string `json:"new_name,omitempty"` +// Slope string `json:"slope,omitempty"` +// } type LibgangliaSinkConfig struct { defaultSinkConfig - GangliaLib string `json:"libganglia_path,omitempty"` - GmondConfig string `json:"gmond_config,omitempty"` - AddGangliaGroup bool `json:"add_ganglia_group,omitempty"` - AddTypeToName bool `json:"add_type_to_name,omitempty"` - AddUnits bool `json:"add_units,omitempty"` - ClusterName string `json:"cluster_name,omitempty"` - SpecialMetrics map[string]LibgangliaSinkSpecialMetric `json:"rename_metrics,omitempty"` // Map to rename metric name from key to value + GangliaLib string `json:"libganglia_path,omitempty"` + GmondConfig string `json:"gmond_config,omitempty"` + AddGangliaGroup bool `json:"add_ganglia_group,omitempty"` + AddTypeToName bool `json:"add_type_to_name,omitempty"` + AddUnits bool `json:"add_units,omitempty"` + ClusterName string `json:"cluster_name,omitempty"` + //SpecialMetrics map[string]LibgangliaSinkSpecialMetric `json:"rename_metrics,omitempty"` // Map to rename metric name from key to value //AddTagsAsDesc bool `json:"add_tags_as_desc,omitempty"` } @@ -109,65 +109,6 @@ type LibgangliaSink struct { cstrCache map[string]*C.char } -func (s *LibgangliaSink) Init(config json.RawMessage) error { - var err error = nil - s.name = "LibgangliaSink" - //s.config.AddTagsAsDesc = false - s.config.AddGangliaGroup = false - s.config.AddTypeToName = false - s.config.AddUnits = true - s.config.GmondConfig = string(GMOND_CONFIG_FILE) - s.config.GangliaLib = string(GANGLIA_LIB_NAME) - if len(config) > 0 { - err = json.Unmarshal(config, &s.config) - if err != nil { - cclog.ComponentError(s.name, "Error reading config:", err.Error()) - return err - } - } - lib := dl.New(s.config.GangliaLib, GANGLIA_LIB_DL_FLAGS) - if lib == nil { - return fmt.Errorf("error instantiating DynamicLibrary for %s", s.config.GangliaLib) - } - err = lib.Open() - if err != nil { - return fmt.Errorf("error opening %s: %v", s.config.GangliaLib, err) - } - - // Set up cache for the C strings - s.cstrCache = make(map[string]*C.char) - // s.cstrCache["globals"] = C.CString("globals") - - // s.cstrCache["override_hostname"] = C.CString("override_hostname") - // s.cstrCache["override_ip"] = C.CString("override_ip") - - // Add some constant strings - s.cstrCache["GROUP"] = C.CString("GROUP") - s.cstrCache["CLUSTER"] = C.CString("CLUSTER") - s.cstrCache[""] = C.CString("") - - // Add cluster name for lookup in Write() - if len(s.config.ClusterName) > 0 { - s.cstrCache[s.config.ClusterName] = C.CString(s.config.ClusterName) - } - // Add supported types for later lookup in Write() - s.cstrCache["double"] = C.CString("double") - s.cstrCache["int32"] = C.CString("int32") - s.cstrCache["string"] = C.CString("string") - - // Create Ganglia pool - s.global_context = C.Ganglia_pool_create(nil) - // Load Ganglia configuration - s.cstrCache[s.config.GmondConfig] = C.CString(s.config.GmondConfig) - s.gmond_config = C.Ganglia_gmond_config_create(s.cstrCache[s.config.GmondConfig], 0) - //globals := C.cfg_getsec(gmond_config, s.cstrCache["globals"]) - //override_hostname := C.cfg_getstr(globals, s.cstrCache["override_hostname"]) - //override_ip := C.cfg_getstr(globals, s.cstrCache["override_ip"]) - - s.send_channels = C.Ganglia_udp_send_channels_create(s.global_context, s.gmond_config) - return nil -} - func (s *LibgangliaSink) Write(point lp.CCMetric) error { var err error = nil var c_name *C.char @@ -184,72 +125,48 @@ func (s *LibgangliaSink) Write(point lp.CCMetric) error { } // Get metric name - metricname := GangliaMetricRename(point) - if s.config.AddTypeToName { - c_name = lookup(GangliaMetricName(point)) - } else { - c_name = lookup(metricname) - } + metricname := GangliaMetricRename(point.Name()) - // Get the value C string and lookup the type string in the cache - value, ok := point.GetField("value") - if !ok { + conf := GetCommonGangliaConfig(point) + if len(conf.Type) == 0 { + conf = GetGangliaConfig(point) + } + if len(conf.Type) == 0 { return fmt.Errorf("metric %s has no 'value' field", metricname) } - switch real := value.(type) { - case float64: - c_value = C.CString(fmt.Sprintf("%f", real)) - c_type = lookup("double") - case float32: - c_value = C.CString(fmt.Sprintf("%f", real)) - c_type = lookup("float") - case int64: - c_value = C.CString(fmt.Sprintf("%d", real)) - c_type = lookup("int32") - case int32: - c_value = C.CString(fmt.Sprintf("%d", real)) - c_type = lookup("int32") - case int: - c_value = C.CString(fmt.Sprintf("%d", real)) - c_type = lookup("int32") - case string: - c_value = C.CString(real) - c_type = lookup("string") - default: - return fmt.Errorf("metric %s has invalid 'value' type for %s", point.Name(), s.name) + + if s.config.AddTypeToName { + metricname = GangliaMetricName(point) } + c_value = C.CString(conf.Value) + c_type = lookup(conf.Type) + c_name = lookup(metricname) + // Add unit + unit := "" if s.config.AddUnits { - if tagunit, tagok := point.GetTag("unit"); tagok { - c_unit = lookup(tagunit) - } else if metaunit, metaok := point.GetMeta("unit"); metaok { - c_unit = lookup(metaunit) - } else { - c_unit = lookup("") - } - } else { - c_unit = lookup("") + unit = conf.Unit } + c_unit = lookup(unit) // Determine the slope of the metric. Ganglia's own collector mostly use // 'both' but the mem and swap total uses 'zero'. - slope := GangliaSlopeType(point) slope_type := C.GANGLIA_SLOPE_BOTH - switch slope { - case 0: + switch conf.Slope { + case "zero": slope_type = C.GANGLIA_SLOPE_ZERO + case "both": + slope_type = C.GANGLIA_SLOPE_BOTH } // Create a new Ganglia metric gmetric := C.Ganglia_metric_create(s.global_context) // Set name, value, type and unit in the Ganglia metric - // Since we don't have this information from the collectors, - // we assume that the metric value can go up and down (slope), - // and there is no maximum for 'dmax' and 'tmax'. - // Ganglia's collectors set 'tmax' but not 'dmax' + // The default slope_type is both directions, so up and down. Some metrics want 'zero' slope, probably constant. + // The 'tmax' value is by default 300. rval := C.int(0) - rval = C.Ganglia_metric_set(gmetric, c_name, c_value, c_type, c_unit, C.uint(slope_type), 0, 0) + rval = C.Ganglia_metric_set(gmetric, c_name, c_value, c_type, c_unit, C.uint(slope_type), C.uint(conf.Tmax), 0) switch rval { case 1: C.free(unsafe.Pointer(c_value)) @@ -259,10 +176,10 @@ func (s *LibgangliaSink) Write(point lp.CCMetric) error { return errors.New("one of your parameters has an invalid character '\"'") case 3: C.free(unsafe.Pointer(c_value)) - return fmt.Errorf("the type parameter \"%s\" is not a valid type", C.GoString(c_type)) + return fmt.Errorf("the type parameter \"%s\" is not a valid type", conf.Type) case 4: C.free(unsafe.Pointer(c_value)) - return fmt.Errorf("the value parameter \"%s\" does not represent a number", C.GoString(c_value)) + return fmt.Errorf("the value parameter \"%s\" does not represent a number", conf.Value) default: } @@ -271,8 +188,8 @@ func (s *LibgangliaSink) Write(point lp.CCMetric) error { C.Ganglia_metadata_add(gmetric, lookup("CLUSTER"), lookup(s.config.ClusterName)) } // Set the group metadata in the Ganglia metric if configured - if group, ok := point.GetMeta("group"); ok && s.config.AddGangliaGroup { - c_group := lookup(group) + if s.config.AddGangliaGroup { + c_group := lookup(conf.Group) C.Ganglia_metadata_add(gmetric, lookup("GROUP"), c_group) } @@ -307,3 +224,63 @@ func (s *LibgangliaSink) Close() { C.free(unsafe.Pointer(cstr)) } } + +func NewLibgangliaSink(name string, config json.RawMessage) (Sink, error) { + s := new(LibgangliaSink) + var err error = nil + s.name = fmt.Sprintf("LibgangliaSink(%s)", name) + //s.config.AddTagsAsDesc = false + s.config.AddGangliaGroup = false + s.config.AddTypeToName = false + s.config.AddUnits = true + s.config.GmondConfig = string(GMOND_CONFIG_FILE) + s.config.GangliaLib = string(GANGLIA_LIB_NAME) + if len(config) > 0 { + err = json.Unmarshal(config, &s.config) + if err != nil { + cclog.ComponentError(s.name, "Error reading config:", err.Error()) + return nil, err + } + } + lib := dl.New(s.config.GangliaLib, GANGLIA_LIB_DL_FLAGS) + if lib == nil { + return nil, fmt.Errorf("error instantiating DynamicLibrary for %s", s.config.GangliaLib) + } + err = lib.Open() + if err != nil { + return nil, fmt.Errorf("error opening %s: %v", s.config.GangliaLib, err) + } + + // Set up cache for the C strings + s.cstrCache = make(map[string]*C.char) + // s.cstrCache["globals"] = C.CString("globals") + + // s.cstrCache["override_hostname"] = C.CString("override_hostname") + // s.cstrCache["override_ip"] = C.CString("override_ip") + + // Add some constant strings + s.cstrCache["GROUP"] = C.CString("GROUP") + s.cstrCache["CLUSTER"] = C.CString("CLUSTER") + s.cstrCache[""] = C.CString("") + + // Add cluster name for lookup in Write() + if len(s.config.ClusterName) > 0 { + s.cstrCache[s.config.ClusterName] = C.CString(s.config.ClusterName) + } + // Add supported types for later lookup in Write() + s.cstrCache["double"] = C.CString("double") + s.cstrCache["int32"] = C.CString("int32") + s.cstrCache["string"] = C.CString("string") + + // Create Ganglia pool + s.global_context = C.Ganglia_pool_create(nil) + // Load Ganglia configuration + s.cstrCache[s.config.GmondConfig] = C.CString(s.config.GmondConfig) + s.gmond_config = C.Ganglia_gmond_config_create(s.cstrCache[s.config.GmondConfig], 0) + //globals := C.cfg_getsec(gmond_config, s.cstrCache["globals"]) + //override_hostname := C.cfg_getstr(globals, s.cstrCache["override_hostname"]) + //override_ip := C.cfg_getstr(globals, s.cstrCache["override_ip"]) + + s.send_channels = C.Ganglia_udp_send_channels_create(s.global_context, s.gmond_config) + return s, nil +} diff --git a/sinks/metricSink.go b/sinks/metricSink.go index d76f5f2..8fe02d7 100644 --- a/sinks/metricSink.go +++ b/sinks/metricSink.go @@ -1,8 +1,6 @@ package sinks import ( - "encoding/json" - lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" ) @@ -17,7 +15,6 @@ type sink struct { } type Sink interface { - Init(config json.RawMessage) error Write(point lp.CCMetric) error Flush() error Close() diff --git a/sinks/natsSink.go b/sinks/natsSink.go index 187157e..0d7987e 100644 --- a/sinks/natsSink.go +++ b/sinks/natsSink.go @@ -53,30 +53,6 @@ func (s *NatsSink) connect() error { return nil } -func (s *NatsSink) Init(config json.RawMessage) error { - s.name = "NatsSink" - if len(config) > 0 { - err := json.Unmarshal(config, &s.config) - if err != nil { - cclog.ComponentError(s.name, "Error reading config for", s.name, ":", err.Error()) - return err - } - } - if len(s.config.Host) == 0 || - len(s.config.Port) == 0 || - len(s.config.Database) == 0 { - return errors.New("not all configuration variables set required by NatsSink") - } - // Setup Influx line protocol - s.buffer = &bytes.Buffer{} - s.buffer.Grow(1025) - s.encoder = influx.NewEncoder(s.buffer) - s.encoder.SetPrecision(time.Second) - s.encoder.SetMaxLineBytes(1024) - // Setup infos for connection - return s.connect() -} - func (s *NatsSink) Write(m lp.CCMetric) error { if s.client != nil { _, err := s.encoder.Encode(m.ToPoint(s.config.MetaAsTags)) @@ -105,3 +81,31 @@ func (s *NatsSink) Close() { s.client.Close() } } + +func NewNatsSink(name string, config json.RawMessage) (Sink, error) { + s := new(NatsSink) + s.name = fmt.Sprintf("NatsSink(%s)", name) + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + cclog.ComponentError(s.name, "Error reading config for", s.name, ":", err.Error()) + return nil, err + } + } + if len(s.config.Host) == 0 || + len(s.config.Port) == 0 || + len(s.config.Database) == 0 { + return nil, errors.New("not all configuration variables set required by NatsSink") + } + // Setup Influx line protocol + s.buffer = &bytes.Buffer{} + s.buffer.Grow(1025) + s.encoder = influx.NewEncoder(s.buffer) + s.encoder.SetPrecision(time.Second) + s.encoder.SetMaxLineBytes(1024) + // Setup infos for connection + if err := s.connect(); err != nil { + return nil, fmt.Errorf("Unable to connect: %v", err) + } + return s, nil +} diff --git a/sinks/prometheusSink.go b/sinks/prometheusSink.go new file mode 100644 index 0000000..5011ac0 --- /dev/null +++ b/sinks/prometheusSink.go @@ -0,0 +1,199 @@ +package sinks + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "net/http" + "strings" + "sync" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" + "github.com/gorilla/mux" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" +) + +type PrometheusSinkConfig struct { + defaultSinkConfig + Host string `json:"host,omitempty"` + Port string `json:"port"` + Path string `json:"path,omitempty"` + GroupAsNameSpace bool `json:"group_as_namespace,omitempty"` + // User string `json:"user,omitempty"` + // Password string `json:"password,omitempty"` + // FlushDelay string `json:"flush_delay,omitempty"` +} + +type PrometheusSink struct { + sink + config PrometheusSinkConfig + labelMetrics map[string]*prometheus.GaugeVec + nodeMetrics map[string]prometheus.Gauge + promWg sync.WaitGroup + promServer *http.Server +} + +func intToFloat64(input interface{}) (float64, error) { + switch value := input.(type) { + case float64: + return value, nil + case float32: + return float64(value), nil + case int: + return float64(value), nil + case int32: + return float64(value), nil + case int64: + return float64(value), nil + } + return 0, errors.New("cannot cast value to float64") +} + +func getLabelValue(metric lp.CCMetric) []string { + labelValues := []string{} + if tid, tidok := metric.GetTag("type-id"); tidok && metric.HasTag("type") { + labelValues = append(labelValues, tid) + } + if d, ok := metric.GetTag("device"); ok { + labelValues = append(labelValues, d) + } else if d, ok := metric.GetMeta("device"); ok { + labelValues = append(labelValues, d) + } + return labelValues +} + +func getLabelNames(metric lp.CCMetric) []string { + labelNames := []string{} + if t, tok := metric.GetTag("type"); tok && metric.HasTag("type-id") { + labelNames = append(labelNames, t) + } + if _, ok := metric.GetTag("device"); ok { + labelNames = append(labelNames, "device") + } else if _, ok := metric.GetMeta("device"); ok { + labelNames = append(labelNames, "device") + } + return labelNames +} + +func (s *PrometheusSink) newMetric(metric lp.CCMetric) error { + var value float64 = 0 + name := metric.Name() + opts := prometheus.GaugeOpts{ + Name: name, + } + labels := getLabelNames(metric) + labelValues := getLabelValue(metric) + if len(labels) > 0 && len(labels) != len(labelValues) { + return fmt.Errorf("cannot detect metric labels for metric %s", name) + } + + if metricValue, ok := metric.GetField("value"); ok { + if floatValue, err := intToFloat64(metricValue); err == nil { + value = floatValue + } else { + return fmt.Errorf("metric %s with value '%v' cannot be casted to float64", name, metricValue) + } + } + if s.config.GroupAsNameSpace && metric.HasMeta("group") { + g, _ := metric.GetMeta("group") + opts.Namespace = strings.ToLower(g) + } + + if len(labels) > 0 { + new := prometheus.NewGaugeVec(opts, labels) + new.WithLabelValues(labelValues...).Set(value) + s.labelMetrics[name] = new + prometheus.Register(new) + } else { + new := prometheus.NewGauge(opts) + new.Set(value) + s.nodeMetrics[name] = new + prometheus.Register(new) + } + return nil +} + +func (s *PrometheusSink) updateMetric(metric lp.CCMetric) error { + var value float64 = 0.0 + name := metric.Name() + labelValues := getLabelValue(metric) + + if metricValue, ok := metric.GetField("value"); ok { + if floatValue, err := intToFloat64(metricValue); err == nil { + value = floatValue + } else { + return fmt.Errorf("metric %s with value '%v' cannot be casted to float64", name, metricValue) + } + } + + if len(labelValues) > 0 { + if _, ok := s.labelMetrics[name]; !ok { + err := s.newMetric(metric) + if err != nil { + return err + } + } + s.labelMetrics[name].WithLabelValues(labelValues...).Set(value) + } else { + if _, ok := s.labelMetrics[name]; !ok { + err := s.newMetric(metric) + if err != nil { + return err + } + } + s.nodeMetrics[name].Set(value) + } + return nil +} + +func (s *PrometheusSink) Write(m lp.CCMetric) error { + return s.updateMetric(m) +} + +func (s *PrometheusSink) Flush() error { + return nil +} + +func (s *PrometheusSink) Close() { + cclog.ComponentDebug(s.name, "CLOSE") + s.promServer.Shutdown(context.Background()) + s.promWg.Wait() +} + +func NewPrometheusSink(name string, config json.RawMessage) (Sink, error) { + s := new(PrometheusSink) + s.name = "PrometheusSink" + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + cclog.ComponentError(s.name, "Error reading config for", s.name, ":", err.Error()) + return nil, err + } + } + if len(s.config.Port) == 0 { + err := errors.New("not all configuration variables set required by PrometheusSink") + cclog.ComponentError(s.name, err.Error()) + return nil, err + } + s.labelMetrics = make(map[string]*prometheus.GaugeVec) + s.nodeMetrics = make(map[string]prometheus.Gauge) + s.promWg.Add(1) + go func() { + router := mux.NewRouter() + // Prometheus endpoint + router.Path("/" + s.config.Path).Handler(promhttp.Handler()) + + url := fmt.Sprintf("%s:%s", s.config.Host, s.config.Port) + cclog.ComponentDebug(s.name, "Serving Prometheus metrics at", fmt.Sprintf("%s:%s/%s", s.config.Host, s.config.Port, s.config.Path)) + s.promServer = &http.Server{Addr: url, Handler: router} + err := s.promServer.ListenAndServe() + if err != nil && err.Error() != "http: Server closed" { + cclog.ComponentError(s.name, err.Error()) + } + s.promWg.Done() + }() + return s, nil +} diff --git a/sinks/prometheusSink.md b/sinks/prometheusSink.md new file mode 100644 index 0000000..0996c43 --- /dev/null +++ b/sinks/prometheusSink.md @@ -0,0 +1,23 @@ +## `prometheus` sink + +The `prometheus` sink publishes all metrics via an HTTP server ready to be scraped by a [Prometheus](https://prometheus.io) server. It creates gauge metrics for all node metrics and gauge vectors for all metrics with a subtype like 'device', 'cpu' or 'socket'. + + +### Configuration structure + +```json +{ + "": { + "type": "prometheus", + "host": "localhost", + "port": "8080", + "path": "metrics" + } +} +``` + +- `type`: makes the sink an `prometheus` sink +- `host`: The HTTP server gets bound to that IP/hostname +- `port`: Portnumber (as string) for the HTTP server +- `path`: Path where the metrics should be servered. The metrics will be published at `host`:`port`/`path` +- `group_as_namespace`: Most metrics contain a group as meta information like 'memory', 'load'. With this the metric names are extended to `group`_`name` if possible. diff --git a/sinks/sampleSink.go b/sinks/sampleSink.go new file mode 100644 index 0000000..be0196a --- /dev/null +++ b/sinks/sampleSink.go @@ -0,0 +1,62 @@ +package sinks + +import ( + "encoding/json" + "fmt" + "log" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +type SampleSinkConfig struct { + defaultSinkConfig // defines JSON tags for 'type' and 'meta_as_tags' + // Add additional options +} + +type SampleSink struct { + sink // declarate 'name' and 'meta_as_tags' + config SampleSinkConfig // entry point to the SampleSinkConfig +} + +// Code to submit a single CCMetric to the sink +func (s *SampleSink) Write(point lp.CCMetric) error { + log.Print(point) + return nil +} + +// If the sink uses batched sends internally, you can tell to flush its buffers +func (s *SampleSink) Flush() error { + return nil +} + +// Close sink: close network connection, close files, close libraries, ... +func (s *SampleSink) Close() { + cclog.ComponentDebug(s.name, "CLOSE") +} + +// New function to create a new instance of the sink +// Initialize the sink by giving it a name and reading in the config JSON +func NewSampleSink(name string, config json.RawMessage) (Sink, error) { + s := new(SampleSink) + s.name = fmt.Sprintf("SampleSink(%s)", name) // Always specify a name here + + // Set defaults in s.config + + // Read in the config JSON + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + return nil, err + } + } + + // Check if all required fields in the config are set + // Use 'len(s.config.Option) > 0' for string settings + + // Establish connection to the server, library, ... + // Check required files exist and lookup path(s) of executable(s) + + // Return (nil, meaningful error message) in case of errors + return s, nil +} diff --git a/sinks/sinkManager.go b/sinks/sinkManager.go index 487e7ca..f531f5d 100644 --- a/sinks/sinkManager.go +++ b/sinks/sinkManager.go @@ -13,14 +13,14 @@ import ( const SINK_MAX_FORWARD = 50 // Map of all available sinks -var AvailableSinks = map[string]Sink{ - "influxdb": new(InfluxSink), - "stdout": new(StdoutSink), - "nats": new(NatsSink), - "http": new(HttpSink), - "ganglia": new(GangliaSink), - "influxasync": new(InfluxAsyncSink), - "libganglia": new(LibgangliaSink), +var AvailableSinks = map[string]func(name string, config json.RawMessage) (Sink, error){ + "ganglia": NewGangliaSink, + "libganglia": NewLibgangliaSink, + "stdout": NewStdoutSink, + "nats": NewNatsSink, + "influxdb": NewInfluxSink, + "influxasync": NewInfluxAsyncSink, + "http": NewHttpSink, } // Metric collector manager data structure @@ -149,8 +149,7 @@ func (sm *sinkManager) AddOutput(name string, rawConfig json.RawMessage) error { cclog.ComponentError("SinkManager", "SKIP", name, "unknown sink:", sinkConfig.Type) return err } - s := AvailableSinks[sinkConfig.Type] - err = s.Init(rawConfig) + s, err := AvailableSinks[sinkConfig.Type](name, rawConfig) if err != nil { cclog.ComponentError("SinkManager", "SKIP", s.Name(), "initialization failed:", err.Error()) return err diff --git a/sinks/stdoutSink.go b/sinks/stdoutSink.go index 5d0761a..acf2621 100644 --- a/sinks/stdoutSink.go +++ b/sinks/stdoutSink.go @@ -19,34 +19,6 @@ type StdoutSink struct { } } -func (s *StdoutSink) Init(config json.RawMessage) error { - s.name = "StdoutSink" - if len(config) > 0 { - err := json.Unmarshal(config, &s.config) - if err != nil { - return err - } - } - - s.output = os.Stdout - if len(s.config.Output) > 0 { - switch strings.ToLower(s.config.Output) { - case "stdout": - s.output = os.Stdout - case "stderr": - s.output = os.Stderr - default: - f, err := os.OpenFile(s.config.Output, os.O_CREATE|os.O_WRONLY, os.FileMode(0600)) - if err != nil { - return err - } - s.output = f - } - } - s.meta_as_tags = s.config.MetaAsTags - return nil -} - func (s *StdoutSink) Write(m lp.CCMetric) error { fmt.Fprint( s.output, @@ -65,3 +37,33 @@ func (s *StdoutSink) Close() { s.output.Close() } } + +func NewStdoutSink(name string, config json.RawMessage) (Sink, error) { + s := new(StdoutSink) + s.name = fmt.Sprintf("StdoutSink(%s)", name) + if len(config) > 0 { + err := json.Unmarshal(config, &s.config) + if err != nil { + return nil, err + } + } + + s.output = os.Stdout + if len(s.config.Output) > 0 { + switch strings.ToLower(s.config.Output) { + case "stdout": + s.output = os.Stdout + case "stderr": + s.output = os.Stderr + default: + f, err := os.OpenFile(s.config.Output, os.O_CREATE|os.O_WRONLY, os.FileMode(0600)) + if err != nil { + return nil, err + } + s.output = f + } + } + s.meta_as_tags = s.config.MetaAsTags + + return s, nil +} From a2f9b23e85ae8ef016274071f55d4d51ae77d1ef Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Mon, 28 Feb 2022 09:39:59 +0100 Subject: [PATCH 02/28] Additional comments --- sinks/sampleSink.go | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/sinks/sampleSink.go b/sinks/sampleSink.go index be0196a..29d1b34 100644 --- a/sinks/sampleSink.go +++ b/sinks/sampleSink.go @@ -10,15 +10,22 @@ import ( ) type SampleSinkConfig struct { - defaultSinkConfig // defines JSON tags for 'type' and 'meta_as_tags' - // Add additional options + // defines JSON tags for 'type' and 'meta_as_tags' + // See: metricSink.go + defaultSinkConfig + // Additional config options, for SampleSink } type SampleSink struct { - sink // declarate 'name' and 'meta_as_tags' + // declares elements 'name' and 'meta_as_tags' + sink config SampleSinkConfig // entry point to the SampleSinkConfig } +// Implement functions required for Sink interface +// Write(...), Flush(), Close() +// See: metricSink.go + // Code to submit a single CCMetric to the sink func (s *SampleSink) Write(point lp.CCMetric) error { log.Print(point) @@ -42,6 +49,7 @@ func NewSampleSink(name string, config json.RawMessage) (Sink, error) { s.name = fmt.Sprintf("SampleSink(%s)", name) // Always specify a name here // Set defaults in s.config + // Allow overwriting these defaults by reading config JSON // Read in the config JSON if len(config) > 0 { @@ -52,7 +60,7 @@ func NewSampleSink(name string, config json.RawMessage) (Sink, error) { } // Check if all required fields in the config are set - // Use 'len(s.config.Option) > 0' for string settings + // E.g. use 'len(s.config.Option) > 0' for string settings // Establish connection to the server, library, ... // Check required files exist and lookup path(s) of executable(s) From 2c08e53be4d7f7d10d203236f4e5b51eddfa75df Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Mon, 28 Feb 2022 09:57:26 +0100 Subject: [PATCH 03/28] Additional comments --- collectors/sampleMetric.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/collectors/sampleMetric.go b/collectors/sampleMetric.go index cd3b4cc..47078a6 100644 --- a/collectors/sampleMetric.go +++ b/collectors/sampleMetric.go @@ -22,6 +22,13 @@ type SampleCollector struct { tags map[string]string // default tags } +// Functions to implement MetricCollector interface +// Init(...), Read(...), Close() +// See: metricCollector.go + +// Init initializes the sample collector +// Called once by the collector manager +// All tags, meta data tags and metrics that do not change over the runtime should be set here func (m *SampleCollector) Init(config json.RawMessage) error { var err error = nil // Always set the name early in Init() to use it in cclog.Component* functions @@ -56,12 +63,14 @@ func (m *SampleCollector) Init(config json.RawMessage) error { return err } +// Read collects all metrics belonging to the sample collector +// and sends them through the output channel to the collector manager func (m *SampleCollector) Read(interval time.Duration, output chan lp.CCMetric) { // Create a sample metric timestamp := time.Now() value := 1.0 - // If you want to measure something for a specific amout of time, use interval + // If you want to measure something for a specific amount of time, use interval // start := readState() // time.Sleep(interval) // stop := readState() @@ -75,6 +84,8 @@ func (m *SampleCollector) Read(interval time.Duration, output chan lp.CCMetric) } +// Close metric collector: close network connection, close files, close libraries, ... +// Called once by the collector manager func (m *SampleCollector) Close() { // Unset flag m.init = false From 33fec95eacd520bcc08cf1f6422b2f5f9e9b4438 Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Mon, 28 Feb 2022 12:16:48 +0100 Subject: [PATCH 04/28] Additional comments --- collectors/metricCollector.go | 27 +++++++++++++++------------ receivers/metricReceiver.go | 9 ++++++--- receivers/sampleReceiver.go | 16 +++++++++++++++- sinks/metricSink.go | 13 +++++++------ sinks/sampleSink.go | 3 +++ 5 files changed, 46 insertions(+), 22 deletions(-) diff --git a/collectors/metricCollector.go b/collectors/metricCollector.go index c71ae16..747772f 100644 --- a/collectors/metricCollector.go +++ b/collectors/metricCollector.go @@ -13,29 +13,30 @@ import ( ) type MetricCollector interface { - Name() string - Init(config json.RawMessage) error - Initialized() bool - Read(duration time.Duration, output chan lp.CCMetric) - Close() + Name() string // Name of the metric collector + Init(config json.RawMessage) error // Initialize metric collector + Initialized() bool // Is metric collector initialized? + Read(duration time.Duration, output chan lp.CCMetric) // Read metrics from metric collector + Close() // Close / finish metric collector } type metricCollector struct { - name string - init bool - meta map[string]string + name string // name of the metric + init bool // is metric collector initialized? + meta map[string]string // static meta data tags } -// Name() returns the name of the metric collector +// Name returns the name of the metric collector func (c *metricCollector) Name() string { return c.name } +// Setup is for future use func (c *metricCollector) setup() error { return nil } -// Initialized() indicates whether the metric collector has been initialized. +// Initialized indicates whether the metric collector has been initialized func (c *metricCollector) Initialized() bool { return c.init } @@ -64,6 +65,7 @@ func stringArrayContains(array []string, str string) (int, bool) { return -1, false } +// SocketList returns the list of physical sockets as read from /proc/cpuinfo func SocketList() []int { buffer, err := ioutil.ReadFile("/proc/cpuinfo") if err != nil { @@ -89,6 +91,7 @@ func SocketList() []int { return packs } +// CpuList returns the list of physical CPUs (in contrast to logical CPUs) as read from /proc/cpuinfo func CpuList() []int { buffer, err := ioutil.ReadFile("/proc/cpuinfo") if err != nil { @@ -117,8 +120,8 @@ func CpuList() []int { // RemoveFromStringList removes the string r from the array of strings s // If r is not contained in the array an error is returned func RemoveFromStringList(s []string, r string) ([]string, error) { - for i, item := range s { - if r == item { + for i := range s { + if r == s[i] { return append(s[:i], s[i+1:]...), nil } } diff --git a/receivers/metricReceiver.go b/receivers/metricReceiver.go index e133354..6b85fd4 100644 --- a/receivers/metricReceiver.go +++ b/receivers/metricReceiver.go @@ -8,6 +8,7 @@ type defaultReceiverConfig struct { Type string `json:"type"` } +// Receiver configuration: Listen address, port type ReceiverConfig struct { Addr string `json:"address"` Port string `json:"port"` @@ -23,15 +24,17 @@ type receiver struct { type Receiver interface { Start() - Close() - Name() string - SetSink(sink chan lp.CCMetric) + Close() // Close / finish metric receiver + Name() string // Name of the metric receiver + SetSink(sink chan lp.CCMetric) // Set sink channel } +// Name returns the name of the metric receiver func (r *receiver) Name() string { return r.name } +// SetSink set the sink channel func (r *receiver) SetSink(sink chan lp.CCMetric) { r.sink = sink } diff --git a/receivers/sampleReceiver.go b/receivers/sampleReceiver.go index e9edc90..2892d56 100644 --- a/receivers/sampleReceiver.go +++ b/receivers/sampleReceiver.go @@ -7,6 +7,7 @@ import ( cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" ) +// SampleReceiver configuration: receiver type, listen address, port type SampleReceiverConfig struct { Type string `json:"type"` Addr string `json:"address"` @@ -24,6 +25,10 @@ type SampleReceiver struct { // wg sync.WaitGroup } +// Implement functions required for Receiver interface +// Start(), Close() +// See: metricReceiver.go + func (r *SampleReceiver) Start() { cclog.ComponentDebug(r.name, "START") @@ -44,6 +49,7 @@ func (r *SampleReceiver) Start() { // }() } +// Close receiver: close network connection, close files, close libraries, ... func (r *SampleReceiver) Close() { cclog.ComponentDebug(r.name, "CLOSE") @@ -54,13 +60,21 @@ func (r *SampleReceiver) Close() { // r.wg.Wait() } +// New function to create a new instance of the receiver +// Initialize the receiver by giving it a name and reading in the config JSON func NewSampleReceiver(name string, config json.RawMessage) (Receiver, error) { r := new(SampleReceiver) - r.name = fmt.Sprintf("HttpReceiver(%s)", name) + + // Set name of SampleReceiver + // The name should be chosen in such a way that different instances of SampleReceiver can be distinguished + r.name = fmt.Sprintf("SampleReceiver(%s)", name) // Set static information r.meta = map[string]string{"source": r.name} + // Set defaults in r.config + // Allow overwriting these defaults by reading config JSON + // Read the sample receiver specific JSON config if len(config) > 0 { err := json.Unmarshal(config, &r.config) diff --git a/sinks/metricSink.go b/sinks/metricSink.go index 8fe02d7..d5356d0 100644 --- a/sinks/metricSink.go +++ b/sinks/metricSink.go @@ -10,17 +10,18 @@ type defaultSinkConfig struct { } type sink struct { - meta_as_tags bool - name string + meta_as_tags bool // Use meta data tags as tags + name string // Name of the sink } type Sink interface { - Write(point lp.CCMetric) error - Flush() error - Close() - Name() string + Write(point lp.CCMetric) error // Write metric to the sink + Flush() error // Flush buffered metrics + Close() // Close / finish metric sink + Name() string // Name of the metric sink } +// Name returns the name of the metric sink func (s *sink) Name() string { return s.name } diff --git a/sinks/sampleSink.go b/sinks/sampleSink.go index 29d1b34..3913a29 100644 --- a/sinks/sampleSink.go +++ b/sinks/sampleSink.go @@ -46,6 +46,9 @@ func (s *SampleSink) Close() { // Initialize the sink by giving it a name and reading in the config JSON func NewSampleSink(name string, config json.RawMessage) (Sink, error) { s := new(SampleSink) + + // Set name of sampleSink + // The name should be chosen in such a way that different instances of SampleSink can be distinguished s.name = fmt.Sprintf("SampleSink(%s)", name) // Always specify a name here // Set defaults in s.config From d40163cf8f39fe2eac91ebbd2bd863ae7c4e0b34 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Mon, 28 Feb 2022 17:26:28 +0100 Subject: [PATCH 05/28] Update README and receiver-specific pages --- receivers/README.md | 24 ++++++------------------ receivers/natsReceiver.md | 21 +++++++++++++++++++++ receivers/prometheusReceiver.md | 2 +- 3 files changed, 28 insertions(+), 19 deletions(-) create mode 100644 receivers/natsReceiver.md diff --git a/receivers/README.md b/receivers/README.md index 808dc74..70b209b 100644 --- a/receivers/README.md +++ b/receivers/README.md @@ -9,32 +9,20 @@ The configuration file for the receivers is a list of configurations. The `type` ```json { "myreceivername" : { + "type": "receiver-type", } } ``` -## Type `nats` +This allows to specify -```json -{ - "type": "nats", - "address": "", - "port" : "", - "subject": "" -} -``` +## Available receivers -The `nats` receiver subscribes to the topic `database` and listens on `address` and `port` for metrics in the InfluxDB line protocol. +- [`nats`](./natsReceiver.md): Receive metrics from the NATS network +- [`prometheus`](./prometheusReceiver.md): Scrape data from a Prometheus client # Contributing own receivers A receiver contains a few functions and is derived from the type `Receiver` (in `metricReceiver.go`): -* `Start() error` -* `Close()` -* `Name() string` -* `SetSink(sink chan lp.CCMetric)` -* `New(name string, config json.RawMessage)` -The data structures should be set up in `Init()` like opening a file or server connection. The `Start()` function should either start a go routine or issue some other asynchronous mechanism for receiving metrics. The `Close()` function should tear down anything created in `Init()`. - -Finally, the receiver needs to be registered in the `receiveManager.go`. There is a list of receivers called `AvailableReceivers` which is a map (`receiver_type_string` -> `pointer to NewReceiver function`). Add a new entry with a descriptive name and the new receiver. +For an example, check the [sample receiver](./sampleReceiver.go) diff --git a/receivers/natsReceiver.md b/receivers/natsReceiver.md new file mode 100644 index 0000000..4a7b7a4 --- /dev/null +++ b/receivers/natsReceiver.md @@ -0,0 +1,21 @@ +## `nats` receiver + +The `nats` receiver can be used receive metrics from the NATS network. The `nats` receiver subscribes to the topic `database` and listens on `address` and `port` for metrics in the InfluxDB line protocol. + +### Configuration structure + +```json +{ + "": { + "type": "nats", + "address" : "nats-server.example.org", + "port" : "4222", + "subject" : "subject" + } +} +``` + +- `type`: makes the receiver a `nats` receiver +- `address`: Address of the NATS control server +- `port`: Port of the NATS control server +- `subject`: Subscribes to this subject and receive metrics diff --git a/receivers/prometheusReceiver.md b/receivers/prometheusReceiver.md index 5fe6e46..83de7f6 100644 --- a/receivers/prometheusReceiver.md +++ b/receivers/prometheusReceiver.md @@ -10,7 +10,7 @@ The `prometheus` receiver can be used to scrape the metrics of a single `prometh "type": "prometheus", "address" : "testpromhost", "port" : "12345", - "port" : "/prometheus", + "path" : "/prometheus", "interval": "5s", "ssl" : true, } From a5325a65350f8ac9f4f3dd0fbfa3d9e993fda063 Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Tue, 1 Mar 2022 15:39:26 +0100 Subject: [PATCH 06/28] GitHub actions (#51) Create new GitHub action which uses unmodified AlmaLinux Docker image --- .github/workflows/AlmaLinux.yml | 60 ++++++++++++++++++++++++++++++++ .github/workflows/runonce.yml | 36 +++++++++++++++---- Makefile | 30 ++++++++++++++++ collectors/Makefile | 17 ++++----- scripts/cc-metric-collector.spec | 2 +- 5 files changed, 127 insertions(+), 18 deletions(-) create mode 100644 .github/workflows/AlmaLinux.yml diff --git a/.github/workflows/AlmaLinux.yml b/.github/workflows/AlmaLinux.yml new file mode 100644 index 0000000..f014e7e --- /dev/null +++ b/.github/workflows/AlmaLinux.yml @@ -0,0 +1,60 @@ +# See: https://docs.github.com/en/actions/using-workflows/workflow-syntax-for-github-actions + +# Workflow name +name: AlmaLinux 8.5 RPM build + +# Run on event push +on: push + +jobs: + + # + # Build on AlmaLinux 8.5 using golang version 1.16 + # + AlmaLinux-RPM-build: + runs-on: ubuntu-latest + container: almalinux:8.5 + steps: + + # Use dnf to install development packages + - name: Install development packages + run: dnf --assumeyes group install "Development Tools" "RPM Development Tools" + + # Checkout git repository and submodules + # fetch-depth must be 0 to use git describe + # See: https://github.com/marketplace/actions/checkout + - name: Checkout + uses: actions/checkout@v2 + with: + submodules: recursive + fetch-depth: 0 + + # Use dnf to install build dependencies + - name: Install build dependencies + run: dnf --assumeyes builddep scripts/cc-metric-collector.spec + + - name: RPM build MetricCollector + id: rpmbuild + run: make RPM + + # See: https://github.com/actions/upload-artifact + - name: Save RPM as artifact + uses: actions/upload-artifact@v2 + with: + name: cc-metric-collector RPM AlmaLinux 8.5 + path: ${{ steps.rpmbuild.outputs.RPM }} + - name: Save SRPM as artifact + uses: actions/upload-artifact@v2 + with: + name: cc-metric-collector SRPM AlmaLinux 8.5 + path: ${{ steps.rpmbuild.outputs.SRPM }} + + # See: https://github.com/softprops/action-gh-release + - name: Release + uses: softprops/action-gh-release@v1 + if: startsWith(github.ref, 'refs/tags/') + with: + name: cc-metric-collector-${{github.ref_name}} + files: | + ${{ steps.rpmbuild.outputs.RPM }} + ${{ steps.rpmbuild.outputs.SRPM }} \ No newline at end of file diff --git a/.github/workflows/runonce.yml b/.github/workflows/runonce.yml index da5b86c..be161ea 100644 --- a/.github/workflows/runonce.yml +++ b/.github/workflows/runonce.yml @@ -1,46 +1,68 @@ +# See: https://docs.github.com/en/actions/using-workflows/workflow-syntax-for-github-actions + +# Workflow name name: Run Test + +# Run on event push on: push jobs: + # + # Job build-1-17 + # Build on latest Ubuntu using golang version 1.17 + # build-1-17: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + # See: https://github.com/marketplace/actions/checkout + # Checkout git repository and submodules + - name: Checkout + uses: actions/checkout@v2 with: submodules: recursive # See: https://github.com/marketplace/actions/setup-go-environment - name: Setup Golang - uses: actions/setup-go@v2.1.5 + uses: actions/setup-go@v2 with: - go-version: '^1.17.6' + go-version: '^1.17.7' + # Install libganglia - name: Setup Ganglia run: sudo apt install ganglia-monitor libganglia1 - name: Build MetricCollector run: make - - name: Run MetricCollector + - name: Run MetricCollector once run: ./cc-metric-collector --once --config .github/ci-config.json + + # + # Job build-1-16 + # Build on latest Ubuntu using golang version 1.16 + # build-1-16: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + # See: https://github.com/marketplace/actions/checkout + # Checkout git repository and submodules + - name: Checkout + uses: actions/checkout@v2 with: submodules: recursive # See: https://github.com/marketplace/actions/setup-go-environment - name: Setup Golang - uses: actions/setup-go@v2.1.5 + uses: actions/setup-go@v2 with: go-version: '^1.16.7' # The version AlmaLinux 8.5 uses + # Install libganglia - name: Setup Ganglia run: sudo apt install ganglia-monitor libganglia1 - name: Build MetricCollector run: make - - name: Run MetricCollectorlibganglia1 + - name: Run MetricCollector once run: ./cc-metric-collector --once --config .github/ci-config.json diff --git a/Makefile b/Makefile index b32fb6b..e0ec475 100644 --- a/Makefile +++ b/Makefile @@ -56,3 +56,33 @@ vet: staticcheck: go install honnef.co/go/tools/cmd/staticcheck@latest $$(go env GOPATH)/bin/staticcheck ./... + +.ONESHELL: +.PHONY: RPM +RPM: scripts/cc-metric-collector.spec + @WORKSPACE="$${PWD}" + @SPECFILE="$${WORKSPACE}/scripts/cc-metric-collector.spec" + # Setup RPM build tree + @eval $$(rpm --eval "ARCH='%{_arch}' RPMDIR='%{_rpmdir}' SOURCEDIR='%{_sourcedir}' SPECDIR='%{_specdir}' SRPMDIR='%{_srcrpmdir}' BUILDDIR='%{_builddir}'") + @mkdir --parents --verbose "$${RPMDIR}" "$${SOURCEDIR}" "$${SPECDIR}" "$${SRPMDIR}" "$${BUILDDIR}" + # Create source tarball + @COMMITISH="HEAD" + @VERS=$$(git describe --tags $${COMMITISH}) + @VERS=$${VERS#v} + @VERS=$${VERS//-/_} + @eval $$(rpmspec --query --queryformat "NAME='%{name}' VERSION='%{version}' RELEASE='%{release}' NVR='%{NVR}' NVRA='%{NVRA}'" --define="VERS $${VERS}" "$${SPECFILE}") + @PREFIX="$${NAME}-$${VERSION}" + @FORMAT="tar.gz" + @SRCFILE="$${SOURCEDIR}/$${PREFIX}.$${FORMAT}" + @git archive --verbose --format "$${FORMAT}" --prefix="$${PREFIX}/" --output="$${SRCFILE}" $${COMMITISH} + # Build RPM and SRPM + @rpmbuild -ba --define="VERS $${VERS}" --rmsource --clean "$${SPECFILE}" + # Report RPMs and SRPMs when in GitHub Workflow + @if [[ "$${GITHUB_ACTIONS}" == true ]]; then + @ RPMFILE="$${RPMDIR}/$${ARCH}/$${NVRA}.rpm" + @ SRPMFILE="$${SRPMDIR}/$${NVR}.src.rpm" + @ echo "RPM: $${RPMFILE}" + @ echo "SRPM: $${SRPMFILE}" + @ echo "::set-output name=SRPM::$${SRPMFILE}" + @ echo "::set-output name=RPM::$${RPMFILE}" + @fi diff --git a/collectors/Makefile b/collectors/Makefile index 0c637b5..379a2d6 100644 --- a/collectors/Makefile +++ b/collectors/Makefile @@ -55,16 +55,13 @@ $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION): $(BUILD_FOLDER)/likwid-$(LIKWID_VERSIO tar -C $(BUILD_FOLDER) -xf $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION).tar.gz $(INSTALL_FOLDER)/liblikwid.a: $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION) $(INSTALL_FOLDER) - sed -i -e s+"PREFIX ?= .*"+"PREFIX = $(LIKWID_BASE)"+g \ - -e s+"SHARED_LIBRARY = .*"+"SHARED_LIBRARY = false"+g \ - -e s+"ACCESSMODE = .*"+"ACCESSMODE = $(ACCESSMODE)"+g \ - -e s+"INSTALLED_ACCESSDAEMON = .*"+"INSTALLED_ACCESSDAEMON = $(DAEMON_INSTALLDIR)/likwid-accessD"+g \ - $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/config.mk - cd $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION) && make - cp $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/liblikwid.a $(INSTALL_FOLDER) - cp $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/ext/hwloc/liblikwid-hwloc.a $(INSTALL_FOLDER) - cp $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/likwid*.h $(INSTALL_FOLDER) - cp $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/bstrlib.h $(INSTALL_FOLDER) + cd "$(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)" && make "PREFIX=$(LIKWID_BASE)" "SHARED_LIBRARY=false" "ACCESSMODE=$(ACCESSMODE)" "INSTALLED_ACCESSDAEMON=$(DAEMON_INSTALLDIR)/likwid-accessD" + cp \ + $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/liblikwid.a \ + $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/ext/hwloc/liblikwid-hwloc.a \ + $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/likwid*.h \ + $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/bstrlib.h \ + $(INSTALL_FOLDER) $(DAEMON_INSTALLDIR)/likwid-accessD: $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/likwid-accessD sudo -u $(DAEMON_USER) -g $(DAEMON_GROUP) install -m 4775 $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/likwid-accessD $(DAEMON_INSTALLDIR)/likwid-accessD diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index 65a9b55..9a1ec3b 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -1,5 +1,5 @@ Name: cc-metric-collector -Version: 0.2 +Version: %{VERS} Release: 1%{?dist} Summary: Metric collection daemon from the ClusterCockpit suite From 33d954f767c58f23c518def9290491eddd2a6110 Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Tue, 1 Mar 2022 17:15:20 +0100 Subject: [PATCH 07/28] Action red hat universal base image (#52) Add Red Hat Universal Base Image 8 Workflow --- .github/workflows/AlmaLinux.yml | 9 +-- .../workflows/RedHatUniversalBaseImage.yml | 61 +++++++++++++++++++ .github/workflows/rpmbuild.yml | 58 ------------------ 3 files changed, 66 insertions(+), 62 deletions(-) create mode 100644 .github/workflows/RedHatUniversalBaseImage.yml delete mode 100644 .github/workflows/rpmbuild.yml diff --git a/.github/workflows/AlmaLinux.yml b/.github/workflows/AlmaLinux.yml index f014e7e..442ad41 100644 --- a/.github/workflows/AlmaLinux.yml +++ b/.github/workflows/AlmaLinux.yml @@ -9,10 +9,11 @@ on: push jobs: # - # Build on AlmaLinux 8.5 using golang version 1.16 + # Build on AlmaLinux 8.5 using go-toolset # AlmaLinux-RPM-build: runs-on: ubuntu-latest + # See: https://hub.docker.com/_/almalinux container: almalinux:8.5 steps: @@ -41,12 +42,12 @@ jobs: - name: Save RPM as artifact uses: actions/upload-artifact@v2 with: - name: cc-metric-collector RPM AlmaLinux 8.5 + name: cc-metric-collector RPM for AlmaLinux 8.5 path: ${{ steps.rpmbuild.outputs.RPM }} - name: Save SRPM as artifact uses: actions/upload-artifact@v2 with: - name: cc-metric-collector SRPM AlmaLinux 8.5 + name: cc-metric-collector SRPM for AlmaLinux 8.5 path: ${{ steps.rpmbuild.outputs.SRPM }} # See: https://github.com/softprops/action-gh-release @@ -57,4 +58,4 @@ jobs: name: cc-metric-collector-${{github.ref_name}} files: | ${{ steps.rpmbuild.outputs.RPM }} - ${{ steps.rpmbuild.outputs.SRPM }} \ No newline at end of file + ${{ steps.rpmbuild.outputs.SRPM }} diff --git a/.github/workflows/RedHatUniversalBaseImage.yml b/.github/workflows/RedHatUniversalBaseImage.yml new file mode 100644 index 0000000..cfd8a1f --- /dev/null +++ b/.github/workflows/RedHatUniversalBaseImage.yml @@ -0,0 +1,61 @@ +# See: https://docs.github.com/en/actions/using-workflows/workflow-syntax-for-github-actions + +# Workflow name +name: Red Hat Universal Base Image 8 RPM build + +# Run on event push +on: push + +jobs: + + # + # Build on UBI 8 using go-toolset + # + UBI-8-RPM-build: + runs-on: ubuntu-latest + # See: https://catalog.redhat.com/software/containers/ubi8/ubi/5c359854d70cc534b3a3784e?container-tabs=gti + container: registry.access.redhat.com/ubi8/ubi:8.5-226.1645809065 + steps: + + # Use dnf to install development packages + - name: Install development packages + run: dnf --assumeyes --disableplugin=subscription-manager install rpm-build go-srpm-macros rpm-build-libs rpm-libs gcc make python38 git + + # Checkout git repository and submodules + # fetch-depth must be 0 to use git describe + # See: https://github.com/marketplace/actions/checkout + - name: Checkout + uses: actions/checkout@v2 + with: + submodules: recursive + fetch-depth: 0 + + # Use dnf to install build dependencies + - name: Install build dependencies + run: dnf --assumeyes --disableplugin=subscription-manager builddep scripts/cc-metric-collector.spec + + - name: RPM build MetricCollector + id: rpmbuild + run: make RPM + + # See: https://github.com/actions/upload-artifact + - name: Save RPM as artifact + uses: actions/upload-artifact@v2 + with: + name: cc-metric-collector RPM for UBI 8 + path: ${{ steps.rpmbuild.outputs.RPM }} + - name: Save SRPM as artifact + uses: actions/upload-artifact@v2 + with: + name: cc-metric-collector SRPM for UBI 8 + path: ${{ steps.rpmbuild.outputs.SRPM }} + + # See: https://github.com/softprops/action-gh-release + - name: Release + uses: softprops/action-gh-release@v1 + if: startsWith(github.ref, 'refs/tags/') + with: + name: cc-metric-collector-${{github.ref_name}} + files: | + ${{ steps.rpmbuild.outputs.RPM }} + ${{ steps.rpmbuild.outputs.SRPM }} diff --git a/.github/workflows/rpmbuild.yml b/.github/workflows/rpmbuild.yml deleted file mode 100644 index 9c6ae13..0000000 --- a/.github/workflows/rpmbuild.yml +++ /dev/null @@ -1,58 +0,0 @@ -name: Run RPM Build -on: - push: - tags: - - '**' - -jobs: - build-alma-8_5: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v2 - - uses: TomTheBear/rpmbuild@alma8.5 - id: rpm - name: Build RPM package on AlmaLinux 8.5 - with: - spec_file: "./scripts/cc-metric-collector.spec" - - name: Save RPM as artifact - uses: actions/upload-artifact@v1.0.0 - with: - name: cc-metric-collector RPM AlmaLinux 8.5 - path: ${{ steps.rpm.outputs.rpm_path }} - - name: Save SRPM as artifact - uses: actions/upload-artifact@v1.0.0 - with: - name: cc-metric-collector SRPM AlmaLinux 8.5 - path: ${{ steps.rpm.outputs.source_rpm_path }} - - name: Release - uses: softprops/action-gh-release@v1 - with: - name: cc-metric-collector-${{github.ref_name}} - files: | - ${{ steps.rpm.outputs.source_rpm_path }} - ${{ steps.rpm.outputs.rpm_path }} - build-rhel-ubi8: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v2 - - uses: TomTheBear/rpmbuild@rh-ubi8 - id: rpm - name: Build RPM package on Red Hat Universal Base Image 8 - with: - spec_file: "./scripts/cc-metric-collector.spec" - - name: Save RPM as artifact - uses: actions/upload-artifact@v1.0.0 - with: - name: cc-metric-collector RPM Red Hat Universal Base Image 8 - path: ${{ steps.rpm.outputs.rpm_path }} - - name: Save SRPM as artifact - uses: actions/upload-artifact@v1.0.0 - with: - name: cc-metric-collector SRPM Red Hat Universal Base Image 8 - path: ${{ steps.rpm.outputs.source_rpm_path }} - - name: Release - uses: softprops/action-gh-release@v1 - with: - files: | - ${{ steps.rpm.outputs.source_rpm_path }} - ${{ steps.rpm.outputs.rpm_path }} From f7e8b526673f955d2e3f099cc702a8f53f09c58b Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Wed, 2 Mar 2022 13:21:54 +0100 Subject: [PATCH 08/28] Run RPM build actions only on tag push --- .github/workflows/AlmaLinux.yml | 7 +++++-- .github/workflows/RedHatUniversalBaseImage.yml | 7 +++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/.github/workflows/AlmaLinux.yml b/.github/workflows/AlmaLinux.yml index 442ad41..dd06dd2 100644 --- a/.github/workflows/AlmaLinux.yml +++ b/.github/workflows/AlmaLinux.yml @@ -3,8 +3,11 @@ # Workflow name name: AlmaLinux 8.5 RPM build -# Run on event push -on: push +# Run on tag push +on: + push: + tags: + - '**' jobs: diff --git a/.github/workflows/RedHatUniversalBaseImage.yml b/.github/workflows/RedHatUniversalBaseImage.yml index cfd8a1f..205a133 100644 --- a/.github/workflows/RedHatUniversalBaseImage.yml +++ b/.github/workflows/RedHatUniversalBaseImage.yml @@ -3,8 +3,11 @@ # Workflow name name: Red Hat Universal Base Image 8 RPM build -# Run on event push -on: push +# Run on tag push +on: + push: + tags: + - '**' jobs: From 092e7f6a71836562190e8eed9455a9f8b11c5bb3 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Wed, 2 Mar 2022 13:54:43 +0100 Subject: [PATCH 09/28] Add section how to temporarly disable LIKWID access to page --- collectors/likwidMetric.md | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/collectors/likwidMetric.md b/collectors/likwidMetric.md index 3ef51f3..2d0b840 100644 --- a/collectors/likwidMetric.md +++ b/collectors/likwidMetric.md @@ -61,7 +61,21 @@ $ scripts/likwid_perfgroup_to_cc_config.py ICX MEM_DP } ``` -You can copy this JSON and add it to the `eventsets` list. If you specify multiple event sets, you can add globally derived metrics in the extra `global_metrics` section with the metric names as variables. +You can copy this JSON and add it to the `eventsets` list. If you specify multiple event sets, you can add globally derived metrics in the extra `global_metrics` section with the metric names as variables. + +### Mixed usage between daemon and users + +LIKWID checks the file `/var/run/likwid.lock` before performing any interfering operations. Who is allowed to access the counters is determined by the owner of the file. If it does not exist, it is created for the current user. So, if you want to temporarly allow counter access to a user (e.g. in a job): + +Before (SLURM prolog, ...) +``` +$ chwon $USER /var/run/likwid.lock +``` + +After (SLURM epilog, ...) +``` +$ chwon root /var/run/likwid.lock +``` ### Example configuration From 0753c811564348b24d44c197d8099cfbfaaa801e Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Wed, 2 Mar 2022 15:10:14 +0100 Subject: [PATCH 10/28] Add/Remove clustercockpit user and group in RPM --- scripts/cc-metric-collector.spec | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index 9a1ec3b..2a34263 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -39,12 +39,26 @@ install -Dpm 0600 scripts/%{name}.config %{buildroot}%{_sysconfdir}/default/%{na %check # go test should be here... :) +%pre +getent group clustercockpit >/dev/null || groupadd -r clustercockpit +getent passwd clustercockpit >/dev/null || \ + useradd -r -g clustercockpit -d /nonexistent -s /sbin/nologin \ + -c "Create system user and group for CC metric collector" clustercockpit +exit 0 + %post %systemd_post %{name}.service %preun %systemd_preun %{name}.service +%postun +if [ "$1" = "1" ]; then +getent passwd clustercockpit >/dev/null && userdel clustercockpit +getent group clustercockpit >/dev/null && groupdel clustercockpit +fi +exit 0 + %files %dir %{_sysconfdir}/%{name} %{_sbindir}/%{name} From 6023abd02877330ef472a53ea21adebe9f95cbac Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 11:02:37 +0100 Subject: [PATCH 11/28] Rename main file to match with executable name --- metric-collector.go => cc-metric-collector.go | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename metric-collector.go => cc-metric-collector.go (100%) diff --git a/metric-collector.go b/cc-metric-collector.go similarity index 100% rename from metric-collector.go rename to cc-metric-collector.go From c61b8d28774b28232e7018dfa1f31481c58ffdd6 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 11:03:51 +0100 Subject: [PATCH 12/28] Set name change in Makefile --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index e0ec475..d747899 100644 --- a/Makefile +++ b/Makefile @@ -1,5 +1,5 @@ APP = cc-metric-collector -GOSRC_APP := metric-collector.go +GOSRC_APP := cc-metric-collector.go GOSRC_COLLECTORS := $(wildcard collectors/*.go) GOSRC_SINKS := $(wildcard sinks/*.go) GOSRC_RECEIVERS := $(wildcard receivers/*.go) From 276c00442a18fc5685dff03be8149a89b87f4e41 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 13:02:00 +0100 Subject: [PATCH 13/28] Add option to LustreCollector to call lctl with sudo --- collectors/lustreMetric.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/collectors/lustreMetric.go b/collectors/lustreMetric.go index 6d6fe26..f98d746 100644 --- a/collectors/lustreMetric.go +++ b/collectors/lustreMetric.go @@ -22,6 +22,7 @@ type LustreCollectorConfig struct { LCtlCommand string `json:"lctl_command"` ExcludeMetrics []string `json:"exclude_metrics"` SendAllMetrics bool `json:"send_all_metrics"` + Sudo bool `json:"use_sudo"` } type LustreCollector struct { @@ -31,11 +32,17 @@ type LustreCollector struct { stats map[string]map[string]int64 config LustreCollectorConfig lctl string + sudoCmd string } func (m *LustreCollector) getDeviceDataCommand(device string) []string { + var command *exec.Cmd statsfile := fmt.Sprintf("llite.%s.stats", device) - command := exec.Command(m.lctl, LCTL_OPTION, statsfile) + if m.config.Sudo { + command = exec.Command(m.sudoCmd, m.lctl, LCTL_OPTION, statsfile) + } else { + command = exec.Command(m.lctl, LCTL_OPTION, statsfile) + } command.Wait() stdout, _ := command.Output() return strings.Split(string(stdout), "\n") @@ -136,6 +143,12 @@ func (m *LustreCollector) Init(config json.RawMessage) error { } } m.lctl = p + if m.config.Sudo { + p, err := exec.LookPath("sudo") + if err != nil { + m.sudoCmd = p + } + } devices := m.getDevices() if len(devices) == 0 { From 60de21c41edd9cbcfa07ed68da851b83a7306aba Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 13:03:58 +0100 Subject: [PATCH 14/28] Switch access mode of LikwidCollector in config file --- collectors/likwidMetric.go | 25 ++++++++++++++++++++++--- collectors/likwidMetric.md | 6 ++++-- 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/collectors/likwidMetric.go b/collectors/likwidMetric.go index 8626d7c..8ab42d5 100644 --- a/collectors/likwidMetric.go +++ b/collectors/likwidMetric.go @@ -2,7 +2,7 @@ package collectors /* #cgo CFLAGS: -I./likwid -#cgo LDFLAGS: -L./likwid -llikwid -llikwid-hwloc -lm -Wl,--unresolved-symbols=ignore-in-object-files +#cgo LDFLAGS: -Wl,--unresolved-symbols=ignore-in-object-files #include #include */ @@ -71,8 +71,9 @@ func GetAllMetricScopes() []MetricScope { } const ( - LIKWID_LIB_NAME = "liblikwid.so" - LIKWID_LIB_DL_FLAGS = dl.RTLD_LAZY | dl.RTLD_GLOBAL + LIKWID_LIB_NAME = "liblikwid.so" + LIKWID_LIB_DL_FLAGS = dl.RTLD_LAZY | dl.RTLD_GLOBAL + LIKWID_DEF_ACCESSMODE = "direct" ) type LikwidCollectorMetricConfig struct { @@ -95,6 +96,8 @@ type LikwidCollectorConfig struct { Metrics []LikwidCollectorMetricConfig `json:"globalmetrics,omitempty"` ForceOverwrite bool `json:"force_overwrite,omitempty"` InvalidToZero bool `json:"invalid_to_zero,omitempty"` + AccessMode string `json:"access_mode,omitempty"` + DaemonPath string `json:"accessdaemon_path,omitempty"` } type LikwidCollector struct { @@ -260,6 +263,7 @@ func (m *LikwidCollector) getResponsiblities() map[MetricScope]map[int]int { func (m *LikwidCollector) Init(config json.RawMessage) error { var ret C.int m.name = "LikwidCollector" + m.config.AccessMode = LIKWID_DEF_ACCESSMODE if len(config) > 0 { err := json.Unmarshal(config, &m.config) if err != nil { @@ -270,6 +274,11 @@ func (m *LikwidCollector) Init(config json.RawMessage) error { if lib == nil { return fmt.Errorf("error instantiating DynamicLibrary for %s", LIKWID_LIB_NAME) } + err := lib.Open() + if err != nil { + return fmt.Errorf("error opening %s: %v", LIKWID_LIB_NAME, err) + } + if m.config.ForceOverwrite { cclog.ComponentDebug(m.name, "Set LIKWID_FORCE=1") os.Setenv("LIKWID_FORCE", "1") @@ -301,6 +310,16 @@ func (m *LikwidCollector) Init(config json.RawMessage) error { m.initGranularity() // Generate map for MetricScope -> scope_id (like socket id) -> responsible id (offset in cpulist) m.scopeRespTids = m.getResponsiblities() + switch m.config.AccessMode { + case "direct": + C.HPMmode(0) + case "accessdaemon": + if len(m.config.DaemonPath) > 0 { + p := os.Getenv("PATH") + os.Setenv("PATH", m.config.DaemonPath+":"+p) + } + C.HPMmode(1) + } cclog.ComponentDebug(m.name, "initialize LIKWID perfmon module") ret = C.perfmon_init(C.int(len(m.cpulist)), &m.cpulist[0]) diff --git a/collectors/likwidMetric.md b/collectors/likwidMetric.md index 2d0b840..1aa4242 100644 --- a/collectors/likwidMetric.md +++ b/collectors/likwidMetric.md @@ -8,6 +8,8 @@ The `likwid` configuration consists of two parts, the "eventsets" and "globalmet - The global metrics are metrics which require data from all event set measurements to be derived. The inputs are the metrics in the event sets. Similar to the metrics in the event sets, the global metrics are defined by a name, a formula, a scope and a publish flag. See event set metrics for details. The only difference is that there is no access to the raw event measurements anymore but only to the metrics. So, the idea is to derive a metric in the "eventsets" section and reuse it in the "globalmetrics" part. If you need a metric only for deriving the global metrics, disable forwarding of the event set metrics. **Be aware** that the combination might be misleading because the "behavior" of a metric changes over time and the multiple measurements might count different computing phases. Additional options: +- `access_mode` : Method to use for hardware performance monitoring (`direct` access as root user, `accessdaemon` for the daemon mode) +- `accessdaemon_path`: Folder with the access daemon `likwid-accessD`, commonly `$LIKWID_INSTALL_LOC/sbin` - `force_overwrite`: Same as setting `LIKWID_FORCE=1`. In case counters are already in-use, LIKWID overwrites their configuration to do its measurements - `invalid_to_zero`: In some cases, the calculations result in `NaN` or `Inf`. With this option, all `NaN` and `Inf` values are replaces with `0.0`. @@ -69,12 +71,12 @@ LIKWID checks the file `/var/run/likwid.lock` before performing any interfering Before (SLURM prolog, ...) ``` -$ chwon $USER /var/run/likwid.lock +$ chwon $JOBUSER /var/run/likwid.lock ``` After (SLURM epilog, ...) ``` -$ chwon root /var/run/likwid.lock +$ chwon $CCUSER /var/run/likwid.lock ``` ### Example configuration From 948c34d74d7c2ba0db3aa209043d7009ff66fed6 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 13:43:43 +0100 Subject: [PATCH 15/28] Add user creation in RPM --- scripts/cc-metric-collector.spec | 18 ++++++------------ scripts/cc-metric-collector.sysusers | 2 ++ 2 files changed, 8 insertions(+), 12 deletions(-) create mode 100644 scripts/cc-metric-collector.sysusers diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index 2a34263..fef1bb8 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -7,6 +7,7 @@ License: MIT Source0: %{name}-%{version}.tar.gz BuildRequires: go-toolset +BuildRequires: systemd-rpm-macros # for internal LIKWID installation BuildRequires: wget perl-Data-Dumper @@ -34,17 +35,14 @@ install -Dpm 0600 receivers.json %{buildroot}%{_sysconfdir}/%{name}/receivers.js install -Dpm 0600 router.json %{buildroot}%{_sysconfdir}/%{name}/router.json install -Dpm 0644 scripts/%{name}.service %{buildroot}%{_unitdir}/%{name}.service install -Dpm 0600 scripts/%{name}.config %{buildroot}%{_sysconfdir}/default/%{name} +install -Dpm 0644 scripts/%{name}.sysusers %{buildroot}%{_sysusersdir}/%{name}.conf %check # go test should be here... :) %pre -getent group clustercockpit >/dev/null || groupadd -r clustercockpit -getent passwd clustercockpit >/dev/null || \ - useradd -r -g clustercockpit -d /nonexistent -s /sbin/nologin \ - -c "Create system user and group for CC metric collector" clustercockpit -exit 0 +%sysusers_create_compat scripts/%{name}.sysusers %post %systemd_post %{name}.service @@ -52,13 +50,6 @@ exit 0 %preun %systemd_preun %{name}.service -%postun -if [ "$1" = "1" ]; then -getent passwd clustercockpit >/dev/null && userdel clustercockpit -getent group clustercockpit >/dev/null && groupdel clustercockpit -fi -exit 0 - %files %dir %{_sysconfdir}/%{name} %{_sbindir}/%{name} @@ -69,8 +60,11 @@ exit 0 %attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/sinks.json %attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/receivers.json %attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/router.json +%{_sysusersdir}/%{name}.conf %changelog +* Thu Mar 03 2022 Thomas Gruber - 0.3 +- Add clustercockpit user installation * Mon Feb 14 2022 Thomas Gruber - 0.2 - Add component specific configuration files - Add %attr to config files diff --git a/scripts/cc-metric-collector.sysusers b/scripts/cc-metric-collector.sysusers new file mode 100644 index 0000000..65646af --- /dev/null +++ b/scripts/cc-metric-collector.sysusers @@ -0,0 +1,2 @@ +#Type Name ID GECOS Home directory Shell +s hpcop - "User for ClusterCockpit" /run/cc-metric-collector /sbin/nologin From db04c8fbae6792dd487cd5e22c01a06148a9428a Mon Sep 17 00:00:00 2001 From: Holger Obermaier <40787752+ho-ob@users.noreply.github.com> Date: Thu, 3 Mar 2022 15:52:50 +0100 Subject: [PATCH 16/28] Removed infinibandPerfQueryMetric.go. infinibandMetric.go offers the same functionality without requiring root privileges. --- collectors/collectorManager.go | 1 - collectors/infinibandPerfQueryMetric.go | 232 ------------------------ collectors/infinibandPerfQueryMetric.md | 28 --- 3 files changed, 261 deletions(-) delete mode 100644 collectors/infinibandPerfQueryMetric.go delete mode 100644 collectors/infinibandPerfQueryMetric.md diff --git a/collectors/collectorManager.go b/collectors/collectorManager.go index 86b423e..46d791a 100644 --- a/collectors/collectorManager.go +++ b/collectors/collectorManager.go @@ -19,7 +19,6 @@ var AvailableCollectors = map[string]MetricCollector{ "memstat": new(MemstatCollector), "netstat": new(NetstatCollector), "ibstat": new(InfinibandCollector), - "ibstat_perfquery": new(InfinibandPerfQueryCollector), "lustrestat": new(LustreCollector), "cpustat": new(CpustatCollector), "topprocs": new(TopProcsCollector), diff --git a/collectors/infinibandPerfQueryMetric.go b/collectors/infinibandPerfQueryMetric.go deleted file mode 100644 index 72f701f..0000000 --- a/collectors/infinibandPerfQueryMetric.go +++ /dev/null @@ -1,232 +0,0 @@ -package collectors - -import ( - "fmt" - "io/ioutil" - "log" - "os/exec" - - lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" - - // "os" - "encoding/json" - "errors" - "path/filepath" - "strconv" - "strings" - "time" -) - -const PERFQUERY = `/usr/sbin/perfquery` - -type InfinibandPerfQueryCollector struct { - metricCollector - tags map[string]string - lids map[string]map[string]string - config struct { - ExcludeDevices []string `json:"exclude_devices,omitempty"` - PerfQueryPath string `json:"perfquery_path"` - } -} - -func (m *InfinibandPerfQueryCollector) Init(config json.RawMessage) error { - var err error - m.name = "InfinibandCollectorPerfQuery" - m.setup() - m.meta = map[string]string{"source": m.name, "group": "Network"} - m.tags = map[string]string{"type": "node"} - if len(config) > 0 { - err = json.Unmarshal(config, &m.config) - if err != nil { - return err - } - } - if len(m.config.PerfQueryPath) == 0 { - path, err := exec.LookPath("perfquery") - if err == nil { - m.config.PerfQueryPath = path - } - } - m.lids = make(map[string]map[string]string) - p := fmt.Sprintf("%s/*/ports/*/lid", string(IB_BASEPATH)) - files, err := filepath.Glob(p) - if err != nil { - return err - } - for _, f := range files { - lid, err := ioutil.ReadFile(f) - if err == nil { - plist := strings.Split(strings.Replace(f, string(IB_BASEPATH), "", -1), "/") - skip := false - for _, d := range m.config.ExcludeDevices { - if d == plist[0] { - skip = true - } - } - if !skip { - m.lids[plist[0]] = make(map[string]string) - m.lids[plist[0]][plist[2]] = string(lid) - } - } - } - - for _, ports := range m.lids { - for port, lid := range ports { - args := fmt.Sprintf("-r %s %s 0xf000", lid, port) - command := exec.Command(m.config.PerfQueryPath, args) - command.Wait() - _, err := command.Output() - if err != nil { - return fmt.Errorf("Failed to execute %s: %v", m.config.PerfQueryPath, err) - } - } - } - - if len(m.lids) == 0 { - return errors.New("No usable IB devices") - } - - m.init = true - return nil -} - -func (m *InfinibandPerfQueryCollector) doPerfQuery(cmd string, dev string, lid string, port string, tags map[string]string, output chan lp.CCMetric) error { - - args := fmt.Sprintf("-r %s %s 0xf000", lid, port) - command := exec.Command(cmd, args) - command.Wait() - stdout, err := command.Output() - if err != nil { - log.Print(err) - return err - } - ll := strings.Split(string(stdout), "\n") - - for _, line := range ll { - if strings.HasPrefix(line, "PortRcvData") || strings.HasPrefix(line, "RcvData") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_recv", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - if strings.HasPrefix(line, "PortXmitData") || strings.HasPrefix(line, "XmtData") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_xmit", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - if strings.HasPrefix(line, "PortRcvPkts") || strings.HasPrefix(line, "RcvPkts") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_recv_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - if strings.HasPrefix(line, "PortXmitPkts") || strings.HasPrefix(line, "XmtPkts") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_xmit_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - if strings.HasPrefix(line, "PortRcvPkts") || strings.HasPrefix(line, "RcvPkts") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_recv_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - if strings.HasPrefix(line, "PortXmitPkts") || strings.HasPrefix(line, "XmtPkts") { - lv := strings.Fields(line) - v, err := strconv.ParseFloat(lv[1], 64) - if err == nil { - y, err := lp.New("ib_xmit_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - } - return nil -} - -func (m *InfinibandPerfQueryCollector) Read(interval time.Duration, output chan lp.CCMetric) { - - if m.init { - for dev, ports := range m.lids { - for port, lid := range ports { - tags := map[string]string{ - "type": "node", - "device": dev, - "port": port, - "lid": lid} - path := fmt.Sprintf("%s/%s/ports/%s/counters/", string(IB_BASEPATH), dev, port) - buffer, err := ioutil.ReadFile(fmt.Sprintf("%s/port_rcv_data", path)) - if err == nil { - data := strings.Replace(string(buffer), "\n", "", -1) - v, err := strconv.ParseFloat(data, 64) - if err == nil { - y, err := lp.New("ib_recv", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - buffer, err = ioutil.ReadFile(fmt.Sprintf("%s/port_xmit_data", path)) - if err == nil { - data := strings.Replace(string(buffer), "\n", "", -1) - v, err := strconv.ParseFloat(data, 64) - if err == nil { - y, err := lp.New("ib_xmit", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - buffer, err = ioutil.ReadFile(fmt.Sprintf("%s/port_rcv_packets", path)) - if err == nil { - data := strings.Replace(string(buffer), "\n", "", -1) - v, err := strconv.ParseFloat(data, 64) - if err == nil { - y, err := lp.New("ib_recv_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - buffer, err = ioutil.ReadFile(fmt.Sprintf("%s/port_xmit_packets", path)) - if err == nil { - data := strings.Replace(string(buffer), "\n", "", -1) - v, err := strconv.ParseFloat(data, 64) - if err == nil { - y, err := lp.New("ib_xmit_pkts", tags, m.meta, map[string]interface{}{"value": float64(v)}, time.Now()) - if err == nil { - output <- y - } - } - } - } - } - } -} - -func (m *InfinibandPerfQueryCollector) Close() { - m.init = false -} diff --git a/collectors/infinibandPerfQueryMetric.md b/collectors/infinibandPerfQueryMetric.md deleted file mode 100644 index 2147963..0000000 --- a/collectors/infinibandPerfQueryMetric.md +++ /dev/null @@ -1,28 +0,0 @@ - -## `ibstat_perfquery` collector - -```json - "ibstat_perfquery": { - "perfquery_path": "/path/to/perfquery", - "exclude_devices": [ - "mlx4" - ] - } -``` - -The `ibstat_perfquery` collector includes all Infiniband devices that can be -found below `/sys/class/infiniband/` and where any of the ports provides a -LID file (`/sys/class/infiniband//ports//lid`) - -The devices can be filtered with the `exclude_devices` option in the configuration. - -For each found LID the collector calls the `perfquery` command. The path to the -`perfquery` command can be configured with the `perfquery_path` option in the configuration - -Metrics: -* `ib_recv` -* `ib_xmit` -* `ib_recv_pkts` -* `ib_xmit_pkts` - -The collector adds a `device` tag to all metrics From b3030a8d44f232906d3903b016c228691b1527df Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 17:24:32 +0100 Subject: [PATCH 17/28] Use right systemd macro to create the user --- scripts/cc-metric-collector.spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index fef1bb8..00a4aa4 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -42,7 +42,7 @@ install -Dpm 0644 scripts/%{name}.sysusers %{buildroot}%{_sysusersdir}/%{name}.c # go test should be here... :) %pre -%sysusers_create_compat scripts/%{name}.sysusers +%sysusers_create_package scripts/%{name}.sysusers %post %systemd_post %{name}.service From 9ece27eec6038dd8cd4d3c567eb6798f9db9ec00 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 17:26:32 +0100 Subject: [PATCH 18/28] Use right systemd macro to create the user --- scripts/cc-metric-collector.spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index fef1bb8..00a4aa4 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -42,7 +42,7 @@ install -Dpm 0644 scripts/%{name}.sysusers %{buildroot}%{_sysusersdir}/%{name}.c # go test should be here... :) %pre -%sysusers_create_compat scripts/%{name}.sysusers +%sysusers_create_package scripts/%{name}.sysusers %post %systemd_post %{name}.service From 31f10b3163deb7929d8613d312627d4a7039ad1b Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Thu, 3 Mar 2022 17:31:15 +0100 Subject: [PATCH 19/28] Fix user creation type --- scripts/cc-metric-collector.sysusers | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/cc-metric-collector.sysusers b/scripts/cc-metric-collector.sysusers index 65646af..5c17ec7 100644 --- a/scripts/cc-metric-collector.sysusers +++ b/scripts/cc-metric-collector.sysusers @@ -1,2 +1,2 @@ #Type Name ID GECOS Home directory Shell -s hpcop - "User for ClusterCockpit" /run/cc-metric-collector /sbin/nologin +u hpcop - "User for ClusterCockpit" /run/cc-metric-collector /sbin/nologin From f1d2828e1dce6145f6f19e57c0b04a72a158dcee Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Fri, 4 Mar 2022 11:32:10 +0100 Subject: [PATCH 20/28] Fix error print in LustreCollector --- collectors/lustreMetric.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/collectors/lustreMetric.go b/collectors/lustreMetric.go index f98d746..66fd3fd 100644 --- a/collectors/lustreMetric.go +++ b/collectors/lustreMetric.go @@ -110,14 +110,16 @@ func (m *LustreCollector) Init(config json.RawMessage) error { "inode_permission": {"lustre_inode_permission": 1}} // Lustre file system statistics can only be queried by user root - user, err := user.Current() - if err != nil { - cclog.ComponentError(m.name, "Failed to get current user:", err.Error()) - return err - } - if user.Uid != "0" { - cclog.ComponentError(m.name, "Lustre file system statistics can only be queried by user root:", err.Error()) - return err + if !m.config.Sudo { + user, err := user.Current() + if err != nil { + cclog.ComponentError(m.name, "Failed to get current user:", err.Error()) + return err + } + if user.Uid != "0" { + cclog.ComponentError(m.name, "Lustre file system statistics can only be queried by user root") + return err + } } m.matches = make(map[string]map[string]int) From 5e64c01c9d3c1cb6895832ac243b66fbc8086cc6 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Fri, 4 Mar 2022 11:51:21 +0100 Subject: [PATCH 21/28] Fix name for ClusterCockpit user --- scripts/cc-metric-collector.sysusers | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/cc-metric-collector.sysusers b/scripts/cc-metric-collector.sysusers index 5c17ec7..6ce3700 100644 --- a/scripts/cc-metric-collector.sysusers +++ b/scripts/cc-metric-collector.sysusers @@ -1,2 +1,2 @@ -#Type Name ID GECOS Home directory Shell -u hpcop - "User for ClusterCockpit" /run/cc-metric-collector /sbin/nologin +#Type Name ID GECOS Home directory Shell +u clustercockpit - "User for ClusterCockpit" /run/cc-metric-collector /sbin/nologin From 7f62975a681607f770fb265867be116fe41d49e3 Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Fri, 4 Mar 2022 11:52:48 +0100 Subject: [PATCH 22/28] Set proper user for files --- scripts/cc-metric-collector.spec | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index 00a4aa4..8b23f20 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -51,15 +51,18 @@ install -Dpm 0644 scripts/%{name}.sysusers %{buildroot}%{_sysusersdir}/%{name}.c %systemd_preun %{name}.service %files +# Binary +%attr(-,clustercockpit,clustercockpit) %{_sbindir}/%{name} +# Config %dir %{_sysconfdir}/%{name} -%{_sbindir}/%{name} +%attr(0600,clustercockpit,clustercockpit) %config(noreplace) %{_sysconfdir}/%{name}/%{name}.json +%attr(0600,clustercockpit,clustercockpit) %config(noreplace) %{_sysconfdir}/%{name}/collectors.json +%attr(0600,clustercockpit,clustercockpit) %config(noreplace) %{_sysconfdir}/%{name}/sinks.json +%attr(0600,clustercockpit,clustercockpit) %config(noreplace) %{_sysconfdir}/%{name}/receivers.json +%attr(0600,clustercockpit,clustercockpit) %config(noreplace) %{_sysconfdir}/%{name}/router.json +# Systemd %{_unitdir}/%{name}.service %{_sysconfdir}/default/%{name} -%attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/%{name}.json -%attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/collectors.json -%attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/sinks.json -%attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/receivers.json -%attr(0600,root,root) %config(noreplace) %{_sysconfdir}/%{name}/router.json %{_sysusersdir}/%{name}.conf %changelog From 547bc0461f77d02002b1b8d46828b51536329dad Mon Sep 17 00:00:00 2001 From: Mehmet Soysal Date: Fri, 4 Mar 2022 14:35:47 +0100 Subject: [PATCH 23/28] Beegfs collector (#50) * added beegfs collectors to collectors/README.md * added beegfs collectors and docs * added new beegfs collectors to AvailableCollectors list * Feedback implemented * changed error type * changed error to only return * changed beegfs lookup path * fixed typo in md files Co-authored-by: Mehmet Soysal --- collectors/README.md | 2 + collectors/beegfsmetaMetric.go | 229 ++++++++++++++++++++++++++++++ collectors/beegfsmetaMetric.md | 75 ++++++++++ collectors/beegfsstorageMetric.go | 221 ++++++++++++++++++++++++++++ collectors/beegfsstorageMetric.md | 55 +++++++ collectors/collectorManager.go | 2 + 6 files changed, 584 insertions(+) create mode 100644 collectors/beegfsmetaMetric.go create mode 100644 collectors/beegfsmetaMetric.md create mode 100644 collectors/beegfsstorageMetric.go create mode 100644 collectors/beegfsstorageMetric.md diff --git a/collectors/README.md b/collectors/README.md index 00e0da7..3fcdd49 100644 --- a/collectors/README.md +++ b/collectors/README.md @@ -37,6 +37,8 @@ In contrast to the configuration files for sinks and receivers, the collectors c * [`cpufreq_cpuinfo`](./cpufreqCpuinfoMetric.md) * [`numastat`](./numastatMetric.md) * [`gpfs`](./gpfsMetric.md) +* [`beegfs_meta`](./beegfsmetaMetric.md) +* [`beegfs_storage`](./beegfsstorageMetric.md) ## Todos diff --git a/collectors/beegfsmetaMetric.go b/collectors/beegfsmetaMetric.go new file mode 100644 index 0000000..57b1e39 --- /dev/null +++ b/collectors/beegfsmetaMetric.go @@ -0,0 +1,229 @@ +package collectors + +import ( + "bufio" + "bytes" + "encoding/json" + "fmt" + "io/ioutil" + "os" + "os/exec" + "os/user" + "regexp" + "strconv" + "strings" + "time" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +const DEFAULT_BEEGFS_CMD = "beegfs-ctl" + +// Struct for the collector-specific JSON config +type BeegfsMetaCollectorConfig struct { + Beegfs string `json:"beegfs_path"` + ExcludeMetrics []string `json:"exclude_metrics,omitempty"` + ExcludeFilesystem []string `json:"exclude_filesystem"` +} + +type BeegfsMetaCollector struct { + metricCollector + tags map[string]string + matches map[string]string + config BeegfsMetaCollectorConfig + skipFS map[string]struct{} +} + +func (m *BeegfsMetaCollector) Init(config json.RawMessage) error { + // Check if already initialized + if m.init { + return nil + } + // Metrics + var nodeMdstat_array = [39]string{ + "sum", "ack", "close", "entInf", + "fndOwn", "mkdir", "create", "rddir", + "refrEn", "mdsInf", "rmdir", "rmLnk", + "mvDirIns", "mvFiIns", "open", "ren", + "sChDrct", "sAttr", "sDirPat", "stat", + "statfs", "trunc", "symlnk", "unlnk", + "lookLI", "statLI", "revalLI", "openLI", + "createLI", "hardlnk", "flckAp", "flckEn", + "flckRg", "dirparent", "listXA", "getXA", + "rmXA", "setXA", "mirror"} + + m.name = "BeegfsMetaCollector" + m.setup() + // Set default beegfs-ctl binary + + m.config.Beegfs = DEFAULT_BEEGFS_CMD + + // Read JSON configuration + if len(config) > 0 { + err := json.Unmarshal(config, &m.config) + if err != nil { + return err + } + } + + //create map with possible variables + m.matches = make(map[string]string) + for _, value := range nodeMdstat_array { + _, skip := stringArrayContains(m.config.ExcludeMetrics, value) + if skip { + m.matches["other"] = "0" + } else { + m.matches["beegfs_cmeta_"+value] = "0" + } + } + + m.meta = map[string]string{ + "source": m.name, + "group": "BeegfsMeta", + } + m.tags = map[string]string{ + "type": "node", + "filesystem": "", + } + m.skipFS = make(map[string]struct{}) + for _, fs := range m.config.ExcludeFilesystem { + m.skipFS[fs] = struct{}{} + } + + // Beegfs file system statistics can only be queried by user root + user, err := user.Current() + if err != nil { + return fmt.Errorf("BeegfsMetaCollector.Init(): Failed to get current user: %v", err) + } + if user.Uid != "0" { + return fmt.Errorf("BeegfsMetaCollector.Init(): BeeGFS file system statistics can only be queried by user root") + } + + // Check if beegfs-ctl is in executable search path + _, err = exec.LookPath(m.config.Beegfs) + if err != nil { + return fmt.Errorf("BeegfsMetaCollector.Init(): Failed to find beegfs-ctl binary '%s': %v", m.config.Beegfs, err) + } + m.init = true + return nil +} + +func (m *BeegfsMetaCollector) Read(interval time.Duration, output chan lp.CCMetric) { + if !m.init { + return + } + //get mounpoint + buffer, _ := ioutil.ReadFile(string("/proc/mounts")) + mounts := strings.Split(string(buffer), "\n") + var mountpoints []string + for _, line := range mounts { + if len(line) == 0 { + continue + } + f := strings.Fields(line) + if strings.Contains(f[0], "beegfs_ondemand") { + // Skip excluded filesystems + if _, skip := m.skipFS[f[1]]; skip { + continue + } + mountpoints = append(mountpoints, f[1]) + } + } + + if len(mountpoints) == 0 { + return + } + + for _, mountpoint := range mountpoints { + m.tags["filesystem"] = mountpoint + + // bwwgfs-ctl: + // --clientstats: Show client IO statistics. + // --nodetype=meta: The node type to query (meta, storage). + // --interval: + // --mount=/mnt/beeond/: Which mount point + //cmd := exec.Command(m.config.Beegfs, "/root/mc/test.txt") + mountoption := "--mount=" + mountpoint + cmd := exec.Command(m.config.Beegfs, "--clientstats", + "--nodetype=meta", mountoption, "--allstats") + cmd.Stdin = strings.NewReader("\n") + cmdStdout := new(bytes.Buffer) + cmdStderr := new(bytes.Buffer) + cmd.Stdout = cmdStdout + cmd.Stderr = cmdStderr + err := cmd.Run() + if err != nil { + fmt.Fprintf(os.Stderr, "BeegfsMetaCollector.Read(): Failed to execute command \"%s\": %s\n", cmd.String(), err.Error()) + fmt.Fprintf(os.Stderr, "BeegfsMetaCollector.Read(): command exit code: \"%d\"\n", cmd.ProcessState.ExitCode()) + data, _ := ioutil.ReadAll(cmdStderr) + fmt.Fprintf(os.Stderr, "BeegfsMetaCollector.Read(): command stderr: \"%s\"\n", string(data)) + data, _ = ioutil.ReadAll(cmdStdout) + fmt.Fprintf(os.Stderr, "BeegfsMetaCollector.Read(): command stdout: \"%s\"\n", string(data)) + return + } + // Read I/O statistics + scanner := bufio.NewScanner(cmdStdout) + + sumLine := regexp.MustCompile(`^Sum:\s+\d+\s+\[[a-zA-Z]+\]+`) + //Line := regexp.MustCompile(`^(.*)\s+(\d)+\s+\[([a-zA-Z]+)\]+`) + statsLine := regexp.MustCompile(`^(.*?)\s+?(\d.*?)$`) + singleSpacePattern := regexp.MustCompile(`\s+`) + removePattern := regexp.MustCompile(`[\[|\]]`) + + for scanner.Scan() { + readLine := scanner.Text() + //fmt.Println(readLine) + // Jump few lines, we only want the I/O stats from nodes + if !sumLine.MatchString(readLine) { + continue + } + + match := statsLine.FindStringSubmatch(readLine) + // nodeName = "Sum:" or would be nodes + // nodeName := match[1] + //Remove multiple whitespaces + dummy := removePattern.ReplaceAllString(match[2], " ") + metaStats := strings.TrimSpace(singleSpacePattern.ReplaceAllString(dummy, " ")) + split := strings.Split(metaStats, " ") + + // fill map with values + // split[i+1] = mdname + // split[i] = amount of md operations + for i := 0; i <= len(split)-1; i += 2 { + if _, ok := m.matches[split[i+1]]; ok { + m.matches["beegfs_cmeta_"+split[i+1]] = split[i] + } else { + f1, err := strconv.ParseFloat(m.matches["other"], 32) + if err != nil { + cclog.ComponentError( + m.name, + fmt.Sprintf("Metric (other): Failed to convert str written '%s' to float: %v", m.matches["other"], err)) + continue + } + f2, err := strconv.ParseFloat(split[i], 32) + if err != nil { + cclog.ComponentError( + m.name, + fmt.Sprintf("Metric (other): Failed to convert str written '%s' to float: %v", m.matches["other"], err)) + continue + } + //mdStat["other"] = fmt.Sprintf("%f", f1+f2) + m.matches["beegfs_cstorage_other"] = fmt.Sprintf("%f", f1+f2) + } + } + + for key, data := range m.matches { + value, _ := strconv.ParseFloat(data, 32) + y, err := lp.New(key, m.tags, m.meta, map[string]interface{}{"value": value}, time.Now()) + if err == nil { + output <- y + } + } + } + } +} + +func (m *BeegfsMetaCollector) Close() { + m.init = false +} diff --git a/collectors/beegfsmetaMetric.md b/collectors/beegfsmetaMetric.md new file mode 100644 index 0000000..932e72f --- /dev/null +++ b/collectors/beegfsmetaMetric.md @@ -0,0 +1,75 @@ +## `BeeGFS on Demand` collector +This Collector is to collect BeeGFS on Demand (BeeOND) metadata clientstats. + +```json + "beegfs_meta": { + "beegfs_path": "/usr/bin/beegfs-ctl", + "exclude_filesystem": [ + "/mnt/ignore_me" + ], + "exclude_metrics": [ + "ack", + "entInf", + "fndOwn" + ] + } +``` + +The `BeeGFS On Demand (BeeOND)` collector uses the `beegfs-ctl` command to read performance metrics for +BeeGFS filesystems. + +The reported filesystems can be filtered with the `exclude_filesystem` option +in the configuration. + +The path to the `beegfs-ctl` command can be configured with the `beegfs_path` option +in the configuration. + +When using the `exclude_metrics` option, the excluded metrics are summed as `other`. + +Important: The metrics listed below, are similar to the naming of BeeGFS. The Collector prefixes these with `beegfs_cstorage`(beegfs client storage). + +For example beegfs metric `open`-> `beegfs_cstorage_open` + +Available Metrics: + +* sum +* ack +* close +* entInf +* fndOwn +* mkdir +* create +* rddir +* refrEnt +* mdsInf +* rmdir +* rmLnk +* mvDirIns +* mvFiIns +* open +* ren +* sChDrct +* sAttr +* sDirPat +* stat +* statfs +* trunc +* symlnk +* unlnk +* lookLI +* statLI +* revalLI +* openLI +* createLI +* hardlnk +* flckAp +* flckEn +* flckRg +* dirparent +* listXA +* getXA +* rmXA +* setXA +* mirror + +The collector adds a `filesystem` tag to all metrics \ No newline at end of file diff --git a/collectors/beegfsstorageMetric.go b/collectors/beegfsstorageMetric.go new file mode 100644 index 0000000..cbc8314 --- /dev/null +++ b/collectors/beegfsstorageMetric.go @@ -0,0 +1,221 @@ +package collectors + +import ( + "bufio" + "bytes" + "encoding/json" + "fmt" + "io/ioutil" + "os" + "os/exec" + "os/user" + "regexp" + "strconv" + "strings" + "time" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" +) + +// Struct for the collector-specific JSON config +type BeegfsStorageCollectorConfig struct { + Beegfs string `json:"beegfs_path"` + ExcludeMetrics []string `json:"exclude_metrics,omitempty"` + ExcludeFilesystem []string `json:"exclude_filesystem"` +} + +type BeegfsStorageCollector struct { + metricCollector + tags map[string]string + matches map[string]string + config BeegfsStorageCollectorConfig + skipFS map[string]struct{} +} + +func (m *BeegfsStorageCollector) Init(config json.RawMessage) error { + // Check if already initialized + if m.init { + return nil + } + // Metrics + var storageStat_array = [18]string{ + "sum", "ack", "sChDrct", "getFSize", + "sAttr", "statfs", "trunc", "close", + "fsync", "ops-rd", "MiB-rd/s", "ops-wr", + "MiB-wr/s", "gendbg", "hrtbeat", "remNode", + "storInf", "unlnk"} + + m.name = "BeegfsStorageCollector" + m.setup() + // Set default beegfs-ctl binary + + m.config.Beegfs = DEFAULT_BEEGFS_CMD + + // Read JSON configuration + if len(config) > 0 { + err := json.Unmarshal(config, &m.config) + if err != nil { + return err + } + } + println(m.config.Beegfs) + //create map with possible variables + m.matches = make(map[string]string) + for _, value := range storageStat_array { + _, skip := stringArrayContains(m.config.ExcludeMetrics, value) + if skip { + m.matches["other"] = "0" + } else { + m.matches["beegfs_cstorage_"+value] = "0" + } + } + + m.meta = map[string]string{ + "source": m.name, + "group": "BeegfsStorage", + } + m.tags = map[string]string{ + "type": "node", + "filesystem": "", + } + m.skipFS = make(map[string]struct{}) + for _, fs := range m.config.ExcludeFilesystem { + m.skipFS[fs] = struct{}{} + } + + // Beegfs file system statistics can only be queried by user root + user, err := user.Current() + if err != nil { + return fmt.Errorf("BeegfsStorageCollector.Init(): Failed to get current user: %v", err) + } + if user.Uid != "0" { + return fmt.Errorf("BeegfsStorageCollector.Init(): BeeGFS file system statistics can only be queried by user root") + } + + // Check if beegfs-ctl is in executable search path + _, err = exec.LookPath(m.config.Beegfs) + if err != nil { + return fmt.Errorf("BeegfsStorageCollector.Init(): Failed to find beegfs-ctl binary '%s': %v", m.config.Beegfs, err) + } + m.init = true + return nil +} + +func (m *BeegfsStorageCollector) Read(interval time.Duration, output chan lp.CCMetric) { + if !m.init { + return + } + //get mounpoint + buffer, _ := ioutil.ReadFile(string("/proc/mounts")) + mounts := strings.Split(string(buffer), "\n") + var mountpoints []string + for _, line := range mounts { + if len(line) == 0 { + continue + } + f := strings.Fields(line) + if strings.Contains(f[0], "beegfs_ondemand") { + // Skip excluded filesystems + if _, skip := m.skipFS[f[1]]; skip { + continue + } + mountpoints = append(mountpoints, f[1]) + } + } + if len(mountpoints) == 0 { + return + } + // collects stats for each BeeGFS on Demand FS + for _, mountpoint := range mountpoints { + m.tags["filesystem"] = mountpoint + + // bwwgfs-ctl: + // --clientstats: Show client IO statistics. + // --nodetype=meta: The node type to query (meta, storage). + // --interval: + // --mount=/mnt/beeond/: Which mount point + //cmd := exec.Command(m.config.Beegfs, "/root/mc/test.txt") + mountoption := "--mount=" + mountpoint + cmd := exec.Command(m.config.Beegfs, "--clientstats", + "--nodetype=storage", mountoption, "--allstats") + cmd.Stdin = strings.NewReader("\n") + cmdStdout := new(bytes.Buffer) + cmdStderr := new(bytes.Buffer) + cmd.Stdout = cmdStdout + cmd.Stderr = cmdStderr + err := cmd.Run() + if err != nil { + fmt.Fprintf(os.Stderr, "BeegfsStorageCollector.Read(): Failed to execute command \"%s\": %s\n", cmd.String(), err.Error()) + fmt.Fprintf(os.Stderr, "BeegfsStorageCollector.Read(): command exit code: \"%d\"\n", cmd.ProcessState.ExitCode()) + data, _ := ioutil.ReadAll(cmdStderr) + fmt.Fprintf(os.Stderr, "BeegfsStorageCollector.Read(): command stderr: \"%s\"\n", string(data)) + data, _ = ioutil.ReadAll(cmdStdout) + fmt.Fprintf(os.Stderr, "BeegfsStorageCollector.Read(): command stdout: \"%s\"\n", string(data)) + return + } + // Read I/O statistics + scanner := bufio.NewScanner(cmdStdout) + + sumLine := regexp.MustCompile(`^Sum:\s+\d+\s+\[[a-zA-Z]+\]+`) + //Line := regexp.MustCompile(`^(.*)\s+(\d)+\s+\[([a-zA-Z]+)\]+`) + statsLine := regexp.MustCompile(`^(.*?)\s+?(\d.*?)$`) + singleSpacePattern := regexp.MustCompile(`\s+`) + removePattern := regexp.MustCompile(`[\[|\]]`) + + for scanner.Scan() { + readLine := scanner.Text() + //fmt.Println(readLine) + // Jump few lines, we only want the I/O stats from nodes + if !sumLine.MatchString(readLine) { + continue + } + + match := statsLine.FindStringSubmatch(readLine) + // nodeName = "Sum:" or would be nodes + // nodeName := match[1] + //Remove multiple whitespaces + dummy := removePattern.ReplaceAllString(match[2], " ") + metaStats := strings.TrimSpace(singleSpacePattern.ReplaceAllString(dummy, " ")) + split := strings.Split(metaStats, " ") + + // fill map with values + // split[i+1] = mdname + // split[i] = amount of operations + for i := 0; i <= len(split)-1; i += 2 { + if _, ok := m.matches[split[i+1]]; ok { + m.matches["beegfs_cstorage_"+split[i+1]] = split[i] + //m.matches[split[i+1]] = split[i] + } else { + f1, err := strconv.ParseFloat(m.matches["other"], 32) + if err != nil { + cclog.ComponentError( + m.name, + fmt.Sprintf("Metric (other): Failed to convert str written '%s' to float: %v", m.matches["other"], err)) + continue + } + f2, err := strconv.ParseFloat(split[i], 32) + if err != nil { + cclog.ComponentError( + m.name, + fmt.Sprintf("Metric (other): Failed to convert str written '%s' to float: %v", m.matches["other"], err)) + continue + } + m.matches["beegfs_cstorage_other"] = fmt.Sprintf("%f", f1+f2) + } + } + + for key, data := range m.matches { + value, _ := strconv.ParseFloat(data, 32) + y, err := lp.New(key, m.tags, m.meta, map[string]interface{}{"value": value}, time.Now()) + if err == nil { + output <- y + } + } + } + } +} + +func (m *BeegfsStorageCollector) Close() { + m.init = false +} diff --git a/collectors/beegfsstorageMetric.md b/collectors/beegfsstorageMetric.md new file mode 100644 index 0000000..519b5bf --- /dev/null +++ b/collectors/beegfsstorageMetric.md @@ -0,0 +1,55 @@ +## `BeeGFS on Demand` collector +This Collector is to collect BeeGFS on Demand (BeeOND) storage stats. + +```json + "beegfs_storage": { + "beegfs_path": "/usr/bin/beegfs-ctl", + "exclude_filesystem": [ + "/mnt/ignore_me" + ], + "exclude_metrics": [ + "ack", + "storInf", + "unlnk" + ] + } +``` + +The `BeeGFS On Demand (BeeOND)` collector uses the `beegfs-ctl` command to read performance metrics for BeeGFS filesystems. + +The reported filesystems can be filtered with the `exclude_filesystem` option +in the configuration. + +The path to the `beegfs-ctl` command can be configured with the `beegfs_path` option +in the configuration. + +When using the `exclude_metrics` option, the excluded metrics are summed as `other`. + +Important: The metrics listed below, are similar to the naming of BeeGFS. The Collector prefixes these with `beegfs_cstorage_`(beegfs client meta). +For example beegfs metric `open`-> `beegfs_cstorage_` + +Note: BeeGFS FS offers many Metadata Information. Probably it makes sense to exlcude most of them. Nevertheless, these excluded metrics will be summed as `beegfs_cstorage_other`. + +Available Metrics: + +* "sum" +* "ack" +* "sChDrct" +* "getFSize" +* "sAttr" +* "statfs" +* "trunc" +* "close" +* "fsync" +* "ops-rd" +* "MiB-rd/s" +* "ops-wr" +* "MiB-wr/s" +* "endbg" +* "hrtbeat" +* "remNode" +* "storInf" +* "unlnk" + + +The collector adds a `filesystem` tag to all metrics \ No newline at end of file diff --git a/collectors/collectorManager.go b/collectors/collectorManager.go index 46d791a..e9ccfe7 100644 --- a/collectors/collectorManager.go +++ b/collectors/collectorManager.go @@ -34,6 +34,8 @@ var AvailableCollectors = map[string]MetricCollector{ "nfs3stat": new(Nfs3Collector), "nfs4stat": new(Nfs4Collector), "numastats": new(NUMAStatsCollector), + "beegfs_meta": new(BeegfsMetaCollector), + "beegfs_storage": new(BeegfsStorageCollector), } // Metric collector manager data structure From 1961edc65968ed9eaae8f98541f68b68f6bb286b Mon Sep 17 00:00:00 2001 From: Thomas Roehl Date: Fri, 4 Mar 2022 15:42:25 +0100 Subject: [PATCH 24/28] Add documentation to help configuring the CC metric collector --- docs/configuration.md | 187 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 187 insertions(+) create mode 100644 docs/configuration.md diff --git a/docs/configuration.md b/docs/configuration.md new file mode 100644 index 0000000..75c7aa5 --- /dev/null +++ b/docs/configuration.md @@ -0,0 +1,187 @@ +# Configuring the CC metric collector + +The configuration of the CC metric collector consists of five configuration files: one global file and four component related files. + +## Global configuration + +The global file contains the paths to the other four files and some global options. + +```json +{ + "sinks": "sinks.json", + "collectors" : "collectors.json", + "receivers" : "receivers.json", + "router" : "router.json", + "interval": 10, + "duration": 1 +} +``` + +Be aware that the paths are relative to the execution folder of the cc-metric-collector binary, so it is recommended to use absolute paths. + +## Component configuration + +The others are mainly list of of subcomponents: the collectors, the receivers, the router and the sinks. Their role is best shown in a picture: + +```mermaid +flowchart LR + + subgraph col ["Collectors"] + direction TB + cpustat["cpustat"] + memstat["memstat"] + tempstat["tempstat"] + misc["..."] + end + + subgraph Receivers ["Receivers"] + direction TB + nats["NATS"] + miscrecv[...] + end + + subgraph calc["Aggregator"] + direction TB + cache["Cache"] + agg["Calculator"] + end + + subgraph sinks ["Sinks"] + direction RL + influx["InfluxDB"] + ganglia["Ganglia"] + logger["Logfile"] + miscsink["..."] + end + + cpustat --> CollectorManager["CollectorManager"] + memstat --> CollectorManager + tempstat --> CollectorManager + misc --> CollectorManager + + nats --> ReceiverManager["ReceiverManager"] + miscrecv --> ReceiverManager + + CollectorManager --> newrouter["Router"] + ReceiverManager -.-> newrouter + calc -.-> newrouter + newrouter --> SinkManager["SinkManager"] + newrouter -.-> calc + + SinkManager --> influx + SinkManager --> ganglia + SinkManager --> logger + SinkManager --> miscsink + + +``` + +There are four parts: +- The collectors read data from files, execute commands and call dynamically loaded library function and send it to the router +- The router can process metrics by cacheing and evaluating functions and conditions on them +- The sinks send the metrics to storage backends +- The receivers can be used to receive metrics from other collectors and forward them to the router. They can be used to create a tree-like structure of collectors. + +(A maybe better differentiation between collectors and receivers is that the collectors are called periodically while the receivers have their own logic and submit metrics at any time) + + +### Collectors configuration file + +The collectors configuration file tells which metrics should be queried from the system. The metric gathering is logically grouped in so called 'Collectors'. So there are Collectors to read CPU, memory or filesystem statistics. The collectors configuration file is a list of these collectors with collector-specific configurations: + +```json +{ + "cpustat" : {}, + "diskstat": { + "exclude_metrics": [ + "disk_total" + ] + } +} +``` + +The first one is the CPU statistics collector without any collector-specific setting. The second one enables disk mount statistics but excludes the metric `disk_total`. + +All names and possible collector-specific configuration options can be found [here](../collectors/README.md). + +Some collectors might dynamically load shared libraries. In order to enable these collectors, make sure that the shared library path is part of the `LD_LIBRARY_PATH` environment variable. + +### Sinks configuration file + +The sinks define the output/sending of metrics. The metrics can be forwarded to multiple sinks, even to sinks of the same type. The sinks configuration file is a list of these sinks, each with an individual name. + +```json +{ + "myinflux" : { + "type" : "influxasync", + "host": "localhost", + "port": "8086", + "organization" : "testorga", + "database" : "testbucket", + "password" : "" + }, + "companyinflux" : { + "type" : "influxasync", + "host": "companyhost", + "port": "8086", + "organization" : "company", + "database" : "main", + "password" : "" + } +} +``` + +The above example configuration file defines two sink, both ot type `influxasync`. They are differentiated internally by the names: `myinflux` and `companyinflux`. + +All types and possible sink-specific configuration options can be found [here](../sinks/README.md). + +Some sinks might dynamically load shared libraries. In order to enable these sinks, make sure that the shared library path is part of the `LD_LIBRARY_PATH` environment variable. + +### Router configuration file + +The collectors and the sinks are connected through the router. The router forwards the metrics to the sinks but enables some data processing. A common example is to tag all passing metrics like adding `cluster=mycluster`. But also aggregations like "take the average of all 'ipc' metrics" (ipc -> Instructions Per Cycle). Since the configurations of these aggregations can be quite complicated, we refer to the router's [README](../internal/metricRouter/README.md). + +A simple router configuration file to start with looks like this: + +```json +{ + "add_tags" : [ + { + "key" : "cluster", + "value" : "mycluster", + "if" : "*" + } + ], + "interval_timestamp" : false, + "num_cache_intervals" : 0 +} +``` + +With the `add_tags` section, we tell to attach the `cluster=mycluster` tag to each (`*` metric). The `interval_timestamp` tell the router to not touch the timestamp of metrics. It is possible to send all metrics within an interval with a common time stamp to avoid later alignment issues. The `num_cache_intervals` diables the cache completely. The cache is only required if you want to do complex metric aggregations. + +All configuration options can be found [here](../internal/metricRouter/README.md). + +### Receivers configuration file + +The receivers are a special feature of the CC Metric Collector to enable simpler integration into exising setups. While collectors query data from the local system, the receivers commonly get data from other systems through some network technology like HTTP or NATS. The idea is keep the current setup but send it to a CC Metric Collector which forwards it to the the destination system (if a sink exists for it). For most setups, the receivers are not required and an the receiver config file should contain only an empty JSON map (`{}`). + +```json +{ + "nats_rack0": { + "type": "nats", + "address" : "nats-server.example.org", + "port" : "4222", + "subject" : "rack0", + }, + "nats_rack1": { + "type": "nats", + "address" : "nats-server.example.org", + "port" : "4222", + "subject" : "rack1", + } +} +``` + +This example configuration creates two receivers with the names `nats_rack0` and `nats_rack1`. While one subscribes to metrics published with the `rack0` subject, the other one subscribes to the `rack0` subject. The NATS server is the same as it manages all subjects in a subnet. (As example, the router could add tags `rack=0` and `rack=1` respectively to the received metrics.) + +All types and possible receiver-specific configuration options can be found [here](../receivers/README.md). \ No newline at end of file From 8c668fcc6fda92b116783588c559646b6268adcf Mon Sep 17 00:00:00 2001 From: Thomas Gruber Date: Fri, 4 Mar 2022 18:33:57 +0100 Subject: [PATCH 25/28] Update configuration.md Add an additional receiver to have better alignment of components --- docs/configuration.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 75c7aa5..a13d3a5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -37,11 +37,12 @@ flowchart LR subgraph Receivers ["Receivers"] direction TB nats["NATS"] + httprecv["HTTP"] miscrecv[...] end subgraph calc["Aggregator"] - direction TB + direction LR cache["Cache"] agg["Calculator"] end @@ -60,6 +61,7 @@ flowchart LR misc --> CollectorManager nats --> ReceiverManager["ReceiverManager"] + httprecv --> ReceiverManager miscrecv --> ReceiverManager CollectorManager --> newrouter["Router"] @@ -184,4 +186,4 @@ The receivers are a special feature of the CC Metric Collector to enable simpler This example configuration creates two receivers with the names `nats_rack0` and `nats_rack1`. While one subscribes to metrics published with the `rack0` subject, the other one subscribes to the `rack0` subject. The NATS server is the same as it manages all subjects in a subnet. (As example, the router could add tags `rack=0` and `rack=1` respectively to the received metrics.) -All types and possible receiver-specific configuration options can be found [here](../receivers/README.md). \ No newline at end of file +All types and possible receiver-specific configuration options can be found [here](../receivers/README.md). From 21864e0ac4a790e4f9ab1a3af9e9ad01a661e751 Mon Sep 17 00:00:00 2001 From: Thomas Gruber Date: Sat, 5 Mar 2022 14:42:04 +0100 Subject: [PATCH 26/28] Change default GpfsCollector command to `mmpmon` (#53) * Set default cmd to 'mmpmon' * Reuse looked up path * Cast const to string --- collectors/gpfsMetric.go | 11 +++++++---- collectors/gpfsMetric.md | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/collectors/gpfsMetric.go b/collectors/gpfsMetric.go index adbc7fb..453704c 100644 --- a/collectors/gpfsMetric.go +++ b/collectors/gpfsMetric.go @@ -17,6 +17,8 @@ import ( lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" ) +const DEFAULT_GPFS_CMD = `mmpmon` + type GpfsCollector struct { metricCollector tags map[string]string @@ -38,7 +40,7 @@ func (m *GpfsCollector) Init(config json.RawMessage) error { m.setup() // Set default mmpmon binary - m.config.Mmpmon = "/usr/lpp/mmfs/bin/mmpmon" + m.config.Mmpmon = string(DEFAULT_GPFS_CMD) // Read JSON configuration if len(config) > 0 { @@ -64,17 +66,18 @@ func (m *GpfsCollector) Init(config json.RawMessage) error { // GPFS / IBM Spectrum Scale file system statistics can only be queried by user root user, err := user.Current() if err != nil { - return fmt.Errorf("Failed to get current user: %v", err) + return fmt.Errorf("failed to get current user: %v", err) } if user.Uid != "0" { return fmt.Errorf("GPFS file system statistics can only be queried by user root") } // Check if mmpmon is in executable search path - _, err = exec.LookPath(m.config.Mmpmon) + p, err := exec.LookPath(m.config.Mmpmon) if err != nil { - return fmt.Errorf("Failed to find mmpmon binary '%s': %v", m.config.Mmpmon, err) + return fmt.Errorf("failed to find mmpmon binary '%s': %v", m.config.Mmpmon, err) } + m.config.Mmpmon = p m.init = true return nil diff --git a/collectors/gpfsMetric.md b/collectors/gpfsMetric.md index 4a6a058..4f2c897 100644 --- a/collectors/gpfsMetric.md +++ b/collectors/gpfsMetric.md @@ -16,7 +16,7 @@ The reported filesystems can be filtered with the `exclude_filesystem` option in the configuration. The path to the `mmpmon` command can be configured with the `mmpmon_path` option -in the configuration. +in the configuration. If nothing is set, the collector searches in `$PATH` for `mmpmon`. Metrics: * `bytes_read` From f2486abeab7628290008bbfbb7b58b013fb99b2c Mon Sep 17 00:00:00 2001 From: Thomas Gruber Date: Sat, 5 Mar 2022 17:30:40 +0100 Subject: [PATCH 27/28] Just download LIKWID to get the headers (#54) * Just download LIKWID to get the headers * Remove perl-Data-Dumper from BuildRequires, only required by LIKWID build --- collectors/Makefile | 88 ++++++-------------------------- scripts/cc-metric-collector.spec | 4 +- 2 files changed, 19 insertions(+), 73 deletions(-) diff --git a/collectors/Makefile b/collectors/Makefile index 379a2d6..b07bccd 100644 --- a/collectors/Makefile +++ b/collectors/Makefile @@ -1,79 +1,25 @@ -# Use central installation -CENTRAL_INSTALL = false -# How to access hardware performance counters through LIKWID. -# Recommended is 'direct' mode -ACCESSMODE = direct -####################################################################### -# if CENTRAL_INSTALL == true -####################################################################### -# Path to central installation (if CENTRAL_INSTALL=true) -LIKWID_BASE=/apps/likwid/5.2.1 -# LIKWID version (should be same major version as central installation, 5.2.x) +all: likwid + + +# LIKWID version LIKWID_VERSION = 5.2.1 -####################################################################### -# if CENTRAL_INSTALL == false and ACCESSMODE == accessdaemon -####################################################################### -# Where to install the accessdaemon -DAEMON_INSTALLDIR = /usr/local -# Which user to use for the accessdaemon -DAEMON_USER = root -# Which group to use for the accessdaemon -DAEMON_GROUP = root +.ONESHELL: +.PHONY: likwid +likwid: + INSTALL_FOLDER="$${PWD}/likwid" + BUILD_FOLDER="$${PWD}/likwidbuild" + if [ -d $${INSTALL_FOLDER} ]; then rm -r $${INSTALL_FOLDER}; fi + mkdir --parents --verbose $${INSTALL_FOLDER} $${BUILD_FOLDER} + wget -P "$${BUILD_FOLDER}" ftp://ftp.rrze.uni-erlangen.de/mirrors/likwid/likwid-$(LIKWID_VERSION).tar.gz + tar -C $${BUILD_FOLDER} -xf $${BUILD_FOLDER}/likwid-$(LIKWID_VERSION).tar.gz + install -Dpm 0644 $${BUILD_FOLDER}/likwid-$(LIKWID_VERSION)/src/includes/likwid*.h $${INSTALL_FOLDER}/ + install -Dpm 0644 $${BUILD_FOLDER}/likwid-$(LIKWID_VERSION)/src/includes/bstrlib.h $${INSTALL_FOLDER}/ + rm -r $${BUILD_FOLDER} - -################################################# -# No need to change anything below this line -################################################# -INSTALL_FOLDER = ./likwid -BUILD_FOLDER = ./likwid/build - -ifneq ($(strip $(CENTRAL_INSTALL)),true) -LIKWID_BASE := $(shell pwd)/$(INSTALL_FOLDER) -DAEMON_BASE := $(LIKWID_BASE) -GROUPS_BASE := $(LIKWID_BASE)/groups -all: $(INSTALL_FOLDER)/liblikwid.a cleanup -else -DAEMON_BASE= $(LIKWID_BASE)/sbin -all: $(INSTALL_FOLDER)/liblikwid.a cleanup -endif - - - -$(BUILD_FOLDER)/likwid-$(LIKWID_VERSION).tar.gz: $(BUILD_FOLDER) - wget -P $(BUILD_FOLDER) ftp://ftp.rrze.uni-erlangen.de/mirrors/likwid/likwid-$(LIKWID_VERSION).tar.gz - -$(BUILD_FOLDER): - mkdir -p $(BUILD_FOLDER) - -$(INSTALL_FOLDER): - mkdir -p $(INSTALL_FOLDER) - -$(BUILD_FOLDER)/likwid-$(LIKWID_VERSION): $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION).tar.gz - tar -C $(BUILD_FOLDER) -xf $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION).tar.gz - -$(INSTALL_FOLDER)/liblikwid.a: $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION) $(INSTALL_FOLDER) - cd "$(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)" && make "PREFIX=$(LIKWID_BASE)" "SHARED_LIBRARY=false" "ACCESSMODE=$(ACCESSMODE)" "INSTALLED_ACCESSDAEMON=$(DAEMON_INSTALLDIR)/likwid-accessD" - cp \ - $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/liblikwid.a \ - $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/ext/hwloc/liblikwid-hwloc.a \ - $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/likwid*.h \ - $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/src/includes/bstrlib.h \ - $(INSTALL_FOLDER) - -$(DAEMON_INSTALLDIR)/likwid-accessD: $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/likwid-accessD - sudo -u $(DAEMON_USER) -g $(DAEMON_GROUP) install -m 4775 $(BUILD_FOLDER)/likwid-$(LIKWID_VERSION)/likwid-accessD $(DAEMON_INSTALLDIR)/likwid-accessD - -prepare_collector: likwidMetric.go - cp likwidMetric.go likwidMetric.go.orig - sed -i -e s+"const GROUPPATH =.*"+"const GROUPPATH = \`$(GROUPS_BASE)\`"+g likwidMetric.go - -cleanup: - rm -rf $(BUILD_FOLDER) - -clean: cleanup +clean: rm -rf likwid .PHONY: clean diff --git a/scripts/cc-metric-collector.spec b/scripts/cc-metric-collector.spec index 8b23f20..9d55b4f 100644 --- a/scripts/cc-metric-collector.spec +++ b/scripts/cc-metric-collector.spec @@ -8,8 +8,8 @@ Source0: %{name}-%{version}.tar.gz BuildRequires: go-toolset BuildRequires: systemd-rpm-macros -# for internal LIKWID installation -BuildRequires: wget perl-Data-Dumper +# for header downloads +BuildRequires: wget Provides: %{name} = %{version} From c0e600269ada6acf069b21ff2ef3bce0733d92c1 Mon Sep 17 00:00:00 2001 From: Thomas Gruber Date: Sat, 5 Mar 2022 17:30:55 +0100 Subject: [PATCH 28/28] Add HttpReceiver as counterpart to the HttpSink (#49) --- receivers/README.md | 1 + receivers/httpReceiver.go | 118 ++++++++++++++++++++++++++++++++++++++ receivers/httpReceiver.md | 23 ++++++++ 3 files changed, 142 insertions(+) create mode 100644 receivers/httpReceiver.go create mode 100644 receivers/httpReceiver.md diff --git a/receivers/README.md b/receivers/README.md index 70b209b..3599a93 100644 --- a/receivers/README.md +++ b/receivers/README.md @@ -21,6 +21,7 @@ This allows to specify - [`nats`](./natsReceiver.md): Receive metrics from the NATS network - [`prometheus`](./prometheusReceiver.md): Scrape data from a Prometheus client +- [`http`](./httpReceiver.md): Listen for HTTP Post requests transporting metrics in InfluxDB line protocol # Contributing own receivers A receiver contains a few functions and is derived from the type `Receiver` (in `metricReceiver.go`): diff --git a/receivers/httpReceiver.go b/receivers/httpReceiver.go new file mode 100644 index 0000000..e66ad5e --- /dev/null +++ b/receivers/httpReceiver.go @@ -0,0 +1,118 @@ +package receivers + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "io/ioutil" + "net/http" + "strings" + "sync" + + cclog "github.com/ClusterCockpit/cc-metric-collector/internal/ccLogger" + lp "github.com/ClusterCockpit/cc-metric-collector/internal/ccMetric" + "github.com/gorilla/mux" + influx "github.com/influxdata/line-protocol" +) + +const HTTP_RECEIVER_PORT = "8080" + +type HttpReceiverConfig struct { + Type string `json:"type"` + Addr string `json:"address"` + Port string `json:"port"` + Path string `json:"path"` +} + +type HttpReceiver struct { + receiver + handler *influx.MetricHandler + parser *influx.Parser + meta map[string]string + config HttpReceiverConfig + router *mux.Router + server *http.Server + wg sync.WaitGroup +} + +func (r *HttpReceiver) Init(name string, config json.RawMessage) error { + r.name = fmt.Sprintf("HttpReceiver(%s)", name) + r.config.Port = HTTP_RECEIVER_PORT + if len(config) > 0 { + err := json.Unmarshal(config, &r.config) + if err != nil { + cclog.ComponentError(r.name, "Error reading config:", err.Error()) + return err + } + } + if len(r.config.Port) == 0 { + return errors.New("not all configuration variables set required by HttpReceiver") + } + r.meta = map[string]string{"source": r.name} + p := r.config.Path + if !strings.HasPrefix(p, "/") { + p = "/" + p + } + uri := fmt.Sprintf("%s:%s%s", r.config.Addr, r.config.Port, p) + cclog.ComponentDebug(r.name, "INIT", uri) + r.handler = influx.NewMetricHandler() + r.parser = influx.NewParser(r.handler) + r.parser.SetTimeFunc(DefaultTime) + + r.router = mux.NewRouter() + r.router.Path(p).HandlerFunc(r.ServerHttp) + r.server = &http.Server{Addr: uri, Handler: r.router} + return nil +} + +func (r *HttpReceiver) Start() { + cclog.ComponentDebug(r.name, "START") + r.wg.Add(1) + go func() { + err := r.server.ListenAndServe() + if err != nil && err.Error() != "http: Server closed" { + cclog.ComponentError(r.name, err.Error()) + } + r.wg.Done() + }() +} + +func (r *HttpReceiver) ServerHttp(w http.ResponseWriter, req *http.Request) { + if req.Method != http.MethodPost { + http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed) + return + } + + body, err := ioutil.ReadAll(req.Body) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + metrics, err := r.parser.Parse(body) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + } + + for _, m := range metrics { + y := lp.FromInfluxMetric(m) + for k, v := range r.meta { + y.AddMeta(k, v) + } + if r.sink != nil { + r.sink <- y + } + } + + w.WriteHeader(http.StatusOK) +} + +func (r *HttpReceiver) Close() { + r.server.Shutdown(context.Background()) +} + +func NewHttpReceiver(name string, config json.RawMessage) (Receiver, error) { + r := new(HttpReceiver) + err := r.Init(name, config) + return r, err +} diff --git a/receivers/httpReceiver.md b/receivers/httpReceiver.md new file mode 100644 index 0000000..ed1e1bf --- /dev/null +++ b/receivers/httpReceiver.md @@ -0,0 +1,23 @@ +## `http` receiver + +The `http` receiver can be used receive metrics through HTTP POST requests. + +### Configuration structure + +```json +{ + "": { + "type": "http", + "address" : "", + "port" : "8080", + "path" : "/write" + } +} +``` + +- `type`: makes the receiver a `http` receiver +- `address`: Listen address +- `port`: Listen port +- `path`: URL path for the write endpoint + +The HTTP endpoint listens to `http://
:/`