2016-12-04 12:16:11 +00:00
|
|
|
package tsdb
|
|
|
|
|
|
|
|
import (
|
2017-01-16 13:18:32 +00:00
|
|
|
"fmt"
|
2017-01-04 13:06:40 +00:00
|
|
|
"math"
|
2017-01-13 15:14:40 +00:00
|
|
|
"math/rand"
|
2017-01-19 10:22:47 +00:00
|
|
|
"os"
|
2016-12-14 17:38:46 +00:00
|
|
|
"sort"
|
2016-12-04 12:16:11 +00:00
|
|
|
"sync"
|
2017-02-04 10:53:52 +00:00
|
|
|
"sync/atomic"
|
2017-01-06 14:18:06 +00:00
|
|
|
"time"
|
2016-12-04 12:16:11 +00:00
|
|
|
|
2017-01-05 14:13:01 +00:00
|
|
|
"github.com/bradfitz/slice"
|
2016-12-04 12:16:11 +00:00
|
|
|
"github.com/fabxc/tsdb/chunks"
|
2016-12-21 08:39:01 +00:00
|
|
|
"github.com/fabxc/tsdb/labels"
|
2017-01-06 14:18:06 +00:00
|
|
|
"github.com/go-kit/kit/log"
|
2017-02-27 09:46:15 +00:00
|
|
|
"github.com/oklog/ulid"
|
2017-01-19 10:22:47 +00:00
|
|
|
"github.com/pkg/errors"
|
2016-12-04 12:16:11 +00:00
|
|
|
)
|
|
|
|
|
2017-01-17 15:33:58 +00:00
|
|
|
var (
|
|
|
|
// ErrNotFound is returned if a looked up resource was not found.
|
|
|
|
ErrNotFound = fmt.Errorf("not found")
|
|
|
|
|
|
|
|
// ErrOutOfOrderSample is returned if an appended sample has a
|
|
|
|
// timestamp larger than the most recent sample.
|
|
|
|
ErrOutOfOrderSample = errors.New("out of order sample")
|
|
|
|
|
|
|
|
// ErrAmendSample is returned if an appended sample has the same timestamp
|
|
|
|
// as the most recent sample but a different value.
|
|
|
|
ErrAmendSample = errors.New("amending sample")
|
|
|
|
|
|
|
|
// ErrOutOfBounds is returned if an appended sample is out of the
|
|
|
|
// writable time range.
|
|
|
|
ErrOutOfBounds = errors.New("out of bounds")
|
|
|
|
)
|
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
// headBlock handles reads and writes of time series data within a time window.
|
|
|
|
type headBlock struct {
|
2017-02-01 14:29:48 +00:00
|
|
|
mtx sync.RWMutex
|
|
|
|
dir string
|
|
|
|
generation uint8
|
|
|
|
wal *WAL
|
2017-01-19 10:22:47 +00:00
|
|
|
|
2017-02-04 10:53:52 +00:00
|
|
|
activeWriters uint64
|
|
|
|
|
2017-02-09 00:13:16 +00:00
|
|
|
symbols map[string]struct{}
|
2016-12-22 00:12:28 +00:00
|
|
|
// descs holds all chunk descs for the head block. Each chunk implicitly
|
|
|
|
// is assigned the index as its ID.
|
2017-01-11 12:02:38 +00:00
|
|
|
series []*memSeries
|
2017-01-05 14:13:01 +00:00
|
|
|
// mapping maps a series ID to its position in an ordered list
|
|
|
|
// of all series. The orderDirty flag indicates that it has gone stale.
|
|
|
|
mapper *positionMapper
|
2016-12-22 00:12:28 +00:00
|
|
|
// hashes contains a collision map of label set hashes of chunks
|
2016-12-31 09:19:02 +00:00
|
|
|
// to their chunk descs.
|
2017-01-11 12:02:38 +00:00
|
|
|
hashes map[uint64][]*memSeries
|
2016-12-22 00:12:28 +00:00
|
|
|
|
|
|
|
values map[string]stringset // label names to possible values
|
|
|
|
postings *memPostings // postings lists for terms
|
2016-12-07 16:10:49 +00:00
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
metamtx sync.RWMutex
|
|
|
|
meta BlockMeta
|
2017-01-19 13:01:38 +00:00
|
|
|
}
|
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
func createHeadBlock(dir string, seq int, l log.Logger, mint, maxt int64) (*headBlock, error) {
|
2017-03-02 08:13:29 +00:00
|
|
|
// Make head block creation appear atomic.
|
|
|
|
tmp := dir + ".tmp"
|
|
|
|
|
|
|
|
if err := os.MkdirAll(tmp, 0777); err != nil {
|
2017-02-27 09:46:15 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
ulid, err := ulid.New(ulid.Now(), entropy)
|
|
|
|
if err != nil {
|
2017-01-19 13:01:38 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
2016-12-22 11:05:24 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
if err := writeMetaFile(tmp, &BlockMeta{
|
2017-02-27 09:46:15 +00:00
|
|
|
ULID: ulid,
|
2017-01-29 07:11:47 +00:00
|
|
|
Sequence: seq,
|
2017-02-01 14:29:48 +00:00
|
|
|
MinTime: mint,
|
|
|
|
MaxTime: maxt,
|
2017-01-29 07:11:47 +00:00
|
|
|
}); err != nil {
|
2017-01-19 13:01:38 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-03-02 08:13:29 +00:00
|
|
|
if err := renameFile(tmp, dir); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-01-19 13:01:38 +00:00
|
|
|
return openHeadBlock(dir, l)
|
2016-12-04 12:16:11 +00:00
|
|
|
}
|
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
// openHeadBlock creates a new empty head block.
|
2017-01-19 13:01:38 +00:00
|
|
|
func openHeadBlock(dir string, l log.Logger) (*headBlock, error) {
|
2017-01-17 15:33:58 +00:00
|
|
|
wal, err := OpenWAL(dir, log.NewContext(l).With("component", "wal"), 5*time.Second)
|
2016-12-22 11:05:24 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-01-19 13:01:38 +00:00
|
|
|
meta, err := readMetaFile(dir)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-12-22 11:05:24 +00:00
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
h := &headBlock{
|
2017-01-10 14:28:22 +00:00
|
|
|
dir: dir,
|
2017-01-19 13:01:38 +00:00
|
|
|
wal: wal,
|
2017-01-11 12:02:38 +00:00
|
|
|
series: []*memSeries{},
|
|
|
|
hashes: map[uint64][]*memSeries{},
|
2016-12-22 00:12:28 +00:00
|
|
|
values: map[string]stringset{},
|
|
|
|
postings: &memPostings{m: make(map[term][]uint32)},
|
2017-01-06 15:27:50 +00:00
|
|
|
mapper: newPositionMapper(nil),
|
2017-01-19 13:01:38 +00:00
|
|
|
meta: *meta,
|
2017-01-07 15:20:32 +00:00
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
|
2017-02-15 05:54:59 +00:00
|
|
|
r := wal.Reader()
|
|
|
|
|
|
|
|
for r.Next() {
|
|
|
|
series, samples := r.At()
|
|
|
|
|
|
|
|
for _, lset := range series {
|
2017-01-19 10:22:47 +00:00
|
|
|
h.create(lset.Hash(), lset)
|
|
|
|
h.meta.Stats.NumSeries++
|
2017-02-15 05:54:59 +00:00
|
|
|
}
|
|
|
|
for _, s := range samples {
|
2017-01-19 10:22:47 +00:00
|
|
|
h.series[s.ref].append(s.t, s.v)
|
2017-01-06 08:26:39 +00:00
|
|
|
|
2017-01-19 13:01:38 +00:00
|
|
|
if !h.inBounds(s.t) {
|
2017-02-15 05:54:59 +00:00
|
|
|
return nil, errors.Wrap(ErrOutOfBounds, "consume WAL")
|
2017-01-19 10:22:47 +00:00
|
|
|
}
|
|
|
|
h.meta.Stats.NumSamples++
|
2017-02-15 05:54:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if err := r.Err(); err != nil {
|
|
|
|
return nil, errors.Wrap(err, "consume WAL")
|
2016-12-22 14:18:33 +00:00
|
|
|
}
|
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
h.updateMapping()
|
2017-01-05 14:13:01 +00:00
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
return h, nil
|
2016-12-22 11:05:24 +00:00
|
|
|
}
|
|
|
|
|
2017-01-19 13:01:38 +00:00
|
|
|
// inBounds returns true if the given timestamp is within the valid
|
|
|
|
// time bounds of the block.
|
|
|
|
func (h *headBlock) inBounds(t int64) bool {
|
2017-02-01 14:29:48 +00:00
|
|
|
return t >= h.meta.MinTime && t <= h.meta.MaxTime
|
2017-01-19 13:01:38 +00:00
|
|
|
}
|
|
|
|
|
2016-12-22 11:05:24 +00:00
|
|
|
// Close syncs all data and closes underlying resources of the head block.
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headBlock) Close() error {
|
2017-03-04 15:50:48 +00:00
|
|
|
// Lock mutex and leave it locked so we panic if there's a bug causing
|
|
|
|
// the block to be used afterwards.
|
|
|
|
h.mtx.Lock()
|
|
|
|
|
2017-02-27 09:46:15 +00:00
|
|
|
if err := h.wal.Close(); err != nil {
|
2017-02-02 06:58:54 +00:00
|
|
|
return err
|
|
|
|
}
|
2017-02-27 09:46:15 +00:00
|
|
|
// Check whether the head block still exists in the underlying dir
|
2017-03-02 13:32:09 +00:00
|
|
|
// or has already been replaced with a compacted version or removed.
|
2017-02-27 09:46:15 +00:00
|
|
|
meta, err := readMetaFile(h.dir)
|
2017-03-02 13:32:09 +00:00
|
|
|
if os.IsNotExist(err) {
|
|
|
|
return nil
|
|
|
|
}
|
2017-02-27 09:46:15 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if meta.ULID == h.meta.ULID {
|
|
|
|
return writeMetaFile(h.dir, &h.meta)
|
|
|
|
}
|
|
|
|
return nil
|
2016-12-09 09:41:51 +00:00
|
|
|
}
|
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
func (h *headBlock) Meta() BlockMeta {
|
|
|
|
h.metamtx.RLock()
|
|
|
|
defer h.metamtx.RUnlock()
|
|
|
|
|
|
|
|
return h.meta
|
|
|
|
}
|
|
|
|
|
2017-02-23 09:50:22 +00:00
|
|
|
func (h *headBlock) Dir() string { return h.dir }
|
|
|
|
func (h *headBlock) Persisted() bool { return false }
|
|
|
|
func (h *headBlock) Index() IndexReader { return &headIndexReader{h} }
|
|
|
|
func (h *headBlock) Chunks() ChunkReader { return &headChunkReader{h} }
|
2017-01-10 14:28:22 +00:00
|
|
|
|
2017-01-12 18:18:51 +00:00
|
|
|
func (h *headBlock) Appender() Appender {
|
2017-02-04 10:53:52 +00:00
|
|
|
atomic.AddUint64(&h.activeWriters, 1)
|
|
|
|
|
2017-01-12 18:18:51 +00:00
|
|
|
h.mtx.RLock()
|
2017-01-12 19:00:36 +00:00
|
|
|
return &headAppender{headBlock: h, samples: getHeadAppendBuffer()}
|
|
|
|
}
|
|
|
|
|
|
|
|
var headPool = sync.Pool{}
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
func getHeadAppendBuffer() []refdSample {
|
2017-01-12 19:00:36 +00:00
|
|
|
b := headPool.Get()
|
|
|
|
if b == nil {
|
2017-01-13 15:14:40 +00:00
|
|
|
return make([]refdSample, 0, 512)
|
2017-01-12 19:00:36 +00:00
|
|
|
}
|
2017-01-13 15:14:40 +00:00
|
|
|
return b.([]refdSample)
|
2017-01-12 19:00:36 +00:00
|
|
|
}
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
func putHeadAppendBuffer(b []refdSample) {
|
2017-01-12 19:00:36 +00:00
|
|
|
headPool.Put(b[:0])
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
type headAppender struct {
|
|
|
|
*headBlock
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
newSeries map[uint64]hashedLabels
|
|
|
|
newHashes map[uint64]uint64
|
|
|
|
refmap map[uint64]uint64
|
2017-01-12 18:18:51 +00:00
|
|
|
newLabels []labels.Labels
|
2017-01-13 14:25:11 +00:00
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
samples []refdSample
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
type hashedLabels struct {
|
|
|
|
hash uint64
|
|
|
|
labels labels.Labels
|
|
|
|
}
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
type refdSample struct {
|
|
|
|
ref uint64
|
|
|
|
t int64
|
|
|
|
v float64
|
|
|
|
}
|
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (uint64, error) {
|
2017-03-06 13:27:33 +00:00
|
|
|
hash := lset.Hash()
|
2017-01-12 18:18:51 +00:00
|
|
|
|
|
|
|
if ms := a.get(hash, lset); ms != nil {
|
2017-02-02 10:09:19 +00:00
|
|
|
return uint64(ms.ref), a.AddFast(uint64(ms.ref), t, v)
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2017-01-13 14:25:11 +00:00
|
|
|
if ref, ok := a.newHashes[hash]; ok {
|
2017-02-02 10:09:19 +00:00
|
|
|
return uint64(ref), a.AddFast(uint64(ref), t, v)
|
2017-01-13 14:25:11 +00:00
|
|
|
}
|
2017-01-12 18:18:51 +00:00
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
// We only know the actual reference after committing. We generate an
|
|
|
|
// intermediate reference only valid for this batch.
|
|
|
|
// It is indicated by the the LSB of the 4th byte being set to 1.
|
|
|
|
// We use a random ID to avoid collisions when new series are created
|
2017-01-17 15:33:58 +00:00
|
|
|
// in two subsequent batches.
|
|
|
|
// TODO(fabxc): Provide method for client to determine whether a ref
|
|
|
|
// is valid beyond the current transaction.
|
2017-01-13 15:14:40 +00:00
|
|
|
ref := uint64(rand.Int31()) | (1 << 32)
|
|
|
|
|
2017-01-12 18:18:51 +00:00
|
|
|
if a.newSeries == nil {
|
2017-01-13 15:14:40 +00:00
|
|
|
a.newSeries = map[uint64]hashedLabels{}
|
|
|
|
a.newHashes = map[uint64]uint64{}
|
|
|
|
a.refmap = map[uint64]uint64{}
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2017-01-13 15:14:40 +00:00
|
|
|
a.newSeries[ref] = hashedLabels{hash: hash, labels: lset}
|
|
|
|
a.newHashes[hash] = ref
|
2017-01-12 18:18:51 +00:00
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
return ref, a.AddFast(ref, t, v)
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
func (a *headAppender) AddFast(ref uint64, t int64, v float64) error {
|
2017-01-19 10:22:47 +00:00
|
|
|
// We only own the last 5 bytes of the reference. Anything before is
|
2017-01-13 15:14:40 +00:00
|
|
|
// used by higher-order appenders. We erase it to avoid issues.
|
2017-01-17 07:40:31 +00:00
|
|
|
ref = (ref << 24) >> 24
|
2017-01-12 18:18:51 +00:00
|
|
|
|
|
|
|
// Distinguish between existing series and series created in
|
|
|
|
// this transaction.
|
2017-01-17 07:40:31 +00:00
|
|
|
if ref&(1<<32) != 0 {
|
2017-01-13 15:14:40 +00:00
|
|
|
if _, ok := a.newSeries[ref]; !ok {
|
2017-01-16 13:18:32 +00:00
|
|
|
return ErrNotFound
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
// TODO(fabxc): we also have to validate here that the
|
|
|
|
// sample sequence is valid.
|
|
|
|
// We also have to revalidate it as we switch locks an create
|
|
|
|
// the new series.
|
2017-01-17 07:40:31 +00:00
|
|
|
} else {
|
|
|
|
ms := a.series[int(ref)]
|
|
|
|
if ms == nil {
|
|
|
|
return ErrNotFound
|
|
|
|
}
|
|
|
|
// TODO(fabxc): memory series should be locked here already.
|
|
|
|
// Only problem is release of locks in case of a rollback.
|
2017-01-19 14:03:57 +00:00
|
|
|
c := ms.head()
|
|
|
|
|
|
|
|
if !a.inBounds(t) {
|
2017-02-01 14:29:48 +00:00
|
|
|
return ErrOutOfBounds
|
2017-01-19 14:03:57 +00:00
|
|
|
}
|
2017-01-17 07:40:31 +00:00
|
|
|
if t < c.maxTime {
|
|
|
|
return ErrOutOfOrderSample
|
|
|
|
}
|
|
|
|
if c.maxTime == t && ms.lastValue != v {
|
|
|
|
return ErrAmendSample
|
|
|
|
}
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
a.samples = append(a.samples, refdSample{
|
|
|
|
ref: ref,
|
2017-01-12 18:18:51 +00:00
|
|
|
t: t,
|
|
|
|
v: v,
|
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) createSeries() {
|
|
|
|
if len(a.newSeries) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
a.newLabels = make([]labels.Labels, 0, len(a.newSeries))
|
2017-01-13 15:14:40 +00:00
|
|
|
base0 := len(a.series)
|
2017-01-12 18:18:51 +00:00
|
|
|
|
|
|
|
a.mtx.RUnlock()
|
|
|
|
a.mtx.Lock()
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
base1 := len(a.series)
|
|
|
|
|
|
|
|
for ref, l := range a.newSeries {
|
2017-01-12 18:18:51 +00:00
|
|
|
// We switched locks and have to re-validate that the series were not
|
|
|
|
// created by another goroutine in the meantime.
|
2017-01-13 15:14:40 +00:00
|
|
|
if base1 > base0 {
|
|
|
|
if ms := a.get(l.hash, l.labels); ms != nil {
|
|
|
|
a.refmap[ref] = uint64(ms.ref)
|
|
|
|
continue
|
|
|
|
}
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
// Series is still new.
|
|
|
|
a.newLabels = append(a.newLabels, l.labels)
|
2017-01-13 15:14:40 +00:00
|
|
|
a.refmap[ref] = uint64(len(a.series))
|
2017-01-12 18:18:51 +00:00
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
a.create(l.hash, l.labels)
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
a.mtx.Unlock()
|
|
|
|
a.mtx.RLock()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) Commit() error {
|
2017-02-04 10:53:52 +00:00
|
|
|
defer atomic.AddUint64(&a.activeWriters, ^uint64(0))
|
2017-01-12 19:00:36 +00:00
|
|
|
defer putHeadAppendBuffer(a.samples)
|
2017-01-12 18:18:51 +00:00
|
|
|
|
|
|
|
a.createSeries()
|
2017-01-13 14:25:11 +00:00
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
for i := range a.samples {
|
|
|
|
s := &a.samples[i]
|
|
|
|
|
|
|
|
if s.ref&(1<<32) > 0 {
|
|
|
|
s.ref = a.refmap[s.ref]
|
|
|
|
}
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2017-01-17 07:40:31 +00:00
|
|
|
|
|
|
|
// Write all new series and samples to the WAL and add it to the
|
|
|
|
// in-mem database on success.
|
|
|
|
if err := a.wal.Log(a.newLabels, a.samples); err != nil {
|
2017-01-19 10:22:47 +00:00
|
|
|
a.mtx.RUnlock()
|
2017-01-17 07:40:31 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2017-01-19 13:01:38 +00:00
|
|
|
var (
|
|
|
|
total = uint64(len(a.samples))
|
|
|
|
mint = int64(math.MaxInt64)
|
|
|
|
maxt = int64(math.MinInt64)
|
|
|
|
)
|
|
|
|
|
2017-01-17 15:33:58 +00:00
|
|
|
for _, s := range a.samples {
|
|
|
|
if !a.series[s.ref].append(s.t, s.v) {
|
|
|
|
total--
|
|
|
|
}
|
2017-02-02 10:09:19 +00:00
|
|
|
|
2017-01-19 13:01:38 +00:00
|
|
|
if s.t < mint {
|
|
|
|
mint = s.t
|
|
|
|
}
|
|
|
|
if s.t > maxt {
|
|
|
|
maxt = s.t
|
|
|
|
}
|
2017-01-17 15:33:58 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
a.mtx.RUnlock()
|
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
a.metamtx.Lock()
|
|
|
|
defer a.metamtx.Unlock()
|
2017-01-12 18:18:51 +00:00
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
a.meta.Stats.NumSamples += total
|
|
|
|
a.meta.Stats.NumSeries += uint64(len(a.newSeries))
|
2017-01-12 18:18:51 +00:00
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) Rollback() error {
|
|
|
|
a.mtx.RUnlock()
|
2017-02-04 10:53:52 +00:00
|
|
|
atomic.AddUint64(&a.activeWriters, ^uint64(0))
|
|
|
|
putHeadAppendBuffer(a.samples)
|
2017-01-12 18:18:51 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-02-23 09:50:22 +00:00
|
|
|
type headChunkReader struct {
|
2017-01-10 14:28:22 +00:00
|
|
|
*headBlock
|
2017-01-07 17:02:17 +00:00
|
|
|
}
|
2016-12-14 17:38:46 +00:00
|
|
|
|
2016-12-15 15:14:33 +00:00
|
|
|
// Chunk returns the chunk for the reference number.
|
2017-02-23 09:50:22 +00:00
|
|
|
func (h *headChunkReader) Chunk(ref uint64) (chunks.Chunk, error) {
|
2017-01-06 16:23:12 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2017-02-18 16:33:20 +00:00
|
|
|
si := ref >> 32
|
|
|
|
ci := (ref << 32) >> 32
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
c := &safeChunk{
|
2017-02-18 16:33:20 +00:00
|
|
|
Chunk: h.series[si].chunks[ci].chunk,
|
|
|
|
s: h.series[si],
|
|
|
|
i: int(ci),
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
2017-01-11 12:02:38 +00:00
|
|
|
return c, nil
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
|
|
|
|
2017-01-09 15:51:39 +00:00
|
|
|
type safeChunk struct {
|
2017-01-11 12:02:38 +00:00
|
|
|
chunks.Chunk
|
|
|
|
s *memSeries
|
|
|
|
i int
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (c *safeChunk) Iterator() chunks.Iterator {
|
2017-01-11 12:02:38 +00:00
|
|
|
c.s.mtx.RLock()
|
|
|
|
defer c.s.mtx.RUnlock()
|
|
|
|
return c.s.iterator(c.i)
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
// func (c *safeChunk) Appender() (chunks.Appender, error) { panic("illegal") }
|
|
|
|
// func (c *safeChunk) Bytes() []byte { panic("illegal") }
|
|
|
|
// func (c *safeChunk) Encoding() chunks.Encoding { panic("illegal") }
|
2017-01-09 15:51:39 +00:00
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
type headIndexReader struct {
|
|
|
|
*headBlock
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// LabelValues returns the possible label values
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headIndexReader) LabelValues(names ...string) (StringTuples, error) {
|
2017-01-06 16:23:12 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-14 17:38:46 +00:00
|
|
|
if len(names) != 1 {
|
|
|
|
return nil, errInvalidSize
|
|
|
|
}
|
|
|
|
var sl []string
|
|
|
|
|
2016-12-22 00:12:28 +00:00
|
|
|
for s := range h.values[names[0]] {
|
2016-12-14 17:38:46 +00:00
|
|
|
sl = append(sl, s)
|
|
|
|
}
|
|
|
|
sort.Strings(sl)
|
|
|
|
|
2017-01-05 14:13:01 +00:00
|
|
|
return &stringTuples{l: len(names), s: sl}, nil
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Postings returns the postings list iterator for the label pair.
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headIndexReader) Postings(name, value string) (Postings, error) {
|
2017-01-06 16:23:12 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-22 00:12:28 +00:00
|
|
|
return h.postings.get(term{name: name, value: value}), nil
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Series returns the series for the given reference.
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headIndexReader) Series(ref uint32) (labels.Labels, []ChunkMeta, error) {
|
2017-01-06 16:23:12 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
if int(ref) >= len(h.series) {
|
2017-01-16 13:18:32 +00:00
|
|
|
return nil, nil, ErrNotFound
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
2017-01-11 12:02:38 +00:00
|
|
|
s := h.series[ref]
|
|
|
|
metas := make([]ChunkMeta, 0, len(s.chunks))
|
|
|
|
|
|
|
|
s.mtx.RLock()
|
|
|
|
defer s.mtx.RUnlock()
|
|
|
|
|
|
|
|
for i, c := range s.chunks {
|
|
|
|
metas = append(metas, ChunkMeta{
|
|
|
|
MinTime: c.minTime,
|
|
|
|
MaxTime: c.maxTime,
|
2017-02-18 16:33:20 +00:00
|
|
|
Ref: (uint64(ref) << 32) | uint64(i),
|
2017-01-11 12:02:38 +00:00
|
|
|
})
|
2017-01-03 14:43:26 +00:00
|
|
|
}
|
2017-01-11 12:02:38 +00:00
|
|
|
|
|
|
|
return s.lset, metas, nil
|
2016-12-31 14:35:08 +00:00
|
|
|
}
|
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headIndexReader) LabelIndices() ([][]string, error) {
|
2017-01-06 16:23:12 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-31 14:35:08 +00:00
|
|
|
res := [][]string{}
|
|
|
|
|
|
|
|
for s := range h.values {
|
|
|
|
res = append(res, []string{s})
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
2016-12-31 14:35:08 +00:00
|
|
|
return res, nil
|
2016-12-14 17:38:46 +00:00
|
|
|
}
|
|
|
|
|
2016-12-04 12:16:11 +00:00
|
|
|
// get retrieves the chunk with the hash and label set and creates
|
|
|
|
// a new one if it doesn't exist yet.
|
2017-01-11 12:02:38 +00:00
|
|
|
func (h *headBlock) get(hash uint64, lset labels.Labels) *memSeries {
|
|
|
|
series := h.hashes[hash]
|
2016-12-22 00:12:28 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
for _, s := range series {
|
|
|
|
if s.lset.Equals(lset) {
|
|
|
|
return s
|
2016-12-04 12:16:11 +00:00
|
|
|
}
|
|
|
|
}
|
2016-12-31 09:19:02 +00:00
|
|
|
return nil
|
2016-12-22 11:05:24 +00:00
|
|
|
}
|
|
|
|
|
2017-01-13 15:14:40 +00:00
|
|
|
func (h *headBlock) create(hash uint64, lset labels.Labels) *memSeries {
|
2017-01-12 18:18:51 +00:00
|
|
|
s := &memSeries{
|
|
|
|
lset: lset,
|
2017-01-13 15:14:40 +00:00
|
|
|
ref: uint32(len(h.series)),
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2017-01-04 13:06:40 +00:00
|
|
|
|
2017-01-12 18:18:51 +00:00
|
|
|
// Allocate empty space until we can insert at the given index.
|
2017-01-13 15:14:40 +00:00
|
|
|
h.series = append(h.series, s)
|
2016-12-22 00:12:28 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
h.hashes[hash] = append(h.hashes[hash], s)
|
2016-12-22 00:12:28 +00:00
|
|
|
|
|
|
|
for _, l := range lset {
|
|
|
|
valset, ok := h.values[l.Name]
|
|
|
|
if !ok {
|
|
|
|
valset = stringset{}
|
|
|
|
h.values[l.Name] = valset
|
|
|
|
}
|
|
|
|
valset.set(l.Value)
|
2017-01-03 14:43:26 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
h.postings.add(s.ref, term{name: l.Name, value: l.Value})
|
2016-12-22 00:12:28 +00:00
|
|
|
}
|
2017-01-03 14:43:26 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
h.postings.add(s.ref, term{})
|
2016-12-04 12:16:11 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
return s
|
2016-12-04 12:16:11 +00:00
|
|
|
}
|
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headBlock) fullness() float64 {
|
2017-01-19 10:22:47 +00:00
|
|
|
h.metamtx.RLock()
|
|
|
|
defer h.metamtx.RUnlock()
|
2017-01-07 17:02:17 +00:00
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
return float64(h.meta.Stats.NumSamples) / float64(h.meta.Stats.NumSeries+1) / 250
|
2017-01-07 17:02:17 +00:00
|
|
|
}
|
|
|
|
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headBlock) updateMapping() {
|
2017-01-09 17:07:45 +00:00
|
|
|
h.mtx.RLock()
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
if h.mapper.sortable != nil && h.mapper.Len() == len(h.series) {
|
2017-01-09 17:34:29 +00:00
|
|
|
h.mtx.RUnlock()
|
2017-01-06 15:27:50 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
series := make([]*memSeries, len(h.series))
|
|
|
|
copy(series, h.series)
|
2017-01-05 14:13:01 +00:00
|
|
|
|
2017-01-09 17:07:45 +00:00
|
|
|
h.mtx.RUnlock()
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
s := slice.SortInterface(series, func(i, j int) bool {
|
|
|
|
return labels.Compare(series[i].lset, series[j].lset) < 0
|
2017-01-05 14:13:01 +00:00
|
|
|
})
|
|
|
|
|
2017-01-06 15:27:50 +00:00
|
|
|
h.mapper.update(s)
|
|
|
|
}
|
|
|
|
|
|
|
|
// remapPostings changes the order of the postings from their ID to the ordering
|
|
|
|
// of the series they reference.
|
|
|
|
// Returned postings have no longer monotonic IDs and MUST NOT be used for regular
|
|
|
|
// postings set operations, i.e. intersect and merge.
|
2017-01-10 14:28:22 +00:00
|
|
|
func (h *headBlock) remapPostings(p Postings) Postings {
|
2017-01-06 15:27:50 +00:00
|
|
|
list, err := expandPostings(p)
|
|
|
|
if err != nil {
|
|
|
|
return errPostings{err: err}
|
|
|
|
}
|
|
|
|
|
2017-01-09 18:24:05 +00:00
|
|
|
h.mapper.mtx.Lock()
|
|
|
|
defer h.mapper.mtx.Unlock()
|
2017-01-06 15:27:50 +00:00
|
|
|
|
2017-01-09 18:24:05 +00:00
|
|
|
h.updateMapping()
|
2017-01-06 15:27:50 +00:00
|
|
|
h.mapper.Sort(list)
|
|
|
|
|
|
|
|
return newListPostings(list)
|
2017-01-05 14:13:01 +00:00
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
type memSeries struct {
|
2017-01-09 15:51:39 +00:00
|
|
|
mtx sync.RWMutex
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
ref uint32
|
|
|
|
lset labels.Labels
|
|
|
|
chunks []*memChunk
|
2017-01-09 15:51:39 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
lastValue float64
|
2017-01-09 15:51:39 +00:00
|
|
|
sampleBuf [4]sample
|
|
|
|
|
2017-03-06 16:34:49 +00:00
|
|
|
app chunks.Appender // Current appender for the chunk.
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
func (s *memSeries) cut() *memChunk {
|
|
|
|
c := &memChunk{
|
|
|
|
chunk: chunks.NewXORChunk(),
|
|
|
|
maxTime: math.MinInt64,
|
|
|
|
}
|
|
|
|
s.chunks = append(s.chunks, c)
|
|
|
|
|
|
|
|
app, err := c.chunk.Appender()
|
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
s.app = app
|
|
|
|
return c
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *memSeries) append(t int64, v float64) bool {
|
|
|
|
var c *memChunk
|
|
|
|
|
2017-02-02 10:09:19 +00:00
|
|
|
if s.app == nil || s.head().samples > 2000 {
|
2017-01-11 12:02:38 +00:00
|
|
|
c = s.cut()
|
|
|
|
c.minTime = t
|
|
|
|
} else {
|
|
|
|
c = s.head()
|
|
|
|
// Skip duplicate samples.
|
|
|
|
if c.maxTime == t && s.lastValue != v {
|
|
|
|
return false
|
|
|
|
}
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
2017-01-11 12:02:38 +00:00
|
|
|
s.app.Append(t, v)
|
|
|
|
|
|
|
|
c.maxTime = t
|
|
|
|
c.samples++
|
|
|
|
|
|
|
|
s.lastValue = v
|
2017-01-09 15:51:39 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
s.sampleBuf[0] = s.sampleBuf[1]
|
|
|
|
s.sampleBuf[1] = s.sampleBuf[2]
|
|
|
|
s.sampleBuf[2] = s.sampleBuf[3]
|
|
|
|
s.sampleBuf[3] = sample{t: t, v: v}
|
2017-01-09 15:51:39 +00:00
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
return true
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
func (s *memSeries) iterator(i int) chunks.Iterator {
|
|
|
|
c := s.chunks[i]
|
|
|
|
|
|
|
|
if i < len(s.chunks)-1 {
|
|
|
|
return c.chunk.Iterator()
|
|
|
|
}
|
|
|
|
|
2017-01-09 15:51:39 +00:00
|
|
|
it := &memSafeIterator{
|
2017-01-11 12:02:38 +00:00
|
|
|
Iterator: c.chunk.Iterator(),
|
2017-01-09 15:51:39 +00:00
|
|
|
i: -1,
|
2017-01-11 12:02:38 +00:00
|
|
|
total: c.samples,
|
|
|
|
buf: s.sampleBuf,
|
2017-01-09 15:51:39 +00:00
|
|
|
}
|
|
|
|
return it
|
|
|
|
}
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
func (s *memSeries) head() *memChunk {
|
|
|
|
return s.chunks[len(s.chunks)-1]
|
|
|
|
}
|
|
|
|
|
|
|
|
type memChunk struct {
|
|
|
|
chunk chunks.Chunk
|
|
|
|
minTime, maxTime int64
|
|
|
|
samples int
|
|
|
|
}
|
|
|
|
|
2017-01-09 15:51:39 +00:00
|
|
|
type memSafeIterator struct {
|
|
|
|
chunks.Iterator
|
|
|
|
|
|
|
|
i int
|
|
|
|
total int
|
|
|
|
buf [4]sample
|
|
|
|
}
|
|
|
|
|
|
|
|
func (it *memSafeIterator) Next() bool {
|
|
|
|
if it.i+1 >= it.total {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
it.i++
|
|
|
|
if it.total-it.i > 4 {
|
|
|
|
return it.Iterator.Next()
|
|
|
|
}
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
func (it *memSafeIterator) At() (int64, float64) {
|
|
|
|
if it.total-it.i > 4 {
|
|
|
|
return it.Iterator.At()
|
|
|
|
}
|
|
|
|
s := it.buf[4-(it.total-it.i)]
|
|
|
|
return s.t, s.v
|
|
|
|
}
|
|
|
|
|
2017-01-05 14:13:01 +00:00
|
|
|
// positionMapper stores a position mapping from unsorted to
|
|
|
|
// sorted indices of a sortable collection.
|
|
|
|
type positionMapper struct {
|
2017-01-06 15:27:50 +00:00
|
|
|
mtx sync.RWMutex
|
2017-01-05 14:13:01 +00:00
|
|
|
sortable sort.Interface
|
|
|
|
iv, fw []int
|
|
|
|
}
|
|
|
|
|
|
|
|
func newPositionMapper(s sort.Interface) *positionMapper {
|
2017-01-06 15:27:50 +00:00
|
|
|
m := &positionMapper{}
|
|
|
|
if s != nil {
|
|
|
|
m.update(s)
|
2017-01-05 14:13:01 +00:00
|
|
|
}
|
|
|
|
return m
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *positionMapper) Len() int { return m.sortable.Len() }
|
|
|
|
func (m *positionMapper) Less(i, j int) bool { return m.sortable.Less(i, j) }
|
|
|
|
|
|
|
|
func (m *positionMapper) Swap(i, j int) {
|
|
|
|
m.sortable.Swap(i, j)
|
|
|
|
|
|
|
|
m.iv[i], m.iv[j] = m.iv[j], m.iv[i]
|
|
|
|
}
|
2017-01-06 15:27:50 +00:00
|
|
|
|
|
|
|
func (m *positionMapper) Sort(l []uint32) {
|
|
|
|
slice.Sort(l, func(i, j int) bool {
|
|
|
|
return m.fw[l[i]] < m.fw[l[j]]
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *positionMapper) update(s sort.Interface) {
|
|
|
|
m.sortable = s
|
|
|
|
|
|
|
|
m.iv = make([]int, s.Len())
|
|
|
|
m.fw = make([]int, s.Len())
|
|
|
|
|
|
|
|
for i := range m.iv {
|
|
|
|
m.iv[i] = i
|
|
|
|
}
|
|
|
|
sort.Sort(m)
|
|
|
|
|
|
|
|
for i, k := range m.iv {
|
|
|
|
m.fw[k] = i
|
|
|
|
}
|
|
|
|
}
|