mirror of
https://github.com/ClusterCockpit/cc-backend
synced 2026-01-28 15:01:46 +01:00
Move metricstore from internal to pkg
This commit is contained in:
305
pkg/metricstore/api.go
Normal file
305
pkg/metricstore/api.go
Normal file
@@ -0,0 +1,305 @@
|
||||
// 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.
|
||||
|
||||
// This file contains the API types and data fetching logic for querying metric data
|
||||
// from the in-memory metric store. It provides structures for building complex queries
|
||||
// with support for aggregation, scaling, padding, and statistics computation.
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/util"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrInvalidTimeRange is returned when a query has 'from' >= 'to'
|
||||
ErrInvalidTimeRange = errors.New("[METRICSTORE]> invalid time range: 'from' must be before 'to'")
|
||||
// ErrEmptyCluster is returned when a query with ForAllNodes has no cluster specified
|
||||
ErrEmptyCluster = errors.New("[METRICSTORE]> cluster name cannot be empty")
|
||||
)
|
||||
|
||||
// APIMetricData represents the response data for a single metric query.
|
||||
//
|
||||
// It contains both the time-series data points and computed statistics (avg, min, max).
|
||||
// If an error occurred during data retrieval, the Error field will be set and other
|
||||
// fields may be incomplete.
|
||||
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"`
|
||||
}
|
||||
|
||||
// APIQueryRequest represents a batch query request for metric data.
|
||||
//
|
||||
// It supports two modes of operation:
|
||||
// 1. Explicit queries via the Queries field
|
||||
// 2. Automatic query generation via ForAllNodes (queries all specified metrics for all nodes in the cluster)
|
||||
//
|
||||
// The request can be customized with flags to include/exclude statistics, raw data, and padding.
|
||||
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"`
|
||||
}
|
||||
|
||||
// APIQueryResponse represents the response to an APIQueryRequest.
|
||||
//
|
||||
// Results is a 2D array where each outer element corresponds to a query,
|
||||
// and each inner element corresponds to a selector within that query
|
||||
// (e.g., multiple CPUs or cores).
|
||||
type APIQueryResponse struct {
|
||||
Queries []APIQuery `json:"queries,omitempty"`
|
||||
Results [][]APIMetricData `json:"results"`
|
||||
}
|
||||
|
||||
// APIQuery represents a single metric query with optional hierarchical selectors.
|
||||
//
|
||||
// The hierarchical selection works as follows:
|
||||
// - Hostname: The node to query
|
||||
// - Type + TypeIds: First level of hierarchy (e.g., "cpu" + ["0", "1", "2"])
|
||||
// - SubType + SubTypeIds: Second level of hierarchy (e.g., "core" + ["0", "1"])
|
||||
//
|
||||
// If Aggregate is true, data from multiple type/subtype IDs will be aggregated according
|
||||
// to the metric's aggregation strategy. Otherwise, separate results are returned for each combination.
|
||||
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"`
|
||||
}
|
||||
|
||||
// AddStats computes and populates the Avg, Min, and Max fields from the Data array.
|
||||
//
|
||||
// NaN values in the data are ignored during computation. If all values are NaN,
|
||||
// the statistics fields will be set to NaN.
|
||||
//
|
||||
// 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
|
||||
}
|
||||
}
|
||||
|
||||
// ScaleBy multiplies all data points and statistics by the given factor.
|
||||
//
|
||||
// This is commonly used for unit conversion (e.g., bytes to gigabytes).
|
||||
// Scaling by 0 or 1 is a no-op for performance reasons.
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
// PadDataWithNull pads the beginning of the data array with NaN values if needed.
|
||||
//
|
||||
// This ensures that the data aligns with the requested 'from' timestamp, even if
|
||||
// the metric store doesn't have data for the earliest time points. This is useful
|
||||
// for maintaining consistent array indexing across multiple queries.
|
||||
//
|
||||
// Parameters:
|
||||
// - ms: MemoryStore instance to lookup metric configuration
|
||||
// - from: The requested start timestamp
|
||||
// - to: The requested end timestamp (unused but kept for API consistency)
|
||||
// - metric: The metric name to lookup frequency information
|
||||
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 range padfront {
|
||||
ndata = append(ndata, schema.NaN)
|
||||
}
|
||||
for j := 0; j < len(data.Data); j++ {
|
||||
ndata = append(ndata, data.Data[j])
|
||||
}
|
||||
data.Data = ndata
|
||||
}
|
||||
}
|
||||
|
||||
// FetchData executes a batch metric query request and returns the results.
|
||||
//
|
||||
// This is the primary API for retrieving metric data from the memory store. It supports:
|
||||
// - Individual queries via req.Queries
|
||||
// - Batch queries for all nodes via req.ForAllNodes
|
||||
// - Hierarchical selector construction (cluster → host → type → subtype)
|
||||
// - Optional statistics computation (avg, min, max)
|
||||
// - Optional data scaling
|
||||
// - Optional data padding with NaN values
|
||||
//
|
||||
// The function constructs selectors based on the query parameters and calls MemoryStore.Read()
|
||||
// for each selector. If a query specifies Aggregate=false with multiple type/subtype IDs,
|
||||
// separate results are returned for each combination.
|
||||
//
|
||||
// Parameters:
|
||||
// - req: The query request containing queries, time range, and options
|
||||
//
|
||||
// Returns:
|
||||
// - APIQueryResponse containing results for each query, or error if validation fails
|
||||
//
|
||||
// Errors:
|
||||
// - ErrInvalidTimeRange if req.From > req.To
|
||||
// - ErrEmptyCluster if req.ForAllNodes is used without specifying a cluster
|
||||
// - Error if MemoryStore is not initialized
|
||||
// - Individual query errors are stored in APIMetricData.Error field
|
||||
func FetchData(req APIQueryRequest) (*APIQueryResponse, error) {
|
||||
if req.From > req.To {
|
||||
return nil, ErrInvalidTimeRange
|
||||
}
|
||||
if req.Cluster == "" && req.ForAllNodes != nil {
|
||||
return nil, ErrEmptyCluster
|
||||
}
|
||||
|
||||
req.WithData = true
|
||||
ms := GetMemoryStore()
|
||||
if ms == nil {
|
||||
return nil, fmt.Errorf("[METRICSTORE]> memorystore not initialized")
|
||||
}
|
||||
|
||||
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},
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
228
pkg/metricstore/archive.go
Normal file
228
pkg/metricstore/archive.go
Normal file
@@ -0,0 +1,228 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"archive/zip"
|
||||
"bufio"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
)
|
||||
|
||||
// Worker for either Archiving or Deleting files
|
||||
|
||||
func CleanUp(wg *sync.WaitGroup, ctx context.Context) {
|
||||
if Keys.Cleanup.Mode == "archive" {
|
||||
// Run as Archiver
|
||||
cleanUpWorker(wg, ctx,
|
||||
Keys.Cleanup.Interval,
|
||||
"archiving",
|
||||
Keys.Cleanup.RootDir,
|
||||
false,
|
||||
)
|
||||
} else {
|
||||
if Keys.Cleanup.Interval == "" {
|
||||
Keys.Cleanup.Interval = Keys.RetentionInMemory
|
||||
}
|
||||
|
||||
// Run as Deleter
|
||||
cleanUpWorker(wg, ctx,
|
||||
Keys.Cleanup.Interval,
|
||||
"deleting",
|
||||
"",
|
||||
true,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// runWorker takes simple values to configure what it does
|
||||
func cleanUpWorker(wg *sync.WaitGroup, ctx context.Context, interval string, mode string, cleanupDir string, delete bool) {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
d, err := time.ParseDuration(interval)
|
||||
if err != nil {
|
||||
cclog.Fatalf("[METRICSTORE]> error parsing %s interval duration: %v\n", mode, err)
|
||||
}
|
||||
if d <= 0 {
|
||||
return
|
||||
}
|
||||
|
||||
ticker := time.NewTicker(d)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
t := time.Now().Add(-d)
|
||||
cclog.Infof("[METRICSTORE]> start %s checkpoints (older than %s)...", mode, t.Format(time.RFC3339))
|
||||
|
||||
n, err := CleanupCheckpoints(Keys.Checkpoints.RootDir, cleanupDir, t.Unix(), delete)
|
||||
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> %s failed: %s", mode, err.Error())
|
||||
} else {
|
||||
if delete && cleanupDir == "" {
|
||||
cclog.Infof("[METRICSTORE]> done: %d checkpoints deleted", n)
|
||||
} else {
|
||||
cclog.Infof("[METRICSTORE]> done: %d files zipped and moved to archive", n)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
var ErrNoNewArchiveData error = errors.New("all data already archived")
|
||||
|
||||
// Delete or ZIP all checkpoint files older than `from` together and write them to the `cleanupDir`,
|
||||
// deleting/moving them from the `checkpointsDir`.
|
||||
func CleanupCheckpoints(checkpointsDir, cleanupDir string, from int64, deleteInstead bool) (int, error) {
|
||||
entries1, err := os.ReadDir(checkpointsDir)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
type workItem struct {
|
||||
cdir, adir string
|
||||
cluster, host string
|
||||
}
|
||||
|
||||
var wg sync.WaitGroup
|
||||
n, errs := int32(0), int32(0)
|
||||
work := make(chan workItem, Keys.NumWorkers)
|
||||
|
||||
wg.Add(Keys.NumWorkers)
|
||||
for worker := 0; worker < Keys.NumWorkers; worker++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for workItem := range work {
|
||||
m, err := cleanupCheckpoints(workItem.cdir, workItem.adir, from, deleteInstead)
|
||||
if err != nil {
|
||||
cclog.Errorf("error while archiving %s/%s: %s", workItem.cluster, workItem.host, err.Error())
|
||||
atomic.AddInt32(&errs, 1)
|
||||
}
|
||||
atomic.AddInt32(&n, int32(m))
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
for _, de1 := range entries1 {
|
||||
entries2, e := os.ReadDir(filepath.Join(checkpointsDir, de1.Name()))
|
||||
if e != nil {
|
||||
err = e
|
||||
}
|
||||
|
||||
for _, de2 := range entries2 {
|
||||
cdir := filepath.Join(checkpointsDir, de1.Name(), de2.Name())
|
||||
adir := filepath.Join(cleanupDir, de1.Name(), de2.Name())
|
||||
work <- workItem{
|
||||
adir: adir, cdir: cdir,
|
||||
cluster: de1.Name(), host: de2.Name(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
close(work)
|
||||
wg.Wait()
|
||||
|
||||
if err != nil {
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
if errs > 0 {
|
||||
return int(n), fmt.Errorf("%d errors happened while archiving (%d successes)", errs, n)
|
||||
}
|
||||
return int(n), nil
|
||||
}
|
||||
|
||||
// Helper function for `CleanupCheckpoints`.
|
||||
func cleanupCheckpoints(dir string, cleanupDir string, from int64, deleteInstead bool) (int, error) {
|
||||
entries, err := os.ReadDir(dir)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
extension := Keys.Checkpoints.FileFormat
|
||||
files, err := findFiles(entries, from, extension, false)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
if deleteInstead {
|
||||
n := 0
|
||||
for _, checkpoint := range files {
|
||||
filename := filepath.Join(dir, checkpoint)
|
||||
if err = os.Remove(filename); err != nil {
|
||||
return n, err
|
||||
}
|
||||
n += 1
|
||||
}
|
||||
return n, nil
|
||||
}
|
||||
|
||||
filename := filepath.Join(cleanupDir, fmt.Sprintf("%d.zip", from))
|
||||
f, err := os.OpenFile(filename, os.O_CREATE|os.O_WRONLY, CheckpointFilePerms)
|
||||
if err != nil && os.IsNotExist(err) {
|
||||
err = os.MkdirAll(cleanupDir, CheckpointDirPerms)
|
||||
if err == nil {
|
||||
f, err = os.OpenFile(filename, os.O_CREATE|os.O_WRONLY, CheckpointFilePerms)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
defer f.Close()
|
||||
bw := bufio.NewWriter(f)
|
||||
defer bw.Flush()
|
||||
zw := zip.NewWriter(bw)
|
||||
defer zw.Close()
|
||||
|
||||
n := 0
|
||||
for _, checkpoint := range files {
|
||||
// Use closure to ensure file is closed immediately after use,
|
||||
// avoiding file descriptor leak from defer in loop
|
||||
err := func() error {
|
||||
filename := filepath.Join(dir, checkpoint)
|
||||
r, err := os.Open(filename)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer r.Close()
|
||||
|
||||
w, err := zw.Create(checkpoint)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if _, err = io.Copy(w, r); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err = os.Remove(filename); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}()
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
n += 1
|
||||
}
|
||||
|
||||
return n, nil
|
||||
}
|
||||
479
pkg/metricstore/avroCheckpoint.go
Normal file
479
pkg/metricstore/avroCheckpoint.go
Normal file
@@ -0,0 +1,479 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"os"
|
||||
"path"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
"github.com/linkedin/goavro/v2"
|
||||
)
|
||||
|
||||
var (
|
||||
NumAvroWorkers int = DefaultAvroWorkers
|
||||
startUp bool = true
|
||||
)
|
||||
|
||||
func (as *AvroStore) ToCheckpoint(dir string, dumpAll bool) (int, error) {
|
||||
levels := make([]*AvroLevel, 0)
|
||||
selectors := make([][]string, 0)
|
||||
as.root.lock.RLock()
|
||||
// Cluster
|
||||
for sel1, l1 := range as.root.children {
|
||||
l1.lock.RLock()
|
||||
// Node
|
||||
for sel2, l2 := range l1.children {
|
||||
l2.lock.RLock()
|
||||
// Frequency
|
||||
for sel3, l3 := range l2.children {
|
||||
levels = append(levels, l3)
|
||||
selectors = append(selectors, []string{sel1, sel2, sel3})
|
||||
}
|
||||
l2.lock.RUnlock()
|
||||
}
|
||||
l1.lock.RUnlock()
|
||||
}
|
||||
as.root.lock.RUnlock()
|
||||
|
||||
type workItem struct {
|
||||
level *AvroLevel
|
||||
dir string
|
||||
selector []string
|
||||
}
|
||||
|
||||
n, errs := int32(0), int32(0)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(NumAvroWorkers)
|
||||
work := make(chan workItem, NumAvroWorkers*2)
|
||||
for range NumAvroWorkers {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
for workItem := range work {
|
||||
from := getTimestamp(workItem.dir)
|
||||
|
||||
if err := workItem.level.toCheckpoint(workItem.dir, from, dumpAll); err != nil {
|
||||
if err == ErrNoNewArchiveData {
|
||||
continue
|
||||
}
|
||||
|
||||
cclog.Errorf("error while checkpointing %#v: %s", workItem.selector, err.Error())
|
||||
atomic.AddInt32(&errs, 1)
|
||||
} else {
|
||||
atomic.AddInt32(&n, 1)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
for i := range len(levels) {
|
||||
dir := path.Join(dir, path.Join(selectors[i]...))
|
||||
work <- workItem{
|
||||
level: levels[i],
|
||||
dir: dir,
|
||||
selector: selectors[i],
|
||||
}
|
||||
}
|
||||
|
||||
close(work)
|
||||
wg.Wait()
|
||||
|
||||
if errs > 0 {
|
||||
return int(n), fmt.Errorf("%d errors happend while creating avro checkpoints (%d successes)", errs, n)
|
||||
}
|
||||
|
||||
startUp = false
|
||||
|
||||
return int(n), nil
|
||||
}
|
||||
|
||||
// getTimestamp returns the timestamp from the directory name
|
||||
func getTimestamp(dir string) int64 {
|
||||
// Extract the resolution and timestamp from the directory name
|
||||
// The existing avro file will be in epoch timestamp format
|
||||
// iterate over all the files in the directory and find the maximum timestamp
|
||||
// and return it
|
||||
|
||||
resolution := path.Base(dir)
|
||||
dir = path.Dir(dir)
|
||||
|
||||
files, err := os.ReadDir(dir)
|
||||
if err != nil {
|
||||
return 0
|
||||
}
|
||||
var maxTS int64 = 0
|
||||
|
||||
if len(files) == 0 {
|
||||
return 0
|
||||
}
|
||||
|
||||
for _, file := range files {
|
||||
if file.IsDir() {
|
||||
continue
|
||||
}
|
||||
name := file.Name()
|
||||
|
||||
if len(name) < 5 || !strings.HasSuffix(name, ".avro") || !strings.HasPrefix(name, resolution+"_") {
|
||||
continue
|
||||
}
|
||||
|
||||
ts, err := strconv.ParseInt(name[strings.Index(name, "_")+1:len(name)-5], 10, 64)
|
||||
if err != nil {
|
||||
fmt.Printf("error while parsing timestamp: %s\n", err.Error())
|
||||
continue
|
||||
}
|
||||
|
||||
if ts > maxTS {
|
||||
maxTS = ts
|
||||
}
|
||||
}
|
||||
|
||||
interval, _ := time.ParseDuration(Keys.Checkpoints.Interval)
|
||||
updateTime := time.Unix(maxTS, 0).Add(interval).Add(time.Duration(CheckpointBufferMinutes-1) * time.Minute).Unix()
|
||||
|
||||
if startUp {
|
||||
return 0
|
||||
}
|
||||
|
||||
if updateTime < time.Now().Unix() {
|
||||
return 0
|
||||
}
|
||||
|
||||
return maxTS
|
||||
}
|
||||
|
||||
func (l *AvroLevel) toCheckpoint(dir string, from int64, dumpAll bool) error {
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
// fmt.Printf("Checkpointing directory: %s\n", dir)
|
||||
// filepath contains the resolution
|
||||
intRes, _ := strconv.Atoi(path.Base(dir))
|
||||
|
||||
// find smallest overall timestamp in l.data map and delete it from l.data
|
||||
minTS := int64(1<<63 - 1)
|
||||
for ts, dat := range l.data {
|
||||
if ts < minTS && len(dat) != 0 {
|
||||
minTS = ts
|
||||
}
|
||||
}
|
||||
|
||||
if from == 0 && minTS != int64(1<<63-1) {
|
||||
from = minTS
|
||||
}
|
||||
|
||||
if from == 0 {
|
||||
return ErrNoNewArchiveData
|
||||
}
|
||||
|
||||
var schema string
|
||||
var codec *goavro.Codec
|
||||
recordList := make([]map[string]any, 0)
|
||||
|
||||
var f *os.File
|
||||
|
||||
filePath := dir + fmt.Sprintf("_%d.avro", from)
|
||||
|
||||
var err error
|
||||
|
||||
fp_, err_ := os.Stat(filePath)
|
||||
if errors.Is(err_, os.ErrNotExist) {
|
||||
err = os.MkdirAll(path.Dir(dir), 0o755)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create directory: %v", err)
|
||||
}
|
||||
} else if fp_.Size() != 0 {
|
||||
f, err = os.Open(filePath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to open existing avro file: %v", err)
|
||||
}
|
||||
|
||||
br := bufio.NewReader(f)
|
||||
|
||||
reader, err := goavro.NewOCFReader(br)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create OCF reader: %v", err)
|
||||
}
|
||||
codec = reader.Codec()
|
||||
schema = codec.Schema()
|
||||
|
||||
f.Close()
|
||||
}
|
||||
|
||||
timeRef := time.Now().Add(time.Duration(-CheckpointBufferMinutes+1) * time.Minute).Unix()
|
||||
|
||||
if dumpAll {
|
||||
timeRef = time.Now().Unix()
|
||||
}
|
||||
|
||||
// Empty values
|
||||
if len(l.data) == 0 {
|
||||
// we checkpoint avro files every 60 seconds
|
||||
repeat := 60 / intRes
|
||||
|
||||
for range repeat {
|
||||
recordList = append(recordList, make(map[string]any))
|
||||
}
|
||||
}
|
||||
|
||||
readFlag := true
|
||||
|
||||
for ts := range l.data {
|
||||
flag := false
|
||||
if ts < timeRef {
|
||||
data := l.data[ts]
|
||||
|
||||
schemaGen, err := generateSchema(data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
flag, schema, err = compareSchema(schema, schemaGen)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to compare read and generated schema: %v", err)
|
||||
}
|
||||
if flag && readFlag && !errors.Is(err_, os.ErrNotExist) {
|
||||
|
||||
f.Close()
|
||||
|
||||
f, err = os.Open(filePath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to open Avro file: %v", err)
|
||||
}
|
||||
|
||||
br := bufio.NewReader(f)
|
||||
|
||||
ocfReader, err := goavro.NewOCFReader(br)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create OCF reader while changing schema: %v", err)
|
||||
}
|
||||
|
||||
for ocfReader.Scan() {
|
||||
record, err := ocfReader.Read()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to read record: %v", err)
|
||||
}
|
||||
|
||||
recordList = append(recordList, record.(map[string]any))
|
||||
}
|
||||
|
||||
f.Close()
|
||||
|
||||
err = os.Remove(filePath)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to delete file: %v", err)
|
||||
}
|
||||
|
||||
readFlag = false
|
||||
}
|
||||
codec, err = goavro.NewCodec(schema)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create codec after merged schema: %v", err)
|
||||
}
|
||||
|
||||
recordList = append(recordList, generateRecord(data))
|
||||
delete(l.data, ts)
|
||||
}
|
||||
}
|
||||
|
||||
if len(recordList) == 0 {
|
||||
return ErrNoNewArchiveData
|
||||
}
|
||||
|
||||
f, err = os.OpenFile(filePath, os.O_CREATE|os.O_APPEND|os.O_RDWR, 0o644)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to append new avro file: %v", err)
|
||||
}
|
||||
|
||||
// fmt.Printf("Codec : %#v\n", codec)
|
||||
|
||||
writer, err := goavro.NewOCFWriter(goavro.OCFConfig{
|
||||
W: f,
|
||||
Codec: codec,
|
||||
CompressionName: goavro.CompressionDeflateLabel,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create OCF writer: %v", err)
|
||||
}
|
||||
|
||||
// Append the new record
|
||||
if err := writer.Append(recordList); err != nil {
|
||||
return fmt.Errorf("failed to append record: %v", err)
|
||||
}
|
||||
|
||||
f.Close()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func compareSchema(schemaRead, schemaGen string) (bool, string, error) {
|
||||
var genSchema, readSchema AvroSchema
|
||||
|
||||
if schemaRead == "" {
|
||||
return false, schemaGen, nil
|
||||
}
|
||||
|
||||
// Unmarshal the schema strings into AvroSchema structs
|
||||
if err := json.Unmarshal([]byte(schemaGen), &genSchema); err != nil {
|
||||
return false, "", fmt.Errorf("failed to parse generated schema: %v", err)
|
||||
}
|
||||
if err := json.Unmarshal([]byte(schemaRead), &readSchema); err != nil {
|
||||
return false, "", fmt.Errorf("failed to parse read schema: %v", err)
|
||||
}
|
||||
|
||||
sort.Slice(genSchema.Fields, func(i, j int) bool {
|
||||
return genSchema.Fields[i].Name < genSchema.Fields[j].Name
|
||||
})
|
||||
|
||||
sort.Slice(readSchema.Fields, func(i, j int) bool {
|
||||
return readSchema.Fields[i].Name < readSchema.Fields[j].Name
|
||||
})
|
||||
|
||||
// Check if schemas are identical
|
||||
schemasEqual := true
|
||||
if len(genSchema.Fields) <= len(readSchema.Fields) {
|
||||
|
||||
for i := range genSchema.Fields {
|
||||
if genSchema.Fields[i].Name != readSchema.Fields[i].Name {
|
||||
schemasEqual = false
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
// If schemas are identical, return the read schema
|
||||
if schemasEqual {
|
||||
return false, schemaRead, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Create a map to hold unique fields from both schemas
|
||||
fieldMap := make(map[string]AvroField)
|
||||
|
||||
// Add fields from the read schema
|
||||
for _, field := range readSchema.Fields {
|
||||
fieldMap[field.Name] = field
|
||||
}
|
||||
|
||||
// Add or update fields from the generated schema
|
||||
for _, field := range genSchema.Fields {
|
||||
fieldMap[field.Name] = field
|
||||
}
|
||||
|
||||
// Create a union schema by collecting fields from the map
|
||||
var mergedFields []AvroField
|
||||
for _, field := range fieldMap {
|
||||
mergedFields = append(mergedFields, field)
|
||||
}
|
||||
|
||||
// Sort fields by name for consistency
|
||||
sort.Slice(mergedFields, func(i, j int) bool {
|
||||
return mergedFields[i].Name < mergedFields[j].Name
|
||||
})
|
||||
|
||||
// Create the merged schema
|
||||
mergedSchema := AvroSchema{
|
||||
Type: "record",
|
||||
Name: genSchema.Name,
|
||||
Fields: mergedFields,
|
||||
}
|
||||
|
||||
// Check if schemas are identical
|
||||
schemasEqual = len(mergedSchema.Fields) == len(readSchema.Fields)
|
||||
if schemasEqual {
|
||||
for i := range mergedSchema.Fields {
|
||||
if mergedSchema.Fields[i].Name != readSchema.Fields[i].Name {
|
||||
schemasEqual = false
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if schemasEqual {
|
||||
return false, schemaRead, nil
|
||||
}
|
||||
}
|
||||
|
||||
// Marshal the merged schema back to JSON
|
||||
mergedSchemaJSON, err := json.Marshal(mergedSchema)
|
||||
if err != nil {
|
||||
return false, "", fmt.Errorf("failed to marshal merged schema: %v", err)
|
||||
}
|
||||
|
||||
return true, string(mergedSchemaJSON), nil
|
||||
}
|
||||
|
||||
func generateSchema(data map[string]schema.Float) (string, error) {
|
||||
// Define the Avro schema structure
|
||||
schema := map[string]any{
|
||||
"type": "record",
|
||||
"name": "DataRecord",
|
||||
"fields": []map[string]any{},
|
||||
}
|
||||
|
||||
fieldTracker := make(map[string]struct{})
|
||||
|
||||
for key := range data {
|
||||
if _, exists := fieldTracker[key]; !exists {
|
||||
key = correctKey(key)
|
||||
|
||||
field := map[string]any{
|
||||
"name": key,
|
||||
"type": "double",
|
||||
"default": -1.0,
|
||||
}
|
||||
schema["fields"] = append(schema["fields"].([]map[string]any), field)
|
||||
fieldTracker[key] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
schemaString, err := json.Marshal(schema)
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("failed to marshal schema: %v", err)
|
||||
}
|
||||
|
||||
return string(schemaString), nil
|
||||
}
|
||||
|
||||
func generateRecord(data map[string]schema.Float) map[string]any {
|
||||
record := make(map[string]any)
|
||||
|
||||
// Iterate through each map in data
|
||||
for key, value := range data {
|
||||
key = correctKey(key)
|
||||
|
||||
// Set the value in the record
|
||||
// avro only accepts basic types
|
||||
record[key] = value.Double()
|
||||
}
|
||||
|
||||
return record
|
||||
}
|
||||
|
||||
func correctKey(key string) string {
|
||||
key = strings.ReplaceAll(key, "_", "_0x5F_")
|
||||
key = strings.ReplaceAll(key, ":", "_0x3A_")
|
||||
key = strings.ReplaceAll(key, ".", "_0x2E_")
|
||||
return key
|
||||
}
|
||||
|
||||
func ReplaceKey(key string) string {
|
||||
key = strings.ReplaceAll(key, "_0x2E_", ".")
|
||||
key = strings.ReplaceAll(key, "_0x3A_", ":")
|
||||
key = strings.ReplaceAll(key, "_0x5F_", "_")
|
||||
return key
|
||||
}
|
||||
129
pkg/metricstore/avroHelper.go
Normal file
129
pkg/metricstore/avroHelper.go
Normal file
@@ -0,0 +1,129 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"context"
|
||||
"slices"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
)
|
||||
|
||||
func DataStaging(wg *sync.WaitGroup, ctx context.Context) {
|
||||
// AvroPool is a pool of Avro writers.
|
||||
go func() {
|
||||
if Keys.Checkpoints.FileFormat == "json" {
|
||||
wg.Done() // Mark this goroutine as done
|
||||
return // Exit the goroutine
|
||||
}
|
||||
|
||||
defer wg.Done()
|
||||
|
||||
var avroLevel *AvroLevel
|
||||
oldSelector := make([]string, 0)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
// Drain any remaining messages in channel before exiting
|
||||
for {
|
||||
select {
|
||||
case val, ok := <-LineProtocolMessages:
|
||||
if !ok {
|
||||
// Channel closed
|
||||
return
|
||||
}
|
||||
// Process remaining message
|
||||
freq, err := GetMetricFrequency(val.MetricName)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
|
||||
metricName := ""
|
||||
for _, selectorName := range val.Selector {
|
||||
metricName += selectorName + SelectorDelimiter
|
||||
}
|
||||
metricName += val.MetricName
|
||||
|
||||
var selector []string
|
||||
selector = append(selector, val.Cluster, val.Node, strconv.FormatInt(freq, 10))
|
||||
|
||||
if !stringSlicesEqual(oldSelector, selector) {
|
||||
avroLevel = avroStore.root.findAvroLevelOrCreate(selector)
|
||||
if avroLevel == nil {
|
||||
cclog.Errorf("Error creating or finding the level with cluster : %s, node : %s, metric : %s\n", val.Cluster, val.Node, val.MetricName)
|
||||
}
|
||||
oldSelector = slices.Clone(selector)
|
||||
}
|
||||
|
||||
if avroLevel != nil {
|
||||
avroLevel.addMetric(metricName, val.Value, val.Timestamp, int(freq))
|
||||
}
|
||||
default:
|
||||
// No more messages, exit
|
||||
return
|
||||
}
|
||||
}
|
||||
case val, ok := <-LineProtocolMessages:
|
||||
if !ok {
|
||||
// Channel closed, exit gracefully
|
||||
return
|
||||
}
|
||||
|
||||
// Fetch the frequency of the metric from the global configuration
|
||||
freq, err := GetMetricFrequency(val.MetricName)
|
||||
if err != nil {
|
||||
cclog.Errorf("Error fetching metric frequency: %s\n", err)
|
||||
continue
|
||||
}
|
||||
|
||||
metricName := ""
|
||||
|
||||
for _, selectorName := range val.Selector {
|
||||
metricName += selectorName + SelectorDelimiter
|
||||
}
|
||||
|
||||
metricName += val.MetricName
|
||||
|
||||
// Create a new selector for the Avro level
|
||||
// The selector is a slice of strings that represents the path to the
|
||||
// Avro level. It is created by appending the cluster, node, and metric
|
||||
// name to the selector.
|
||||
var selector []string
|
||||
selector = append(selector, val.Cluster, val.Node, strconv.FormatInt(freq, 10))
|
||||
|
||||
if !stringSlicesEqual(oldSelector, selector) {
|
||||
// Get the Avro level for the metric
|
||||
avroLevel = avroStore.root.findAvroLevelOrCreate(selector)
|
||||
|
||||
// If the Avro level is nil, create a new one
|
||||
if avroLevel == nil {
|
||||
cclog.Errorf("Error creating or finding the level with cluster : %s, node : %s, metric : %s\n", val.Cluster, val.Node, val.MetricName)
|
||||
}
|
||||
oldSelector = slices.Clone(selector)
|
||||
}
|
||||
|
||||
if avroLevel != nil {
|
||||
avroLevel.addMetric(metricName, val.Value, val.Timestamp, int(freq))
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func stringSlicesEqual(a, b []string) bool {
|
||||
if len(a) != len(b) {
|
||||
return false
|
||||
}
|
||||
for i := range a {
|
||||
if a[i] != b[i] {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
167
pkg/metricstore/avroStruct.go
Normal file
167
pkg/metricstore/avroStruct.go
Normal file
@@ -0,0 +1,167 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
)
|
||||
|
||||
var (
|
||||
LineProtocolMessages = make(chan *AvroStruct)
|
||||
// SelectorDelimiter separates hierarchical selector components in metric names for Avro encoding
|
||||
SelectorDelimiter = "_SEL_"
|
||||
)
|
||||
|
||||
var CheckpointBufferMinutes = DefaultCheckpointBufferMin
|
||||
|
||||
type AvroStruct struct {
|
||||
MetricName string
|
||||
Cluster string
|
||||
Node string
|
||||
Selector []string
|
||||
Value schema.Float
|
||||
Timestamp int64
|
||||
}
|
||||
|
||||
type AvroStore struct {
|
||||
root AvroLevel
|
||||
}
|
||||
|
||||
var avroStore AvroStore
|
||||
|
||||
type AvroLevel struct {
|
||||
children map[string]*AvroLevel
|
||||
data map[int64]map[string]schema.Float
|
||||
lock sync.RWMutex
|
||||
}
|
||||
|
||||
type AvroField struct {
|
||||
Name string `json:"name"`
|
||||
Type any `json:"type"`
|
||||
Default any `json:"default,omitempty"`
|
||||
}
|
||||
|
||||
type AvroSchema struct {
|
||||
Type string `json:"type"`
|
||||
Name string `json:"name"`
|
||||
Fields []AvroField `json:"fields"`
|
||||
}
|
||||
|
||||
func (l *AvroLevel) findAvroLevelOrCreate(selector []string) *AvroLevel {
|
||||
if len(selector) == 0 {
|
||||
return l
|
||||
}
|
||||
|
||||
// Allow concurrent reads:
|
||||
l.lock.RLock()
|
||||
var child *AvroLevel
|
||||
var ok bool
|
||||
if l.children == nil {
|
||||
// Children map needs to be created...
|
||||
l.lock.RUnlock()
|
||||
} else {
|
||||
child, ok := l.children[selector[0]]
|
||||
l.lock.RUnlock()
|
||||
if ok {
|
||||
return child.findAvroLevelOrCreate(selector[1:])
|
||||
}
|
||||
}
|
||||
|
||||
// The level does not exist, take write lock for unique access:
|
||||
l.lock.Lock()
|
||||
// While this thread waited for the write lock, another thread
|
||||
// could have created the child node.
|
||||
if l.children != nil {
|
||||
child, ok = l.children[selector[0]]
|
||||
if ok {
|
||||
l.lock.Unlock()
|
||||
return child.findAvroLevelOrCreate(selector[1:])
|
||||
}
|
||||
}
|
||||
|
||||
child = &AvroLevel{
|
||||
data: make(map[int64]map[string]schema.Float, 0),
|
||||
children: nil,
|
||||
}
|
||||
|
||||
if l.children != nil {
|
||||
l.children[selector[0]] = child
|
||||
} else {
|
||||
l.children = map[string]*AvroLevel{selector[0]: child}
|
||||
}
|
||||
l.lock.Unlock()
|
||||
return child.findAvroLevelOrCreate(selector[1:])
|
||||
}
|
||||
|
||||
func (l *AvroLevel) addMetric(metricName string, value schema.Float, timestamp int64, Freq int) {
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
KeyCounter := int(CheckpointBufferMinutes * 60 / Freq)
|
||||
|
||||
// Create keys in advance for the given amount of time
|
||||
if len(l.data) != KeyCounter {
|
||||
if len(l.data) == 0 {
|
||||
for i := range KeyCounter {
|
||||
l.data[timestamp+int64(i*Freq)] = make(map[string]schema.Float, 0)
|
||||
}
|
||||
} else {
|
||||
// Get the last timestamp
|
||||
var lastTS int64
|
||||
for ts := range l.data {
|
||||
if ts > lastTS {
|
||||
lastTS = ts
|
||||
}
|
||||
}
|
||||
// Create keys for the next KeyCounter timestamps
|
||||
l.data[lastTS+int64(Freq)] = make(map[string]schema.Float, 0)
|
||||
}
|
||||
}
|
||||
|
||||
closestTS := int64(0)
|
||||
minDiff := int64(Freq) + 1 // Start with diff just outside the valid range
|
||||
found := false
|
||||
|
||||
// Iterate over timestamps and choose the one which is within range.
|
||||
// Since its epoch time, we check if the difference is less than 60 seconds.
|
||||
for ts, dat := range l.data {
|
||||
// Check if timestamp is within range
|
||||
diff := timestamp - ts
|
||||
if diff < -int64(Freq) || diff > int64(Freq) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Metric already present at this timestamp — skip
|
||||
if _, ok := dat[metricName]; ok {
|
||||
continue
|
||||
}
|
||||
|
||||
// Check if this is the closest timestamp so far
|
||||
if Abs(diff) < minDiff {
|
||||
minDiff = Abs(diff)
|
||||
closestTS = ts
|
||||
found = true
|
||||
}
|
||||
}
|
||||
|
||||
if found {
|
||||
l.data[closestTS][metricName] = value
|
||||
}
|
||||
}
|
||||
|
||||
func GetAvroStore() *AvroStore {
|
||||
return &avroStore
|
||||
}
|
||||
|
||||
// Abs returns the absolute value of x.
|
||||
func Abs(x int64) int64 {
|
||||
if x < 0 {
|
||||
return -x
|
||||
}
|
||||
return x
|
||||
}
|
||||
318
pkg/metricstore/buffer.go
Normal file
318
pkg/metricstore/buffer.go
Normal file
@@ -0,0 +1,318 @@
|
||||
// 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 metricstore provides buffer.go: Time-series data buffer implementation.
|
||||
//
|
||||
// # Buffer Architecture
|
||||
//
|
||||
// Each metric at each hierarchical level (cluster/host/cpu/etc.) uses a linked-list
|
||||
// chain of fixed-size buffers to store time-series data. This design:
|
||||
//
|
||||
// - Avoids reallocation/copying when growing (new links added instead)
|
||||
// - Enables efficient pooling (buffers returned to sync.Pool)
|
||||
// - Supports traversal back in time (via prev pointers)
|
||||
// - Maintains temporal ordering (newer data in later buffers)
|
||||
//
|
||||
// # Buffer Chain Example
|
||||
//
|
||||
// [oldest buffer] <- prev -- [older] <- prev -- [newest buffer (head)]
|
||||
// start=1000 start=1512 start=2024
|
||||
// data=[v0...v511] data=[v0...v511] data=[v0...v42]
|
||||
//
|
||||
// When the head buffer reaches capacity (BufferCap = 512), a new buffer becomes
|
||||
// the new head and the old head is linked via prev.
|
||||
//
|
||||
// # Pooling Strategy
|
||||
//
|
||||
// sync.Pool reduces GC pressure for the common case (BufferCap-sized allocations).
|
||||
// Non-standard capacity buffers are not pooled (e.g., from checkpoint deserialization).
|
||||
//
|
||||
// # Time Alignment
|
||||
//
|
||||
// Timestamps are aligned to measurement frequency intervals:
|
||||
//
|
||||
// index = (timestamp - buffer.start) / buffer.frequency
|
||||
// actualTime = buffer.start + (frequency / 2) + (index * frequency)
|
||||
//
|
||||
// Missing data points are represented as NaN values. The read() function performs
|
||||
// linear interpolation where possible.
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
)
|
||||
|
||||
// BufferCap is the default buffer capacity.
|
||||
// buffer.data will only ever grow up to its capacity and a new link
|
||||
// in the buffer chain will be created if needed so that no copying
|
||||
// of data or reallocation needs to happen on writes.
|
||||
const BufferCap int = DefaultBufferCapacity
|
||||
|
||||
var bufferPool sync.Pool = sync.Pool{
|
||||
New: func() any {
|
||||
return &buffer{
|
||||
data: make([]schema.Float, 0, BufferCap),
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrNoData indicates no time-series data exists for the requested metric/level.
|
||||
ErrNoData error = errors.New("[METRICSTORE]> no data for this metric/level")
|
||||
|
||||
// ErrDataDoesNotAlign indicates that aggregated data from child scopes
|
||||
// does not align with the parent scope's expected timestamps/intervals.
|
||||
ErrDataDoesNotAlign error = errors.New("[METRICSTORE]> data from lower granularities does not align")
|
||||
)
|
||||
|
||||
// buffer stores time-series data for a single metric at a specific hierarchical level.
|
||||
//
|
||||
// Buffers form doubly-linked chains ordered by time. When capacity is reached,
|
||||
// a new buffer becomes the head and the old head is linked via prev/next.
|
||||
//
|
||||
// Fields:
|
||||
// - prev: Link to older buffer in the chain (nil if this is oldest)
|
||||
// - next: Link to newer buffer in the chain (nil if this is newest/head)
|
||||
// - data: Time-series values (schema.Float supports NaN for missing data)
|
||||
// - frequency: Measurement interval in seconds
|
||||
// - start: Start timestamp (adjusted by -frequency/2 for alignment)
|
||||
// - archived: True if data has been persisted to disk archive
|
||||
// - closed: True if buffer is no longer accepting writes
|
||||
//
|
||||
// Index calculation: index = (timestamp - start) / frequency
|
||||
// Actual data timestamp: start + (frequency / 2) + (index * frequency)
|
||||
type buffer struct {
|
||||
prev *buffer
|
||||
next *buffer
|
||||
data []schema.Float
|
||||
frequency int64
|
||||
start int64
|
||||
archived bool
|
||||
closed bool
|
||||
}
|
||||
|
||||
func newBuffer(ts, freq int64) *buffer {
|
||||
b := bufferPool.Get().(*buffer)
|
||||
b.frequency = freq
|
||||
b.start = ts - (freq / 2)
|
||||
b.prev = nil
|
||||
b.next = nil
|
||||
b.archived = false
|
||||
b.closed = false
|
||||
b.data = b.data[:0]
|
||||
return b
|
||||
}
|
||||
|
||||
// write appends a timestamped value to the buffer chain.
|
||||
//
|
||||
// Returns the head buffer (which may be newly created if capacity was reached).
|
||||
// Timestamps older than the buffer's start are rejected. If the calculated index
|
||||
// exceeds capacity, a new buffer is allocated and linked as the new head.
|
||||
//
|
||||
// Missing timestamps are automatically filled with NaN values to maintain alignment.
|
||||
// Overwrites are allowed if the index is already within the existing data slice.
|
||||
//
|
||||
// Parameters:
|
||||
// - ts: Unix timestamp in seconds
|
||||
// - value: Metric value (can be schema.NaN for missing data)
|
||||
//
|
||||
// Returns:
|
||||
// - *buffer: The new head buffer (same as b if no new buffer created)
|
||||
// - error: Non-nil if timestamp is before buffer start
|
||||
func (b *buffer) write(ts int64, value schema.Float) (*buffer, error) {
|
||||
if ts < b.start {
|
||||
return nil, errors.New("[METRICSTORE]> cannot write value to buffer from past")
|
||||
}
|
||||
|
||||
// idx := int((ts - b.start + (b.frequency / 3)) / b.frequency)
|
||||
idx := int((ts - b.start) / b.frequency)
|
||||
if idx >= cap(b.data) {
|
||||
newbuf := newBuffer(ts, b.frequency)
|
||||
newbuf.prev = b
|
||||
b.next = newbuf
|
||||
b = newbuf
|
||||
idx = 0
|
||||
}
|
||||
|
||||
// Overwriting value or writing value from past
|
||||
if idx < len(b.data) {
|
||||
b.data[idx] = value
|
||||
return b, nil
|
||||
}
|
||||
|
||||
// Fill up unwritten slots with NaN
|
||||
for i := len(b.data); i < idx; i++ {
|
||||
b.data = append(b.data, schema.NaN)
|
||||
}
|
||||
|
||||
b.data = append(b.data, value)
|
||||
return b, nil
|
||||
}
|
||||
|
||||
func (b *buffer) end() int64 {
|
||||
return b.firstWrite() + int64(len(b.data))*b.frequency
|
||||
}
|
||||
|
||||
func (b *buffer) firstWrite() int64 {
|
||||
return b.start + (b.frequency / 2)
|
||||
}
|
||||
|
||||
// read retrieves time-series data from the buffer chain for the specified time range.
|
||||
//
|
||||
// Traverses the buffer chain backwards (via prev links) if 'from' precedes the current
|
||||
// buffer's start. Missing data points are represented as NaN. Values are accumulated
|
||||
// into the provided 'data' slice (using +=, so caller must zero-initialize if needed).
|
||||
//
|
||||
// The function adjusts the actual time range returned if data is unavailable at the
|
||||
// boundaries (returned via adjusted from/to timestamps).
|
||||
//
|
||||
// Parameters:
|
||||
// - from: Start timestamp (Unix seconds)
|
||||
// - to: End timestamp (Unix seconds, exclusive)
|
||||
// - data: Pre-allocated slice to accumulate results (must be large enough)
|
||||
//
|
||||
// Returns:
|
||||
// - []schema.Float: Slice of data (may be shorter than input 'data' slice)
|
||||
// - int64: Actual start timestamp with available data
|
||||
// - int64: Actual end timestamp (exclusive)
|
||||
// - error: Non-nil on failure
|
||||
//
|
||||
// Panics if 'data' slice is too small to hold all values in [from, to).
|
||||
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)
|
||||
}
|
||||
from = b.firstWrite()
|
||||
}
|
||||
|
||||
i := 0
|
||||
t := from
|
||||
for ; t < to; t += b.frequency {
|
||||
idx := int((t - b.start) / b.frequency)
|
||||
if idx >= cap(b.data) {
|
||||
if b.next == nil {
|
||||
break
|
||||
}
|
||||
b = b.next
|
||||
idx = 0
|
||||
}
|
||||
|
||||
if idx >= len(b.data) {
|
||||
if b.next == nil || to <= b.next.start {
|
||||
break
|
||||
}
|
||||
data[i] += schema.NaN
|
||||
} else if t < b.start {
|
||||
data[i] += schema.NaN
|
||||
} else {
|
||||
data[i] += b.data[idx]
|
||||
}
|
||||
i++
|
||||
}
|
||||
|
||||
return data[:i], from, t, nil
|
||||
}
|
||||
|
||||
// free removes buffers older than the specified timestamp from the chain.
|
||||
//
|
||||
// Recursively traverses backwards (via prev) and unlinks buffers whose end time
|
||||
// is before the retention threshold. Freed buffers are returned to the pool if
|
||||
// they have the standard capacity (BufferCap).
|
||||
//
|
||||
// Parameters:
|
||||
// - t: Retention threshold timestamp (Unix seconds)
|
||||
//
|
||||
// Returns:
|
||||
// - delme: True if the current buffer itself should be deleted by caller
|
||||
// - n: Number of buffers freed in this subtree
|
||||
func (b *buffer) free(t int64) (delme bool, n int) {
|
||||
if b.prev != nil {
|
||||
delme, m := b.prev.free(t)
|
||||
n += m
|
||||
if delme {
|
||||
b.prev.next = nil
|
||||
if cap(b.prev.data) == BufferCap {
|
||||
bufferPool.Put(b.prev)
|
||||
}
|
||||
b.prev = nil
|
||||
}
|
||||
}
|
||||
|
||||
end := b.end()
|
||||
if end < t {
|
||||
return true, n + 1
|
||||
}
|
||||
|
||||
return false, n
|
||||
}
|
||||
|
||||
// forceFreeOldest recursively finds the end of the linked list (the oldest buffer)
|
||||
// and removes it.
|
||||
// Returns:
|
||||
//
|
||||
// delme: true if 'b' itself is the oldest and should be removed by the caller
|
||||
// n: the number of buffers freed (will be 1 or 0)
|
||||
func (b *buffer) forceFreeOldest() (delme bool, n int) {
|
||||
// If there is a previous buffer, recurse down to find the oldest
|
||||
if b.prev != nil {
|
||||
delPrev, freed := b.prev.forceFreeOldest()
|
||||
|
||||
// If the previous buffer signals it should be deleted:
|
||||
if delPrev {
|
||||
// Clear links on the dying buffer to prevent leaks
|
||||
b.prev.next = nil
|
||||
b.prev.data = nil // Release the underlying float slice immediately
|
||||
|
||||
// Remove the link from the current buffer
|
||||
b.prev = nil
|
||||
}
|
||||
return false, freed
|
||||
}
|
||||
|
||||
// If b.prev is nil, THIS buffer is the oldest.
|
||||
// We return true so the parent (or the Level loop) knows to delete reference to 'b'.
|
||||
return true, 1
|
||||
}
|
||||
|
||||
// iterFromTo invokes callback on every buffer in the chain that overlaps [from, to].
|
||||
//
|
||||
// Traverses backwards (via prev) first, then processes current buffer if it overlaps
|
||||
// the time range. Used for checkpoint/archive operations that need to serialize buffers
|
||||
// within a specific time window.
|
||||
//
|
||||
// Parameters:
|
||||
// - from: Start timestamp (Unix seconds, inclusive)
|
||||
// - to: End timestamp (Unix seconds, inclusive)
|
||||
// - callback: Function to invoke on each overlapping buffer
|
||||
//
|
||||
// Returns:
|
||||
// - error: First error returned by callback, or nil if all succeeded
|
||||
func (b *buffer) iterFromTo(from, to int64, callback func(b *buffer) error) error {
|
||||
if b == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := b.prev.iterFromTo(from, to, callback); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if from <= b.end() && b.start <= to {
|
||||
return callback(b)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *buffer) count() int64 {
|
||||
res := int64(len(b.data))
|
||||
if b.prev != nil {
|
||||
res += b.prev.count()
|
||||
}
|
||||
return res
|
||||
}
|
||||
853
pkg/metricstore/checkpoint.go
Normal file
853
pkg/metricstore/checkpoint.go
Normal file
@@ -0,0 +1,853 @@
|
||||
// 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.
|
||||
|
||||
// This file implements checkpoint persistence for the in-memory metric store.
|
||||
//
|
||||
// Checkpoints enable graceful restarts by periodically saving in-memory metric
|
||||
// data to disk in either JSON or Avro format. The checkpoint system:
|
||||
//
|
||||
// Key Features:
|
||||
// - Periodic background checkpointing via the Checkpointing() worker
|
||||
// - Two formats: JSON (human-readable) and Avro (compact, efficient)
|
||||
// - Parallel checkpoint creation and loading using worker pools
|
||||
// - Hierarchical file organization: checkpoint_dir/cluster/host/timestamp.{json|avro}
|
||||
// - Only saves unarchived data (archived data is already persisted elsewhere)
|
||||
// - Automatic format detection and fallback during loading
|
||||
// - GC optimization during loading to prevent excessive heap growth
|
||||
//
|
||||
// Checkpoint Workflow:
|
||||
// 1. Init() loads checkpoints within retention window at startup
|
||||
// 2. Checkpointing() worker periodically saves new data
|
||||
// 3. Shutdown() writes final checkpoint before exit
|
||||
//
|
||||
// File Organization:
|
||||
//
|
||||
// checkpoints/
|
||||
// cluster1/
|
||||
// host001/
|
||||
// 1234567890.json (timestamp = checkpoint start time)
|
||||
// 1234567950.json
|
||||
// host002/
|
||||
// ...
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/fs"
|
||||
"os"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
"github.com/linkedin/goavro/v2"
|
||||
)
|
||||
|
||||
const (
|
||||
CheckpointFilePerms = 0o644 // File permissions for checkpoint files
|
||||
CheckpointDirPerms = 0o755 // Directory permissions for checkpoint directories
|
||||
GCTriggerInterval = DefaultGCTriggerInterval // Interval for triggering GC during checkpoint loading
|
||||
)
|
||||
|
||||
// CheckpointMetrics represents metric data in a checkpoint file.
|
||||
// Whenever the structure changes, update MarshalJSON as well!
|
||||
type CheckpointMetrics struct {
|
||||
Data []schema.Float `json:"data"`
|
||||
Frequency int64 `json:"frequency"`
|
||||
Start int64 `json:"start"`
|
||||
}
|
||||
|
||||
// CheckpointFile represents the hierarchical structure of a checkpoint file.
|
||||
// It mirrors the Level tree structure from the MemoryStore.
|
||||
type CheckpointFile struct {
|
||||
Metrics map[string]*CheckpointMetrics `json:"metrics"`
|
||||
Children map[string]*CheckpointFile `json:"children"`
|
||||
From int64 `json:"from"`
|
||||
To int64 `json:"to"`
|
||||
}
|
||||
|
||||
// lastCheckpoint tracks the timestamp of the last checkpoint creation.
|
||||
var (
|
||||
lastCheckpoint time.Time
|
||||
lastCheckpointMu sync.Mutex
|
||||
)
|
||||
|
||||
// Checkpointing starts a background worker that periodically saves metric data to disk.
|
||||
//
|
||||
// The behavior depends on the configured file format:
|
||||
// - JSON: Periodic checkpointing based on Keys.Checkpoints.Interval
|
||||
// - Avro: Initial delay + periodic checkpointing at DefaultAvroCheckpointInterval
|
||||
//
|
||||
// The worker respects context cancellation and signals completion via the WaitGroup.
|
||||
func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
|
||||
lastCheckpointMu.Lock()
|
||||
lastCheckpoint = time.Now()
|
||||
lastCheckpointMu.Unlock()
|
||||
|
||||
if Keys.Checkpoints.FileFormat == "json" {
|
||||
ms := GetMemoryStore()
|
||||
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
d, err := time.ParseDuration(Keys.Checkpoints.Interval)
|
||||
if err != nil {
|
||||
cclog.Fatalf("[METRICSTORE]> invalid checkpoint interval '%s': %s", Keys.Checkpoints.Interval, err.Error())
|
||||
}
|
||||
if d <= 0 {
|
||||
cclog.Warnf("[METRICSTORE]> checkpoint interval is zero or negative (%s), checkpointing disabled", d)
|
||||
return
|
||||
}
|
||||
|
||||
ticker := time.NewTicker(d)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
lastCheckpointMu.Lock()
|
||||
from := lastCheckpoint
|
||||
lastCheckpointMu.Unlock()
|
||||
|
||||
cclog.Infof("[METRICSTORE]> start checkpointing (starting at %s)...", from.Format(time.RFC3339))
|
||||
now := time.Now()
|
||||
n, err := ms.ToCheckpoint(Keys.Checkpoints.RootDir,
|
||||
from.Unix(), now.Unix())
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> checkpointing failed: %s", err.Error())
|
||||
} else {
|
||||
cclog.Infof("[METRICSTORE]> done: %d checkpoint files created", n)
|
||||
lastCheckpointMu.Lock()
|
||||
lastCheckpoint = now
|
||||
lastCheckpointMu.Unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
} else {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-time.After(time.Duration(CheckpointBufferMinutes) * time.Minute):
|
||||
GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, false)
|
||||
}
|
||||
|
||||
ticker := time.NewTicker(DefaultAvroCheckpointInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, false)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// MarshalJSON provides optimized JSON encoding for CheckpointMetrics.
|
||||
//
|
||||
// Since schema.Float has custom MarshalJSON, serializing []Float has significant overhead.
|
||||
// This method manually constructs JSON to avoid allocations and interface conversions.
|
||||
func (cm *CheckpointMetrics) MarshalJSON() ([]byte, error) {
|
||||
buf := make([]byte, 0, 128+len(cm.Data)*8)
|
||||
buf = append(buf, `{"frequency":`...)
|
||||
buf = strconv.AppendInt(buf, cm.Frequency, 10)
|
||||
buf = append(buf, `,"start":`...)
|
||||
buf = strconv.AppendInt(buf, cm.Start, 10)
|
||||
buf = append(buf, `,"data":[`...)
|
||||
for i, x := range cm.Data {
|
||||
if i != 0 {
|
||||
buf = append(buf, ',')
|
||||
}
|
||||
if x.IsNaN() {
|
||||
buf = append(buf, `null`...)
|
||||
} else {
|
||||
buf = strconv.AppendFloat(buf, float64(x), 'f', 1, 32)
|
||||
}
|
||||
}
|
||||
buf = append(buf, `]}`...)
|
||||
return buf, nil
|
||||
}
|
||||
|
||||
// ToCheckpoint writes metric data to checkpoint files in parallel.
|
||||
//
|
||||
// Metrics at root and cluster levels are skipped. One file per host is created.
|
||||
// Uses worker pool (Keys.NumWorkers) for parallel processing. Only locks one host
|
||||
// at a time, allowing concurrent writes/reads to other hosts.
|
||||
//
|
||||
// Returns the number of checkpoint files created and any errors encountered.
|
||||
func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
|
||||
// Pre-calculate capacity by counting cluster/host pairs
|
||||
m.root.lock.RLock()
|
||||
totalHosts := 0
|
||||
for _, l1 := range m.root.children {
|
||||
l1.lock.RLock()
|
||||
totalHosts += len(l1.children)
|
||||
l1.lock.RUnlock()
|
||||
}
|
||||
m.root.lock.RUnlock()
|
||||
|
||||
levels := make([]*Level, 0, totalHosts)
|
||||
selectors := make([][]string, 0, totalHosts)
|
||||
|
||||
m.root.lock.RLock()
|
||||
for sel1, l1 := range m.root.children {
|
||||
l1.lock.RLock()
|
||||
for sel2, l2 := range l1.children {
|
||||
levels = append(levels, l2)
|
||||
selectors = append(selectors, []string{sel1, sel2})
|
||||
}
|
||||
l1.lock.RUnlock()
|
||||
}
|
||||
m.root.lock.RUnlock()
|
||||
|
||||
type workItem struct {
|
||||
level *Level
|
||||
dir string
|
||||
selector []string
|
||||
}
|
||||
|
||||
n, errs := int32(0), int32(0)
|
||||
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(Keys.NumWorkers)
|
||||
work := make(chan workItem, Keys.NumWorkers*2)
|
||||
for worker := 0; worker < Keys.NumWorkers; worker++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
|
||||
for workItem := range work {
|
||||
if err := workItem.level.toCheckpoint(workItem.dir, from, to, m); err != nil {
|
||||
if err == ErrNoNewArchiveData {
|
||||
continue
|
||||
}
|
||||
|
||||
cclog.Errorf("[METRICSTORE]> error while checkpointing %#v: %s", workItem.selector, err.Error())
|
||||
atomic.AddInt32(&errs, 1)
|
||||
} else {
|
||||
atomic.AddInt32(&n, 1)
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
for i := 0; i < len(levels); i++ {
|
||||
dir := path.Join(dir, path.Join(selectors[i]...))
|
||||
work <- workItem{
|
||||
level: levels[i],
|
||||
dir: dir,
|
||||
selector: selectors[i],
|
||||
}
|
||||
}
|
||||
|
||||
close(work)
|
||||
wg.Wait()
|
||||
|
||||
if errs > 0 {
|
||||
return int(n), fmt.Errorf("[METRICSTORE]> %d errors happened while creating checkpoints (%d successes)", errs, n)
|
||||
}
|
||||
return int(n), nil
|
||||
}
|
||||
|
||||
// toCheckpointFile recursively converts a Level tree to CheckpointFile structure.
|
||||
// Skips metrics that are already archived. Returns nil if no unarchived data exists.
|
||||
func (l *Level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFile, error) {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
retval := &CheckpointFile{
|
||||
From: from,
|
||||
To: to,
|
||||
Metrics: make(map[string]*CheckpointMetrics),
|
||||
Children: make(map[string]*CheckpointFile),
|
||||
}
|
||||
|
||||
for metric, minfo := range m.Metrics {
|
||||
b := l.metrics[minfo.offset]
|
||||
if b == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
allArchived := true
|
||||
b.iterFromTo(from, to, func(b *buffer) error {
|
||||
if !b.archived {
|
||||
allArchived = false
|
||||
return fmt.Errorf("stop") // Early termination signal
|
||||
}
|
||||
return nil
|
||||
})
|
||||
|
||||
if allArchived {
|
||||
continue
|
||||
}
|
||||
|
||||
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] = schema.NaN
|
||||
}
|
||||
|
||||
retval.Metrics[metric] = &CheckpointMetrics{
|
||||
Frequency: b.frequency,
|
||||
Start: start,
|
||||
Data: data,
|
||||
}
|
||||
}
|
||||
|
||||
for name, child := range l.children {
|
||||
val, err := child.toCheckpointFile(from, to, m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if val != nil {
|
||||
retval.Children[name] = val
|
||||
}
|
||||
}
|
||||
|
||||
if len(retval.Children) == 0 && len(retval.Metrics) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return retval, nil
|
||||
}
|
||||
|
||||
// toCheckpoint writes a Level's data to a JSON checkpoint file.
|
||||
// Creates directory if needed. Returns ErrNoNewArchiveData if nothing to save.
|
||||
func (l *Level) toCheckpoint(dir string, from, to int64, m *MemoryStore) error {
|
||||
cf, err := l.toCheckpointFile(from, to, m)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if cf == nil {
|
||||
return ErrNoNewArchiveData
|
||||
}
|
||||
|
||||
filepath := path.Join(dir, fmt.Sprintf("%d.json", from))
|
||||
f, err := os.OpenFile(filepath, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, CheckpointFilePerms)
|
||||
if err != nil && os.IsNotExist(err) {
|
||||
err = os.MkdirAll(dir, CheckpointDirPerms)
|
||||
if err == nil {
|
||||
f, err = os.OpenFile(filepath, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, CheckpointFilePerms)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
bw := bufio.NewWriter(f)
|
||||
if err = json.NewEncoder(bw).Encode(cf); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return bw.Flush()
|
||||
}
|
||||
|
||||
// enqueueCheckpointHosts traverses checkpoint directory and enqueues cluster/host pairs.
|
||||
// Returns error if directory structure is invalid.
|
||||
func enqueueCheckpointHosts(dir string, work chan<- [2]string) error {
|
||||
clustersDir, err := os.ReadDir(dir)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
gcCounter := 0
|
||||
for _, clusterDir := range clustersDir {
|
||||
if !clusterDir.IsDir() {
|
||||
return errors.New("[METRICSTORE]> expected only directories at first level of checkpoints/ directory")
|
||||
}
|
||||
|
||||
hostsDir, err := os.ReadDir(filepath.Join(dir, clusterDir.Name()))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, hostDir := range hostsDir {
|
||||
if !hostDir.IsDir() {
|
||||
return errors.New("[METRICSTORE]> expected only directories at second level of checkpoints/ directory")
|
||||
}
|
||||
|
||||
gcCounter++
|
||||
if gcCounter%GCTriggerInterval == 0 {
|
||||
// Forcing garbage collection runs here regulary during the loading of checkpoints
|
||||
// will decrease the total heap size after loading everything back to memory is done.
|
||||
// While loading data, the heap will grow fast, so the GC target size will double
|
||||
// almost always. By forcing GCs here, we can keep it growing more slowly so that
|
||||
// at the end, less memory is wasted.
|
||||
runtime.GC()
|
||||
}
|
||||
|
||||
work <- [2]string{clusterDir.Name(), hostDir.Name()}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// FromCheckpoint loads checkpoint files from disk into memory in parallel.
|
||||
//
|
||||
// Uses worker pool to load cluster/host combinations. Periodically triggers GC
|
||||
// to prevent excessive heap growth. Returns number of files loaded and any errors.
|
||||
func (m *MemoryStore) FromCheckpoint(dir string, from int64, extension string) (int, error) {
|
||||
var wg sync.WaitGroup
|
||||
work := make(chan [2]string, Keys.NumWorkers*4)
|
||||
n, errs := int32(0), int32(0)
|
||||
|
||||
wg.Add(Keys.NumWorkers)
|
||||
for worker := 0; worker < Keys.NumWorkers; worker++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for host := range work {
|
||||
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 {
|
||||
cclog.Errorf("[METRICSTORE]> error while loading checkpoints for %s/%s: %s", host[0], host[1], err.Error())
|
||||
atomic.AddInt32(&errs, 1)
|
||||
}
|
||||
atomic.AddInt32(&n, int32(nn))
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
err := enqueueCheckpointHosts(dir, work)
|
||||
close(work)
|
||||
wg.Wait()
|
||||
|
||||
if err != nil {
|
||||
return int(n), err
|
||||
}
|
||||
|
||||
if errs > 0 {
|
||||
return int(n), fmt.Errorf("[METRICSTORE]> %d errors happened while creating checkpoints (%d successes)", errs, n)
|
||||
}
|
||||
return int(n), nil
|
||||
}
|
||||
|
||||
// FromCheckpointFiles is the main entry point for loading checkpoints at startup.
|
||||
//
|
||||
// Automatically detects checkpoint format (JSON vs Avro) and falls back if needed.
|
||||
// Creates checkpoint directory if it doesn't exist. This function must be called
|
||||
// before any writes or reads, and can only be called once.
|
||||
func (m *MemoryStore) FromCheckpointFiles(dir string, from int64) (int, error) {
|
||||
if _, err := os.Stat(dir); os.IsNotExist(err) {
|
||||
// The directory does not exist, so create it using os.MkdirAll()
|
||||
err := os.MkdirAll(dir, CheckpointDirPerms) // CheckpointDirPerms sets the permissions for the directory
|
||||
if err != nil {
|
||||
cclog.Fatalf("[METRICSTORE]> Error creating directory: %#v\n", err)
|
||||
}
|
||||
cclog.Debugf("[METRICSTORE]> %#v Directory created successfully", dir)
|
||||
}
|
||||
|
||||
// Config read (replace with your actual config read)
|
||||
fileFormat := Keys.Checkpoints.FileFormat
|
||||
if fileFormat == "" {
|
||||
fileFormat = "avro"
|
||||
}
|
||||
|
||||
// Map to easily get the fallback format
|
||||
oppositeFormat := map[string]string{
|
||||
"json": "avro",
|
||||
"avro": "json",
|
||||
}
|
||||
|
||||
// First, attempt to load the specified format
|
||||
if found, err := checkFilesWithExtension(dir, fileFormat); err != nil {
|
||||
return 0, fmt.Errorf("[METRICSTORE]> error checking files with extension: %v", err)
|
||||
} else if found {
|
||||
cclog.Infof("[METRICSTORE]> Loading %s files because fileformat is %s", 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("[METRICSTORE]> error checking files with extension: %v", err)
|
||||
} else if found {
|
||||
cclog.Infof("[METRICSTORE]> Loading %s files but fileformat is %s", altFormat, fileFormat)
|
||||
return m.FromCheckpoint(dir, from, altFormat)
|
||||
}
|
||||
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
// checkFilesWithExtension walks a directory tree to check if files with the given extension exist.
|
||||
func checkFilesWithExtension(dir string, extension string) (bool, error) {
|
||||
found := false
|
||||
|
||||
err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error {
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error accessing path %s: %v", path, err)
|
||||
}
|
||||
if !info.IsDir() && filepath.Ext(info.Name()) == "."+extension {
|
||||
found = true
|
||||
return nil
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("[METRICSTORE]> error walking through directories: %s", err)
|
||||
}
|
||||
|
||||
return found, nil
|
||||
}
|
||||
|
||||
func (l *Level) loadAvroFile(m *MemoryStore, f *os.File, from int64) error {
|
||||
br := bufio.NewReader(f)
|
||||
|
||||
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("[METRICSTORE]> error while reading avro file (resolution parsing) : %s", err)
|
||||
}
|
||||
|
||||
fromTimestamp, err := strconv.ParseInt(fileName[strings.Index(fileName, "_")+1:len(fileName)-5], 10, 64)
|
||||
|
||||
// Same logic according to lineprotocol
|
||||
fromTimestamp -= (resolution / 2)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error converting timestamp from the avro file : %s", err)
|
||||
}
|
||||
|
||||
// fmt.Printf("File : %s with resolution : %d\n", fileName, resolution)
|
||||
|
||||
var recordCounter int64 = 0
|
||||
|
||||
// Create a new OCF reader from the buffered reader
|
||||
ocfReader, err := goavro.NewOCFReader(br)
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error creating OCF reader: %w", err)
|
||||
}
|
||||
|
||||
metricsData := make(map[string]schema.FloatArray)
|
||||
|
||||
for ocfReader.Scan() {
|
||||
datum, err := ocfReader.Read()
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error while reading avro file : %s", err)
|
||||
}
|
||||
|
||||
record, ok := datum.(map[string]any)
|
||||
if !ok {
|
||||
return fmt.Errorf("[METRICSTORE]> failed to assert datum as map[string]interface{}")
|
||||
}
|
||||
|
||||
for key, value := range record {
|
||||
metricsData[key] = append(metricsData[key], schema.ConvertToFloat(value.(float64)))
|
||||
}
|
||||
|
||||
recordCounter += 1
|
||||
}
|
||||
|
||||
to := (fromTimestamp + (recordCounter / (60 / resolution) * 60))
|
||||
if to < from {
|
||||
return nil
|
||||
}
|
||||
|
||||
for key, floatArray := range metricsData {
|
||||
metricName := ReplaceKey(key)
|
||||
|
||||
if strings.Contains(metricName, SelectorDelimiter) {
|
||||
subString := strings.Split(metricName, SelectorDelimiter)
|
||||
|
||||
lvl := l
|
||||
|
||||
for i := 0; i < len(subString)-1; i++ {
|
||||
|
||||
sel := subString[i]
|
||||
|
||||
if lvl.children == nil {
|
||||
lvl.children = make(map[string]*Level)
|
||||
}
|
||||
|
||||
child, ok := lvl.children[sel]
|
||||
if !ok {
|
||||
child = &Level{
|
||||
metrics: make([]*buffer, len(m.Metrics)),
|
||||
children: nil,
|
||||
}
|
||||
lvl.children[sel] = child
|
||||
}
|
||||
lvl = child
|
||||
}
|
||||
|
||||
leafMetricName := subString[len(subString)-1]
|
||||
err = lvl.createBuffer(m, leafMetricName, floatArray, fromTimestamp, resolution)
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error while creating buffers from avroReader : %s", err)
|
||||
}
|
||||
} else {
|
||||
err = l.createBuffer(m, metricName, floatArray, fromTimestamp, resolution)
|
||||
if err != nil {
|
||||
return fmt.Errorf("[METRICSTORE]> error while creating buffers from avroReader : %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *Level) createBuffer(m *MemoryStore, metricName string, floatArray schema.FloatArray, from int64, resolution int64) error {
|
||||
n := len(floatArray)
|
||||
b := &buffer{
|
||||
frequency: resolution,
|
||||
start: from,
|
||||
data: floatArray[0:n:n],
|
||||
prev: nil,
|
||||
next: nil,
|
||||
archived: true,
|
||||
}
|
||||
|
||||
minfo, ok := m.Metrics[metricName]
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
prev := l.metrics[minfo.offset]
|
||||
if prev == nil {
|
||||
l.metrics[minfo.offset] = b
|
||||
} else {
|
||||
if prev.start > b.start {
|
||||
return fmt.Errorf("[METRICSTORE]> buffer start time %d is before previous buffer start %d", b.start, prev.start)
|
||||
}
|
||||
|
||||
b.prev = prev
|
||||
prev.next = b
|
||||
|
||||
missingCount := ((int(b.start) - int(prev.start)) - len(prev.data)*int(b.frequency))
|
||||
if missingCount > 0 {
|
||||
missingCount /= int(b.frequency)
|
||||
|
||||
for range missingCount {
|
||||
prev.data = append(prev.data, schema.NaN)
|
||||
}
|
||||
|
||||
prev.data = prev.data[0:len(prev.data):len(prev.data)]
|
||||
}
|
||||
}
|
||||
l.metrics[minfo.offset] = b
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *Level) loadJSONFile(m *MemoryStore, f *os.File, from int64) error {
|
||||
br := bufio.NewReader(f)
|
||||
cf := &CheckpointFile{}
|
||||
if err := json.NewDecoder(br).Decode(cf); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if cf.To != 0 && cf.To < from {
|
||||
return nil
|
||||
}
|
||||
|
||||
if err := l.loadFile(cf, m); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *Level) loadFile(cf *CheckpointFile, m *MemoryStore) error {
|
||||
for name, metric := range cf.Metrics {
|
||||
n := len(metric.Data)
|
||||
b := &buffer{
|
||||
frequency: metric.Frequency,
|
||||
start: metric.Start,
|
||||
data: metric.Data[0:n:n],
|
||||
prev: nil,
|
||||
next: nil,
|
||||
archived: true,
|
||||
}
|
||||
|
||||
minfo, ok := m.Metrics[name]
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
||||
prev := l.metrics[minfo.offset]
|
||||
if prev == nil {
|
||||
l.metrics[minfo.offset] = b
|
||||
} else {
|
||||
if prev.start > b.start {
|
||||
return fmt.Errorf("[METRICSTORE]> buffer start time %d is before previous buffer start %d", b.start, prev.start)
|
||||
}
|
||||
|
||||
b.prev = prev
|
||||
prev.next = b
|
||||
}
|
||||
l.metrics[minfo.offset] = b
|
||||
}
|
||||
|
||||
if len(cf.Children) > 0 && l.children == nil {
|
||||
l.children = make(map[string]*Level)
|
||||
}
|
||||
|
||||
for sel, childCf := range cf.Children {
|
||||
child, ok := l.children[sel]
|
||||
if !ok {
|
||||
child = &Level{
|
||||
metrics: make([]*buffer, len(m.Metrics)),
|
||||
children: nil,
|
||||
}
|
||||
l.children[sel] = child
|
||||
}
|
||||
|
||||
if err := child.loadFile(childCf, m); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *Level) fromCheckpoint(m *MemoryStore, dir string, from int64, extension string) (int, error) {
|
||||
direntries, err := os.ReadDir(dir)
|
||||
if err != nil {
|
||||
if os.IsNotExist(err) {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
return 0, err
|
||||
}
|
||||
|
||||
allFiles := make([]fs.DirEntry, 0)
|
||||
filesLoaded := 0
|
||||
for _, e := range direntries {
|
||||
if e.IsDir() {
|
||||
child := &Level{
|
||||
metrics: make([]*buffer, len(m.Metrics)),
|
||||
children: make(map[string]*Level),
|
||||
}
|
||||
|
||||
files, err := child.fromCheckpoint(m, path.Join(dir, e.Name()), from, extension)
|
||||
filesLoaded += files
|
||||
if err != nil {
|
||||
return filesLoaded, err
|
||||
}
|
||||
|
||||
l.children[e.Name()] = child
|
||||
} else if strings.HasSuffix(e.Name(), "."+extension) {
|
||||
allFiles = append(allFiles, e)
|
||||
} else {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
files, err := findFiles(allFiles, from, extension, true)
|
||||
if err != nil {
|
||||
return filesLoaded, err
|
||||
}
|
||||
|
||||
loaders := map[string]func(*MemoryStore, *os.File, int64) error{
|
||||
"json": l.loadJSONFile,
|
||||
"avro": l.loadAvroFile,
|
||||
}
|
||||
|
||||
loader := loaders[extension]
|
||||
|
||||
for _, filename := range files {
|
||||
// Use a closure to ensure file is closed immediately after use
|
||||
err := func() error {
|
||||
f, err := os.Open(path.Join(dir, filename))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer f.Close()
|
||||
|
||||
return loader(m, f, from)
|
||||
}()
|
||||
if err != nil {
|
||||
return filesLoaded, err
|
||||
}
|
||||
|
||||
filesLoaded += 1
|
||||
}
|
||||
|
||||
return filesLoaded, nil
|
||||
}
|
||||
|
||||
// This will probably get very slow over time!
|
||||
// A solution could be some sort of an index file in which all other files
|
||||
// and the timespan they contain is listed.
|
||||
func findFiles(direntries []fs.DirEntry, t int64, extension string, findMoreRecentFiles bool) ([]string, error) {
|
||||
nums := map[string]int64{}
|
||||
for _, e := range direntries {
|
||||
if !strings.HasSuffix(e.Name(), "."+extension) {
|
||||
continue
|
||||
}
|
||||
|
||||
ts, err := strconv.ParseInt(e.Name()[strings.Index(e.Name(), "_")+1:len(e.Name())-5], 10, 64)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
nums[e.Name()] = ts
|
||||
}
|
||||
|
||||
sort.Slice(direntries, func(i, j int) bool {
|
||||
a, b := direntries[i], direntries[j]
|
||||
return nums[a.Name()] < nums[b.Name()]
|
||||
})
|
||||
|
||||
if len(nums) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
filenames := make([]string, 0)
|
||||
|
||||
for i, e := range direntries {
|
||||
ts1 := nums[e.Name()]
|
||||
|
||||
// Logic to look for files in forward or direction
|
||||
// If logic: All files greater than or after
|
||||
// the given timestamp will be selected
|
||||
// Else If logic: All files less than or before
|
||||
// the given timestamp will be selected
|
||||
if findMoreRecentFiles && t <= ts1 {
|
||||
filenames = append(filenames, e.Name())
|
||||
} else if !findMoreRecentFiles && ts1 <= t && ts1 != 0 {
|
||||
filenames = append(filenames, e.Name())
|
||||
}
|
||||
if i == len(direntries)-1 {
|
||||
continue
|
||||
}
|
||||
|
||||
enext := direntries[i+1]
|
||||
ts2 := nums[enext.Name()]
|
||||
|
||||
if findMoreRecentFiles {
|
||||
if ts1 < t && t < ts2 {
|
||||
filenames = append(filenames, e.Name())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return filenames, nil
|
||||
}
|
||||
261
pkg/metricstore/config.go
Normal file
261
pkg/metricstore/config.go
Normal file
@@ -0,0 +1,261 @@
|
||||
// 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 metricstore provides config.go: Configuration structures and metric management.
|
||||
//
|
||||
// # Configuration Hierarchy
|
||||
//
|
||||
// The metricstore package uses nested configuration structures:
|
||||
//
|
||||
// MetricStoreConfig (Keys)
|
||||
// ├─ NumWorkers: Parallel checkpoint/archive workers
|
||||
// ├─ RetentionInMemory: How long to keep data in RAM
|
||||
// ├─ MemoryCap: Memory limit in bytes (triggers forceFree)
|
||||
// ├─ Checkpoints: Persistence configuration
|
||||
// │ ├─ FileFormat: "avro" or "json"
|
||||
// │ ├─ Interval: How often to save (e.g., "1h")
|
||||
// │ └─ RootDir: Checkpoint storage path
|
||||
// ├─ Cleanup: Long-term storage configuration
|
||||
// │ ├─ Interval: How often to delete/archive
|
||||
// │ ├─ RootDir: Archive storage path
|
||||
// │ └─ Mode: "delete" or "archive"
|
||||
// ├─ Debug: Development/debugging options
|
||||
// └─ Subscriptions: NATS topic subscriptions for metric ingestion
|
||||
//
|
||||
// # Metric Configuration
|
||||
//
|
||||
// Each metric (e.g., "cpu_load", "mem_used") has a MetricConfig entry in the global
|
||||
// Metrics map, defining:
|
||||
//
|
||||
// - Frequency: Measurement interval in seconds
|
||||
// - Aggregation: How to combine values (sum/avg/none) when transforming scopes
|
||||
// - offset: Internal index into Level.metrics slice (assigned during Init)
|
||||
//
|
||||
// # AggregationStrategy
|
||||
//
|
||||
// Determines how to combine metric values when aggregating from finer to coarser scopes:
|
||||
//
|
||||
// - NoAggregation: Do not combine (incompatible scopes)
|
||||
// - SumAggregation: Add values (e.g., power consumption: core→socket)
|
||||
// - AvgAggregation: Average values (e.g., temperature: core→socket)
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
DefaultMaxWorkers = 10
|
||||
DefaultBufferCapacity = 512
|
||||
DefaultGCTriggerInterval = 100
|
||||
DefaultAvroWorkers = 4
|
||||
DefaultCheckpointBufferMin = 3
|
||||
DefaultAvroCheckpointInterval = time.Minute
|
||||
DefaultMemoryUsageTrackerInterval = 1 * time.Hour
|
||||
)
|
||||
|
||||
// Checkpoints configures periodic persistence of in-memory metric data.
|
||||
//
|
||||
// Fields:
|
||||
// - FileFormat: "avro" (default, binary, compact) or "json" (human-readable, slower)
|
||||
// - Interval: Duration string (e.g., "1h", "30m") between checkpoint saves
|
||||
// - RootDir: Filesystem path for checkpoint files (created if missing)
|
||||
type Checkpoints struct {
|
||||
FileFormat string `json:"file-format"`
|
||||
Interval string `json:"interval"`
|
||||
RootDir string `json:"directory"`
|
||||
}
|
||||
|
||||
// Debug provides development and profiling options.
|
||||
//
|
||||
// Fields:
|
||||
// - DumpToFile: Path to dump checkpoint data for inspection (empty = disabled)
|
||||
// - EnableGops: Enable gops agent for live runtime debugging (https://github.com/google/gops)
|
||||
type Debug struct {
|
||||
DumpToFile string `json:"dump-to-file"`
|
||||
EnableGops bool `json:"gops"`
|
||||
}
|
||||
|
||||
// Archive configures long-term storage of old metric data.
|
||||
//
|
||||
// Data older than RetentionInMemory is archived to disk or deleted.
|
||||
//
|
||||
// Fields:
|
||||
// - ArchiveInterval: Duration string (e.g., "24h") between archive operations
|
||||
// - RootDir: Filesystem path for archived data (created if missing)
|
||||
// - DeleteInstead: If true, delete old data instead of archiving (saves disk space)
|
||||
type Cleanup struct {
|
||||
Interval string `json:"interval"`
|
||||
RootDir string `json:"directory"`
|
||||
Mode string `json:"mode"`
|
||||
}
|
||||
|
||||
// Subscriptions defines NATS topics to subscribe to for metric ingestion.
|
||||
//
|
||||
// Each subscription receives metrics via NATS messaging, enabling real-time
|
||||
// data collection from compute nodes.
|
||||
//
|
||||
// Fields:
|
||||
// - SubscribeTo: NATS subject/channel name (e.g., "metrics.compute.*")
|
||||
// - ClusterTag: Default cluster name for metrics without cluster tag (optional)
|
||||
type Subscriptions []struct {
|
||||
// Channel name
|
||||
SubscribeTo string `json:"subscribe-to"`
|
||||
|
||||
// Allow lines without a cluster tag, use this as default, optional
|
||||
ClusterTag string `json:"cluster-tag"`
|
||||
}
|
||||
|
||||
// MetricStoreConfig defines the main configuration for the metricstore.
|
||||
//
|
||||
// Loaded from cc-backend's config.json "metricstore" section. Controls memory usage,
|
||||
// persistence, archiving, and metric ingestion.
|
||||
//
|
||||
// Fields:
|
||||
// - NumWorkers: Parallel workers for checkpoint/archive (0 = auto: min(NumCPU/2+1, 10))
|
||||
// - RetentionInMemory: Duration string (e.g., "48h") for in-memory data retention
|
||||
// - MemoryCap: Max bytes for buffer data (0 = unlimited); triggers forceFree when exceeded
|
||||
// - Checkpoints: Periodic persistence configuration
|
||||
// - Debug: Development/profiling options (nil = disabled)
|
||||
// - Archive: Long-term storage configuration (nil = disabled)
|
||||
// - Subscriptions: NATS topics for metric ingestion (nil = polling only)
|
||||
type MetricStoreConfig struct {
|
||||
// Number of concurrent workers for checkpoint and archive operations.
|
||||
// If not set or 0, defaults to min(runtime.NumCPU()/2+1, 10)
|
||||
NumWorkers int `json:"num-workers"`
|
||||
RetentionInMemory string `json:"retention-in-memory"`
|
||||
MemoryCap int `json:"memory-cap"`
|
||||
Checkpoints Checkpoints `json:"checkpoints"`
|
||||
Debug *Debug `json:"debug"`
|
||||
Cleanup *Cleanup `json:"cleanup"`
|
||||
Subscriptions *Subscriptions `json:"nats-subscriptions"`
|
||||
}
|
||||
|
||||
// Keys is the global metricstore configuration instance.
|
||||
//
|
||||
// Initialized with defaults, then overwritten by cc-backend's config.json.
|
||||
// Accessed by Init(), Checkpointing(), and other lifecycle functions.
|
||||
var Keys MetricStoreConfig = MetricStoreConfig{
|
||||
Checkpoints: Checkpoints{
|
||||
FileFormat: "avro",
|
||||
RootDir: "./var/checkpoints",
|
||||
},
|
||||
Cleanup: &Cleanup{
|
||||
Mode: "delete",
|
||||
},
|
||||
}
|
||||
|
||||
// AggregationStrategy defines how to combine metric values across hierarchy levels.
|
||||
//
|
||||
// Used when transforming data from finer-grained scopes (e.g., core) to coarser scopes
|
||||
// (e.g., socket). This is SPATIAL aggregation, not TEMPORAL (time-based) aggregation.
|
||||
//
|
||||
// Values:
|
||||
// - NoAggregation: Do not aggregate (incompatible scopes or non-aggregatable metrics)
|
||||
// - SumAggregation: Add values (e.g., power: sum core power → socket power)
|
||||
// - AvgAggregation: Average values (e.g., temperature: average core temps → socket temp)
|
||||
type AggregationStrategy int
|
||||
|
||||
const (
|
||||
NoAggregation AggregationStrategy = iota // Do not aggregate
|
||||
SumAggregation // Sum values (e.g., power, energy)
|
||||
AvgAggregation // Average values (e.g., temperature, utilization)
|
||||
)
|
||||
|
||||
// AssignAggregationStrategy parses a string into an AggregationStrategy value.
|
||||
//
|
||||
// Used when loading metric configurations from JSON/YAML files.
|
||||
//
|
||||
// Parameters:
|
||||
// - str: "sum", "avg", or "" (empty string for NoAggregation)
|
||||
//
|
||||
// Returns:
|
||||
// - AggregationStrategy: Parsed value
|
||||
// - error: Non-nil if str is unrecognized
|
||||
func AssignAggregationStrategy(str string) (AggregationStrategy, error) {
|
||||
switch str {
|
||||
case "":
|
||||
return NoAggregation, nil
|
||||
case "sum":
|
||||
return SumAggregation, nil
|
||||
case "avg":
|
||||
return AvgAggregation, nil
|
||||
default:
|
||||
return NoAggregation, fmt.Errorf("[METRICSTORE]> unknown aggregation strategy: %s", str)
|
||||
}
|
||||
}
|
||||
|
||||
// MetricConfig defines configuration for a single metric type.
|
||||
//
|
||||
// Stored in the global Metrics map, keyed by metric name (e.g., "cpu_load").
|
||||
//
|
||||
// Fields:
|
||||
// - Frequency: Measurement interval in seconds (e.g., 60 for 1-minute granularity)
|
||||
// - Aggregation: How to combine values across hierarchy levels (sum/avg/none)
|
||||
// - offset: Internal index into Level.metrics slice (assigned during Init)
|
||||
type MetricConfig struct {
|
||||
// Interval in seconds at which measurements are stored
|
||||
Frequency int64
|
||||
|
||||
// Can be 'sum', 'avg' or null. Describes how to aggregate metrics from the same timestep over the hierarchy.
|
||||
Aggregation AggregationStrategy
|
||||
|
||||
// Private, used internally...
|
||||
offset int
|
||||
}
|
||||
|
||||
// Metrics is the global map of metric configurations.
|
||||
//
|
||||
// Keyed by metric name (e.g., "cpu_load", "mem_used"). Populated during Init()
|
||||
// from cluster configuration and checkpoint restoration. Each MetricConfig.offset
|
||||
// corresponds to the buffer slice index in Level.metrics.
|
||||
var Metrics map[string]MetricConfig
|
||||
|
||||
// GetMetricFrequency retrieves the measurement interval for a metric.
|
||||
//
|
||||
// Parameters:
|
||||
// - metricName: Metric name (e.g., "cpu_load")
|
||||
//
|
||||
// Returns:
|
||||
// - int64: Frequency in seconds
|
||||
// - error: Non-nil if metric not found in Metrics map
|
||||
func GetMetricFrequency(metricName string) (int64, error) {
|
||||
if metric, ok := Metrics[metricName]; ok {
|
||||
return metric.Frequency, nil
|
||||
}
|
||||
return 0, fmt.Errorf("[METRICSTORE]> metric %s not found", metricName)
|
||||
}
|
||||
|
||||
// AddMetric registers a new metric or updates an existing one.
|
||||
//
|
||||
// If the metric already exists with a different frequency, uses the higher frequency
|
||||
// (finer granularity). This handles cases where different clusters report the same
|
||||
// metric at different intervals.
|
||||
//
|
||||
// Parameters:
|
||||
// - name: Metric name (e.g., "cpu_load")
|
||||
// - metric: Configuration (frequency, aggregation strategy)
|
||||
//
|
||||
// Returns:
|
||||
// - error: Always nil (signature for future error handling)
|
||||
func AddMetric(name string, metric MetricConfig) error {
|
||||
if Metrics == nil {
|
||||
Metrics = make(map[string]MetricConfig, 0)
|
||||
}
|
||||
|
||||
if existingMetric, ok := Metrics[name]; ok {
|
||||
if existingMetric.Frequency != metric.Frequency {
|
||||
if existingMetric.Frequency < metric.Frequency {
|
||||
existingMetric.Frequency = metric.Frequency
|
||||
Metrics[name] = existingMetric
|
||||
}
|
||||
}
|
||||
} else {
|
||||
Metrics[name] = metric
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
90
pkg/metricstore/configSchema.go
Normal file
90
pkg/metricstore/configSchema.go
Normal file
@@ -0,0 +1,90 @@
|
||||
// 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 metricstore
|
||||
|
||||
const configSchema = `{
|
||||
"type": "object",
|
||||
"description": "Configuration specific to built-in metric-store.",
|
||||
"properties": {
|
||||
"num-workers": {
|
||||
"description": "Number of concurrent workers for checkpoint and archive operations",
|
||||
"type": "integer"
|
||||
},
|
||||
"checkpoints": {
|
||||
"description": "Configuration for checkpointing the metrics buffers",
|
||||
"type": "object",
|
||||
"properties": {
|
||||
"file-format": {
|
||||
"description": "Specify the format for checkpoint files. There are 2 variants: 'avro' and 'json'. If nothing is specified, 'avro' is default.",
|
||||
"type": "string"
|
||||
},
|
||||
"interval": {
|
||||
"description": "Interval at which the metrics should be checkpointed.",
|
||||
"type": "string"
|
||||
},
|
||||
"directory": {
|
||||
"description": "Path in which the checkpointed files should be placed.",
|
||||
"type": "string"
|
||||
}
|
||||
},
|
||||
"required": ["interval"]
|
||||
},
|
||||
"cleanup": {
|
||||
"description": "Configuration for the cleanup process.",
|
||||
"type": "object",
|
||||
"properties": {
|
||||
"mode": {
|
||||
"description": "The operation mode (e.g., 'archive' or 'delete').",
|
||||
"type": "string",
|
||||
"enum": ["archive", "delete"]
|
||||
},
|
||||
"interval": {
|
||||
"description": "Interval at which the cleanup runs.",
|
||||
"type": "string"
|
||||
},
|
||||
"directory": {
|
||||
"description": "Target directory for operations.",
|
||||
"type": "string"
|
||||
}
|
||||
},
|
||||
"if": {
|
||||
"properties": {
|
||||
"mode": { "const": "archive" }
|
||||
}
|
||||
},
|
||||
"then": {
|
||||
"required": ["interval", "directory"]
|
||||
}
|
||||
},
|
||||
"retention-in-memory": {
|
||||
"description": "Keep the metrics within memory for given time interval. Retention for X hours, then the metrics would be freed.",
|
||||
"type": "string"
|
||||
},
|
||||
"memory-cap": {
|
||||
"description": "Upper memory capacity limit used by metricstore in GB",
|
||||
"type": "integer"
|
||||
},
|
||||
"nats-subscriptions": {
|
||||
"description": "Array of various subscriptions. Allows to subscribe to different subjects and publishers.",
|
||||
"type": "array",
|
||||
"items": {
|
||||
"type": "object",
|
||||
"properties": {
|
||||
"subscribe-to": {
|
||||
"description": "Subject name",
|
||||
"type": "string"
|
||||
},
|
||||
"cluster-tag": {
|
||||
"description": "Optional: Allow lines without a cluster tag, use this as default",
|
||||
"type": "string"
|
||||
}
|
||||
}
|
||||
"required": ["subscribe-to"]
|
||||
}
|
||||
}
|
||||
},
|
||||
"required": ["checkpoints", "retention-in-memory", "memory-cap"]
|
||||
}`
|
||||
112
pkg/metricstore/debug.go
Normal file
112
pkg/metricstore/debug.go
Normal file
@@ -0,0 +1,112 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"fmt"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
func (b *buffer) debugDump(buf []byte) []byte {
|
||||
if b.prev != nil {
|
||||
buf = b.prev.debugDump(buf)
|
||||
}
|
||||
|
||||
start, len, end := b.start, len(b.data), b.start+b.frequency*int64(len(b.data))
|
||||
buf = append(buf, `{"start":`...)
|
||||
buf = strconv.AppendInt(buf, start, 10)
|
||||
buf = append(buf, `,"len":`...)
|
||||
buf = strconv.AppendInt(buf, int64(len), 10)
|
||||
buf = append(buf, `,"end":`...)
|
||||
buf = strconv.AppendInt(buf, end, 10)
|
||||
if b.archived {
|
||||
buf = append(buf, `,"saved":true`...)
|
||||
}
|
||||
if b.next != nil {
|
||||
buf = append(buf, `},`...)
|
||||
} else {
|
||||
buf = append(buf, `}`...)
|
||||
}
|
||||
return buf
|
||||
}
|
||||
|
||||
func (l *Level) debugDump(m *MemoryStore, w *bufio.Writer, lvlname string, buf []byte, depth int) ([]byte, error) {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
for i := 0; i < depth; i++ {
|
||||
buf = append(buf, '\t')
|
||||
}
|
||||
buf = append(buf, '"')
|
||||
buf = append(buf, lvlname...)
|
||||
buf = append(buf, "\":{\n"...)
|
||||
depth += 1
|
||||
objitems := 0
|
||||
for name, mc := range m.Metrics {
|
||||
if b := l.metrics[mc.offset]; b != nil {
|
||||
for i := 0; i < depth; i++ {
|
||||
buf = append(buf, '\t')
|
||||
}
|
||||
|
||||
buf = append(buf, '"')
|
||||
buf = append(buf, name...)
|
||||
buf = append(buf, `":[`...)
|
||||
buf = b.debugDump(buf)
|
||||
buf = append(buf, "],\n"...)
|
||||
objitems++
|
||||
}
|
||||
}
|
||||
|
||||
for name, lvl := range l.children {
|
||||
_, err := w.Write(buf)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
buf = buf[0:0]
|
||||
buf, err = lvl.debugDump(m, w, name, buf, depth)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
buf = append(buf, ',', '\n')
|
||||
objitems++
|
||||
}
|
||||
|
||||
// remove final `,`:
|
||||
if objitems > 0 {
|
||||
buf = append(buf[0:len(buf)-1], '\n')
|
||||
}
|
||||
|
||||
depth -= 1
|
||||
for i := 0; i < depth; i++ {
|
||||
buf = append(buf, '\t')
|
||||
}
|
||||
buf = append(buf, '}')
|
||||
return buf, nil
|
||||
}
|
||||
|
||||
func (m *MemoryStore) DebugDump(w *bufio.Writer, selector []string) error {
|
||||
lvl := m.root.findLevel(selector)
|
||||
if lvl == nil {
|
||||
return fmt.Errorf("[METRICSTORE]> not found: %#v", selector)
|
||||
}
|
||||
|
||||
buf := make([]byte, 0, 2048)
|
||||
buf = append(buf, "{"...)
|
||||
|
||||
buf, err := lvl.debugDump(m, w, "data", buf, 0)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
buf = append(buf, "}\n"...)
|
||||
if _, err = w.Write(buf); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return w.Flush()
|
||||
}
|
||||
92
pkg/metricstore/healthcheck.go
Normal file
92
pkg/metricstore/healthcheck.go
Normal file
@@ -0,0 +1,92 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"fmt"
|
||||
"time"
|
||||
)
|
||||
|
||||
// MaxMissingDataPoints is a threshold that allows a node to be healthy with certain number of data points missing.
|
||||
// Suppose a node does not receive last 5 data points, then healthCheck endpoint will still say a
|
||||
// node is healthy. Anything more than 5 missing points in metrics of the node will deem the node unhealthy.
|
||||
const MaxMissingDataPoints int64 = 5
|
||||
|
||||
// MaxUnhealthyMetrics is a threshold which allows upto certain number of metrics in a node to be unhealthly.
|
||||
// Works with MaxMissingDataPoints. Say 5 metrics (including submetrics) do not receive the last
|
||||
// MaxMissingDataPoints data points, then the node will be deemed healthy. Any more metrics that does
|
||||
// not receive data for MaxMissingDataPoints data points will deem the node unhealthy.
|
||||
const MaxUnhealthyMetrics int64 = 5
|
||||
|
||||
func (b *buffer) healthCheck() int64 {
|
||||
// Check if the buffer is empty
|
||||
if b.data == nil {
|
||||
return 1
|
||||
}
|
||||
|
||||
bufferEnd := b.start + b.frequency*int64(len(b.data))
|
||||
t := time.Now().Unix()
|
||||
|
||||
// Check if the buffer is too old
|
||||
if t-bufferEnd > MaxMissingDataPoints*b.frequency {
|
||||
return 1
|
||||
}
|
||||
|
||||
return 0
|
||||
}
|
||||
|
||||
func (l *Level) healthCheck(m *MemoryStore, count int64) (int64, error) {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
for _, mc := range m.Metrics {
|
||||
if b := l.metrics[mc.offset]; b != nil {
|
||||
count += b.healthCheck()
|
||||
}
|
||||
}
|
||||
|
||||
for _, lvl := range l.children {
|
||||
c, err := lvl.healthCheck(m, 0)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
count += c
|
||||
}
|
||||
|
||||
return count, nil
|
||||
}
|
||||
|
||||
func (m *MemoryStore) HealthCheck(w *bufio.Writer, selector []string) error {
|
||||
lvl := m.root.findLevel(selector)
|
||||
if lvl == nil {
|
||||
return fmt.Errorf("[METRICSTORE]> not found: %#v", selector)
|
||||
}
|
||||
|
||||
buf := make([]byte, 0, 25)
|
||||
// buf = append(buf, "{"...)
|
||||
|
||||
var count int64 = 0
|
||||
|
||||
unhealthyMetricsCount, err := lvl.healthCheck(m, count)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if unhealthyMetricsCount < MaxUnhealthyMetrics {
|
||||
buf = append(buf, "Healthy"...)
|
||||
} else {
|
||||
buf = append(buf, "Unhealthy"...)
|
||||
}
|
||||
|
||||
// buf = append(buf, "}\n"...)
|
||||
|
||||
if _, err = w.Write(buf); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return w.Flush()
|
||||
}
|
||||
388
pkg/metricstore/level.go
Normal file
388
pkg/metricstore/level.go
Normal file
@@ -0,0 +1,388 @@
|
||||
// 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 metricstore provides level.go: Hierarchical tree structure for metric storage.
|
||||
//
|
||||
// # Level Architecture
|
||||
//
|
||||
// The Level type forms a tree structure where each node represents a level in the
|
||||
// ClusterCockpit hierarchy: cluster → host → socket → core → hwthread, with special
|
||||
// nodes for memory domains and accelerators.
|
||||
//
|
||||
// Structure:
|
||||
//
|
||||
// Root Level (cluster="emmy")
|
||||
// ├─ Level (host="node001")
|
||||
// │ ├─ Level (socket="0")
|
||||
// │ │ ├─ Level (core="0") [stores cpu0 metrics]
|
||||
// │ │ └─ Level (core="1") [stores cpu1 metrics]
|
||||
// │ └─ Level (socket="1")
|
||||
// │ └─ ...
|
||||
// └─ Level (host="node002")
|
||||
// └─ ...
|
||||
//
|
||||
// Each Level can:
|
||||
// - Hold data (metrics slice of buffer pointers)
|
||||
// - Have child nodes (children map[string]*Level)
|
||||
// - Both simultaneously (inner nodes can store aggregated metrics)
|
||||
//
|
||||
// # Selector Paths
|
||||
//
|
||||
// Selectors are hierarchical paths: []string{"cluster", "host", "component"}.
|
||||
// Example: []string{"emmy", "node001", "cpu0"} navigates to the cpu0 core level.
|
||||
//
|
||||
// # Concurrency
|
||||
//
|
||||
// RWMutex protects children map and metrics slice. Read-heavy workload (metric reads)
|
||||
// uses RLock. Writes (new levels, buffer updates) use Lock. Double-checked locking
|
||||
// prevents races during level creation.
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/util"
|
||||
)
|
||||
|
||||
// Level represents a node in the hierarchical metric storage tree.
|
||||
//
|
||||
// Can be both a leaf or inner node. Inner nodes hold data in 'metrics' for aggregated
|
||||
// values (e.g., socket-level metrics derived from core-level data). Named "Level"
|
||||
// instead of "node" to avoid confusion with cluster nodes (hosts).
|
||||
//
|
||||
// Fields:
|
||||
// - children: Map of child level names to Level pointers (e.g., "cpu0" → Level)
|
||||
// - metrics: Slice of buffer pointers (one per metric, indexed by MetricConfig.offset)
|
||||
// - lock: RWMutex for concurrent access (read-heavy, write-rare)
|
||||
type Level struct {
|
||||
children map[string]*Level
|
||||
metrics []*buffer
|
||||
lock sync.RWMutex
|
||||
}
|
||||
|
||||
// findLevelOrCreate navigates to or creates the level specified by selector.
|
||||
//
|
||||
// Recursively descends the tree, creating missing levels as needed. Uses double-checked
|
||||
// locking: RLock first (fast path), then Lock if creation needed (slow path), then
|
||||
// re-check after acquiring Lock to handle races.
|
||||
//
|
||||
// Example selector: []string{"emmy", "node001", "cpu0"}
|
||||
// Navigates: root → emmy → node001 → cpu0, creating levels as needed.
|
||||
//
|
||||
// Parameters:
|
||||
// - selector: Hierarchical path (consumed recursively, decreasing depth)
|
||||
// - nMetrics: Number of metric slots to allocate in new levels
|
||||
//
|
||||
// Returns:
|
||||
// - *Level: The target level (existing or newly created)
|
||||
//
|
||||
// Note: sync.Map may improve performance for high-concurrency writes, but current
|
||||
// approach suffices for read-heavy workload.
|
||||
func (l *Level) findLevelOrCreate(selector []string, nMetrics int) *Level {
|
||||
if len(selector) == 0 {
|
||||
return l
|
||||
}
|
||||
|
||||
// Allow concurrent reads:
|
||||
l.lock.RLock()
|
||||
var child *Level
|
||||
var ok bool
|
||||
if l.children == nil {
|
||||
// Children map needs to be created...
|
||||
l.lock.RUnlock()
|
||||
} else {
|
||||
child, ok = l.children[selector[0]]
|
||||
l.lock.RUnlock()
|
||||
if ok {
|
||||
return child.findLevelOrCreate(selector[1:], nMetrics)
|
||||
}
|
||||
}
|
||||
|
||||
// The level does not exist, take write lock for unique access:
|
||||
l.lock.Lock()
|
||||
// While this thread waited for the write lock, another thread
|
||||
// could have created the child node.
|
||||
if l.children != nil {
|
||||
child, ok = l.children[selector[0]]
|
||||
if ok {
|
||||
l.lock.Unlock()
|
||||
return child.findLevelOrCreate(selector[1:], nMetrics)
|
||||
}
|
||||
}
|
||||
|
||||
child = &Level{
|
||||
metrics: make([]*buffer, nMetrics),
|
||||
children: nil,
|
||||
}
|
||||
|
||||
if l.children != nil {
|
||||
l.children[selector[0]] = child
|
||||
} else {
|
||||
l.children = map[string]*Level{selector[0]: child}
|
||||
}
|
||||
l.lock.Unlock()
|
||||
return child.findLevelOrCreate(selector[1:], nMetrics)
|
||||
}
|
||||
|
||||
// collectPaths gathers all selector paths at the specified depth in the tree.
|
||||
//
|
||||
// Recursively traverses children, collecting paths when currentDepth+1 == targetDepth.
|
||||
// Each path is a selector that can be used with findLevel() or findBuffers().
|
||||
//
|
||||
// Explicitly copies slices to avoid shared underlying arrays between siblings, preventing
|
||||
// unintended mutations.
|
||||
//
|
||||
// Parameters:
|
||||
// - currentDepth: Depth of current level (0 = root)
|
||||
// - targetDepth: Depth to collect paths from
|
||||
// - currentPath: Path accumulated so far
|
||||
// - results: Output slice (appended to)
|
||||
//
|
||||
// Example: collectPaths(0, 2, []string{}, &results) collects all 2-level paths
|
||||
// like []string{"emmy", "node001"}, []string{"emmy", "node002"}, etc.
|
||||
func (l *Level) collectPaths(currentDepth, targetDepth int, currentPath []string, results *[][]string) {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
for key, child := range l.children {
|
||||
if child == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// We explicitly make a new slice and copy data to avoid sharing underlying arrays between siblings
|
||||
newPath := make([]string, len(currentPath))
|
||||
copy(newPath, currentPath)
|
||||
newPath = append(newPath, key)
|
||||
|
||||
// Check depth, and just return if depth reached
|
||||
if currentDepth+1 == targetDepth {
|
||||
*results = append(*results, newPath)
|
||||
} else {
|
||||
child.collectPaths(currentDepth+1, targetDepth, newPath, results)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// free removes buffers older than the retention threshold from the entire subtree.
|
||||
//
|
||||
// Recursively frees buffers in this level's metrics and all child levels. Buffers
|
||||
// with standard capacity (BufferCap) are returned to the pool. Called by the
|
||||
// retention worker to enforce retention policies.
|
||||
//
|
||||
// Parameters:
|
||||
// - t: Retention threshold timestamp (Unix seconds)
|
||||
//
|
||||
// Returns:
|
||||
// - int: Total number of buffers freed in this subtree
|
||||
// - error: Non-nil on failure (propagated from children)
|
||||
func (l *Level) free(t int64) (int, error) {
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
n := 0
|
||||
for i, b := range l.metrics {
|
||||
if b != nil {
|
||||
delme, m := b.free(t)
|
||||
n += m
|
||||
if delme {
|
||||
if cap(b.data) == BufferCap {
|
||||
bufferPool.Put(b)
|
||||
}
|
||||
l.metrics[i] = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for _, l := range l.children {
|
||||
m, err := l.free(t)
|
||||
n += m
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
}
|
||||
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// forceFree removes the oldest buffer from each metric chain in the subtree.
|
||||
//
|
||||
// Unlike free(), which removes based on time threshold, this unconditionally removes
|
||||
// the oldest buffer in each chain. Used by MemoryUsageTracker when memory cap is
|
||||
// exceeded and time-based retention is insufficient.
|
||||
//
|
||||
// Recursively processes current level's metrics and all child levels.
|
||||
//
|
||||
// Returns:
|
||||
// - int: Total number of buffers freed in this subtree
|
||||
// - error: Non-nil on failure (propagated from children)
|
||||
func (l *Level) forceFree() (int, error) {
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
n := 0
|
||||
|
||||
// Iterate over metrics in the current level
|
||||
for i, b := range l.metrics {
|
||||
if b != nil {
|
||||
// Attempt to free the oldest buffer in this chain
|
||||
delme, freedCount := b.forceFreeOldest()
|
||||
n += freedCount
|
||||
|
||||
// If delme is true, it means 'b' itself (the head) was the oldest
|
||||
// and needs to be removed from the slice.
|
||||
if delme {
|
||||
// Nil out fields to ensure no hanging references
|
||||
|
||||
b.next = nil
|
||||
b.prev = nil
|
||||
b.data = nil
|
||||
|
||||
l.metrics[i] = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Recursively traverse children
|
||||
for _, child := range l.children {
|
||||
m, err := child.forceFree()
|
||||
n += m
|
||||
if err != nil {
|
||||
return n, err
|
||||
}
|
||||
}
|
||||
|
||||
return n, nil
|
||||
}
|
||||
|
||||
// sizeInBytes calculates the total memory usage of all buffers in the subtree.
|
||||
//
|
||||
// Recursively sums buffer data sizes (count of Float values × sizeof(Float)) across
|
||||
// this level's metrics and all child levels. Used by MemoryUsageTracker to enforce
|
||||
// memory cap limits.
|
||||
//
|
||||
// Returns:
|
||||
// - int64: Total bytes used by buffer data in this subtree
|
||||
func (l *Level) sizeInBytes() int64 {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
size := int64(0)
|
||||
|
||||
for _, b := range l.metrics {
|
||||
if b != nil {
|
||||
size += b.count() * int64(unsafe.Sizeof(util.Float(0)))
|
||||
}
|
||||
}
|
||||
|
||||
for _, child := range l.children {
|
||||
size += child.sizeInBytes()
|
||||
}
|
||||
|
||||
return size
|
||||
}
|
||||
|
||||
// findLevel navigates to the level specified by selector, returning nil if not found.
|
||||
//
|
||||
// Read-only variant of findLevelOrCreate. Does not create missing levels.
|
||||
// Recursively descends the tree following the selector path.
|
||||
//
|
||||
// Parameters:
|
||||
// - selector: Hierarchical path (e.g., []string{"emmy", "node001", "cpu0"})
|
||||
//
|
||||
// Returns:
|
||||
// - *Level: The target level, or nil if any component in the path does not exist
|
||||
func (l *Level) findLevel(selector []string) *Level {
|
||||
if len(selector) == 0 {
|
||||
return l
|
||||
}
|
||||
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
lvl := l.children[selector[0]]
|
||||
if lvl == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return lvl.findLevel(selector[1:])
|
||||
}
|
||||
|
||||
// findBuffers invokes callback on all buffers matching the selector pattern.
|
||||
//
|
||||
// Supports flexible selector patterns (from cc-lib/util.Selector):
|
||||
// - Exact match: Selector element with String set (e.g., "node001")
|
||||
// - Group match: Selector element with Group set (e.g., ["cpu0", "cpu2", "cpu4"])
|
||||
// - Wildcard: Selector element with Any=true (matches all children)
|
||||
//
|
||||
// Empty selector (len==0) matches current level's buffer at 'offset' and recursively
|
||||
// all descendant buffers at the same offset (used for aggregation queries).
|
||||
//
|
||||
// Parameters:
|
||||
// - selector: Pattern to match (consumed recursively)
|
||||
// - offset: Metric index in metrics slice (from MetricConfig.offset)
|
||||
// - f: Callback invoked on each matching buffer
|
||||
//
|
||||
// Returns:
|
||||
// - error: First error returned by callback, or nil if all succeeded
|
||||
//
|
||||
// Example:
|
||||
//
|
||||
// // Find all cpu0 buffers across all hosts:
|
||||
// findBuffers([]Selector{{Any: true}, {String: "cpu0"}}, metricOffset, callback)
|
||||
func (l *Level) findBuffers(selector util.Selector, offset int, f func(b *buffer) error) error {
|
||||
l.lock.RLock()
|
||||
defer l.lock.RUnlock()
|
||||
|
||||
if len(selector) == 0 {
|
||||
b := l.metrics[offset]
|
||||
if b != nil {
|
||||
return f(b)
|
||||
}
|
||||
|
||||
for _, lvl := range l.children {
|
||||
err := lvl.findBuffers(nil, offset, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
sel := selector[0]
|
||||
if len(sel.String) != 0 && l.children != nil {
|
||||
lvl, ok := l.children[sel.String]
|
||||
if ok {
|
||||
err := lvl.findBuffers(selector[1:], offset, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
if sel.Group != nil && l.children != nil {
|
||||
for _, key := range sel.Group {
|
||||
lvl, ok := l.children[key]
|
||||
if ok {
|
||||
err := lvl.findBuffers(selector[1:], offset, f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
if sel.Any && l.children != nil {
|
||||
for _, lvl := range l.children {
|
||||
if err := lvl.findBuffers(selector[1:], offset, f); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
263
pkg/metricstore/lineprotocol.go
Normal file
263
pkg/metricstore/lineprotocol.go
Normal file
@@ -0,0 +1,263 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ClusterCockpit/cc-backend/pkg/nats"
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
"github.com/influxdata/line-protocol/v2/lineprotocol"
|
||||
)
|
||||
|
||||
func ReceiveNats(ms *MemoryStore,
|
||||
workers int,
|
||||
ctx context.Context,
|
||||
) error {
|
||||
nc := nats.GetClient()
|
||||
|
||||
if nc == nil {
|
||||
cclog.Warn("NATS client not initialized")
|
||||
return nil
|
||||
}
|
||||
|
||||
var wg sync.WaitGroup
|
||||
msgs := make(chan []byte, workers*2)
|
||||
|
||||
for _, sc := range *Keys.Subscriptions {
|
||||
clusterTag := sc.ClusterTag
|
||||
if workers > 1 {
|
||||
wg.Add(workers)
|
||||
|
||||
for range workers {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
for m := range msgs {
|
||||
dec := lineprotocol.NewDecoderWithBytes(m)
|
||||
if err := DecodeLine(dec, ms, clusterTag); err != nil {
|
||||
cclog.Errorf("error: %s", err.Error())
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
nc.Subscribe(sc.SubscribeTo, func(subject string, data []byte) {
|
||||
select {
|
||||
case msgs <- data:
|
||||
case <-ctx.Done():
|
||||
}
|
||||
})
|
||||
} else {
|
||||
nc.Subscribe(sc.SubscribeTo, func(subject string, data []byte) {
|
||||
dec := lineprotocol.NewDecoderWithBytes(data)
|
||||
if err := DecodeLine(dec, ms, clusterTag); err != nil {
|
||||
cclog.Errorf("error: %s", err.Error())
|
||||
}
|
||||
})
|
||||
}
|
||||
cclog.Infof("NATS subscription to '%s' established", sc.SubscribeTo)
|
||||
}
|
||||
|
||||
go func() {
|
||||
<-ctx.Done()
|
||||
close(msgs)
|
||||
}()
|
||||
|
||||
wg.Wait()
|
||||
|
||||
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 := range n {
|
||||
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:
|
||||
}
|
||||
}
|
||||
|
||||
// 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 Keys.Checkpoints.FileFormat != "json" {
|
||||
LineProtocolMessages <- &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
|
||||
}
|
||||
790
pkg/metricstore/metricstore.go
Normal file
790
pkg/metricstore/metricstore.go
Normal file
@@ -0,0 +1,790 @@
|
||||
// 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 metricstore provides an efficient in-memory time-series metric storage system
|
||||
// with support for hierarchical data organization, checkpointing, and archiving.
|
||||
//
|
||||
// The package organizes metrics in a tree structure (cluster → host → component) and
|
||||
// provides concurrent read/write access to metric data with configurable aggregation strategies.
|
||||
// Background goroutines handle periodic checkpointing (JSON or Avro format), archiving old data,
|
||||
// and enforcing retention policies.
|
||||
//
|
||||
// Key features:
|
||||
// - In-memory metric storage with configurable retention
|
||||
// - Hierarchical data organization (selectors)
|
||||
// - Concurrent checkpoint/archive workers
|
||||
// - Support for sum and average aggregation
|
||||
// - NATS integration for metric ingestion
|
||||
package metricstore
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"runtime"
|
||||
"slices"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/ClusterCockpit/cc-backend/internal/config"
|
||||
"github.com/ClusterCockpit/cc-backend/pkg/archive"
|
||||
cclog "github.com/ClusterCockpit/cc-lib/v2/ccLogger"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/resampler"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
"github.com/ClusterCockpit/cc-lib/v2/util"
|
||||
)
|
||||
|
||||
// Define a struct to hold your globals and the mutex
|
||||
type GlobalState struct {
|
||||
mu sync.RWMutex
|
||||
lastRetentionTime int64
|
||||
selectorsExcluded bool
|
||||
}
|
||||
|
||||
var (
|
||||
singleton sync.Once
|
||||
msInstance *MemoryStore
|
||||
// shutdownFunc stores the context cancellation function created in Init
|
||||
// and is called during Shutdown to cancel all background goroutines
|
||||
shutdownFunc context.CancelFunc
|
||||
shutdownFuncMu sync.Mutex // Protects shutdownFunc from concurrent access
|
||||
// Create a global instance
|
||||
state = &GlobalState{}
|
||||
)
|
||||
|
||||
// NodeProvider provides information about nodes currently in use by running jobs.
|
||||
//
|
||||
// This interface allows metricstore to query job information without directly
|
||||
// depending on the repository package, breaking the import cycle.
|
||||
//
|
||||
// Implementations should return nodes that are actively processing jobs started
|
||||
// before the given timestamp. These nodes will be excluded from retention-based
|
||||
// garbage collection to prevent data loss for jobs that are still running or
|
||||
// recently completed.
|
||||
type NodeProvider interface {
|
||||
// GetUsedNodes returns a map of cluster names to sorted lists of unique hostnames
|
||||
// that are currently in use by jobs that started before the given timestamp.
|
||||
//
|
||||
// Parameters:
|
||||
// - ts: Unix timestamp threshold - returns nodes with jobs started before this time
|
||||
//
|
||||
// Returns:
|
||||
// - Map of cluster names to lists of node hostnames that should be excluded from garbage collection
|
||||
// - Error if the query fails
|
||||
GetUsedNodes(ts int64) (map[string][]string, error)
|
||||
}
|
||||
|
||||
// Metric represents a single metric data point to be written to the store.
|
||||
type Metric struct {
|
||||
Name string
|
||||
Value schema.Float
|
||||
// MetricConfig contains frequency and aggregation settings for this metric.
|
||||
// If Frequency is 0, configuration will be looked up from MemoryStore.Metrics during Write().
|
||||
MetricConfig MetricConfig
|
||||
}
|
||||
|
||||
// MemoryStore is the main in-memory time-series metric storage implementation.
|
||||
//
|
||||
// It organizes metrics in a hierarchical tree structure where each level represents
|
||||
// a component of the system hierarchy (e.g., cluster → host → CPU). Each level can
|
||||
// store multiple metrics as time-series buffers.
|
||||
//
|
||||
// The store is initialized as a singleton via InitMetrics() and accessed via GetMemoryStore().
|
||||
// All public methods are safe for concurrent use.
|
||||
type MemoryStore struct {
|
||||
Metrics map[string]MetricConfig
|
||||
root Level
|
||||
nodeProvider NodeProvider
|
||||
}
|
||||
|
||||
// Init initializes the metric store from configuration and starts background workers.
|
||||
//
|
||||
// This function must be called exactly once before any other metricstore operations.
|
||||
// It performs the following initialization steps:
|
||||
// 1. Validates and decodes the metric store configuration
|
||||
// 2. Configures worker pool size (defaults to NumCPU/2+1, max 10)
|
||||
// 3. Loads metric configurations from all registered clusters
|
||||
// 4. Restores checkpoints within the retention window
|
||||
// 5. Starts background workers for retention, checkpointing, archiving, and monitoring
|
||||
// 6. Optionally subscribes to NATS for real-time metric ingestion
|
||||
//
|
||||
// Parameters:
|
||||
// - rawConfig: JSON configuration for the metric store (see MetricStoreConfig)
|
||||
// - wg: WaitGroup that will be incremented for each background goroutine started
|
||||
//
|
||||
// The function will call cclog.Fatal on critical errors during initialization.
|
||||
// Use Shutdown() to cleanly stop all background workers started by Init().
|
||||
//
|
||||
// Note: Signal handling must be implemented by the caller. Call Shutdown() when
|
||||
// receiving termination signals to ensure checkpoint data is persisted.
|
||||
func Init(rawConfig json.RawMessage, wg *sync.WaitGroup) {
|
||||
startupTime := time.Now()
|
||||
|
||||
if rawConfig != nil {
|
||||
config.Validate(configSchema, rawConfig)
|
||||
dec := json.NewDecoder(bytes.NewReader(rawConfig))
|
||||
dec.DisallowUnknownFields()
|
||||
if err := dec.Decode(&Keys); err != nil {
|
||||
cclog.Abortf("[METRICSTORE]> Metric Store Config Init: Could not decode config file '%s'.\nError: %s\n", rawConfig, err.Error())
|
||||
}
|
||||
}
|
||||
|
||||
// Set NumWorkers from config or use default
|
||||
if Keys.NumWorkers <= 0 {
|
||||
Keys.NumWorkers = min(runtime.NumCPU()/2+1, DefaultMaxWorkers)
|
||||
}
|
||||
cclog.Debugf("[METRICSTORE]> Using %d workers for checkpoint/archive operations\n", Keys.NumWorkers)
|
||||
|
||||
// Helper function to add metric configuration
|
||||
addMetricConfig := func(mc *schema.MetricConfig) {
|
||||
agg, err := AssignAggregationStrategy(mc.Aggregation)
|
||||
if err != nil {
|
||||
cclog.Warnf("Could not find aggregation strategy for metric config '%s': %s", mc.Name, err.Error())
|
||||
}
|
||||
|
||||
AddMetric(mc.Name, MetricConfig{
|
||||
Frequency: int64(mc.Timestep),
|
||||
Aggregation: agg,
|
||||
})
|
||||
}
|
||||
|
||||
for _, c := range archive.Clusters {
|
||||
for _, mc := range c.MetricConfig {
|
||||
addMetricConfig(mc)
|
||||
}
|
||||
|
||||
for _, sc := range c.SubClusters {
|
||||
for _, mc := range sc.MetricConfig {
|
||||
addMetricConfig(mc)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Pass the config.MetricStoreKeys
|
||||
InitMetrics(Metrics)
|
||||
|
||||
ms := GetMemoryStore()
|
||||
|
||||
d, err := time.ParseDuration(Keys.RetentionInMemory)
|
||||
if err != nil {
|
||||
cclog.Fatal(err)
|
||||
}
|
||||
|
||||
restoreFrom := startupTime.Add(-d)
|
||||
cclog.Infof("[METRICSTORE]> Loading checkpoints newer than %s\n", restoreFrom.Format(time.RFC3339))
|
||||
files, err := ms.FromCheckpointFiles(Keys.Checkpoints.RootDir, restoreFrom.Unix())
|
||||
loadedData := ms.SizeInBytes() / 1024 / 1024 // In MB
|
||||
if err != nil {
|
||||
cclog.Fatalf("[METRICSTORE]> Loading checkpoints failed: %s\n", err.Error())
|
||||
} else {
|
||||
cclog.Infof("[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
|
||||
// lowering the target percentage. The default of 100 means
|
||||
// that only once the ratio of new allocations execeds the
|
||||
// previously active heap, a GC is triggered.
|
||||
// Forcing a GC here will set the "previously active heap"
|
||||
// to a minumum.
|
||||
runtime.GC()
|
||||
|
||||
ctx, shutdown := context.WithCancel(context.Background())
|
||||
|
||||
retentionGoroutines := 1
|
||||
checkpointingGoroutines := 1
|
||||
dataStagingGoroutines := 1
|
||||
archivingGoroutines := 1
|
||||
memoryUsageTracker := 1
|
||||
|
||||
totalGoroutines := retentionGoroutines +
|
||||
checkpointingGoroutines +
|
||||
dataStagingGoroutines +
|
||||
archivingGoroutines +
|
||||
memoryUsageTracker
|
||||
|
||||
wg.Add(totalGoroutines)
|
||||
|
||||
Retention(wg, ctx)
|
||||
Checkpointing(wg, ctx)
|
||||
CleanUp(wg, ctx)
|
||||
DataStaging(wg, ctx)
|
||||
MemoryUsageTracker(wg, ctx)
|
||||
|
||||
// Note: Signal handling has been removed from this function.
|
||||
// The caller is responsible for handling shutdown signals and calling
|
||||
// the shutdown() function when appropriate.
|
||||
// Store the shutdown function for later use by Shutdown()
|
||||
shutdownFuncMu.Lock()
|
||||
shutdownFunc = shutdown
|
||||
shutdownFuncMu.Unlock()
|
||||
|
||||
if Keys.Subscriptions != nil {
|
||||
err = ReceiveNats(ms, 1, ctx)
|
||||
if err != nil {
|
||||
cclog.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// InitMetrics initializes the singleton MemoryStore instance with the given metric configurations.
|
||||
//
|
||||
// This function must be called before GetMemoryStore() and can only be called once due to
|
||||
// the singleton pattern. It assigns each metric an internal offset for efficient buffer indexing.
|
||||
//
|
||||
// Parameters:
|
||||
// - metrics: Map of metric names to their configurations (frequency and aggregation strategy)
|
||||
//
|
||||
// Panics if any metric has Frequency == 0, which indicates an invalid configuration.
|
||||
//
|
||||
// After this call, the global msInstance is ready for use via GetMemoryStore().
|
||||
func InitMetrics(metrics map[string]MetricConfig) {
|
||||
singleton.Do(func() {
|
||||
offset := 0
|
||||
for key, cfg := range metrics {
|
||||
if cfg.Frequency == 0 {
|
||||
panic("[METRICSTORE]> invalid frequency")
|
||||
}
|
||||
|
||||
metrics[key] = MetricConfig{
|
||||
Frequency: cfg.Frequency,
|
||||
Aggregation: cfg.Aggregation,
|
||||
offset: offset,
|
||||
}
|
||||
offset += 1
|
||||
}
|
||||
|
||||
msInstance = &MemoryStore{
|
||||
root: Level{
|
||||
metrics: make([]*buffer, len(metrics)),
|
||||
children: make(map[string]*Level),
|
||||
},
|
||||
Metrics: metrics,
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// GetMemoryStore returns the singleton MemoryStore instance.
|
||||
//
|
||||
// Returns the initialized MemoryStore singleton. Calls cclog.Fatal if InitMetrics() was not called first.
|
||||
//
|
||||
// This function is safe for concurrent use after initialization.
|
||||
func GetMemoryStore() *MemoryStore {
|
||||
if msInstance == nil {
|
||||
cclog.Fatalf("[METRICSTORE]> MemoryStore not initialized!")
|
||||
}
|
||||
|
||||
return msInstance
|
||||
}
|
||||
|
||||
// SetNodeProvider sets the NodeProvider implementation for the MemoryStore.
|
||||
// This must be called during initialization to provide job state information
|
||||
// for selective buffer retention during Free operations.
|
||||
// If not set, the Free function will fall back to freeing all buffers.
|
||||
func (ms *MemoryStore) SetNodeProvider(provider NodeProvider) {
|
||||
ms.nodeProvider = provider
|
||||
}
|
||||
|
||||
// Shutdown performs a graceful shutdown of the metric store.
|
||||
//
|
||||
// This function cancels all background goroutines started by Init() and writes
|
||||
// a final checkpoint to disk before returning. It should be called when the
|
||||
// application receives a termination signal.
|
||||
//
|
||||
// The function will:
|
||||
// 1. Cancel the context to stop all background workers
|
||||
// 2. Close NATS message channels if using Avro format
|
||||
// 3. Write a final checkpoint to preserve in-memory data
|
||||
// 4. Log any errors encountered during shutdown
|
||||
//
|
||||
// Note: This function blocks until the final checkpoint is written.
|
||||
func Shutdown() {
|
||||
shutdownFuncMu.Lock()
|
||||
defer shutdownFuncMu.Unlock()
|
||||
if shutdownFunc != nil {
|
||||
shutdownFunc()
|
||||
}
|
||||
|
||||
if Keys.Checkpoints.FileFormat != "json" {
|
||||
close(LineProtocolMessages)
|
||||
}
|
||||
|
||||
cclog.Infof("[METRICSTORE]> Writing to '%s'...\n", Keys.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())
|
||||
} else {
|
||||
files, err = GetAvroStore().ToCheckpoint(Keys.Checkpoints.RootDir, true)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> Writing checkpoint failed: %s\n", err.Error())
|
||||
}
|
||||
cclog.Infof("[METRICSTORE]> Done! (%d files written)\n", files)
|
||||
}
|
||||
|
||||
// Retention starts a background goroutine that periodically frees old metric data.
|
||||
//
|
||||
// This worker runs at half the retention interval and calls Free() to remove buffers
|
||||
// older than the configured retention time. It respects the NodeProvider to preserve
|
||||
// data for nodes with active jobs.
|
||||
//
|
||||
// Parameters:
|
||||
// - wg: WaitGroup to signal completion when context is cancelled
|
||||
// - ctx: Context for cancellation signal
|
||||
//
|
||||
// The goroutine exits when ctx is cancelled.
|
||||
func Retention(wg *sync.WaitGroup, ctx context.Context) {
|
||||
ms := GetMemoryStore()
|
||||
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
d, err := time.ParseDuration(Keys.RetentionInMemory)
|
||||
if err != nil {
|
||||
cclog.Fatal(err)
|
||||
}
|
||||
if d <= 0 {
|
||||
return
|
||||
}
|
||||
|
||||
tickInterval := d / 2
|
||||
if tickInterval <= 0 {
|
||||
return
|
||||
}
|
||||
ticker := time.NewTicker(tickInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
state.mu.Lock()
|
||||
|
||||
t := time.Now().Add(-d)
|
||||
|
||||
state.lastRetentionTime = t.Unix()
|
||||
|
||||
cclog.Infof("[METRICSTORE]> start freeing buffers (older than %s)...\n", t.Format(time.RFC3339))
|
||||
|
||||
freed, err := Free(ms, t)
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> freeing up buffers failed: %s\n", err.Error())
|
||||
} else {
|
||||
cclog.Infof("[METRICSTORE]> done: %d buffers freed\n", freed)
|
||||
}
|
||||
|
||||
state.mu.Unlock()
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// MemoryUsageTracker starts a background goroutine that monitors memory usage.
|
||||
//
|
||||
// This worker checks memory usage every minute and force-frees buffers if memory
|
||||
// exceeds the configured cap. It protects against infinite loops by limiting
|
||||
// iterations and forcing garbage collection between attempts.
|
||||
//
|
||||
// Parameters:
|
||||
// - wg: WaitGroup to signal completion when context is cancelled
|
||||
// - ctx: Context for cancellation signal
|
||||
//
|
||||
// The goroutine exits when ctx is cancelled.
|
||||
func MemoryUsageTracker(wg *sync.WaitGroup, ctx context.Context) {
|
||||
ms := GetMemoryStore()
|
||||
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
d := DefaultMemoryUsageTrackerInterval
|
||||
|
||||
if d <= 0 {
|
||||
return
|
||||
}
|
||||
|
||||
ticker := time.NewTicker(d)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
state.mu.RLock()
|
||||
|
||||
memoryUsageGB := ms.SizeInGB()
|
||||
cclog.Infof("[METRICSTORE]> current memory usage: %.2f GB\n", memoryUsageGB)
|
||||
|
||||
freedTotal := 0
|
||||
var err error
|
||||
|
||||
// First force-free all the checkpoints that were
|
||||
if state.lastRetentionTime != 0 && state.selectorsExcluded {
|
||||
freedTotal, err = ms.Free(nil, state.lastRetentionTime)
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> error while force-freeing the excluded buffers: %s", err)
|
||||
}
|
||||
|
||||
// Calling runtime.GC() twice in succession tp completely empty a bufferPool (sync.Pool)
|
||||
runtime.GC()
|
||||
runtime.GC()
|
||||
|
||||
cclog.Infof("[METRICSTORE]> done: %d excluded buffers force-freed\n", freedTotal)
|
||||
}
|
||||
|
||||
state.mu.RUnlock()
|
||||
|
||||
memoryUsageGB = ms.SizeInGB()
|
||||
|
||||
if memoryUsageGB > float64(Keys.MemoryCap) {
|
||||
cclog.Warnf("[METRICSTORE]> memory usage is still greater than the Memory Cap: %d GB\n", Keys.MemoryCap)
|
||||
cclog.Warnf("[METRICSTORE]> starting to force-free the buffers from the Metric Store\n")
|
||||
|
||||
const maxIterations = 100
|
||||
|
||||
for range maxIterations {
|
||||
memoryUsageGB = ms.SizeInGB()
|
||||
if memoryUsageGB < float64(Keys.MemoryCap) {
|
||||
break
|
||||
}
|
||||
|
||||
freed, err := ms.ForceFree()
|
||||
if err != nil {
|
||||
cclog.Errorf("[METRICSTORE]> error while force-freeing the buffers: %s", err)
|
||||
}
|
||||
if freed == 0 {
|
||||
cclog.Errorf("[METRICSTORE]> 0 buffers force-freed in last try, %d total buffers force-freed, memory usage of %.2f GB remains higher than the memory cap of %d GB and there are no buffers left to force-free\n", freedTotal, memoryUsageGB, Keys.MemoryCap)
|
||||
break
|
||||
}
|
||||
freedTotal += freed
|
||||
|
||||
runtime.GC()
|
||||
}
|
||||
|
||||
if memoryUsageGB >= float64(Keys.MemoryCap) {
|
||||
cclog.Errorf("[METRICSTORE]> reached maximum iterations (%d) or no more buffers to free, current memory usage: %.2f GB\n", maxIterations, memoryUsageGB)
|
||||
} else {
|
||||
cclog.Infof("[METRICSTORE]> done: %d buffers force-freed\n", freedTotal)
|
||||
cclog.Infof("[METRICSTORE]> current memory usage after force-freeing the buffers: %.2f GB\n", memoryUsageGB)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// Free removes metric data older than the given time while preserving data for active nodes.
|
||||
//
|
||||
// This function implements intelligent retention by consulting the NodeProvider (if configured)
|
||||
// to determine which nodes are currently in use by running jobs. Data for these nodes is
|
||||
// preserved even if older than the retention time.
|
||||
//
|
||||
// Parameters:
|
||||
// - ms: The MemoryStore instance
|
||||
// - t: Time threshold - buffers with data older than this will be freed
|
||||
//
|
||||
// Returns:
|
||||
// - Number of buffers freed
|
||||
// - Error if NodeProvider query fails
|
||||
//
|
||||
// Behavior:
|
||||
// - If no NodeProvider is set: frees all buffers older than t
|
||||
// - If NodeProvider returns empty map: frees all buffers older than t
|
||||
// - Otherwise: preserves buffers for nodes returned by GetUsedNodes(), frees others
|
||||
func Free(ms *MemoryStore, t time.Time) (int, error) {
|
||||
// If no NodeProvider is configured, free all buffers older than t
|
||||
if ms.nodeProvider == nil {
|
||||
return ms.Free(nil, t.Unix())
|
||||
}
|
||||
|
||||
excludeSelectors, err := ms.nodeProvider.GetUsedNodes(t.Unix())
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
switch lenMap := len(excludeSelectors); lenMap {
|
||||
|
||||
// If the length of the map returned by GetUsedNodes() is 0,
|
||||
// then use default Free method with nil selector
|
||||
case 0:
|
||||
state.selectorsExcluded = false
|
||||
return ms.Free(nil, t.Unix())
|
||||
|
||||
// Else formulate selectors, exclude those from the map
|
||||
// and free the rest of the selectors
|
||||
default:
|
||||
state.selectorsExcluded = true
|
||||
selectors := GetSelectors(ms, excludeSelectors)
|
||||
return FreeSelected(ms, selectors, t)
|
||||
}
|
||||
}
|
||||
|
||||
// FreeSelected frees buffers for specific selectors while preserving others.
|
||||
//
|
||||
// This function is used when we want to retain some specific nodes beyond the retention time.
|
||||
// It iterates through the provided selectors and frees their associated buffers.
|
||||
//
|
||||
// Parameters:
|
||||
// - ms: The MemoryStore instance
|
||||
// - selectors: List of selector paths to free (e.g., [["cluster1", "node1"], ["cluster2", "node2"]])
|
||||
// - t: Time threshold for freeing buffers
|
||||
//
|
||||
// Returns the total number of buffers freed and any error encountered.
|
||||
func FreeSelected(ms *MemoryStore, selectors [][]string, t time.Time) (int, error) {
|
||||
freed := 0
|
||||
|
||||
for _, selector := range selectors {
|
||||
|
||||
freedBuffers, err := ms.Free(selector, t.Unix())
|
||||
if err != nil {
|
||||
cclog.Errorf("error while freeing selected buffers: %#v", err)
|
||||
}
|
||||
freed += freedBuffers
|
||||
|
||||
}
|
||||
|
||||
return freed, nil
|
||||
}
|
||||
|
||||
// GetSelectors returns all selectors at depth 2 (cluster/node level) that are NOT in the exclusion map.
|
||||
//
|
||||
// This function generates a list of selectors whose buffers should be freed by excluding
|
||||
// selectors that correspond to nodes currently in use by running jobs.
|
||||
//
|
||||
// Parameters:
|
||||
// - ms: The MemoryStore instance
|
||||
// - excludeSelectors: Map of cluster names to node hostnames that should NOT be freed
|
||||
//
|
||||
// Returns a list of selectors ([]string paths) that can be safely freed.
|
||||
//
|
||||
// Example:
|
||||
//
|
||||
// If the tree has paths ["emmy", "node001"] and ["emmy", "node002"],
|
||||
// and excludeSelectors contains {"emmy": ["node001"]},
|
||||
// then only [["emmy", "node002"]] is returned.
|
||||
func GetSelectors(ms *MemoryStore, excludeSelectors map[string][]string) [][]string {
|
||||
allSelectors := ms.GetPaths(2)
|
||||
|
||||
filteredSelectors := make([][]string, 0, len(allSelectors))
|
||||
|
||||
for _, path := range allSelectors {
|
||||
if len(path) < 2 {
|
||||
continue
|
||||
}
|
||||
|
||||
key := path[0] // The "Key" (Level 1)
|
||||
value := path[1] // The "Value" (Level 2)
|
||||
|
||||
exclude := false
|
||||
|
||||
// Check if the key exists in our exclusion map
|
||||
if excludedValues, exists := excludeSelectors[key]; exists {
|
||||
// The key exists, now check if the specific value is in the exclusion list
|
||||
if slices.Contains(excludedValues, value) {
|
||||
exclude = true
|
||||
}
|
||||
}
|
||||
|
||||
if !exclude {
|
||||
filteredSelectors = append(filteredSelectors, path)
|
||||
}
|
||||
}
|
||||
|
||||
return filteredSelectors
|
||||
}
|
||||
|
||||
// GetPaths returns a list of lists (paths) to the specified depth.
|
||||
func (ms *MemoryStore) GetPaths(targetDepth int) [][]string {
|
||||
var results [][]string
|
||||
|
||||
// Start recursion. Initial path is empty.
|
||||
// We treat Root as depth 0.
|
||||
ms.root.collectPaths(0, targetDepth, []string{}, &results)
|
||||
|
||||
return results
|
||||
}
|
||||
|
||||
// Write all values in `metrics` to the level specified by `selector` for time `ts`.
|
||||
// Look at `findLevelOrCreate` for how selectors work.
|
||||
func (m *MemoryStore) Write(selector []string, ts int64, metrics []Metric) error {
|
||||
var ok bool
|
||||
for i, metric := range metrics {
|
||||
if metric.MetricConfig.Frequency == 0 {
|
||||
metric.MetricConfig, ok = m.Metrics[metric.Name]
|
||||
if !ok {
|
||||
cclog.Debugf("[METRICSTORE]> Unknown metric '%s' in Write() - skipping", metric.Name)
|
||||
metric.MetricConfig.Frequency = 0
|
||||
}
|
||||
metrics[i] = metric
|
||||
}
|
||||
}
|
||||
|
||||
return m.WriteToLevel(&m.root, selector, ts, metrics)
|
||||
}
|
||||
|
||||
func (m *MemoryStore) GetLevel(selector []string) *Level {
|
||||
return m.root.findLevelOrCreate(selector, len(m.Metrics))
|
||||
}
|
||||
|
||||
// WriteToLevel assumes that `minfo` in `metrics` is filled in
|
||||
func (m *MemoryStore) WriteToLevel(l *Level, selector []string, ts int64, metrics []Metric) error {
|
||||
l = l.findLevelOrCreate(selector, len(m.Metrics))
|
||||
l.lock.Lock()
|
||||
defer l.lock.Unlock()
|
||||
|
||||
for _, metric := range metrics {
|
||||
if metric.MetricConfig.Frequency == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
b := l.metrics[metric.MetricConfig.offset]
|
||||
if b == nil {
|
||||
// First write to this metric and level
|
||||
b = newBuffer(ts, metric.MetricConfig.Frequency)
|
||||
l.metrics[metric.MetricConfig.offset] = b
|
||||
}
|
||||
|
||||
nb, err := b.write(ts, metric.Value)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Last write created a new buffer...
|
||||
if b != nb {
|
||||
l.metrics[metric.MetricConfig.offset] = nb
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Read returns all values for metric `metric` from `from` to `to` for the selected level(s).
|
||||
// If the level does not hold the metric itself, the data will be aggregated recursively from the children.
|
||||
// The second and third return value are the actual from/to for the data. Those can be different from
|
||||
// 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("[METRICSTORE]> invalid time range")
|
||||
}
|
||||
|
||||
minfo, ok := m.Metrics[metric]
|
||||
if !ok {
|
||||
return nil, 0, 0, 0, errors.New("[METRICSTORE]> unknown metric: " + metric)
|
||||
}
|
||||
|
||||
n, data := 0, make([]schema.Float, (to-from)/minfo.Frequency+1)
|
||||
|
||||
err := m.root.findBuffers(selector, minfo.offset, func(b *buffer) error {
|
||||
cdata, cfrom, cto, err := b.read(from, to, data)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if n == 0 {
|
||||
from, to = cfrom, cto
|
||||
} else if from != cfrom || to != cto || len(data) != len(cdata) {
|
||||
missingfront, missingback := int((from-cfrom)/minfo.Frequency), int((to-cto)/minfo.Frequency)
|
||||
if missingfront != 0 {
|
||||
return ErrDataDoesNotAlign
|
||||
}
|
||||
|
||||
newlen := len(cdata) - missingback
|
||||
if newlen < 1 {
|
||||
return ErrDataDoesNotAlign
|
||||
}
|
||||
cdata = cdata[0:newlen]
|
||||
if len(cdata) != len(data) {
|
||||
return ErrDataDoesNotAlign
|
||||
}
|
||||
|
||||
from, to = cfrom, cto
|
||||
}
|
||||
|
||||
data = cdata
|
||||
n += 1
|
||||
return nil
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return nil, 0, 0, 0, err
|
||||
} else if n == 0 {
|
||||
return nil, 0, 0, 0, errors.New("[METRICSTORE]> metric or host not found")
|
||||
} else if n > 1 {
|
||||
if minfo.Aggregation == 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("[METRICSTORE]> invalid aggregation")
|
||||
}
|
||||
}
|
||||
|
||||
data, resolution, err = resampler.LargestTriangleThreeBucket(data, minfo.Frequency, resolution)
|
||||
if err != nil {
|
||||
return nil, 0, 0, 0, err
|
||||
}
|
||||
|
||||
return data, from, to, resolution, nil
|
||||
}
|
||||
|
||||
// Free releases all buffers for the selected level and all its children that
|
||||
// contain only values older than `t`.
|
||||
func (m *MemoryStore) Free(selector []string, t int64) (int, error) {
|
||||
return m.GetLevel(selector).free(t)
|
||||
}
|
||||
|
||||
// Free releases all buffers for the selected level and all its children that
|
||||
// contain only values older than `t`.
|
||||
func (m *MemoryStore) ForceFree() (int, error) {
|
||||
return m.GetLevel(nil).forceFree()
|
||||
}
|
||||
|
||||
func (m *MemoryStore) FreeAll() error {
|
||||
for k := range m.root.children {
|
||||
delete(m.root.children, k)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *MemoryStore) SizeInBytes() int64 {
|
||||
return m.root.sizeInBytes()
|
||||
}
|
||||
|
||||
func (m *MemoryStore) SizeInGB() float64 {
|
||||
return float64(m.root.sizeInBytes()) / 1e9
|
||||
}
|
||||
|
||||
// ListChildren , given a selector, returns a list of all children of the level
|
||||
// selected.
|
||||
func (m *MemoryStore) ListChildren(selector []string) []string {
|
||||
lvl := &m.root
|
||||
for lvl != nil && len(selector) != 0 {
|
||||
lvl.lock.RLock()
|
||||
next := lvl.children[selector[0]]
|
||||
lvl.lock.RUnlock()
|
||||
lvl = next
|
||||
selector = selector[1:]
|
||||
}
|
||||
|
||||
if lvl == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
lvl.lock.RLock()
|
||||
defer lvl.lock.RUnlock()
|
||||
|
||||
children := make([]string, 0, len(lvl.children))
|
||||
for child := range lvl.children {
|
||||
children = append(children, child)
|
||||
}
|
||||
|
||||
return children
|
||||
}
|
||||
156
pkg/metricstore/metricstore_test.go
Normal file
156
pkg/metricstore/metricstore_test.go
Normal file
@@ -0,0 +1,156 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/schema"
|
||||
)
|
||||
|
||||
func TestAssignAggregationStrategy(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
input string
|
||||
expected AggregationStrategy
|
||||
wantErr bool
|
||||
}{
|
||||
{"empty string", "", NoAggregation, false},
|
||||
{"sum", "sum", SumAggregation, false},
|
||||
{"avg", "avg", AvgAggregation, false},
|
||||
{"invalid", "invalid", NoAggregation, true},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
result, err := AssignAggregationStrategy(tt.input)
|
||||
if (err != nil) != tt.wantErr {
|
||||
t.Errorf("AssignAggregationStrategy(%q) error = %v, wantErr %v", tt.input, err, tt.wantErr)
|
||||
return
|
||||
}
|
||||
if result != tt.expected {
|
||||
t.Errorf("AssignAggregationStrategy(%q) = %v, want %v", tt.input, result, tt.expected)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestAddMetric(t *testing.T) {
|
||||
// Reset Metrics before test
|
||||
Metrics = make(map[string]MetricConfig)
|
||||
|
||||
err := AddMetric("test_metric", MetricConfig{
|
||||
Frequency: 60,
|
||||
Aggregation: SumAggregation,
|
||||
})
|
||||
if err != nil {
|
||||
t.Errorf("AddMetric() error = %v", err)
|
||||
}
|
||||
|
||||
if _, ok := Metrics["test_metric"]; !ok {
|
||||
t.Error("AddMetric() did not add metric to Metrics map")
|
||||
}
|
||||
|
||||
// Test updating with higher frequency
|
||||
err = AddMetric("test_metric", MetricConfig{
|
||||
Frequency: 120,
|
||||
Aggregation: SumAggregation,
|
||||
})
|
||||
if err != nil {
|
||||
t.Errorf("AddMetric() error = %v", err)
|
||||
}
|
||||
|
||||
if Metrics["test_metric"].Frequency != 120 {
|
||||
t.Errorf("AddMetric() frequency = %d, want 120", Metrics["test_metric"].Frequency)
|
||||
}
|
||||
|
||||
// Test updating with lower frequency (should not update)
|
||||
err = AddMetric("test_metric", MetricConfig{
|
||||
Frequency: 30,
|
||||
Aggregation: SumAggregation,
|
||||
})
|
||||
if err != nil {
|
||||
t.Errorf("AddMetric() error = %v", err)
|
||||
}
|
||||
|
||||
if Metrics["test_metric"].Frequency != 120 {
|
||||
t.Errorf("AddMetric() frequency = %d, want 120 (should not downgrade)", Metrics["test_metric"].Frequency)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetMetricFrequency(t *testing.T) {
|
||||
// Reset Metrics before test
|
||||
Metrics = map[string]MetricConfig{
|
||||
"test_metric": {
|
||||
Frequency: 60,
|
||||
Aggregation: SumAggregation,
|
||||
},
|
||||
}
|
||||
|
||||
freq, err := GetMetricFrequency("test_metric")
|
||||
if err != nil {
|
||||
t.Errorf("GetMetricFrequency() error = %v", err)
|
||||
}
|
||||
if freq != 60 {
|
||||
t.Errorf("GetMetricFrequency() = %d, want 60", freq)
|
||||
}
|
||||
|
||||
_, err = GetMetricFrequency("nonexistent")
|
||||
if err == nil {
|
||||
t.Error("GetMetricFrequency() expected error for nonexistent metric")
|
||||
}
|
||||
}
|
||||
|
||||
func TestBufferWrite(t *testing.T) {
|
||||
b := newBuffer(100, 10)
|
||||
|
||||
// Test writing value
|
||||
nb, err := b.write(100, schema.Float(42.0))
|
||||
if err != nil {
|
||||
t.Errorf("buffer.write() error = %v", err)
|
||||
}
|
||||
if nb != b {
|
||||
t.Error("buffer.write() created new buffer unexpectedly")
|
||||
}
|
||||
if len(b.data) != 1 {
|
||||
t.Errorf("buffer.write() len(data) = %d, want 1", len(b.data))
|
||||
}
|
||||
if b.data[0] != schema.Float(42.0) {
|
||||
t.Errorf("buffer.write() data[0] = %v, want 42.0", b.data[0])
|
||||
}
|
||||
|
||||
// Test writing value from past (should error)
|
||||
_, err = b.write(50, schema.Float(10.0))
|
||||
if err == nil {
|
||||
t.Error("buffer.write() expected error for past timestamp")
|
||||
}
|
||||
}
|
||||
|
||||
func TestBufferRead(t *testing.T) {
|
||||
b := newBuffer(100, 10)
|
||||
|
||||
// Write some test data
|
||||
b.write(100, schema.Float(1.0))
|
||||
b.write(110, schema.Float(2.0))
|
||||
b.write(120, schema.Float(3.0))
|
||||
|
||||
// Read data
|
||||
data := make([]schema.Float, 3)
|
||||
result, from, to, err := b.read(100, 130, data)
|
||||
if err != nil {
|
||||
t.Errorf("buffer.read() error = %v", err)
|
||||
}
|
||||
// Buffer read should return from as firstWrite (start + freq/2)
|
||||
if from != 100 {
|
||||
t.Errorf("buffer.read() from = %d, want 100", from)
|
||||
}
|
||||
if to != 130 {
|
||||
t.Errorf("buffer.read() to = %d, want 130", to)
|
||||
}
|
||||
if len(result) != 3 {
|
||||
t.Errorf("buffer.read() len(result) = %d, want 3", len(result))
|
||||
}
|
||||
}
|
||||
1248
pkg/metricstore/query.go
Normal file
1248
pkg/metricstore/query.go
Normal file
File diff suppressed because it is too large
Load Diff
124
pkg/metricstore/stats.go
Normal file
124
pkg/metricstore/stats.go
Normal file
@@ -0,0 +1,124 @@
|
||||
// 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 metricstore
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"math"
|
||||
|
||||
"github.com/ClusterCockpit/cc-lib/v2/util"
|
||||
)
|
||||
|
||||
type Stats struct {
|
||||
Samples int
|
||||
Avg util.Float
|
||||
Min util.Float
|
||||
Max util.Float
|
||||
}
|
||||
|
||||
func (b *buffer) stats(from, to int64) (Stats, int64, int64, error) {
|
||||
if from < b.start {
|
||||
if b.prev != nil {
|
||||
return b.prev.stats(from, to)
|
||||
}
|
||||
from = b.start
|
||||
}
|
||||
|
||||
// TODO: Check if b.closed and if so and the full buffer is queried,
|
||||
// use b.statistics instead of iterating over the buffer.
|
||||
|
||||
samples := 0
|
||||
sum, min, max := 0.0, math.MaxFloat32, -math.MaxFloat32
|
||||
|
||||
var t int64
|
||||
for t = from; t < to; t += b.frequency {
|
||||
idx := int((t - b.start) / b.frequency)
|
||||
if idx >= cap(b.data) {
|
||||
b = b.next
|
||||
if b == nil {
|
||||
break
|
||||
}
|
||||
idx = 0
|
||||
}
|
||||
|
||||
if t < b.start || idx >= len(b.data) {
|
||||
continue
|
||||
}
|
||||
|
||||
xf := float64(b.data[idx])
|
||||
if math.IsNaN(xf) {
|
||||
continue
|
||||
}
|
||||
|
||||
samples += 1
|
||||
sum += xf
|
||||
min = math.Min(min, xf)
|
||||
max = math.Max(max, xf)
|
||||
}
|
||||
|
||||
return Stats{
|
||||
Samples: samples,
|
||||
Avg: util.Float(sum) / util.Float(samples),
|
||||
Min: util.Float(min),
|
||||
Max: util.Float(max),
|
||||
}, from, t, nil
|
||||
}
|
||||
|
||||
// Returns statistics for the requested metric on the selected node/level.
|
||||
// Data is aggregated to the selected level the same way as in `MemoryStore.Read`.
|
||||
// If `Stats.Samples` is zero, the statistics should not be considered as valid.
|
||||
func (m *MemoryStore) Stats(selector util.Selector, metric string, from, to int64) (*Stats, int64, int64, error) {
|
||||
if from > to {
|
||||
return nil, 0, 0, errors.New("invalid time range")
|
||||
}
|
||||
|
||||
minfo, ok := m.Metrics[metric]
|
||||
if !ok {
|
||||
return nil, 0, 0, errors.New("unknown metric: " + metric)
|
||||
}
|
||||
|
||||
n, samples := 0, 0
|
||||
avg, min, max := util.Float(0), math.MaxFloat32, -math.MaxFloat32
|
||||
err := m.root.findBuffers(selector, minfo.offset, func(b *buffer) error {
|
||||
stats, cfrom, cto, err := b.stats(from, to)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if n == 0 {
|
||||
from, to = cfrom, cto
|
||||
} else if from != cfrom || to != cto {
|
||||
return ErrDataDoesNotAlign
|
||||
}
|
||||
|
||||
samples += stats.Samples
|
||||
avg += stats.Avg
|
||||
min = math.Min(min, float64(stats.Min))
|
||||
max = math.Max(max, float64(stats.Max))
|
||||
n += 1
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, 0, 0, err
|
||||
}
|
||||
|
||||
if n == 0 {
|
||||
return nil, 0, 0, ErrNoData
|
||||
}
|
||||
|
||||
if minfo.Aggregation == AvgAggregation {
|
||||
avg /= util.Float(n)
|
||||
} else if n > 1 && minfo.Aggregation != SumAggregation {
|
||||
return nil, 0, 0, errors.New("invalid aggregation")
|
||||
}
|
||||
|
||||
return &Stats{
|
||||
Samples: samples,
|
||||
Avg: avg,
|
||||
Min: util.Float(min),
|
||||
Max: util.Float(max),
|
||||
}, from, to, nil
|
||||
}
|
||||
Reference in New Issue
Block a user