Add documentation

This commit is contained in:
2026-01-16 08:27:46 +01:00
parent 93dcfee8c5
commit 9a97d0e8eb
6 changed files with 782 additions and 111 deletions

View File

@@ -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