mirror of
https://github.com/ClusterCockpit/cc-backend
synced 2026-01-16 09:41:47 +01:00
Add documentation
This commit is contained in:
@@ -42,30 +42,75 @@ var (
|
||||
msInstance *MemoryStore
|
||||
// shutdownFunc stores the context cancellation function created in Init
|
||||
// and is called during Shutdown to cancel all background goroutines
|
||||
shutdownFunc context.CancelFunc
|
||||
shutdownFunc context.CancelFunc
|
||||
shutdownFuncMu sync.Mutex // Protects shutdownFunc from concurrent access
|
||||
)
|
||||
|
||||
// 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
|
||||
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 // Injected dependency for querying running jobs
|
||||
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()
|
||||
|
||||
@@ -163,7 +208,9 @@ func Init(rawConfig json.RawMessage, wg *sync.WaitGroup) {
|
||||
// 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)
|
||||
@@ -173,8 +220,17 @@ func Init(rawConfig json.RawMessage, wg *sync.WaitGroup) {
|
||||
}
|
||||
}
|
||||
|
||||
// InitMetrics creates a new, initialized instance of a MemoryStore.
|
||||
// Will panic if values in the metric configurations are invalid.
|
||||
// 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
|
||||
@@ -201,6 +257,11 @@ func InitMetrics(metrics map[string]MetricConfig) {
|
||||
})
|
||||
}
|
||||
|
||||
// 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!")
|
||||
@@ -217,7 +278,22 @@ 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()
|
||||
}
|
||||
@@ -244,6 +320,17 @@ func Shutdown() {
|
||||
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()
|
||||
|
||||
@@ -283,6 +370,17 @@ func Retention(wg *sync.WaitGroup, ctx context.Context) {
|
||||
}()
|
||||
}
|
||||
|
||||
// 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()
|
||||
|
||||
@@ -303,15 +401,16 @@ func MemoryUsageTracker(wg *sync.WaitGroup, ctx context.Context) {
|
||||
return
|
||||
case <-ticker.C:
|
||||
memoryUsageGB := ms.SizeInGB()
|
||||
cclog.Infof("[METRICSTORE]> current memory usage: %.2f\n", memoryUsageGB)
|
||||
cclog.Infof("[METRICSTORE]> current memory usage: %.2f GB\n", memoryUsageGB)
|
||||
|
||||
if memoryUsageGB > float64(Keys.MemoryCap) {
|
||||
cclog.Warnf("[METRICSTORE]> current memory usage is greater than the Memory Cap: %d\n", Keys.MemoryCap)
|
||||
cclog.Warnf("[METRICSTORE]> current memory usage is greater than the Memory Cap: %d GB\n", Keys.MemoryCap)
|
||||
cclog.Warnf("[METRICSTORE]> starting to force-free the buffers from the Metric Store\n")
|
||||
|
||||
freedTotal := 0
|
||||
const maxIterations = 100
|
||||
|
||||
for {
|
||||
for range maxIterations {
|
||||
memoryUsageGB = ms.SizeInGB()
|
||||
if memoryUsageGB < float64(Keys.MemoryCap) {
|
||||
break
|
||||
@@ -319,16 +418,23 @@ func MemoryUsageTracker(wg *sync.WaitGroup, ctx context.Context) {
|
||||
|
||||
freed, err := ms.ForceFree()
|
||||
if err != nil {
|
||||
cclog.Errorf("error while force-freeing the buffers: %s", err)
|
||||
cclog.Errorf("[METRICSTORE]> error while force-freeing the buffers: %s", err)
|
||||
}
|
||||
if freed == 0 {
|
||||
cclog.Fatalf("0 buffers force-freed in last try, %d total buffers force-freed, memory usage of %.2f remains higher than the memory cap and there are no buffers left to force-free\n", freedTotal, memoryUsageGB)
|
||||
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()
|
||||
}
|
||||
|
||||
cclog.Infof("[METRICSTORE]> done: %d buffers freed\n", freedTotal)
|
||||
cclog.Infof("[METRICSTORE]> current memory usage after force-freeing the buffers: %.2f\n", memoryUsageGB)
|
||||
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 freed\n", freedTotal)
|
||||
cclog.Infof("[METRICSTORE]> current memory usage after force-freeing the buffers: %.2f GB\n", memoryUsageGB)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -336,6 +442,24 @@ func MemoryUsageTracker(wg *sync.WaitGroup, ctx context.Context) {
|
||||
}()
|
||||
}
|
||||
|
||||
// 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 {
|
||||
@@ -362,8 +486,17 @@ func Free(ms *MemoryStore, t time.Time) (int, error) {
|
||||
}
|
||||
}
|
||||
|
||||
// A function to free specific selectors. Used when we want to retain some specific nodes
|
||||
// beyond the retention time.
|
||||
// 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
|
||||
|
||||
@@ -380,8 +513,22 @@ func FreeSelected(ms *MemoryStore, selectors [][]string, t time.Time) (int, erro
|
||||
return freed, nil
|
||||
}
|
||||
|
||||
// This function will populate all the second last levels - meaning nodes
|
||||
// From that we can exclude the specific selectosr/node we want to retain.
|
||||
// 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)
|
||||
|
||||
@@ -432,6 +579,7 @@ func (m *MemoryStore) Write(selector []string, ts int64, metrics []Metric) error
|
||||
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
|
||||
|
||||
Reference in New Issue
Block a user