mirror of
https://github.com/ClusterCockpit/cc-backend
synced 2026-01-16 09:41:47 +01:00
Add documentation
This commit is contained in:
@@ -3,6 +3,34 @@
|
||||
// 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 (
|
||||
@@ -29,18 +57,21 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
CheckpointFilePerms = 0o644
|
||||
CheckpointDirPerms = 0o755
|
||||
GCTriggerInterval = DefaultGCTriggerInterval
|
||||
CheckpointFilePerms = 0o644 // File permissions for checkpoint files
|
||||
CheckpointDirPerms = 0o755 // Directory permissions for checkpoint directories
|
||||
GCTriggerInterval = DefaultGCTriggerInterval // Interval for triggering GC during checkpoint loading
|
||||
)
|
||||
|
||||
// Whenever changed, update MarshalJSON as well!
|
||||
// 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"`
|
||||
@@ -48,10 +79,23 @@ type CheckpointFile struct {
|
||||
To int64 `json:"to"`
|
||||
}
|
||||
|
||||
var lastCheckpoint time.Time
|
||||
// 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()
|
||||
@@ -60,9 +104,10 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
|
||||
defer wg.Done()
|
||||
d, err := time.ParseDuration(Keys.Checkpoints.Interval)
|
||||
if err != nil {
|
||||
cclog.Fatal(err)
|
||||
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
|
||||
}
|
||||
|
||||
@@ -74,15 +119,21 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
cclog.Infof("[METRICSTORE]> start checkpointing (starting at %s)...", lastCheckpoint.Format(time.RFC3339))
|
||||
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,
|
||||
lastCheckpoint.Unix(), now.Unix())
|
||||
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()
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -113,9 +164,10 @@ func Checkpointing(wg *sync.WaitGroup, ctx context.Context) {
|
||||
}
|
||||
}
|
||||
|
||||
// As `Float` implements a custom MarshalJSON() function,
|
||||
// serializing an array of such types has more overhead
|
||||
// than one would assume (because of extra allocations, interfaces and so on).
|
||||
// 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":`...)
|
||||
@@ -137,13 +189,27 @@ func (cm *CheckpointMetrics) MarshalJSON() ([]byte, error) {
|
||||
return buf, nil
|
||||
}
|
||||
|
||||
// Metrics stored at the lowest 2 levels are not stored away (root and cluster)!
|
||||
// On a per-host basis a new JSON file is created. I have no idea if this will scale.
|
||||
// The good thing: Only a host at a time is locked, so this function can run
|
||||
// in parallel to writes/reads.
|
||||
// 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) {
|
||||
levels := make([]*Level, 0)
|
||||
selectors := make([][]string, 0)
|
||||
// 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()
|
||||
@@ -203,6 +269,8 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
|
||||
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()
|
||||
@@ -224,6 +292,7 @@ func (l *Level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFil
|
||||
b.iterFromTo(from, to, func(b *buffer) error {
|
||||
if !b.archived {
|
||||
allArchived = false
|
||||
return fmt.Errorf("stop") // Early termination signal
|
||||
}
|
||||
return nil
|
||||
})
|
||||
@@ -267,6 +336,8 @@ func (l *Level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFil
|
||||
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 {
|
||||
@@ -278,11 +349,11 @@ func (l *Level) toCheckpoint(dir string, from, to int64, m *MemoryStore) error {
|
||||
}
|
||||
|
||||
filepath := path.Join(dir, fmt.Sprintf("%d.json", from))
|
||||
f, err := os.OpenFile(filepath, os.O_CREATE|os.O_WRONLY, CheckpointFilePerms)
|
||||
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, CheckpointFilePerms)
|
||||
f, err = os.OpenFile(filepath, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, CheckpointFilePerms)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
@@ -298,9 +369,54 @@ func (l *Level) toCheckpoint(dir string, from, to int64, m *MemoryStore) error {
|
||||
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)
|
||||
work := make(chan [2]string, Keys.NumWorkers*4)
|
||||
n, errs := int32(0), int32(0)
|
||||
|
||||
wg.Add(Keys.NumWorkers)
|
||||
@@ -319,40 +435,7 @@ func (m *MemoryStore) FromCheckpoint(dir string, from int64, extension string) (
|
||||
}()
|
||||
}
|
||||
|
||||
i := 0
|
||||
clustersDir, err := os.ReadDir(dir)
|
||||
for _, clusterDir := range clustersDir {
|
||||
if !clusterDir.IsDir() {
|
||||
err = errors.New("[METRICSTORE]> expected only directories at first level of checkpoints/ directory")
|
||||
goto done
|
||||
}
|
||||
|
||||
hostsDir, e := os.ReadDir(filepath.Join(dir, clusterDir.Name()))
|
||||
if e != nil {
|
||||
err = e
|
||||
goto done
|
||||
}
|
||||
|
||||
for _, hostDir := range hostsDir {
|
||||
if !hostDir.IsDir() {
|
||||
err = errors.New("[METRICSTORE]> expected only directories at second level of checkpoints/ directory")
|
||||
goto done
|
||||
}
|
||||
|
||||
i++
|
||||
if i%Keys.NumWorkers == 0 && i > GCTriggerInterval {
|
||||
// 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()}
|
||||
}
|
||||
}
|
||||
done:
|
||||
err := enqueueCheckpointHosts(dir, work)
|
||||
close(work)
|
||||
wg.Wait()
|
||||
|
||||
@@ -366,9 +449,11 @@ done:
|
||||
return int(n), nil
|
||||
}
|
||||
|
||||
// Metrics stored at the lowest 2 levels are not loaded (root and cluster)!
|
||||
// This function can only be called once and before the very first write or read.
|
||||
// Different host's data is loaded to memory in parallel.
|
||||
// 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()
|
||||
@@ -411,6 +496,7 @@ func (m *MemoryStore) FromCheckpointFiles(dir string, from int64) (int, error) {
|
||||
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
|
||||
|
||||
|
||||
Reference in New Issue
Block a user