mirror of
				https://github.com/ClusterCockpit/cc-metric-store.git
				synced 2025-11-04 10:45:07 +01:00 
			
		
		
		
	Reintroduce JSON support for compability/transition
This commit is contained in:
		@@ -1,4 +1,4 @@
 | 
			
		||||
package main
 | 
			
		||||
package memstore
 | 
			
		||||
 | 
			
		||||
import (
 | 
			
		||||
	"archive/zip"
 | 
			
		||||
@@ -18,19 +18,21 @@ import (
 | 
			
		||||
	"strings"
 | 
			
		||||
	"sync"
 | 
			
		||||
	"sync/atomic"
 | 
			
		||||
 | 
			
		||||
	"github.com/ClusterCockpit/cc-metric-store/internal/types"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// Whenever changed, update MarshalJSON as well!
 | 
			
		||||
type CheckpointMetrics struct {
 | 
			
		||||
type JSONCheckpointMetrics struct {
 | 
			
		||||
	Frequency int64         `json:"frequency"`
 | 
			
		||||
	Start     int64         `json:"start"`
 | 
			
		||||
	Data      []Float `json:"data"`
 | 
			
		||||
	Data      []types.Float `json:"data"`
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// 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).
 | 
			
		||||
func (cm *CheckpointMetrics) MarshalJSON() ([]byte, error) {
 | 
			
		||||
func (cm *JSONCheckpointMetrics) MarshalJSON() ([]byte, error) {
 | 
			
		||||
	buf := make([]byte, 0, 128+len(cm.Data)*8)
 | 
			
		||||
	buf = append(buf, `{"frequency":`...)
 | 
			
		||||
	buf = strconv.AppendInt(buf, cm.Frequency, 10)
 | 
			
		||||
@@ -51,11 +53,11 @@ func (cm *CheckpointMetrics) MarshalJSON() ([]byte, error) {
 | 
			
		||||
	return buf, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
type CheckpointFile struct {
 | 
			
		||||
type JSONCheckpointFile struct {
 | 
			
		||||
	From     int64                             `json:"from"`
 | 
			
		||||
	To       int64                             `json:"to"`
 | 
			
		||||
	Metrics  map[string]*CheckpointMetrics `json:"metrics"`
 | 
			
		||||
	Children map[string]*CheckpointFile    `json:"children"`
 | 
			
		||||
	Metrics  map[string]*JSONCheckpointMetrics `json:"metrics"`
 | 
			
		||||
	Children map[string]*JSONCheckpointFile    `json:"children"`
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
var ErrNoNewData error = errors.New("all data already archived")
 | 
			
		||||
@@ -74,13 +76,13 @@ func init() {
 | 
			
		||||
// 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.
 | 
			
		||||
func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
 | 
			
		||||
	levels := make([]*level, 0)
 | 
			
		||||
func (m *MemoryStore) ToJSONCheckpoint(dir string, from, to int64) (int, error) {
 | 
			
		||||
	levels := make([]*Level, 0)
 | 
			
		||||
	selectors := make([][]string, 0)
 | 
			
		||||
	m.root.lock.RLock()
 | 
			
		||||
	for sel1, l1 := range m.root.children {
 | 
			
		||||
	for sel1, l1 := range m.root.sublevels {
 | 
			
		||||
		l1.lock.RLock()
 | 
			
		||||
		for sel2, l2 := range l1.children {
 | 
			
		||||
		for sel2, l2 := range l1.sublevels {
 | 
			
		||||
			levels = append(levels, l2)
 | 
			
		||||
			selectors = append(selectors, []string{sel1, sel2})
 | 
			
		||||
		}
 | 
			
		||||
@@ -89,7 +91,7 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
 | 
			
		||||
	m.root.lock.RUnlock()
 | 
			
		||||
 | 
			
		||||
	type workItem struct {
 | 
			
		||||
		level    *level
 | 
			
		||||
		level    *Level
 | 
			
		||||
		dir      string
 | 
			
		||||
		selector []string
 | 
			
		||||
	}
 | 
			
		||||
@@ -104,7 +106,7 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
 | 
			
		||||
			defer wg.Done()
 | 
			
		||||
 | 
			
		||||
			for workItem := range work {
 | 
			
		||||
				if err := workItem.level.toCheckpoint(workItem.dir, from, to, m); err != nil {
 | 
			
		||||
				if err := workItem.level.toJSONCheckpoint(workItem.dir, from, to, m); err != nil {
 | 
			
		||||
					if err == ErrNoNewData {
 | 
			
		||||
						continue
 | 
			
		||||
					}
 | 
			
		||||
@@ -136,54 +138,42 @@ func (m *MemoryStore) ToCheckpoint(dir string, from, to int64) (int, error) {
 | 
			
		||||
	return int(n), nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (l *level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFile, error) {
 | 
			
		||||
func (l *Level) toJSONCheckpointFile(from, to int64, m *MemoryStore) (*JSONCheckpointFile, error) {
 | 
			
		||||
	l.lock.RLock()
 | 
			
		||||
	defer l.lock.RUnlock()
 | 
			
		||||
 | 
			
		||||
	retval := &CheckpointFile{
 | 
			
		||||
	retval := &JSONCheckpointFile{
 | 
			
		||||
		From:     from,
 | 
			
		||||
		To:       to,
 | 
			
		||||
		Metrics:  make(map[string]*CheckpointMetrics),
 | 
			
		||||
		Children: make(map[string]*CheckpointFile),
 | 
			
		||||
		Metrics:  make(map[string]*JSONCheckpointMetrics),
 | 
			
		||||
		Children: make(map[string]*JSONCheckpointFile),
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for metric, minfo := range m.metrics {
 | 
			
		||||
		b := l.metrics[minfo.offset]
 | 
			
		||||
		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 nil
 | 
			
		||||
		})
 | 
			
		||||
 | 
			
		||||
		if allArchived {
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		data := make([]Float, (to-from)/b.frequency+1)
 | 
			
		||||
		data := make([]types.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] = NaN
 | 
			
		||||
			data[i] = types.NaN
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		retval.Metrics[metric] = &CheckpointMetrics{
 | 
			
		||||
		retval.Metrics[metric] = &JSONCheckpointMetrics{
 | 
			
		||||
			Frequency: b.frequency,
 | 
			
		||||
			Start:     start,
 | 
			
		||||
			Data:      data,
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for name, child := range l.children {
 | 
			
		||||
		val, err := child.toCheckpointFile(from, to, m)
 | 
			
		||||
	for name, child := range l.sublevels {
 | 
			
		||||
		val, err := child.toJSONCheckpointFile(from, to, m)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
@@ -200,8 +190,8 @@ func (l *level) toCheckpointFile(from, to int64, m *MemoryStore) (*CheckpointFil
 | 
			
		||||
	return retval, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (l *level) toCheckpoint(dir string, from, to int64, m *MemoryStore) error {
 | 
			
		||||
	cf, err := l.toCheckpointFile(from, to, m)
 | 
			
		||||
func (l *Level) toJSONCheckpoint(dir string, from, to int64, m *MemoryStore) error {
 | 
			
		||||
	cf, err := l.toJSONCheckpointFile(from, to, m)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
@@ -234,7 +224,7 @@ func (l *level) toCheckpoint(dir string, from, to int64, m *MemoryStore) error {
 | 
			
		||||
// 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.
 | 
			
		||||
func (m *MemoryStore) FromCheckpoint(dir string, from int64) (int, error) {
 | 
			
		||||
func (m *MemoryStore) FromJSONCheckpoint(dir string, from int64) (int, error) {
 | 
			
		||||
	var wg sync.WaitGroup
 | 
			
		||||
	work := make(chan [2]string, NumWorkers)
 | 
			
		||||
	n, errs := int32(0), int32(0)
 | 
			
		||||
@@ -245,7 +235,7 @@ func (m *MemoryStore) FromCheckpoint(dir string, from int64) (int, error) {
 | 
			
		||||
			defer wg.Done()
 | 
			
		||||
			for host := range work {
 | 
			
		||||
				lvl := m.root.findLevelOrCreate(host[:], len(m.metrics))
 | 
			
		||||
				nn, err := lvl.fromCheckpoint(filepath.Join(dir, host[0], host[1]), from, m)
 | 
			
		||||
				nn, err := lvl.fromJSONCheckpoint(filepath.Join(dir, host[0], host[1]), from, m)
 | 
			
		||||
				if err != nil {
 | 
			
		||||
					log.Fatalf("error while loading checkpoints: %s", err.Error())
 | 
			
		||||
					atomic.AddInt32(&errs, 1)
 | 
			
		||||
@@ -302,54 +292,53 @@ done:
 | 
			
		||||
	return int(n), nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (l *level) loadFile(cf *CheckpointFile, m *MemoryStore) error {
 | 
			
		||||
func (l *Level) loadJSONCheckpointFile(cf *JSONCheckpointFile, m *MemoryStore) error {
 | 
			
		||||
	for name, metric := range cf.Metrics {
 | 
			
		||||
		n := len(metric.Data)
 | 
			
		||||
		b := &buffer{
 | 
			
		||||
		c := &chunk{
 | 
			
		||||
			frequency:    metric.Frequency,
 | 
			
		||||
			start:        metric.Start,
 | 
			
		||||
			data:         metric.Data[0:n:n], // Space is wasted here :(
 | 
			
		||||
			prev:         nil,
 | 
			
		||||
			next:         nil,
 | 
			
		||||
			archived:  true,
 | 
			
		||||
			checkpointed: true,
 | 
			
		||||
		}
 | 
			
		||||
		b.close()
 | 
			
		||||
 | 
			
		||||
		minfo, ok := m.metrics[name]
 | 
			
		||||
		mc, ok := m.metrics[name]
 | 
			
		||||
		if !ok {
 | 
			
		||||
			continue
 | 
			
		||||
			// return errors.New("Unkown metric: " + name)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		prev := l.metrics[minfo.offset]
 | 
			
		||||
		prev := l.metrics[mc.Offset]
 | 
			
		||||
		if prev == nil {
 | 
			
		||||
			l.metrics[minfo.offset] = b
 | 
			
		||||
			l.metrics[mc.Offset] = c
 | 
			
		||||
		} else {
 | 
			
		||||
			if prev.start > b.start {
 | 
			
		||||
			if prev.start > c.start {
 | 
			
		||||
				return errors.New("wooops")
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			b.prev = prev
 | 
			
		||||
			prev.next = b
 | 
			
		||||
			c.prev = prev
 | 
			
		||||
			prev.next = c
 | 
			
		||||
		}
 | 
			
		||||
		l.metrics[minfo.offset] = b
 | 
			
		||||
		l.metrics[mc.Offset] = c
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if len(cf.Children) > 0 && l.children == nil {
 | 
			
		||||
		l.children = make(map[string]*level)
 | 
			
		||||
	if len(cf.Children) > 0 && l.sublevels == nil {
 | 
			
		||||
		l.sublevels = make(map[string]*Level)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	for sel, childCf := range cf.Children {
 | 
			
		||||
		child, ok := l.children[sel]
 | 
			
		||||
		child, ok := l.sublevels[sel]
 | 
			
		||||
		if !ok {
 | 
			
		||||
			child = &level{
 | 
			
		||||
				metrics:  make([]*buffer, len(m.metrics)),
 | 
			
		||||
				children: nil,
 | 
			
		||||
			child = &Level{
 | 
			
		||||
				metrics:   make([]*chunk, len(m.metrics)),
 | 
			
		||||
				sublevels: nil,
 | 
			
		||||
			}
 | 
			
		||||
			l.children[sel] = child
 | 
			
		||||
			l.sublevels[sel] = child
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if err := child.loadFile(childCf, m); err != nil {
 | 
			
		||||
		if err := child.loadJSONCheckpointFile(childCf, m); err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
@@ -357,7 +346,7 @@ func (l *level) loadFile(cf *CheckpointFile, m *MemoryStore) error {
 | 
			
		||||
	return nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (l *level) fromCheckpoint(dir string, from int64, m *MemoryStore) (int, error) {
 | 
			
		||||
func (l *Level) fromJSONCheckpoint(dir string, from int64, m *MemoryStore) (int, error) {
 | 
			
		||||
	direntries, err := os.ReadDir(dir)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		if os.IsNotExist(err) {
 | 
			
		||||
@@ -371,18 +360,18 @@ func (l *level) fromCheckpoint(dir string, from int64, m *MemoryStore) (int, err
 | 
			
		||||
	filesLoaded := 0
 | 
			
		||||
	for _, e := range direntries {
 | 
			
		||||
		if e.IsDir() {
 | 
			
		||||
			child := &level{
 | 
			
		||||
				metrics:  make([]*buffer, len(m.metrics)),
 | 
			
		||||
				children: make(map[string]*level),
 | 
			
		||||
			child := &Level{
 | 
			
		||||
				metrics:   make([]*chunk, len(m.metrics)),
 | 
			
		||||
				sublevels: make(map[string]*Level),
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			files, err := child.fromCheckpoint(path.Join(dir, e.Name()), from, m)
 | 
			
		||||
			files, err := child.fromJSONCheckpoint(path.Join(dir, e.Name()), from, m)
 | 
			
		||||
			filesLoaded += files
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				return filesLoaded, err
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			l.children[e.Name()] = child
 | 
			
		||||
			l.sublevels[e.Name()] = child
 | 
			
		||||
		} else if strings.HasSuffix(e.Name(), ".json") {
 | 
			
		||||
			jsonFiles = append(jsonFiles, e)
 | 
			
		||||
		} else {
 | 
			
		||||
@@ -403,7 +392,7 @@ func (l *level) fromCheckpoint(dir string, from int64, m *MemoryStore) (int, err
 | 
			
		||||
		defer f.Close()
 | 
			
		||||
 | 
			
		||||
		br := bufio.NewReader(f)
 | 
			
		||||
		cf := &CheckpointFile{}
 | 
			
		||||
		cf := &JSONCheckpointFile{}
 | 
			
		||||
		if err = json.NewDecoder(br).Decode(cf); err != nil {
 | 
			
		||||
			return filesLoaded, err
 | 
			
		||||
		}
 | 
			
		||||
@@ -412,7 +401,7 @@ func (l *level) fromCheckpoint(dir string, from int64, m *MemoryStore) (int, err
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if err = l.loadFile(cf, m); err != nil {
 | 
			
		||||
		if err = l.loadJSONCheckpointFile(cf, m); err != nil {
 | 
			
		||||
			return filesLoaded, err
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
@@ -480,12 +469,13 @@ func ArchiveCheckpoints(checkpointsDir, archiveDir string, from int64, deleteIns
 | 
			
		||||
		cluster, host string
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	numWorkers := 2
 | 
			
		||||
	var wg sync.WaitGroup
 | 
			
		||||
	n, errs := int32(0), int32(0)
 | 
			
		||||
	work := make(chan workItem, NumWorkers)
 | 
			
		||||
	work := make(chan workItem, numWorkers)
 | 
			
		||||
 | 
			
		||||
	wg.Add(NumWorkers)
 | 
			
		||||
	for worker := 0; worker < NumWorkers; worker++ {
 | 
			
		||||
	wg.Add(numWorkers)
 | 
			
		||||
	for worker := 0; worker < numWorkers; worker++ {
 | 
			
		||||
		go func() {
 | 
			
		||||
			defer wg.Done()
 | 
			
		||||
			for workItem := range work {
 | 
			
		||||
		Reference in New Issue
	
	Block a user