Combined metricstore api and functions

This commit is contained in:
Aditya Ujeniya
2025-09-08 11:29:27 +02:00
parent bca176170c
commit 62565b9ae2
26 changed files with 1248 additions and 430 deletions

419
internal/memorystore/api.go Normal file
View File

@@ -0,0 +1,419 @@
// Copyright (C) NHR@FAU, University Erlangen-Nuremberg.
// All rights reserved.
// Use of this source code is governed by a MIT-style
// license that can be found in the LICENSE file.
package memorystore
import (
"bufio"
"encoding/json"
"errors"
"fmt"
"io"
"log"
"math"
"net/http"
"strconv"
"strings"
"github.com/ClusterCockpit/cc-lib/schema"
"github.com/ClusterCockpit/cc-lib/util"
"github.com/influxdata/line-protocol/v2/lineprotocol"
)
// @title cc-metric-store REST API
// @version 1.0.0
// @description API for cc-metric-store
// @contact.name ClusterCockpit Project
// @contact.url https://clustercockpit.org
// @contact.email support@clustercockpit.org
// @license.name MIT License
// @license.url https://opensource.org/licenses/MIT
// @host localhost:8082
// @basePath /api/
// @securityDefinitions.apikey ApiKeyAuth
// @in header
// @name X-Auth-Token
// ErrorResponse model
type ErrorResponse struct {
// Statustext of Errorcode
Status string `json:"status"`
Error string `json:"error"` // Error Message
}
type ApiMetricData struct {
Error *string `json:"error,omitempty"`
Data schema.FloatArray `json:"data,omitempty"`
From int64 `json:"from"`
To int64 `json:"to"`
Resolution int64 `json:"resolution"`
Avg schema.Float `json:"avg"`
Min schema.Float `json:"min"`
Max schema.Float `json:"max"`
}
func handleError(err error, statusCode int, rw http.ResponseWriter) {
// log.Warnf("REST ERROR : %s", err.Error())
rw.Header().Add("Content-Type", "application/json")
rw.WriteHeader(statusCode)
json.NewEncoder(rw).Encode(ErrorResponse{
Status: http.StatusText(statusCode),
Error: err.Error(),
})
}
// TODO: Optimize this, just like the stats endpoint!
func (data *ApiMetricData) AddStats() {
n := 0
sum, min, max := 0.0, math.MaxFloat64, -math.MaxFloat64
for _, x := range data.Data {
if x.IsNaN() {
continue
}
n += 1
sum += float64(x)
min = math.Min(min, float64(x))
max = math.Max(max, float64(x))
}
if n > 0 {
avg := sum / float64(n)
data.Avg = schema.Float(avg)
data.Min = schema.Float(min)
data.Max = schema.Float(max)
} else {
data.Avg, data.Min, data.Max = schema.NaN, schema.NaN, schema.NaN
}
}
func (data *ApiMetricData) ScaleBy(f schema.Float) {
if f == 0 || f == 1 {
return
}
data.Avg *= f
data.Min *= f
data.Max *= f
for i := 0; i < len(data.Data); i++ {
data.Data[i] *= f
}
}
func (data *ApiMetricData) PadDataWithNull(ms *MemoryStore, from, to int64, metric string) {
minfo, ok := ms.Metrics[metric]
if !ok {
return
}
if (data.From / minfo.Frequency) > (from / minfo.Frequency) {
padfront := int((data.From / minfo.Frequency) - (from / minfo.Frequency))
ndata := make([]schema.Float, 0, padfront+len(data.Data))
for i := 0; i < padfront; i++ {
ndata = append(ndata, schema.NaN)
}
for j := 0; j < len(data.Data); j++ {
ndata = append(ndata, data.Data[j])
}
data.Data = ndata
}
}
// handleFree godoc
// @summary
// @tags free
// @description This endpoint allows the users to free the Buffers from the
// metric store. This endpoint offers the users to remove then systematically
// and also allows then to prune the data under node, if they do not want to
// remove the whole node.
// @produce json
// @param to query string false "up to timestamp"
// @success 200 {string} string "ok"
// @failure 400 {object} api.ErrorResponse "Bad Request"
// @failure 401 {object} api.ErrorResponse "Unauthorized"
// @failure 403 {object} api.ErrorResponse "Forbidden"
// @failure 500 {object} api.ErrorResponse "Internal Server Error"
// @security ApiKeyAuth
// @router /free/ [post]
func HandleFree(rw http.ResponseWriter, r *http.Request) {
rawTo := r.URL.Query().Get("to")
if rawTo == "" {
handleError(errors.New("'to' is a required query parameter"), http.StatusBadRequest, rw)
return
}
to, err := strconv.ParseInt(rawTo, 10, 64)
if err != nil {
handleError(err, http.StatusInternalServerError, rw)
return
}
// // TODO: lastCheckpoint might be modified by different go-routines.
// // Load it using the sync/atomic package?
// freeUpTo := lastCheckpoint.Unix()
// if to < freeUpTo {
// freeUpTo = to
// }
bodyDec := json.NewDecoder(r.Body)
var selectors [][]string
err = bodyDec.Decode(&selectors)
if err != nil {
http.Error(rw, err.Error(), http.StatusBadRequest)
return
}
ms := GetMemoryStore()
n := 0
for _, sel := range selectors {
bn, err := ms.Free(sel, to)
if err != nil {
handleError(err, http.StatusInternalServerError, rw)
return
}
n += bn
}
rw.WriteHeader(http.StatusOK)
fmt.Fprintf(rw, "buffers freed: %d\n", n)
}
// handleWrite godoc
// @summary Receive metrics in InfluxDB line-protocol
// @tags write
// @description Write data to the in-memory store in the InfluxDB line-protocol using [this format](https://github.com/ClusterCockpit/cc-specifications/blob/master/metrics/lineprotocol_alternative.md)
// @accept plain
// @produce json
// @param cluster query string false "If the lines in the body do not have a cluster tag, use this value instead."
// @success 200 {string} string "ok"
// @failure 400 {object} api.ErrorResponse "Bad Request"
// @failure 401 {object} api.ErrorResponse "Unauthorized"
// @failure 403 {object} api.ErrorResponse "Forbidden"
// @failure 500 {object} api.ErrorResponse "Internal Server Error"
// @security ApiKeyAuth
// @router /write/ [post]
func HandleWrite(rw http.ResponseWriter, r *http.Request) {
bytes, err := io.ReadAll(r.Body)
rw.Header().Add("Content-Type", "application/json")
if err != nil {
handleError(err, http.StatusInternalServerError, rw)
return
}
ms := GetMemoryStore()
dec := lineprotocol.NewDecoderWithBytes(bytes)
if err := decodeLine(dec, ms, r.URL.Query().Get("cluster")); err != nil {
log.Printf("/api/write error: %s", err.Error())
handleError(err, http.StatusBadRequest, rw)
return
}
rw.WriteHeader(http.StatusOK)
}
type ApiQueryRequest struct {
Cluster string `json:"cluster"`
Queries []ApiQuery `json:"queries"`
ForAllNodes []string `json:"for-all-nodes"`
From int64 `json:"from"`
To int64 `json:"to"`
WithStats bool `json:"with-stats"`
WithData bool `json:"with-data"`
WithPadding bool `json:"with-padding"`
}
type ApiQueryResponse struct {
Queries []ApiQuery `json:"queries,omitempty"`
Results [][]ApiMetricData `json:"results"`
}
type ApiQuery struct {
Type *string `json:"type,omitempty"`
SubType *string `json:"subtype,omitempty"`
Metric string `json:"metric"`
Hostname string `json:"host"`
Resolution int64 `json:"resolution"`
TypeIds []string `json:"type-ids,omitempty"`
SubTypeIds []string `json:"subtype-ids,omitempty"`
ScaleFactor schema.Float `json:"scale-by,omitempty"`
Aggregate bool `json:"aggreg"`
}
func FetchData(req ApiQueryRequest) (*ApiQueryResponse, error) {
req.WithData = true
req.WithData = true
req.WithData = true
ms := GetMemoryStore()
response := ApiQueryResponse{
Results: make([][]ApiMetricData, 0, len(req.Queries)),
}
if req.ForAllNodes != nil {
nodes := ms.ListChildren([]string{req.Cluster})
for _, node := range nodes {
for _, metric := range req.ForAllNodes {
q := ApiQuery{
Metric: metric,
Hostname: node,
}
req.Queries = append(req.Queries, q)
response.Queries = append(response.Queries, q)
}
}
}
for _, query := range req.Queries {
sels := make([]util.Selector, 0, 1)
if query.Aggregate || query.Type == nil {
sel := util.Selector{{String: req.Cluster}, {String: query.Hostname}}
if query.Type != nil {
if len(query.TypeIds) == 1 {
sel = append(sel, util.SelectorElement{String: *query.Type + query.TypeIds[0]})
} else {
ids := make([]string, len(query.TypeIds))
for i, id := range query.TypeIds {
ids[i] = *query.Type + id
}
sel = append(sel, util.SelectorElement{Group: ids})
}
if query.SubType != nil {
if len(query.SubTypeIds) == 1 {
sel = append(sel, util.SelectorElement{String: *query.SubType + query.SubTypeIds[0]})
} else {
ids := make([]string, len(query.SubTypeIds))
for i, id := range query.SubTypeIds {
ids[i] = *query.SubType + id
}
sel = append(sel, util.SelectorElement{Group: ids})
}
}
}
sels = append(sels, sel)
} else {
for _, typeId := range query.TypeIds {
if query.SubType != nil {
for _, subTypeId := range query.SubTypeIds {
sels = append(sels, util.Selector{
{String: req.Cluster},
{String: query.Hostname},
{String: *query.Type + typeId},
{String: *query.SubType + subTypeId},
})
}
} else {
sels = append(sels, util.Selector{
{String: req.Cluster},
{String: query.Hostname},
{String: *query.Type + typeId},
})
}
}
}
// log.Printf("query: %#v\n", query)
// log.Printf("sels: %#v\n", sels)
var err error
res := make([]ApiMetricData, 0, len(sels))
for _, sel := range sels {
data := ApiMetricData{}
data.Data, data.From, data.To, data.Resolution, err = ms.Read(sel, query.Metric, req.From, req.To, query.Resolution)
if err != nil {
msg := err.Error()
data.Error = &msg
res = append(res, data)
continue
}
if req.WithStats {
data.AddStats()
}
if query.ScaleFactor != 0 {
data.ScaleBy(query.ScaleFactor)
}
if req.WithPadding {
data.PadDataWithNull(ms, req.From, req.To, query.Metric)
}
if !req.WithData {
data.Data = nil
}
res = append(res, data)
}
response.Results = append(response.Results, res)
}
return &response, nil
}
// handleDebug godoc
// @summary Debug endpoint
// @tags debug
// @description This endpoint allows the users to print the content of
// nodes/clusters/metrics to review the state of the data.
// @produce json
// @param selector query string false "Selector"
// @success 200 {string} string "Debug dump"
// @failure 400 {object} api.ErrorResponse "Bad Request"
// @failure 401 {object} api.ErrorResponse "Unauthorized"
// @failure 403 {object} api.ErrorResponse "Forbidden"
// @failure 500 {object} api.ErrorResponse "Internal Server Error"
// @security ApiKeyAuth
// @router /debug/ [post]
func HandleDebug(rw http.ResponseWriter, r *http.Request) {
raw := r.URL.Query().Get("selector")
rw.Header().Add("Content-Type", "application/json")
selector := []string{}
if len(raw) != 0 {
selector = strings.Split(raw, ":")
}
ms := GetMemoryStore()
if err := ms.DebugDump(bufio.NewWriter(rw), selector); err != nil {
handleError(err, http.StatusBadRequest, rw)
return
}
}
// handleHealthCheck godoc
// @summary HealthCheck endpoint
// @tags healthcheck
// @description This endpoint allows the users to check if a node is healthy
// @produce json
// @param selector query string false "Selector"
// @success 200 {string} string "Debug dump"
// @failure 400 {object} api.ErrorResponse "Bad Request"
// @failure 401 {object} api.ErrorResponse "Unauthorized"
// @failure 403 {object} api.ErrorResponse "Forbidden"
// @failure 500 {object} api.ErrorResponse "Internal Server Error"
// @security ApiKeyAuth
// @router /healthcheck/ [get]
func HandleHealthCheck(rw http.ResponseWriter, r *http.Request) {
rawCluster := r.URL.Query().Get("cluster")
rawNode := r.URL.Query().Get("node")
if rawCluster == "" || rawNode == "" {
handleError(errors.New("'cluster' and 'node' are required query parameter"), http.StatusBadRequest, rw)
return
}
rw.Header().Add("Content-Type", "application/json")
selector := []string{rawCluster, rawNode}
ms := GetMemoryStore()
if err := ms.HealthCheck(bufio.NewWriter(rw), selector); err != nil {
handleError(err, http.StatusBadRequest, rw)
return
}
}

