2016-11-15 01:34:25 -08:00
|
|
|
// Package tsdb implements a time series storage for float64 sample data.
|
|
|
|
package tsdb
|
|
|
|
|
|
|
|
import (
|
2016-12-10 09:08:50 -08:00
|
|
|
"bytes"
|
2016-12-04 04:16:11 -08:00
|
|
|
"fmt"
|
|
|
|
"os"
|
2016-12-08 08:43:10 -08:00
|
|
|
"path/filepath"
|
2016-12-15 02:56:41 -08:00
|
|
|
"reflect"
|
2016-12-14 23:31:26 -08:00
|
|
|
"strconv"
|
2016-12-08 08:43:10 -08:00
|
|
|
"sync"
|
2016-11-15 01:34:25 -08:00
|
|
|
"time"
|
2016-12-15 02:56:41 -08:00
|
|
|
"unsafe"
|
2016-11-15 01:34:25 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
"github.com/coreos/etcd/pkg/fileutil"
|
2016-12-08 08:43:10 -08:00
|
|
|
"github.com/fabxc/tsdb/chunks"
|
2016-12-21 00:39:01 -08:00
|
|
|
"github.com/fabxc/tsdb/labels"
|
2016-12-14 09:38:46 -08:00
|
|
|
"github.com/go-kit/kit/log"
|
2017-01-03 06:43:26 -08:00
|
|
|
"github.com/pkg/errors"
|
2016-12-31 00:48:49 -08:00
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2016-11-15 01:34:25 -08:00
|
|
|
)
|
|
|
|
|
2016-12-09 01:00:14 -08:00
|
|
|
// DefaultOptions used for the DB. They are sane for setups using
|
|
|
|
// millisecond precision timestamps.
|
2016-11-15 01:34:25 -08:00
|
|
|
var DefaultOptions = &Options{
|
2016-12-26 07:55:44 -08:00
|
|
|
Retention: 15 * 24 * 3600 * 1000, // 15 days
|
|
|
|
DisableWAL: false,
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Options of the DB storage.
|
|
|
|
type Options struct {
|
2016-12-26 07:55:44 -08:00
|
|
|
Retention int64
|
|
|
|
DisableWAL bool
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// DB is a time series storage.
|
|
|
|
type DB struct {
|
|
|
|
logger log.Logger
|
|
|
|
opts *Options
|
2016-12-08 08:43:10 -08:00
|
|
|
path string
|
2016-11-15 01:34:25 -08:00
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
partitions []*Partition
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
2016-12-04 04:16:11 -08:00
|
|
|
// TODO(fabxc): make configurable
|
|
|
|
const (
|
2017-01-05 23:08:02 -08:00
|
|
|
partitionShift = 0
|
|
|
|
numPartitions = 1 << partitionShift
|
|
|
|
maxChunkSize = 1024
|
2016-12-04 04:16:11 -08:00
|
|
|
)
|
|
|
|
|
2016-11-15 01:34:25 -08:00
|
|
|
// Open or create a new DB.
|
|
|
|
func Open(path string, l log.Logger, opts *Options) (*DB, error) {
|
|
|
|
if opts == nil {
|
|
|
|
opts = DefaultOptions
|
|
|
|
}
|
2016-12-04 04:16:11 -08:00
|
|
|
if err := os.MkdirAll(path, 0777); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-12-14 09:38:46 -08:00
|
|
|
if l == nil {
|
|
|
|
l = log.NewLogfmtLogger(os.Stdout)
|
|
|
|
l = log.NewContext(l).With("ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
|
|
|
|
}
|
2016-11-15 01:34:25 -08:00
|
|
|
|
|
|
|
c := &DB{
|
2016-12-04 04:16:11 -08:00
|
|
|
logger: l,
|
|
|
|
opts: opts,
|
2016-12-08 08:43:10 -08:00
|
|
|
path: path,
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// Initialize vertical partitions.
|
|
|
|
// TODO(fabxc): validate partition number to be power of 2, which is required
|
|
|
|
// for the bitshift-modulo when finding the right partition.
|
|
|
|
for i := 0; i < numPartitions; i++ {
|
|
|
|
l := log.NewContext(l).With("partition", i)
|
|
|
|
d := partitionDir(path, i)
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
s, err := OpenPartition(d, i, l)
|
2016-12-14 23:31:26 -08:00
|
|
|
if err != nil {
|
2017-01-05 23:08:02 -08:00
|
|
|
return nil, fmt.Errorf("initializing partition %q failed: %s", d, err)
|
2016-12-14 23:31:26 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
c.partitions = append(c.partitions, s)
|
2016-12-04 04:16:11 -08:00
|
|
|
}
|
|
|
|
|
2016-11-15 01:34:25 -08:00
|
|
|
return c, nil
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func partitionDir(base string, i int) string {
|
2016-12-14 23:31:26 -08:00
|
|
|
return filepath.Join(base, strconv.Itoa(i))
|
|
|
|
}
|
|
|
|
|
2016-12-04 04:16:11 -08:00
|
|
|
// Close the database.
|
|
|
|
func (db *DB) Close() error {
|
2016-12-14 23:31:26 -08:00
|
|
|
var g errgroup.Group
|
2016-12-09 01:00:14 -08:00
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
for _, partition := range db.partitions {
|
|
|
|
g.Go(partition.Close)
|
2016-12-07 08:10:49 -08:00
|
|
|
}
|
2016-12-09 01:00:14 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
return g.Wait()
|
2016-12-04 04:16:11 -08:00
|
|
|
}
|
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
// Appender allows committing batches of samples to a database.
|
|
|
|
// The data held by the appender is reset after Commit returns.
|
2016-12-10 09:08:50 -08:00
|
|
|
type Appender interface {
|
2016-12-20 15:02:37 -08:00
|
|
|
// AddSeries registers a new known series label set with the appender
|
|
|
|
// and returns a reference number used to add samples to it over the
|
|
|
|
// life time of the Appender.
|
|
|
|
// AddSeries(Labels) uint64
|
2016-12-10 09:08:50 -08:00
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
// Add adds a sample pair for the referenced series.
|
2016-12-29 02:03:39 -08:00
|
|
|
Add(lset labels.Labels, t int64, v float64) error
|
2016-12-20 15:02:37 -08:00
|
|
|
|
|
|
|
// Commit submits the collected samples and purges the batch.
|
2016-12-10 09:08:50 -08:00
|
|
|
Commit() error
|
|
|
|
}
|
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
// Appender returns a new appender against the database.
|
|
|
|
func (db *DB) Appender() Appender {
|
|
|
|
return &bucketAppender{
|
|
|
|
db: db,
|
2017-01-05 23:08:02 -08:00
|
|
|
buckets: make([][]hashedSample, numPartitions),
|
2016-12-20 15:02:37 -08:00
|
|
|
}
|
2016-12-10 09:08:50 -08:00
|
|
|
}
|
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
type bucketAppender struct {
|
|
|
|
db *DB
|
|
|
|
buckets [][]hashedSample
|
2016-12-10 09:08:50 -08:00
|
|
|
}
|
|
|
|
|
2016-12-29 02:03:39 -08:00
|
|
|
func (ba *bucketAppender) Add(lset labels.Labels, t int64, v float64) error {
|
2016-12-10 09:08:50 -08:00
|
|
|
h := lset.Hash()
|
2017-01-05 23:08:02 -08:00
|
|
|
s := h >> (64 - partitionShift)
|
2016-12-10 09:08:50 -08:00
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
ba.buckets[s] = append(ba.buckets[s], hashedSample{
|
|
|
|
hash: h,
|
|
|
|
labels: lset,
|
|
|
|
t: t,
|
|
|
|
v: v,
|
2016-12-10 09:08:50 -08:00
|
|
|
})
|
2016-12-29 02:03:39 -08:00
|
|
|
|
|
|
|
return nil
|
2016-12-10 09:08:50 -08:00
|
|
|
}
|
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
func (ba *bucketAppender) reset() {
|
|
|
|
for i := range ba.buckets {
|
|
|
|
ba.buckets[i] = ba.buckets[i][:0]
|
2016-12-09 04:41:38 -08:00
|
|
|
}
|
2016-12-20 15:02:37 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (ba *bucketAppender) Commit() error {
|
|
|
|
defer ba.reset()
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
var merr MultiError
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// Spill buckets into partitions.
|
2016-12-20 15:02:37 -08:00
|
|
|
for s, b := range ba.buckets {
|
2017-01-05 23:08:02 -08:00
|
|
|
merr.Add(ba.db.partitions[s].appendBatch(b))
|
2016-12-20 15:02:37 -08:00
|
|
|
}
|
|
|
|
return merr.Err()
|
2016-12-09 04:41:38 -08:00
|
|
|
}
|
|
|
|
|
2016-12-20 15:02:37 -08:00
|
|
|
type hashedSample struct {
|
|
|
|
hash uint64
|
2016-12-21 00:39:01 -08:00
|
|
|
labels labels.Labels
|
2016-12-22 03:05:24 -08:00
|
|
|
ref uint32
|
2016-12-20 15:02:37 -08:00
|
|
|
|
|
|
|
t int64
|
|
|
|
v float64
|
2016-12-09 07:54:38 -08:00
|
|
|
}
|
|
|
|
|
2016-12-09 01:00:14 -08:00
|
|
|
const sep = '\xff'
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// Partition handles reads and writes of time series falling into
|
|
|
|
// a hashed partition of a series.
|
|
|
|
type Partition struct {
|
2017-01-02 13:24:35 -08:00
|
|
|
path string
|
|
|
|
logger log.Logger
|
2017-01-05 23:08:02 -08:00
|
|
|
metrics *partitionMetrics
|
2016-12-09 01:00:14 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
mtx sync.RWMutex
|
2017-01-03 06:43:26 -08:00
|
|
|
persisted []*persistedBlock
|
|
|
|
heads []*HeadBlock
|
2017-01-02 01:34:55 -08:00
|
|
|
compactor *compactor
|
2017-01-02 13:24:35 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
donec chan struct{}
|
|
|
|
cutc chan struct{}
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
type partitionMetrics struct {
|
2016-12-31 00:48:49 -08:00
|
|
|
persistences prometheus.Counter
|
|
|
|
persistenceDuration prometheus.Histogram
|
|
|
|
samplesAppended prometheus.Counter
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func newPartitionMetrics(r prometheus.Registerer, i int) *partitionMetrics {
|
|
|
|
partitionLabel := prometheus.Labels{
|
|
|
|
"partition": fmt.Sprintf("%d", i),
|
2016-12-31 00:48:49 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
m := &partitionMetrics{}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
|
|
|
m.persistences = prometheus.NewCounter(prometheus.CounterOpts{
|
2017-01-05 23:08:02 -08:00
|
|
|
Name: "tsdb_partition_persistences_total",
|
2017-01-03 06:43:26 -08:00
|
|
|
Help: "Total number of head persistances that ran so far.",
|
2017-01-05 23:08:02 -08:00
|
|
|
ConstLabels: partitionLabel,
|
2017-01-03 06:43:26 -08:00
|
|
|
})
|
|
|
|
m.persistenceDuration = prometheus.NewHistogram(prometheus.HistogramOpts{
|
2017-01-05 23:08:02 -08:00
|
|
|
Name: "tsdb_partition_persistence_duration_seconds",
|
2017-01-03 06:43:26 -08:00
|
|
|
Help: "Duration of persistences in seconds.",
|
2017-01-05 23:08:02 -08:00
|
|
|
ConstLabels: partitionLabel,
|
2017-01-03 06:43:26 -08:00
|
|
|
Buckets: prometheus.ExponentialBuckets(0.25, 2, 5),
|
|
|
|
})
|
|
|
|
m.samplesAppended = prometheus.NewCounter(prometheus.CounterOpts{
|
2017-01-05 23:08:02 -08:00
|
|
|
Name: "tsdb_partition_samples_appended_total",
|
|
|
|
Help: "Total number of appended samples for the partition.",
|
|
|
|
ConstLabels: partitionLabel,
|
2017-01-03 06:43:26 -08:00
|
|
|
})
|
2016-12-31 00:48:49 -08:00
|
|
|
|
|
|
|
if r != nil {
|
|
|
|
r.MustRegister(
|
|
|
|
m.persistences,
|
|
|
|
m.persistenceDuration,
|
|
|
|
m.samplesAppended,
|
|
|
|
)
|
|
|
|
}
|
|
|
|
return m
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// OpenPartition returns a new Partition.
|
|
|
|
func OpenPartition(path string, i int, logger log.Logger) (*Partition, error) {
|
|
|
|
// Create directory if partition is new.
|
2016-12-14 23:31:26 -08:00
|
|
|
if _, err := os.Stat(path); os.IsNotExist(err) {
|
|
|
|
if err := os.MkdirAll(path, 0777); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Initialize previously persisted blocks.
|
2017-01-03 06:43:26 -08:00
|
|
|
persisted, heads, err := findBlocks(path)
|
2016-12-14 23:31:26 -08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2016-12-22 03:05:24 -08:00
|
|
|
// TODO(fabxc): get time from client-defined `now` function.
|
2016-12-27 23:50:20 -08:00
|
|
|
baset := time.Unix(0, 0).UnixNano() / int64(time.Millisecond)
|
2017-01-03 06:43:26 -08:00
|
|
|
if len(persisted) > 0 {
|
|
|
|
baset = persisted[len(persisted)-1].bstats.MaxTime
|
2016-12-22 03:05:24 -08:00
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
if len(heads) == 0 {
|
|
|
|
head, err := OpenHeadBlock(filepath.Join(path, fmt.Sprintf("%d", baset)), baset)
|
2016-12-22 06:54:39 -08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
heads = []*HeadBlock{head}
|
2016-12-22 03:05:24 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
s := &Partition{
|
2016-12-09 04:41:38 -08:00
|
|
|
path: path,
|
|
|
|
logger: logger,
|
2017-01-05 23:08:02 -08:00
|
|
|
metrics: newPartitionMetrics(nil, i),
|
2017-01-03 06:43:26 -08:00
|
|
|
heads: heads,
|
|
|
|
persisted: persisted,
|
|
|
|
cutc: make(chan struct{}, 1),
|
2017-01-02 13:24:35 -08:00
|
|
|
donec: make(chan struct{}),
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
if s.compactor, err = newCompactor(i, s, logger); err != nil {
|
2017-01-02 01:34:55 -08:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-01-02 13:24:35 -08:00
|
|
|
go s.run()
|
2017-01-02 01:34:55 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
return s, nil
|
2016-12-09 04:41:38 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) run() {
|
2017-01-03 06:43:26 -08:00
|
|
|
for range s.cutc {
|
|
|
|
// if err := s.cut(); err != nil {
|
|
|
|
// s.logger.Log("msg", "cut error", "err", err)
|
|
|
|
// }
|
|
|
|
// select {
|
|
|
|
// case <-s.cutc:
|
|
|
|
// default:
|
|
|
|
// }
|
|
|
|
// start := time.Now()
|
|
|
|
|
|
|
|
// if err := s.persist(); err != nil {
|
|
|
|
// s.logger.Log("msg", "persistence error", "err", err)
|
|
|
|
// }
|
|
|
|
|
|
|
|
// s.metrics.persistenceDuration.Observe(time.Since(start).Seconds())
|
|
|
|
// s.metrics.persistences.Inc()
|
2017-01-02 13:24:35 -08:00
|
|
|
}
|
|
|
|
close(s.donec)
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// Close the partition.
|
|
|
|
func (s *Partition) Close() error {
|
2017-01-03 06:43:26 -08:00
|
|
|
close(s.cutc)
|
2017-01-02 13:24:35 -08:00
|
|
|
<-s.donec
|
2016-12-28 02:41:44 -08:00
|
|
|
|
2017-01-02 13:24:35 -08:00
|
|
|
var merr MultiError
|
|
|
|
merr.Add(s.compactor.Close())
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2017-01-02 13:24:35 -08:00
|
|
|
s.mtx.Lock()
|
|
|
|
defer s.mtx.Unlock()
|
2017-01-02 01:34:55 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
for _, pb := range s.persisted {
|
2017-01-02 13:24:35 -08:00
|
|
|
merr.Add(pb.Close())
|
2016-12-14 23:31:26 -08:00
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
for _, hb := range s.heads {
|
|
|
|
merr.Add(hb.Close())
|
|
|
|
}
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2017-01-02 13:24:35 -08:00
|
|
|
return merr.Err()
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) appendBatch(samples []hashedSample) error {
|
2016-12-31 06:35:08 -08:00
|
|
|
if len(samples) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
s.mtx.Lock()
|
|
|
|
defer s.mtx.Unlock()
|
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
head := s.heads[len(s.heads)-1]
|
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
// TODO(fabxc): distinguish samples between concurrent heads for
|
|
|
|
// different time blocks. Those may occurr during transition to still
|
|
|
|
// allow late samples to arrive for a previous block.
|
2017-01-03 06:43:26 -08:00
|
|
|
err := head.appendBatch(samples)
|
2016-12-31 06:35:08 -08:00
|
|
|
if err == nil {
|
2016-12-31 01:19:02 -08:00
|
|
|
s.metrics.samplesAppended.Add(float64(len(samples)))
|
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
// TODO(fabxc): randomize over time and use better scoring function.
|
2017-01-04 12:11:15 -08:00
|
|
|
if head.bstats.SampleCount/(uint64(head.bstats.ChunkCount)+1) > 250 {
|
2017-01-03 06:43:26 -08:00
|
|
|
if err := s.cut(); err != nil {
|
|
|
|
s.logger.Log("msg", "cut failed", "err", err)
|
2016-12-09 04:41:38 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
return err
|
2016-12-09 04:41:38 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) lock() sync.Locker {
|
2017-01-03 06:43:26 -08:00
|
|
|
return &s.mtx
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) headForDir(dir string) (int, bool) {
|
2017-01-03 06:43:26 -08:00
|
|
|
for i, b := range s.heads {
|
|
|
|
if b.dir() == dir {
|
|
|
|
return i, true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return -1, false
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) persistedForDir(dir string) (int, bool) {
|
2017-01-03 06:43:26 -08:00
|
|
|
for i, b := range s.persisted {
|
|
|
|
if b.dir() == dir {
|
|
|
|
return i, true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return -1, false
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) reinit(dir string) error {
|
2017-01-03 06:43:26 -08:00
|
|
|
if !fileutil.Exist(dir) {
|
|
|
|
if i, ok := s.headForDir(dir); ok {
|
|
|
|
if err := s.heads[i].Close(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.heads = append(s.heads[:i], s.heads[i+1:]...)
|
|
|
|
}
|
|
|
|
if i, ok := s.persistedForDir(dir); ok {
|
|
|
|
if err := s.persisted[i].Close(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.persisted = append(s.persisted[:i], s.persisted[i+1:]...)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Remove a previous head block.
|
|
|
|
if i, ok := s.headForDir(dir); ok {
|
|
|
|
if err := s.heads[i].Close(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.heads = append(s.heads[:i], s.heads[i+1:]...)
|
|
|
|
}
|
|
|
|
// Close an old persisted block.
|
|
|
|
i, ok := s.persistedForDir(dir)
|
|
|
|
if ok {
|
|
|
|
if err := s.persisted[i].Close(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
pb, err := newPersistedBlock(dir)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrap(err, "open persisted block")
|
|
|
|
}
|
|
|
|
if i >= 0 {
|
|
|
|
s.persisted[i] = pb
|
|
|
|
} else {
|
|
|
|
s.persisted = append(s.persisted, pb)
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) compactable() []block {
|
2017-01-03 06:43:26 -08:00
|
|
|
var blocks []block
|
|
|
|
for _, pb := range s.persisted {
|
2017-01-04 12:11:15 -08:00
|
|
|
blocks = append([]block{pb}, blocks...)
|
2017-01-03 06:43:26 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// threshold := s.heads[len(s.heads)-1].bstats.MaxTime - headGracePeriod
|
|
|
|
|
|
|
|
// for _, hb := range s.heads {
|
|
|
|
// if hb.bstats.MaxTime < threshold {
|
|
|
|
// blocks = append(blocks, hb)
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
for _, hb := range s.heads[:len(s.heads)-1] {
|
2017-01-04 12:11:15 -08:00
|
|
|
blocks = append([]block{hb}, blocks...)
|
2017-01-03 06:43:26 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
return blocks
|
|
|
|
}
|
|
|
|
|
2016-12-15 07:14:33 -08:00
|
|
|
func intervalOverlap(amin, amax, bmin, bmax int64) bool {
|
|
|
|
if bmin >= amin && bmin <= amax {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
if amin >= bmin && amin <= bmax {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2016-12-16 03:13:17 -08:00
|
|
|
func intervalContains(min, max, t int64) bool {
|
|
|
|
return t >= min && t <= max
|
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// blocksForRange returns all blocks within the partition that may contain
|
2016-12-13 06:26:58 -08:00
|
|
|
// data for the given time range.
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) blocksForInterval(mint, maxt int64) []block {
|
2016-12-15 07:14:33 -08:00
|
|
|
var bs []block
|
|
|
|
|
|
|
|
for _, b := range s.persisted {
|
|
|
|
bmin, bmax := b.interval()
|
|
|
|
|
|
|
|
if intervalOverlap(mint, maxt, bmin, bmax) {
|
|
|
|
bs = append(bs, b)
|
|
|
|
}
|
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
for _, b := range s.heads {
|
|
|
|
bmin, bmax := b.interval()
|
2016-12-15 07:14:33 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
if intervalOverlap(mint, maxt, bmin, bmax) {
|
|
|
|
bs = append(bs, b)
|
|
|
|
}
|
2016-12-15 07:14:33 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
return bs
|
2016-12-13 06:26:58 -08:00
|
|
|
}
|
|
|
|
|
2016-12-09 04:41:38 -08:00
|
|
|
// TODO(fabxc): make configurable.
|
2017-01-03 06:43:26 -08:00
|
|
|
const headGracePeriod = 60 * 1000 // 60 seconds for millisecond scale
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
// cut starts a new head block to append to. The completed head block
|
|
|
|
// will still be appendable for the configured grace period.
|
2017-01-05 23:08:02 -08:00
|
|
|
func (s *Partition) cut() error {
|
2016-12-09 04:41:38 -08:00
|
|
|
// Set new head block.
|
2017-01-03 06:43:26 -08:00
|
|
|
head := s.heads[len(s.heads)-1]
|
|
|
|
|
|
|
|
newHead, err := OpenHeadBlock(filepath.Join(s.path, fmt.Sprintf("%d", head.bstats.MaxTime)), head.bstats.MaxTime)
|
2016-12-22 03:05:24 -08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
s.heads = append(s.heads, newHead)
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
s.compactor.trigger()
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
return nil
|
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// func (s *Partition) persist() error {
|
2017-01-03 06:43:26 -08:00
|
|
|
// s.mtx.Lock()
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
// // Set new head block.
|
|
|
|
// head := s.head
|
|
|
|
// newHead, err := OpenHeadBlock(filepath.Join(s.path, fmt.Sprintf("%d", head.bstats.MaxTime)), head.bstats.MaxTime)
|
|
|
|
// if err != nil {
|
|
|
|
// s.mtx.Unlock()
|
|
|
|
// return err
|
|
|
|
// }
|
|
|
|
// s.head = newHead
|
2016-12-18 05:43:27 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
// s.mtx.Unlock()
|
2016-12-18 05:43:27 -08:00
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// // TODO(fabxc): add grace period where we can still append to old head partition
|
2017-01-03 06:43:26 -08:00
|
|
|
// // before actually persisting it.
|
|
|
|
// dir := filepath.Join(s.path, fmt.Sprintf("%d", head.stats.MinTime))
|
2017-01-02 01:34:55 -08:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
// if err := persist(dir, head.persist); err != nil {
|
|
|
|
// return err
|
|
|
|
// }
|
|
|
|
// s.logger.Log("samples", head.stats.SampleCount, "chunks", head.stats.ChunkCount, "msg", "persisted head")
|
|
|
|
|
|
|
|
// // Reopen block as persisted block for querying.
|
|
|
|
// pb, err := newPersistedBlock(dir)
|
|
|
|
// if err != nil {
|
|
|
|
// return err
|
|
|
|
// }
|
|
|
|
|
|
|
|
// s.mtx.Lock()
|
|
|
|
// s.persisted = append(s.persisted, pb)
|
|
|
|
// s.mtx.Unlock()
|
|
|
|
|
|
|
|
// s.compactor.trigger()
|
|
|
|
|
|
|
|
// return nil
|
|
|
|
// }
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2016-12-09 01:00:14 -08:00
|
|
|
// chunkDesc wraps a plain data chunk and provides cached meta data about it.
|
|
|
|
type chunkDesc struct {
|
2016-12-31 01:19:02 -08:00
|
|
|
ref uint32
|
2016-12-21 00:39:01 -08:00
|
|
|
lset labels.Labels
|
2016-12-09 01:00:14 -08:00
|
|
|
chunk chunks.Chunk
|
|
|
|
|
|
|
|
// Caching fields.
|
2017-01-04 05:06:40 -08:00
|
|
|
firstTimestamp int64
|
|
|
|
lastTimestamp int64
|
|
|
|
lastValue float64
|
|
|
|
numSamples int
|
2016-12-09 01:00:14 -08:00
|
|
|
|
|
|
|
app chunks.Appender // Current appender for the chunks.
|
|
|
|
}
|
|
|
|
|
2016-12-31 01:10:27 -08:00
|
|
|
func (cd *chunkDesc) append(ts int64, v float64) {
|
|
|
|
if cd.numSamples == 0 {
|
2017-01-04 05:06:40 -08:00
|
|
|
cd.firstTimestamp = ts
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
2016-12-31 01:10:27 -08:00
|
|
|
cd.app.Append(ts, v)
|
2016-12-09 01:00:14 -08:00
|
|
|
|
|
|
|
cd.lastTimestamp = ts
|
|
|
|
cd.lastValue = v
|
2016-12-22 11:57:00 -08:00
|
|
|
cd.numSamples++
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
// The MultiError type implements the error interface, and contains the
|
|
|
|
// Errors used to construct it.
|
|
|
|
type MultiError []error
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
// Returns a concatenated string of the contained errors
|
|
|
|
func (es MultiError) Error() string {
|
|
|
|
var buf bytes.Buffer
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2017-01-04 12:11:15 -08:00
|
|
|
if len(es) > 1 {
|
2016-12-10 09:08:50 -08:00
|
|
|
fmt.Fprintf(&buf, "%d errors: ", len(es))
|
2016-12-08 01:04:24 -08:00
|
|
|
}
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
for i, err := range es {
|
|
|
|
if i != 0 {
|
|
|
|
buf.WriteString("; ")
|
|
|
|
}
|
|
|
|
buf.WriteString(err.Error())
|
|
|
|
}
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
return buf.String()
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2016-12-15 02:56:41 -08:00
|
|
|
// Add adds the error to the error list if it is not nil.
|
2016-12-31 06:35:08 -08:00
|
|
|
func (es *MultiError) Add(err error) {
|
|
|
|
if err == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if merr, ok := err.(MultiError); ok {
|
|
|
|
*es = append(*es, merr...)
|
|
|
|
} else {
|
|
|
|
*es = append(*es, err)
|
2016-12-14 23:31:26 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-12-15 02:56:41 -08:00
|
|
|
// Err returns the error list as an error or nil if it is empty.
|
2016-12-14 23:31:26 -08:00
|
|
|
func (es MultiError) Err() error {
|
|
|
|
if len(es) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return es
|
|
|
|
}
|
2016-12-15 02:56:41 -08:00
|
|
|
|
|
|
|
func yoloString(b []byte) string {
|
2017-01-02 01:34:55 -08:00
|
|
|
sh := (*reflect.SliceHeader)(unsafe.Pointer(&b))
|
|
|
|
|
2016-12-15 02:56:41 -08:00
|
|
|
h := reflect.StringHeader{
|
2017-01-02 01:34:55 -08:00
|
|
|
Data: sh.Data,
|
|
|
|
Len: sh.Len,
|
2016-12-15 02:56:41 -08:00
|
|
|
}
|
|
|
|
return *((*string)(unsafe.Pointer(&h)))
|
|
|
|
}
|
|
|
|
|
|
|
|
func yoloBytes(s string) []byte {
|
|
|
|
sh := (*reflect.StringHeader)(unsafe.Pointer(&s))
|
|
|
|
|
|
|
|
h := reflect.SliceHeader{
|
|
|
|
Cap: sh.Len,
|
|
|
|
Len: sh.Len,
|
|
|
|
Data: sh.Data,
|
|
|
|
}
|
|
|
|
return *((*[]byte)(unsafe.Pointer(&h)))
|
|
|
|
}
|