View File

@@ -17,15 +17,16 @@ import (
"sync/atomic"
"time"
"github.com/ClusterCockpit/cc-backend/internal/config"
cclog "github.com/ClusterCockpit/cc-lib/ccLogger"
)
func Archiving(wg *sync.WaitGroup, ctx context.Context) {
go func() {
defer wg.Done()
d, err := time.ParseDuration(Keys.Archive.Interval)
d, err := time.ParseDuration(config.MetricStoreKeys.Archive.Interval)
if err != nil {
cclog.Fatalf("error parsing archive interval duration: %v\n", err)
cclog.Fatalf("[METRICSTORE]> error parsing archive interval duration: %v\n", err)
}
if d <= 0 {
return
@@ -43,14 +44,14 @@ func Archiving(wg *sync.WaitGroup, ctx context.Context) {
return
case <-ticks:
t := time.Now().Add(-d)
cclog.Infof("start archiving checkpoints (older than %s)...\n", t.Format(time.RFC3339))
n, err := ArchiveCheckpoints(Keys.Checkpoints.RootDir,
Keys.Archive.RootDir, t.Unix(), Keys.Archive.DeleteInstead)
cclog.Infof("[METRICSTORE]> start archiving checkpoints (older than %s)...\n", t.Format(time.RFC3339))
n, err := ArchiveCheckpoints(config.MetricStoreKeys.Checkpoints.RootDir,
config.MetricStoreKeys.Archive.RootDir, t.Unix(), config.MetricStoreKeys.Archive.DeleteInstead)
if err != nil {
cclog.Warnf("archiving failed: %s\n", err.Error())
cclog.Warnf("[METRICSTORE]> archiving failed: %s\n", err.Error())
} else {
cclog.Infof("done: %d files zipped and moved to archive\n", n)
cclog.Infof("[METRICSTORE]> done: %d files zipped and moved to archive\n", n)
}
}
}
@@ -127,7 +128,7 @@ func archiveCheckpoints(dir string, archiveDir string, from int64, deleteInstead
return 0, err
}
extension := Keys.Checkpoints.FileFormat
extension := config.MetricStoreKeys.Checkpoints.FileFormat
files, err := findFiles(entries, from, extension, false)
if err != nil {
return 0, err

View File

@@ -4,7 +4,7 @@ import (
"errors"
"sync"
"github.com/ClusterCockpit/cc-lib/util"
"github.com/ClusterCockpit/cc-lib/schema"
)
// Default buffer capacity.
@@ -19,14 +19,14 @@ const (
var bufferPool sync.Pool = sync.Pool{
New: func() interface{} {
return &buffer{
data: make([]util.Float, 0, BUFFER_CAP),
data: make([]schema.Float, 0, BUFFER_CAP),
}
},
}
var (
ErrNoData error = errors.New("no data for this metric/level")
ErrDataDoesNotAlign error = errors.New("data from lower granularities does not align")
ErrNoData error = errors.New("[METRICSTORE]> no data for this metric/level")
ErrDataDoesNotAlign error = errors.New("[METRICSTORE]> data from lower granularities does not align")
)
// Each metric on each level has it's own buffer.
@@ -36,7 +36,7 @@ var (
type buffer struct {
prev *buffer
next *buffer
data []util.Float
data []schema.Float
frequency int64
start int64
archived bool
@@ -59,9 +59,9 @@ func newBuffer(ts, freq int64) *buffer {
// Otherwise, the existing buffer is returnd.
// Normaly, only "newer" data should be written, but if the value would
// end up in the same buffer anyways it is allowed.
func (b *buffer) write(ts int64, value util.Float) (*buffer, error) {
func (b *buffer) write(ts int64, value schema.Float) (*buffer, error) {
if ts < b.start {
return nil, errors.New("cannot write value to buffer from past")
return nil, errors.New("[METRICSTORE]> cannot write value to buffer from past")
}
// idx := int((ts - b.start + (b.frequency / 3)) / b.frequency)
@@ -83,7 +83,7 @@ func (b *buffer) write(ts int64, value util.Float) (*buffer, error) {
// Fill up unwritten slots with NaN
for i := len(b.data); i < idx; i++ {
b.data = append(b.data, util.NaN)
b.data = append(b.data, schema.NaN)
}
b.data = append(b.data, value)
@@ -147,7 +147,7 @@ func (b *buffer) close() {
// This function goes back the buffer chain if `from` is older than the currents buffer start.
// The loaded values are added to `data` and `data` is returned, possibly with a shorter length.
// If `data` is not long enough to hold all values, this function will panic!
func (b *buffer) read(from, to int64, data []util.Float) ([]util.Float, int64, int64, error) {
func (b *buffer) read(from, to int64, data []schema.Float) ([]schema.Float, int64, int64, error) {
if from < b.firstWrite() {
if b.prev != nil {
return b.prev.read(from, to, data)
@@ -171,9 +171,9 @@ func (b *buffer) read(from, to int64, data []util.Float) ([]util.Float, int64, i
if b.next == nil || to <= b.next.start {
break
}
data[i] += util.NaN
data[i] += schema.NaN
} else if t < b.start {
data[i] += util.NaN
data[i] += schema.NaN
// } else if b.data[idx].IsNaN() {
// data[i] += interpolate(idx, b.data)
} else {

View File

@@ -20,15 +20,16 @@ import (
"time"
"github.com/ClusterCockpit/cc-backend/internal/avro"
"github.com/ClusterCockpit/cc-lib/util"
"github.com/ClusterCockpit/cc-backend/internal/config"
"github.com/ClusterCockpit/cc-lib/schema"
"github.com/linkedin/goavro/v2"
)
// Whenever changed, update MarshalJSON as well!
type CheckpointMetrics struct {
Data []util.Float `json:"data"`
Frequency int64 `json:"frequency"`
Start int64 `json:"start"`
Data []schema.Float `json:"data"`
Frequency int64 `json:"frequency"`
Start int64 `json:"start"`
}
type CheckpointFile struct {
@@ -43,12 +44,12 @@ var lastCheckpoint time.Time
func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
lastCheckpoint = time.Now()
if Keys.Checkpoints.FileFormat == "json" {
if config.MetricStoreKeys.Checkpoints.FileFormat == "json" {
ms := GetMemoryStore()
go func() {
defer wg.Done()
d, err := time.ParseDuration(Keys.Checkpoints.Interval)
d, err := time.ParseDuration(config.MetricStoreKeys.Checkpoints.Interval)
if err != nil {
log.Fatal(err)
}
@@ -67,14 +68,14 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
case <-ctx.Done():
return
case <-ticks:
log.Printf("start checkpointing (starting at %s)...\n", lastCheckpoint.Format(time.RFC3339))
log.Printf("[METRICSTORE]> start checkpointing (starting at %s)...\n", lastCheckpoint.Format(time.RFC3339))
now := time.Now()
n, err := ms.ToCheckpoint(Keys.Checkpoints.RootDir,
n, err := ms.ToCheckpoint(config.MetricStoreKeys.Checkpoints.RootDir,
lastCheckpoint.Unix(), now.Unix())
if err != nil {
log.Printf("checkpointing failed: %s\n", err.Error())
log.Printf("[METRICSTORE]> checkpointing failed: %s\n", err.Error())
} else {
log.Printf("done: %d checkpoint files created\n", n)
log.Printf("[METRICSTORE]> done: %d checkpoint files created\n", n)
lastCheckpoint = now
}
}
@@ -90,7 +91,7 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
return
case <-time.After(time.Duration(avro.CheckpointBufferMinutes) * time.Minute):
// This is the first tick untill we collect the data for given minutes.
avro.GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, false)
avro.GetAvroStore().ToCheckpoint(config.MetricStoreKeys.Checkpoints.RootDir, false)
// log.Printf("Checkpointing %d avro files", count)
}
@@ -108,7 +109,7 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
return
case <-ticks:
// Regular ticks of 1 minute to write data.
avro.GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, false)
avro.GetAvroStore().ToCheckpoint(config.MetricStoreKeys.Checkpoints.RootDir, false)
// log.Printf("Checkpointing %d avro files", count)
}
}
@@ -179,7 +180,7 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
continue
}
log.Printf("error while checkpointing %#v: %s", workItem.selector, err.Error())
log.Printf("[METRICSTORE]> error while checkpointing %#v: %s", workItem.selector, err.Error())
atomic.AddInt32(&errs, 1)
} else {
atomic.AddInt32(&n, 1)
@@ -201,7 +202,7 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
wg.Wait()
if errs > 0 {
return int(n), fmt.Errorf("%d errors happend while creating checkpoints (%d successes)", errs, n)
return int(n), fmt.Errorf("[METRICSTORE]> %d errors happend while creating checkpoints (%d successes)", errs, n)
}
return int(n), nil
}
@@ -235,14 +236,14 @@ func (l *Level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFil
continue
}
data := make([]util.Float, (to-from)/b.frequency+1)
data := make([]schema.Float, (to-from)/b.frequency+1)
data, start, end, err := b.read(from, to, data)
if err != nil {
return nil, err
}
for i := int((end - start) / b.frequency); i < len(data); i++ {
data[i] = util.NaN
data[i] = schema.NaN
}
retval.Metrics[metric] = &CheckpointMetrics{
@@ -314,7 +315,7 @@ func (m *MemoryStore) FromCheckpoint(dir string, from int64, extension string) (
lvl := m.root.findLevelOrCreate(host[:], len(m.Metrics))
nn, err := lvl.fromCheckpoint(m, filepath.Join(dir, host[0], host[1]), from, extension)
if err != nil {
log.Fatalf("error while loading checkpoints: %s", err.Error())
log.Fatalf("[METRICSTORE]> error while loading checkpoints: %s", err.Error())
atomic.AddInt32(&errs, 1)
}
atomic.AddInt32(&n, int32(nn))
@@ -326,7 +327,7 @@ func (m *MemoryStore) FromCheckpoint(dir string, from int64, extension string) (
clustersDir, err := os.ReadDir(dir)
for _, clusterDir := range clustersDir {
if !clusterDir.IsDir() {
err = errors.New("expected only directories at first level of checkpoints/ directory")
err = errors.New("[METRICSTORE]> expected only directories at first level of checkpoints/ directory")
goto done
}
@@ -338,7 +339,7 @@ func (m *MemoryStore) FromCheckpoint(dir string, from int64, extension string) (
for _, hostDir := range hostsDir {
if !hostDir.IsDir() {
err = errors.New("expected only directories at second level of checkpoints/ directory")
err = errors.New("[METRICSTORE]> expected only directories at second level of checkpoints/ directory")
goto done
}
@@ -364,7 +365,7 @@ done:
}
if errs > 0 {
return int(n), fmt.Errorf("%d errors happend while creating checkpoints (%d successes)", errs, n)
return int(n), fmt.Errorf("[METRICSTORE]> %d errors happend while creating checkpoints (%d successes)", errs, n)
}
return int(n), nil
}
@@ -377,13 +378,13 @@ func (m *MemoryStore) FromCheckpointFiles(dir string, from int64) (int, error) {
// The directory does not exist, so create it using os.MkdirAll()
err := os.MkdirAll(dir, 0755) // 0755 sets the permissions for the directory
if err != nil {
log.Fatalf("Error creating directory: %#v\n", err)
log.Fatalf("[METRICSTORE]> Error creating directory: %#v\n", err)
}
fmt.Printf("%#v Directory created successfully.\n", dir)
fmt.Printf("[METRICSTORE]> %#v Directory created successfully.\n", dir)
}
// Config read (replace with your actual config read)
fileFormat := Keys.Checkpoints.FileFormat
fileFormat := config.MetricStoreKeys.Checkpoints.FileFormat
if fileFormat == "" {
fileFormat = "avro"
}
@@ -396,22 +397,22 @@ func (m *MemoryStore) FromCheckpointFiles(dir string, from int64) (int, error) {
// First, attempt to load the specified format
if found, err := checkFilesWithExtension(dir, fileFormat); err != nil {
return 0, fmt.Errorf("error checking files with extension: %v", err)
return 0, fmt.Errorf("[METRICSTORE]> error checking files with extension: %v", err)
} else if found {
log.Printf("Loading %s files because fileformat is %s\n", fileFormat, fileFormat)
log.Printf("[METRICSTORE]> Loading %s files because fileformat is %s\n", fileFormat, fileFormat)
return m.FromCheckpoint(dir, from, fileFormat)
}
// If not found, attempt the opposite format
altFormat := oppositeFormat[fileFormat]
if found, err := checkFilesWithExtension(dir, altFormat); err != nil {
return 0, fmt.Errorf("error checking files with extension: %v", err)
return 0, fmt.Errorf("[METRICSTORE]> error checking files with extension: %v", err)
} else if found {
log.Printf("Loading %s files but fileformat is %s\n", altFormat, fileFormat)
log.Printf("[METRICSTORE]> Loading %s files but fileformat is %s\n", altFormat, fileFormat)
return m.FromCheckpoint(dir, from, altFormat)
}
log.Println("No valid checkpoint files found in the directory.")
log.Println("[METRICSTORE]> No valid checkpoint files found in the directory.")
return 0, nil
}
@@ -420,7 +421,7 @@ func checkFilesWithExtension(dir string, extension string) (bool, error) {
err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error {
if err != nil {
return fmt.Errorf("error accessing path %s: %v", path, err)
return fmt.Errorf("[METRICSTORE]> error accessing path %s: %v", path, err)
}
if !info.IsDir() && filepath.Ext(info.Name()) == "."+extension {
found = true
@@ -429,7 +430,7 @@ func checkFilesWithExtension(dir string, extension string) (bool, error) {
return nil
})
if err != nil {
return false, fmt.Errorf("error walking through directories: %s", err)
return false, fmt.Errorf("[METRICSTORE]> error walking through directories: %s", err)
}
return found, nil
@@ -441,7 +442,7 @@ func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
fileName := f.Name()[strings.LastIndex(f.Name(), "/")+1:]
resolution, err := strconv.ParseInt(fileName[0:strings.Index(fileName, "_")], 10, 64)
if err != nil {
return fmt.Errorf("error while reading avro file (resolution parsing) : %s", err)
return fmt.Errorf("[METRICSTORE]> error while reading avro file (resolution parsing) : %s", err)
}
from_timestamp, err := strconv.ParseInt(fileName[strings.Index(fileName, "_")+1:len(fileName)-5], 10, 64)
@@ -450,7 +451,7 @@ func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
from_timestamp -= (resolution / 2)
if err != nil {
return fmt.Errorf("error converting timestamp from the avro file : %s", err)
return fmt.Errorf("[METRICSTORE]> error converting timestamp from the avro file : %s", err)
}
// fmt.Printf("File : %s with resolution : %d\n", fileName, resolution)
@@ -463,21 +464,21 @@ func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
panic(err)
}
metricsData := make(map[string]util.FloatArray)
metricsData := make(map[string]schema.FloatArray)
for ocfReader.Scan() {
datum, err := ocfReader.Read()
if err != nil {
return fmt.Errorf("error while reading avro file : %s", err)
return fmt.Errorf("[METRICSTORE]> error while reading avro file : %s", err)
}
record, ok := datum.(map[string]interface{})
if !ok {
panic("failed to assert datum as map[string]interface{}")
panic("[METRICSTORE]> failed to assert datum as map[string]interface{}")
}
for key, value := range record {
metricsData[key] = append(metricsData[key], util.ConvertToFloat(value.(float64)))
metricsData[key] = append(metricsData[key], schema.ConvertToFloat(value.(float64)))
}
recordCounter += 1
@@ -518,12 +519,12 @@ func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
leafMetricName := subString[len(subString)-1]
err = lvl.createBuffer(m, leafMetricName, floatArray, from_timestamp, resolution)
if err != nil {
return fmt.Errorf("error while creating buffers from avroReader : %s", err)
return fmt.Errorf("[METRICSTORE]> error while creating buffers from avroReader : %s", err)
}
} else {
err = l.createBuffer(m, metricName, floatArray, from_timestamp, resolution)
if err != nil {
return fmt.Errorf("error while creating buffers from avroReader : %s", err)
return fmt.Errorf("[METRICSTORE]> error while creating buffers from avroReader : %s", err)
}
}
@@ -532,7 +533,7 @@ func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
return nil
}
func (l *Level) createBuffer(m *MemoryStore, metricName string, floatArray util.FloatArray, from int64, resolution int64) error {
func (l *Level) createBuffer(m *MemoryStore, metricName string, floatArray schema.FloatArray, from int64, resolution int64) error {
n := len(floatArray)
b := &buffer{
frequency: resolution,
@@ -566,7 +567,7 @@ func (l *Level) createBuffer(m *MemoryStore, metricName string, floatArray util.
missingCount /= int(b.frequency)
for range missingCount {
prev.data = append(prev.data, util.NaN)
prev.data = append(prev.data, schema.NaN)
}
prev.data = prev.data[0:len(prev.data):len(prev.data)]

View File

@@ -1,26 +0,0 @@
// Copyright (C) NHR@FAU, University Erlangen-Nuremberg.
// All rights reserved. This file is part of cc-backend.
// Use of this source code is governed by a MIT-style
// license that can be found in the LICENSE file.
package memorystore
type MetricStoreConfig struct {
Checkpoints struct {
FileFormat string `json:"file-format"`
Interval string `json:"interval"`
RootDir string `json:"directory"`
Restore string `json:"restore"`
} `json:"checkpoints"`
Debug struct {
DumpToFile string `json:"dump-to-file"`
EnableGops bool `json:"gops"`
} `json:"debug"`
RetentionInMemory string `json:"retention-in-memory"`
Archive struct {
Interval string `json:"interval"`
RootDir string `json:"directory"`
DeleteInstead bool `json:"delete-instead"`
} `json:"archive"`
}
var Keys MetricStoreConfig

View File

@@ -87,7 +87,7 @@ func (l *Level) debugDump(m *MemoryStore, w *bufio.Writer, lvlname string, buf [
func (m *MemoryStore) DebugDump(w *bufio.Writer, selector []string) error {
lvl := m.root.findLevel(selector)
if lvl == nil {
return fmt.Errorf("not found: %#v", selector)
return fmt.Errorf("[METRICSTORE]> not found: %#v", selector)
}
buf := make([]byte, 0, 2048)

View File

@@ -59,7 +59,7 @@ func (l *Level) healthCheck(m *MemoryStore, count int64) (int64, error) {
func (m *MemoryStore) HealthCheck(w *bufio.Writer, selector []string) error {
lvl := m.root.findLevel(selector)
if lvl == nil {
return fmt.Errorf("not found: %#v", selector)
return fmt.Errorf("[METRICSTORE]> not found: %#v", selector)
}
buf := make([]byte, 0, 25)

View File

@@ -0,0 +1,349 @@
package memorystore
import (
"context"
"errors"
"fmt"
"log"
"net"
"sync"
"time"
"github.com/ClusterCockpit/cc-backend/internal/avro"
"github.com/ClusterCockpit/cc-backend/internal/config"
"github.com/ClusterCockpit/cc-lib/schema"
"github.com/influxdata/line-protocol/v2/lineprotocol"
"github.com/nats-io/nats.go"
)
// Each connection is handled in it's own goroutine. This is a blocking function.
func ReceiveRaw(ctx context.Context,
listener net.Listener,
handleLine func(*lineprotocol.Decoder, string) error,
) error {
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
<-ctx.Done()
if err := listener.Close(); err != nil {
log.Printf("listener.Close(): %s", err.Error())
}
}()
for {
conn, err := listener.Accept()
if err != nil {
if errors.Is(err, net.ErrClosed) {
break
}
log.Printf("listener.Accept(): %s", err.Error())
}
wg.Add(2)
go func() {
defer wg.Done()
defer conn.Close()
dec := lineprotocol.NewDecoder(conn)
connctx, cancel := context.WithCancel(context.Background())
defer cancel()
go func() {
defer wg.Done()
select {
case <-connctx.Done():
conn.Close()
case <-ctx.Done():
conn.Close()
}
}()
if err := handleLine(dec, "default"); err != nil {
if errors.Is(err, net.ErrClosed) {
return
}
log.Printf("%s: %s", conn.RemoteAddr().String(), err.Error())
errmsg := make([]byte, 128)
errmsg = append(errmsg, `error: `...)
errmsg = append(errmsg, err.Error()...)
errmsg = append(errmsg, '\n')
conn.Write(errmsg)
}
}()
}
wg.Wait()
return nil
}
// Connect to a nats server and subscribe to "updates". This is a blocking
// function. handleLine will be called for each line recieved via nats.
// Send `true` through the done channel for gracefull termination.
func ReceiveNats(conf *(config.NatsConfig),
ms *MemoryStore,
workers int,
ctx context.Context,
) error {
var opts []nats.Option
if conf.Username != "" && conf.Password != "" {
opts = append(opts, nats.UserInfo(conf.Username, conf.Password))
}
if conf.Credsfilepath != "" {
opts = append(opts, nats.UserCredentials(conf.Credsfilepath))
}
nc, err := nats.Connect(conf.Address, opts...)
if err != nil {
return err
}
defer nc.Close()
var wg sync.WaitGroup
var subs []*nats.Subscription
msgs := make(chan *nats.Msg, workers*2)
for _, sc := range conf.Subscriptions {
clusterTag := sc.ClusterTag
var sub *nats.Subscription
if workers > 1 {
wg.Add(workers)
for i := 0; i < workers; i++ {
go func() {
for m := range msgs {
dec := lineprotocol.NewDecoderWithBytes(m.Data)
if err := decodeLine(dec, ms, clusterTag); err != nil {
log.Printf("error: %s\n", err.Error())
}
}
wg.Done()
}()
}
sub, err = nc.Subscribe(sc.SubscribeTo, func(m *nats.Msg) {
msgs <- m
})
} else {
sub, err = nc.Subscribe(sc.SubscribeTo, func(m *nats.Msg) {
dec := lineprotocol.NewDecoderWithBytes(m.Data)
if err := decodeLine(dec, ms, clusterTag); err != nil {
log.Printf("error: %s\n", err.Error())
}
})
}
if err != nil {
return err
}
log.Printf("NATS subscription to '%s' on '%s' established\n", sc.SubscribeTo, conf.Address)
subs = append(subs, sub)
}
<-ctx.Done()
for _, sub := range subs {
err = sub.Unsubscribe()
if err != nil {
log.Printf("NATS unsubscribe failed: %s", err.Error())
}
}
close(msgs)
wg.Wait()
nc.Close()
log.Println("NATS connection closed")
return nil
}
// Place `prefix` in front of `buf` but if possible,
// do that inplace in `buf`.
func reorder(buf, prefix []byte) []byte {
n := len(prefix)
m := len(buf)
if cap(buf) < m+n {
return append(prefix[:n:n], buf...)
} else {
buf = buf[:n+m]
for i := m - 1; i >= 0; i-- {
buf[i+n] = buf[i]
}
for i := 0; i < n; i++ {
buf[i] = prefix[i]
}
return buf
}
}
// Decode lines using dec and make write calls to the MemoryStore.
// If a line is missing its cluster tag, use clusterDefault as default.
func decodeLine(dec *lineprotocol.Decoder,
ms *MemoryStore,
clusterDefault string,
) error {
// Reduce allocations in loop:
t := time.Now()
metric, metricBuf := Metric{}, make([]byte, 0, 16)
selector := make([]string, 0, 4)
typeBuf, subTypeBuf := make([]byte, 0, 16), make([]byte, 0)
// Optimize for the case where all lines in a "batch" are about the same
// cluster and host. By using `WriteToLevel` (level = host), we do not need
// to take the root- and cluster-level lock as often.
var lvl *Level = nil
prevCluster, prevHost := "", ""
var ok bool
for dec.Next() {
rawmeasurement, err := dec.Measurement()
if err != nil {
return err
}
// Needs to be copied because another call to dec.* would
// invalidate the returned slice.
metricBuf = append(metricBuf[:0], rawmeasurement...)
// The go compiler optimizes map[string(byteslice)] lookups:
metric.MetricConfig, ok = ms.Metrics[string(rawmeasurement)]
if !ok {
continue
}
typeBuf, subTypeBuf := typeBuf[:0], subTypeBuf[:0]
cluster, host := clusterDefault, ""
for {
key, val, err := dec.NextTag()
if err != nil {
return err
}
if key == nil {
break
}
// The go compiler optimizes string([]byte{...}) == "...":
switch string(key) {
case "cluster":
if string(val) == prevCluster {
cluster = prevCluster
} else {
cluster = string(val)
lvl = nil
}
case "hostname", "host":
if string(val) == prevHost {
host = prevHost
} else {
host = string(val)
lvl = nil
}
case "type":
if string(val) == "node" {
break
}
// We cannot be sure that the "type" tag comes before the "type-id" tag:
if len(typeBuf) == 0 {
typeBuf = append(typeBuf, val...)
} else {
typeBuf = reorder(typeBuf, val)
}
case "type-id":
typeBuf = append(typeBuf, val...)
case "subtype":
// We cannot be sure that the "subtype" tag comes before the "stype-id" tag:
if len(subTypeBuf) == 0 {
subTypeBuf = append(subTypeBuf, val...)
} else {
subTypeBuf = reorder(subTypeBuf, val)
// subTypeBuf = reorder(typeBuf, val)
}
case "stype-id":
subTypeBuf = append(subTypeBuf, val...)
default:
// Ignore unkown tags (cc-metric-collector might send us a unit for example that we do not need)
// return fmt.Errorf("unkown tag: '%s' (value: '%s')", string(key), string(val))
}
}
// If the cluster or host changed, the lvl was set to nil
if lvl == nil {
selector = selector[:2]
selector[0], selector[1] = cluster, host
lvl = ms.GetLevel(selector)
prevCluster, prevHost = cluster, host
}
// subtypes:
selector = selector[:0]
if len(typeBuf) > 0 {
selector = append(selector, string(typeBuf)) // <- Allocation :(
if len(subTypeBuf) > 0 {
selector = append(selector, string(subTypeBuf))
}
}
for {
key, val, err := dec.NextField()
if err != nil {
return err
}
if key == nil {
break
}
if string(key) != "value" {
return fmt.Errorf("host %s: unknown field: '%s' (value: %#v)", host, string(key), val)
}
if val.Kind() == lineprotocol.Float {
metric.Value = schema.Float(val.FloatV())
} else if val.Kind() == lineprotocol.Int {
metric.Value = schema.Float(val.IntV())
} else if val.Kind() == lineprotocol.Uint {
metric.Value = schema.Float(val.UintV())
} else {
return fmt.Errorf("host %s: unsupported value type in message: %s", host, val.Kind().String())
}
}
if t, err = dec.Time(lineprotocol.Second, t); err != nil {
t = time.Now()
if t, err = dec.Time(lineprotocol.Millisecond, t); err != nil {
t = time.Now()
if t, err = dec.Time(lineprotocol.Microsecond, t); err != nil {
t = time.Now()
if t, err = dec.Time(lineprotocol.Nanosecond, t); err != nil {
return fmt.Errorf("host %s: timestamp : %#v with error : %#v", host, t, err.Error())
}
}
}
}
if err != nil {
return fmt.Errorf("host %s: timestamp : %#v with error : %#v", host, t, err.Error())
}
time := t.Unix()
if config.MetricStoreKeys.Checkpoints.FileFormat != "json" {
avro.LineProtocolMessages <- &avro.AvroStruct{
MetricName: string(metricBuf),
Cluster: cluster,
Node: host,
Selector: append([]string{}, selector...),
Value: metric.Value,
Timestamp: time}
}
if err := ms.WriteToLevel(lvl, selector, time, []Metric{metric}); err != nil {
return err
}
}
return nil
}

View File

@@ -2,16 +2,19 @@ package memorystore
import (
"context"
"encoding/json"
"errors"
"fmt"
"log"
"os"
"os/signal"
"runtime"
"sync"
"syscall"
"time"
"github.com/ClusterCockpit/cc-backend/internal/avro"
"github.com/ClusterCockpit/cc-backend/internal/config"
"github.com/ClusterCockpit/cc-lib/resampler"
"github.com/ClusterCockpit/cc-lib/runtimeEnv"
"github.com/ClusterCockpit/cc-lib/schema"
"github.com/ClusterCockpit/cc-lib/util"
)
@@ -21,6 +24,8 @@ var (
msInstance *MemoryStore
)
var Clusters = make([]string, 0)
var NumWorkers int = 4
func init() {
@@ -31,77 +36,38 @@ func init() {
}
}
// For aggregation over multiple values at different cpus/sockets/..., not time!
type AggregationStrategy int
const (
NoAggregation AggregationStrategy = iota
SumAggregation
AvgAggregation
)
func (as *AggregationStrategy) UnmarshalJSON(data []byte) error {
var str string
if err := json.Unmarshal(data, &str); err != nil {
return err
}
switch str {
case "":
*as = NoAggregation
case "sum":
*as = SumAggregation
case "avg":
*as = AvgAggregation
default:
return fmt.Errorf("invalid aggregation strategy: %#v", str)
}
return nil
}
type MetricConfig struct {
// Interval in seconds at which measurements will arive.
Frequency int64 `json:"frequency"`
// Can be 'sum', 'avg' or null. Describes how to aggregate metrics from the same timestep over the hierarchy.
Aggregation AggregationStrategy `json:"aggregation"`
// Private, used internally...
Offset int
}
type Metric struct {
Name string
Value util.Float
MetricConfig MetricConfig
Value schema.Float
MetricConfig config.MetricConfig
}
type MemoryStore struct {
Metrics map[string]MetricConfig
Metrics map[string]config.MetricConfig
root Level
}
func Init() {
func Init(wg sync.WaitGroup) {
startupTime := time.Now()
//Pass the keys from cluster config
InitMetrics()
//Pass the config.MetricStoreKeys
InitMetrics(config.Metrics)
ms := GetMemoryStore()
d, err := time.ParseDuration(Keys.Checkpoints.Restore)
d, err := time.ParseDuration(config.MetricStoreKeys.Checkpoints.Restore)
if err != nil {
log.Fatal(err)
}
restoreFrom := startupTime.Add(-d)
log.Printf("Loading checkpoints newer than %s\n", restoreFrom.Format(time.RFC3339))
files, err := ms.FromCheckpointFiles(Keys.Checkpoints.RootDir, restoreFrom.Unix())
log.Printf("[METRICSTORE]> Loading checkpoints newer than %s\n", restoreFrom.Format(time.RFC3339))
files, err := ms.FromCheckpointFiles(config.MetricStoreKeys.Checkpoints.RootDir, restoreFrom.Unix())
loadedData := ms.SizeInBytes() / 1024 / 1024 // In MB
if err != nil {
log.Fatalf("Loading checkpoints failed: %s\n", err.Error())
log.Fatalf("[METRICSTORE]> Loading checkpoints failed: %s\n", err.Error())
} else {
log.Printf("Checkpoints loaded (%d files, %d MB, that took %fs)\n", files, loadedData, time.Since(startupTime).Seconds())
log.Printf("[METRICSTORE]> Checkpoints loaded (%d files, %d MB, that took %fs)\n", files, loadedData, time.Since(startupTime).Seconds())
}
// Try to use less memory by forcing a GC run here and then
@@ -112,28 +78,53 @@ func Init() {
// to a minumum.
runtime.GC()
ctx, _ := context.WithCancel(context.Background())
ctx, shutdown := context.WithCancel(context.Background())
var wg sync.WaitGroup
wg.Add(4)
Retention(&wg, ctx)
Checkpointing(&wg, ctx)
Archiving(&wg, ctx)
avro.DataStaging(&wg, ctx)
wg.Add(1)
sigs := make(chan os.Signal, 1)
signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM)
go func() {
defer wg.Done()
<-sigs
runtimeEnv.SystemdNotifiy(false, "[METRICSTORE]> Shutting down ...")
shutdown()
}()
if config.MetricStoreKeys.Nats != nil {
for _, natsConf := range config.MetricStoreKeys.Nats {
// TODO: When multiple nats configs share a URL, do a single connect.
wg.Add(1)
nc := natsConf
go func() {
// err := ReceiveNats(conf.Nats, decodeLine, runtime.NumCPU()-1, ctx)
err := ReceiveNats(nc, ms, 1, ctx)
if err != nil {
log.Fatal(err)
}
wg.Done()
}()
}
}
}
// Create a new, initialized instance of a MemoryStore.
// Will panic if values in the metric configurations are invalid.
func InitMetrics(metrics map[string]MetricConfig) {
func InitMetrics(metrics map[string]config.MetricConfig) {
singleton.Do(func() {
offset := 0
for key, cfg := range metrics {
if cfg.Frequency == 0 {
panic("invalid frequency")
panic("[METRICSTORE]> invalid frequency")
}
metrics[key] = MetricConfig{
metrics[key] = config.MetricConfig{
Frequency: cfg.Frequency,
Aggregation: cfg.Aggregation,
Offset: offset,
@@ -153,30 +144,30 @@ func InitMetrics(metrics map[string]MetricConfig) {
func GetMemoryStore() *MemoryStore {
if msInstance == nil {
log.Fatalf("MemoryStore not initialized!")
log.Fatalf("[METRICSTORE]> MemoryStore not initialized!")
}
return msInstance
}
func Shutdown() {
log.Printf("Writing to '%s'...\n", Keys.Checkpoints.RootDir)
log.Printf("[METRICSTORE]> Writing to '%s'...\n", config.MetricStoreKeys.Checkpoints.RootDir)
var files int
var err error
ms := GetMemoryStore()
if Keys.Checkpoints.FileFormat == "json" {
files, err = ms.ToCheckpoint(Keys.Checkpoints.RootDir, lastCheckpoint.Unix(), time.Now().Unix())
if config.MetricStoreKeys.Checkpoints.FileFormat == "json" {
files, err = ms.ToCheckpoint(config.MetricStoreKeys.Checkpoints.RootDir, lastCheckpoint.Unix(), time.Now().Unix())
} else {
files, err = avro.GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, true)
files, err = avro.GetAvroStore().ToCheckpoint(config.MetricStoreKeys.Checkpoints.RootDir, true)
close(avro.LineProtocolMessages)
}
if err != nil {
log.Printf("Writing checkpoint failed: %s\n", err.Error())
log.Printf("[METRICSTORE]> Writing checkpoint failed: %s\n", err.Error())
}
log.Printf("Done! (%d files written)\n", files)
log.Printf("[METRICSTORE]> Done! (%d files written)\n", files)
// ms.PrintHeirarchy()
}
@@ -255,7 +246,7 @@ func Retention(wg *sync.WaitGroup, ctx context.Context) {
go func() {
defer wg.Done()
d, err := time.ParseDuration(Keys.RetentionInMemory)
d, err := time.ParseDuration(config.MetricStoreKeys.RetentionInMemory)
if err != nil {
log.Fatal(err)
}
@@ -276,12 +267,12 @@ func Retention(wg *sync.WaitGroup, ctx context.Context) {
return
case <-ticks:
t := time.Now().Add(-d)
log.Printf("start freeing buffers (older than %s)...\n", t.Format(time.RFC3339))
log.Printf("[METRICSTORE]> start freeing buffers (older than %s)...\n", t.Format(time.RFC3339))
freed, err := ms.Free(nil, t.Unix())
if err != nil {
log.Printf("freeing up buffers failed: %s\n", err.Error())
log.Printf("[METRICSTORE]> freeing up buffers failed: %s\n", err.Error())
} else {
log.Printf("done: %d buffers freed\n", freed)
log.Printf("[METRICSTORE]> done: %d buffers freed\n", freed)
}
}
}
@@ -346,12 +337,12 @@ func (m *MemoryStore) WriteToLevel(l *Level, selector []string, ts int64, metric
// the range asked for if no data was available.
func (m *MemoryStore) Read(selector util.Selector, metric string, from, to, resolution int64) ([]schema.Float, int64, int64, int64, error) {
if from > to {
return nil, 0, 0, 0, errors.New("invalid time range")
return nil, 0, 0, 0, errors.New("[METRICSTORE]> invalid time range")
}
minfo, ok := m.Metrics[metric]
if !ok {
return nil, 0, 0, 0, errors.New("unkown metric: " + metric)
return nil, 0, 0, 0, errors.New("[METRICSTORE]> unkown metric: " + metric)
}
n, data := 0, make([]schema.Float, (to-from)/minfo.Frequency+1)
@@ -390,15 +381,15 @@ func (m *MemoryStore) Read(selector util.Selector, metric string, from, to, reso
if err != nil {
return nil, 0, 0, 0, err
} else if n == 0 {
return nil, 0, 0, 0, errors.New("metric or host not found")
return nil, 0, 0, 0, errors.New("[METRICSTORE]> metric or host not found")
} else if n > 1 {
if minfo.Aggregation == AvgAggregation {
if minfo.Aggregation == config.AvgAggregation {
normalize := 1. / schema.Float(n)
for i := 0; i < len(data); i++ {
data[i] *= normalize
}
} else if minfo.Aggregation != SumAggregation {
return nil, 0, 0, 0, errors.New("invalid aggregation")
} else if minfo.Aggregation != config.SumAggregation {
return nil, 0, 0, 0, errors.New("[METRICSTORE]> invalid aggregation")
}
}

View File

@@ -4,6 +4,7 @@ import (
"errors"
"math"
"github.com/ClusterCockpit/cc-backend/internal/config"
"github.com/ClusterCockpit/cc-lib/util"
)
@@ -104,9 +105,9 @@ func (m *MemoryStore) Stats(selector util.Selector, metric string, from, to int6
return nil, 0, 0, ErrNoData
}
if minfo.Aggregation == AvgAggregation {
if minfo.Aggregation == config.AvgAggregation {
avg /= util.Float(n)
} else if n > 1 && minfo.Aggregation != SumAggregation {
} else if n > 1 && minfo.Aggregation != config.SumAggregation {
return nil, 0, 0, errors.New("invalid aggregation")
}