2016-12-04 04:16:11 -08:00
|
|
|
package tsdb
|
|
|
|
|
|
|
|
import (
|
2016-12-31 06:35:08 -08:00
|
|
|
"errors"
|
2017-01-04 05:06:40 -08:00
|
|
|
"math"
|
2016-12-14 09:38:46 -08:00
|
|
|
"sort"
|
2016-12-04 04:16:11 -08:00
|
|
|
"sync"
|
2017-01-12 10:18:51 -08:00
|
|
|
"sync/atomic"
|
2017-01-06 06:18:06 -08:00
|
|
|
"time"
|
2016-12-04 04:16:11 -08:00
|
|
|
|
2017-01-05 06:13:01 -08:00
|
|
|
"github.com/bradfitz/slice"
|
2016-12-04 04:16:11 -08:00
|
|
|
"github.com/fabxc/tsdb/chunks"
|
2016-12-21 00:39:01 -08:00
|
|
|
"github.com/fabxc/tsdb/labels"
|
2017-01-06 06:18:06 -08:00
|
|
|
"github.com/go-kit/kit/log"
|
2016-12-04 04:16:11 -08:00
|
|
|
)
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
// headBlock handles reads and writes of time series data within a time window.
|
|
|
|
type headBlock struct {
|
2016-12-21 16:12:28 -08:00
|
|
|
mtx sync.RWMutex
|
2017-01-10 06:28:22 -08:00
|
|
|
dir string
|
2016-12-21 16:12:28 -08:00
|
|
|
|
|
|
|
// descs holds all chunk descs for the head block. Each chunk implicitly
|
|
|
|
// is assigned the index as its ID.
|
2017-01-11 04:02:38 -08:00
|
|
|
series []*memSeries
|
2017-01-05 06:13:01 -08: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-21 16:12:28 -08:00
|
|
|
// hashes contains a collision map of label set hashes of chunks
|
2016-12-31 01:19:02 -08:00
|
|
|
// to their chunk descs.
|
2017-01-11 04:02:38 -08:00
|
|
|
hashes map[uint64][]*memSeries
|
2016-12-21 16:12:28 -08:00
|
|
|
|
2017-01-12 10:18:51 -08:00
|
|
|
nextSeriesID uint64
|
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
values map[string]stringset // label names to possible values
|
|
|
|
postings *memPostings // postings lists for terms
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-22 03:05:24 -08:00
|
|
|
wal *WAL
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
stats *BlockStats
|
2016-12-04 04:16:11 -08:00
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
// openHeadBlock creates a new empty head block.
|
|
|
|
func openHeadBlock(dir string, l log.Logger) (*headBlock, error) {
|
2017-01-06 06:18:06 -08:00
|
|
|
wal, err := OpenWAL(dir, log.NewContext(l).With("component", "wal"), 15*time.Second)
|
2016-12-22 03:05:24 -08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
b := &headBlock{
|
|
|
|
dir: dir,
|
2017-01-11 04:02:38 -08:00
|
|
|
series: []*memSeries{},
|
|
|
|
hashes: map[uint64][]*memSeries{},
|
2016-12-21 16:12:28 -08:00
|
|
|
values: map[string]stringset{},
|
|
|
|
postings: &memPostings{m: make(map[term][]uint32)},
|
2016-12-22 03:05:24 -08:00
|
|
|
wal: wal,
|
2017-01-06 07:27:50 -08:00
|
|
|
mapper: newPositionMapper(nil),
|
2016-12-09 01:41:51 -08:00
|
|
|
}
|
2017-01-10 06:28:22 -08:00
|
|
|
b.stats = &BlockStats{
|
|
|
|
MinTime: math.MinInt64,
|
|
|
|
MaxTime: math.MaxInt64,
|
2017-01-07 07:20:32 -08:00
|
|
|
}
|
2017-01-06 03:37:28 -08:00
|
|
|
|
2016-12-22 06:18:33 -08:00
|
|
|
err = wal.ReadAll(&walHandler{
|
|
|
|
series: func(lset labels.Labels) {
|
2017-01-12 10:18:51 -08:00
|
|
|
b.create(uint32(b.nextSeriesID), lset.Hash(), lset)
|
|
|
|
b.nextSeriesID++
|
2017-01-10 06:28:22 -08:00
|
|
|
b.stats.SeriesCount++
|
2016-12-22 06:18:33 -08:00
|
|
|
},
|
|
|
|
sample: func(s hashedSample) {
|
2017-01-11 04:02:38 -08:00
|
|
|
si := s.ref
|
|
|
|
|
|
|
|
cd := b.series[si]
|
2017-01-06 00:26:39 -08:00
|
|
|
cd.append(s.t, s.v)
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
if s.t > b.stats.MaxTime {
|
|
|
|
b.stats.MaxTime = s.t
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-01-10 06:28:22 -08:00
|
|
|
b.stats.SampleCount++
|
2016-12-22 06:18:33 -08:00
|
|
|
},
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2017-01-06 07:27:50 -08:00
|
|
|
b.updateMapping()
|
2017-01-05 06:13:01 -08:00
|
|
|
|
2016-12-22 03:05:24 -08:00
|
|
|
return b, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close syncs all data and closes underlying resources of the head block.
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headBlock) Close() error {
|
2016-12-22 03:05:24 -08:00
|
|
|
return h.wal.Close()
|
2016-12-09 01:41:51 -08:00
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08: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) Series() SeriesReader { return &headSeriesReader{h} }
|
|
|
|
|
|
|
|
// Stats returns statisitics about the indexed data.
|
|
|
|
func (h *headBlock) Stats() BlockStats {
|
|
|
|
h.stats.mtx.RLock()
|
|
|
|
defer h.stats.mtx.RUnlock()
|
2017-01-07 09:02:17 -08:00
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
return *h.stats
|
|
|
|
}
|
2017-01-07 09:02:17 -08:00
|
|
|
|
2017-01-12 10:18:51 -08:00
|
|
|
func (h *headBlock) Appender() Appender {
|
|
|
|
h.mtx.RLock()
|
|
|
|
return &headAppender{headBlock: h}
|
|
|
|
}
|
|
|
|
|
|
|
|
type headAppender struct {
|
|
|
|
*headBlock
|
|
|
|
|
|
|
|
newSeries map[uint32]hashedLabels
|
|
|
|
newLabels []labels.Labels
|
|
|
|
samples []hashedSample
|
|
|
|
}
|
|
|
|
|
|
|
|
type hashedLabels struct {
|
|
|
|
hash uint64
|
|
|
|
labels labels.Labels
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) SetSeries(lset labels.Labels) (uint64, error) {
|
|
|
|
return a.setSeries(lset.Hash(), lset)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) setSeries(hash uint64, lset labels.Labels) (uint64, error) {
|
|
|
|
if ms := a.get(hash, lset); ms != nil {
|
|
|
|
return uint64(ms.ref), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
id := atomic.AddUint64(&a.nextSeriesID, 1) - 1
|
|
|
|
if a.newSeries == nil {
|
|
|
|
a.newSeries = map[uint32]hashedLabels{}
|
|
|
|
}
|
|
|
|
a.newSeries[uint32(id)] = hashedLabels{hash: hash, labels: lset}
|
|
|
|
|
|
|
|
return id, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) Add(ref uint64, t int64, v float64) error {
|
|
|
|
// We only act on the last 4 bytes. Anything before is used by higher-order
|
|
|
|
// appenders. We erase it to avoid issues.
|
|
|
|
ref = (ref << 32) >> 32
|
|
|
|
|
|
|
|
// Distinguish between existing series and series created in
|
|
|
|
// this transaction.
|
|
|
|
if int(ref) >= len(a.series) {
|
|
|
|
if _, ok := a.newSeries[uint32(ref)]; !ok {
|
|
|
|
return errNotFound
|
|
|
|
}
|
|
|
|
// 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.
|
|
|
|
a.samples = append(a.samples, hashedSample{
|
|
|
|
ref: uint32(ref),
|
|
|
|
t: t,
|
|
|
|
v: v,
|
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
ms := a.series[int(ref)]
|
|
|
|
if ms == nil {
|
|
|
|
return errNotFound
|
|
|
|
}
|
|
|
|
c := ms.head()
|
|
|
|
|
|
|
|
// TODO(fabxc): memory series should be locked here already.
|
|
|
|
// Only problem is release of locks in case of a rollback.
|
|
|
|
if t < c.maxTime {
|
|
|
|
return ErrOutOfOrderSample
|
|
|
|
}
|
|
|
|
if c.maxTime == t && ms.lastValue != v {
|
|
|
|
return ErrAmendSample
|
|
|
|
}
|
|
|
|
|
|
|
|
a.samples = append(a.samples, hashedSample{
|
|
|
|
ref: uint32(ref),
|
|
|
|
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))
|
|
|
|
|
|
|
|
a.mtx.RUnlock()
|
|
|
|
a.mtx.Lock()
|
|
|
|
|
|
|
|
for id, l := range a.newSeries {
|
|
|
|
// We switched locks and have to re-validate that the series were not
|
|
|
|
// created by another goroutine in the meantime.
|
|
|
|
if int(id) < len(a.series) && a.series[id] != nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
// Series is still new.
|
|
|
|
a.newLabels = append(a.newLabels, l.labels)
|
|
|
|
|
|
|
|
a.create(id, l.hash, l.labels)
|
|
|
|
}
|
|
|
|
|
|
|
|
a.mtx.Unlock()
|
|
|
|
a.mtx.RLock()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) Commit() error {
|
|
|
|
defer a.mtx.RUnlock()
|
|
|
|
|
|
|
|
// 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 {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
a.createSeries()
|
|
|
|
var (
|
|
|
|
total = uint64(len(a.samples))
|
|
|
|
mint = int64(math.MaxInt64)
|
|
|
|
maxt = int64(math.MinInt64)
|
|
|
|
)
|
|
|
|
|
|
|
|
for _, s := range a.samples {
|
|
|
|
if !a.series[s.ref].append(s.t, s.v) {
|
|
|
|
total--
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
a.stats.mtx.Lock()
|
|
|
|
defer a.stats.mtx.Unlock()
|
|
|
|
|
|
|
|
a.stats.SampleCount += total
|
|
|
|
a.stats.SeriesCount += uint64(len(a.newSeries))
|
|
|
|
|
|
|
|
if mint < a.stats.MinTime {
|
|
|
|
a.stats.MinTime = mint
|
|
|
|
}
|
|
|
|
if maxt > a.stats.MaxTime {
|
|
|
|
a.stats.MaxTime = maxt
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (a *headAppender) Rollback() error {
|
|
|
|
a.mtx.RUnlock()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
type headSeriesReader struct {
|
|
|
|
*headBlock
|
2017-01-07 09:02:17 -08:00
|
|
|
}
|
2016-12-14 09:38:46 -08:00
|
|
|
|
2016-12-15 07:14:33 -08:00
|
|
|
// Chunk returns the chunk for the reference number.
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headSeriesReader) Chunk(ref uint32) (chunks.Chunk, error) {
|
2017-01-06 08:23:12 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
c := &safeChunk{
|
|
|
|
Chunk: h.series[ref>>8].chunks[int((ref<<24)>>24)].chunk,
|
|
|
|
s: h.series[ref>>8],
|
|
|
|
i: int((ref << 24) >> 24),
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
return c, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
|
|
|
|
2017-01-09 07:51:39 -08:00
|
|
|
type safeChunk struct {
|
2017-01-11 04:02:38 -08:00
|
|
|
chunks.Chunk
|
|
|
|
s *memSeries
|
|
|
|
i int
|
2017-01-09 07:51:39 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (c *safeChunk) Iterator() chunks.Iterator {
|
2017-01-11 04:02:38 -08:00
|
|
|
c.s.mtx.RLock()
|
|
|
|
defer c.s.mtx.RUnlock()
|
|
|
|
return c.s.iterator(c.i)
|
2017-01-09 07:51:39 -08:00
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08: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 07:51:39 -08:00
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
type headIndexReader struct {
|
|
|
|
*headBlock
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// LabelValues returns the possible label values
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headIndexReader) LabelValues(names ...string) (StringTuples, error) {
|
2017-01-06 08:23:12 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-14 09:38:46 -08:00
|
|
|
if len(names) != 1 {
|
|
|
|
return nil, errInvalidSize
|
|
|
|
}
|
|
|
|
var sl []string
|
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
for s := range h.values[names[0]] {
|
2016-12-14 09:38:46 -08:00
|
|
|
sl = append(sl, s)
|
|
|
|
}
|
|
|
|
sort.Strings(sl)
|
|
|
|
|
2017-01-05 06:13:01 -08:00
|
|
|
return &stringTuples{l: len(names), s: sl}, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Postings returns the postings list iterator for the label pair.
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headIndexReader) Postings(name, value string) (Postings, error) {
|
2017-01-06 08:23:12 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-21 16:12:28 -08:00
|
|
|
return h.postings.get(term{name: name, value: value}), nil
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Series returns the series for the given reference.
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headIndexReader) Series(ref uint32) (labels.Labels, []ChunkMeta, error) {
|
2017-01-06 08:23:12 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
if int(ref) >= len(h.series) {
|
2016-12-31 06:35:08 -08:00
|
|
|
return nil, nil, errNotFound
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
2017-01-11 04:02:38 -08: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,
|
|
|
|
Ref: (ref << 8) | uint32(i),
|
|
|
|
})
|
2017-01-03 06:43:26 -08:00
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
|
|
return s.lset, metas, nil
|
2016-12-31 06:35:08 -08:00
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headIndexReader) LabelIndices() ([][]string, error) {
|
2017-01-06 08:23:12 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
defer h.mtx.RUnlock()
|
|
|
|
|
2016-12-31 06:35:08 -08:00
|
|
|
res := [][]string{}
|
|
|
|
|
|
|
|
for s := range h.values {
|
|
|
|
res = append(res, []string{s})
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
2016-12-31 06:35:08 -08:00
|
|
|
return res, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headIndexReader) Stats() (BlockStats, error) {
|
|
|
|
h.stats.mtx.RLock()
|
|
|
|
defer h.stats.mtx.RUnlock()
|
|
|
|
return *h.stats, nil
|
|
|
|
}
|
|
|
|
|
2016-12-04 04:16:11 -08: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 04:02:38 -08:00
|
|
|
func (h *headBlock) get(hash uint64, lset labels.Labels) *memSeries {
|
|
|
|
series := h.hashes[hash]
|
2016-12-21 16:12:28 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
for _, s := range series {
|
|
|
|
if s.lset.Equals(lset) {
|
|
|
|
return s
|
2016-12-04 04:16:11 -08:00
|
|
|
}
|
|
|
|
}
|
2016-12-31 01:19:02 -08:00
|
|
|
return nil
|
2016-12-22 03:05:24 -08:00
|
|
|
}
|
|
|
|
|
2017-01-12 10:18:51 -08:00
|
|
|
func (h *headBlock) create(ref uint32, hash uint64, lset labels.Labels) *memSeries {
|
|
|
|
s := &memSeries{
|
|
|
|
ref: ref,
|
|
|
|
lset: lset,
|
|
|
|
}
|
2017-01-04 05:06:40 -08:00
|
|
|
|
2017-01-12 10:18:51 -08:00
|
|
|
// Allocate empty space until we can insert at the given index.
|
|
|
|
for int(ref) >= len(h.series) {
|
|
|
|
h.series = append(h.series, nil)
|
|
|
|
}
|
|
|
|
h.series[ref] = s
|
2016-12-21 16:12:28 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
h.hashes[hash] = append(h.hashes[hash], s)
|
2016-12-21 16:12:28 -08: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 06:43:26 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
h.postings.add(s.ref, term{name: l.Name, value: l.Value})
|
2016-12-21 16:12:28 -08:00
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
h.postings.add(s.ref, term{})
|
2016-12-04 04:16:11 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
return s
|
2016-12-04 04:16:11 -08:00
|
|
|
}
|
|
|
|
|
2016-12-31 06:35:08 -08:00
|
|
|
var (
|
2017-01-02 13:24:35 -08:00
|
|
|
// ErrOutOfOrderSample is returned if an appended sample has a
|
|
|
|
// timestamp larger than the most recent sample.
|
2016-12-31 06:35:08 -08:00
|
|
|
ErrOutOfOrderSample = errors.New("out of order sample")
|
2017-01-02 13:24:35 -08:00
|
|
|
|
|
|
|
// 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")
|
2017-01-10 06:28:22 -08:00
|
|
|
|
|
|
|
ErrOutOfBounds = errors.New("out of bounds")
|
2016-12-31 06:35:08 -08:00
|
|
|
)
|
|
|
|
|
2017-01-12 10:18:51 -08:00
|
|
|
// func (h *headBlock) appendBatch(samples []hashedSample) (int, error) {
|
|
|
|
// // Find head chunks for all samples and allocate new IDs/refs for
|
|
|
|
// // ones we haven't seen before.
|
|
|
|
|
|
|
|
// var (
|
|
|
|
// newSeries = map[uint64][]*hashedSample{}
|
|
|
|
// newLabels []labels.Labels
|
|
|
|
// )
|
|
|
|
|
|
|
|
// h.mtx.RLock()
|
|
|
|
// defer h.mtx.RUnlock()
|
|
|
|
|
|
|
|
// for i := range samples {
|
|
|
|
// s := &samples[i]
|
|
|
|
|
|
|
|
// ms := h.get(s.hash, s.labels)
|
|
|
|
// if ms != nil {
|
|
|
|
// c := ms.head()
|
|
|
|
|
|
|
|
// if s.t < c.maxTime {
|
|
|
|
// return 0, ErrOutOfOrderSample
|
|
|
|
// }
|
|
|
|
// if c.maxTime == s.t && ms.lastValue != s.v {
|
|
|
|
// return 0, ErrAmendSample
|
|
|
|
// }
|
|
|
|
// // TODO(fabxc): sample refs are only scoped within a block for
|
|
|
|
// // now and we ignore any previously set value
|
|
|
|
// s.ref = ms.ref
|
|
|
|
// continue
|
|
|
|
// }
|
|
|
|
|
|
|
|
// // TODO(fabxc): technically there's still collision probability here.
|
|
|
|
// // Extract the hashmap of the head block and use an instance of it here as well.
|
|
|
|
// newSeries[s.hash] = append(newSeries[s.hash], s)
|
|
|
|
// }
|
|
|
|
|
|
|
|
// // After the samples were successfully written to the WAL, there may
|
|
|
|
// // be no further failures.
|
|
|
|
// if len(newSeries) > 0 {
|
|
|
|
// newLabels = make([]labels.Labels, 0, len(newSeries))
|
|
|
|
// base0 := len(h.series)
|
|
|
|
|
|
|
|
// h.mtx.RUnlock()
|
|
|
|
// h.mtx.Lock()
|
|
|
|
|
|
|
|
// base1 := len(h.series)
|
|
|
|
// i := 0
|
|
|
|
|
|
|
|
// for hash, ser := range newSeries {
|
|
|
|
// lset := ser[0].labels
|
|
|
|
// // We switched locks and have to re-validate that the series were not
|
|
|
|
// // created by another goroutine in the meantime.
|
|
|
|
// if base1 != base0 {
|
|
|
|
// if ms := h.get(hash, lset); ms != nil {
|
|
|
|
// for _, s := range ser {
|
|
|
|
// s.ref = ms.ref
|
|
|
|
// }
|
|
|
|
// continue
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// // Series is still new.
|
|
|
|
// newLabels = append(newLabels, lset)
|
|
|
|
|
|
|
|
// h.create(hash, lset)
|
|
|
|
// // Set sample references to the series we just created.
|
|
|
|
// for _, s := range ser {
|
|
|
|
// s.ref = uint32(base1 + i)
|
|
|
|
// }
|
|
|
|
// i++
|
|
|
|
// }
|
|
|
|
|
|
|
|
// h.mtx.Unlock()
|
|
|
|
// h.mtx.RLock()
|
|
|
|
// }
|
|
|
|
// // Write all new series and samples to the WAL and add it to the
|
|
|
|
// // in-mem database on success.
|
|
|
|
// if err := h.wal.Log(newLabels, samples); err != nil {
|
|
|
|
// return 0, err
|
|
|
|
// }
|
|
|
|
|
|
|
|
// var (
|
|
|
|
// total = uint64(len(samples))
|
|
|
|
// mint = int64(math.MaxInt64)
|
|
|
|
// maxt = int64(math.MinInt64)
|
|
|
|
// )
|
|
|
|
// for _, s := range samples {
|
|
|
|
// ser := h.series[s.ref]
|
|
|
|
|
|
|
|
// ser.mtx.Lock()
|
|
|
|
// ok := ser.append(s.t, s.v)
|
|
|
|
// ser.mtx.Unlock()
|
|
|
|
|
|
|
|
// if !ok {
|
|
|
|
// total--
|
|
|
|
// continue
|
|
|
|
// }
|
|
|
|
// if mint > s.t {
|
|
|
|
// mint = s.t
|
|
|
|
// }
|
|
|
|
// if maxt < s.t {
|
|
|
|
// maxt = s.t
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
|
|
|
|
// h.stats.mtx.Lock()
|
|
|
|
// defer h.stats.mtx.Unlock()
|
|
|
|
|
|
|
|
// h.stats.SampleCount += total
|
|
|
|
// h.stats.SeriesCount += uint64(len(newSeries))
|
|
|
|
// h.stats.ChunkCount += uint64(len(newSeries)) // head block has one chunk/series
|
|
|
|
|
|
|
|
// if mint < h.stats.MinTime {
|
|
|
|
// h.stats.MinTime = mint
|
|
|
|
// }
|
|
|
|
// if maxt > h.stats.MaxTime {
|
|
|
|
// h.stats.MaxTime = maxt
|
|
|
|
// }
|
|
|
|
|
|
|
|
// return int(total), nil
|
|
|
|
// }
|
2017-01-05 06:13:01 -08:00
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headBlock) fullness() float64 {
|
|
|
|
h.stats.mtx.RLock()
|
|
|
|
defer h.stats.mtx.RUnlock()
|
2017-01-07 09:02:17 -08:00
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
return float64(h.stats.SampleCount) / float64(h.stats.SeriesCount+1) / 250
|
2017-01-07 09:02:17 -08:00
|
|
|
}
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
func (h *headBlock) updateMapping() {
|
2017-01-09 09:07:45 -08:00
|
|
|
h.mtx.RLock()
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
if h.mapper.sortable != nil && h.mapper.Len() == len(h.series) {
|
2017-01-09 09:34:29 -08:00
|
|
|
h.mtx.RUnlock()
|
2017-01-06 07:27:50 -08:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
series := make([]*memSeries, len(h.series))
|
|
|
|
copy(series, h.series)
|
2017-01-05 06:13:01 -08:00
|
|
|
|
2017-01-09 09:07:45 -08:00
|
|
|
h.mtx.RUnlock()
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
s := slice.SortInterface(series, func(i, j int) bool {
|
|
|
|
return labels.Compare(series[i].lset, series[j].lset) < 0
|
2017-01-05 06:13:01 -08:00
|
|
|
})
|
|
|
|
|
2017-01-06 07:27:50 -08: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 06:28:22 -08:00
|
|
|
func (h *headBlock) remapPostings(p Postings) Postings {
|
2017-01-06 07:27:50 -08:00
|
|
|
list, err := expandPostings(p)
|
|
|
|
if err != nil {
|
|
|
|
return errPostings{err: err}
|
|
|
|
}
|
|
|
|
|
2017-01-09 10:24:05 -08:00
|
|
|
h.mapper.mtx.Lock()
|
|
|
|
defer h.mapper.mtx.Unlock()
|
2017-01-06 07:27:50 -08:00
|
|
|
|
2017-01-09 10:24:05 -08:00
|
|
|
h.updateMapping()
|
2017-01-06 07:27:50 -08:00
|
|
|
h.mapper.Sort(list)
|
|
|
|
|
|
|
|
return newListPostings(list)
|
2017-01-05 06:13:01 -08:00
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
type memSeries struct {
|
2017-01-09 07:51:39 -08:00
|
|
|
mtx sync.RWMutex
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
ref uint32
|
|
|
|
lset labels.Labels
|
|
|
|
chunks []*memChunk
|
2017-01-09 07:51:39 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
lastValue float64
|
2017-01-09 07:51:39 -08:00
|
|
|
sampleBuf [4]sample
|
|
|
|
|
|
|
|
app chunks.Appender // Current appender for the chunkdb.
|
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08: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
|
|
|
|
|
|
|
|
if s.app == nil || s.head().samples > 10050 {
|
|
|
|
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 07:51:39 -08:00
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
s.app.Append(t, v)
|
|
|
|
|
|
|
|
c.maxTime = t
|
|
|
|
c.samples++
|
|
|
|
|
|
|
|
s.lastValue = v
|
2017-01-09 07:51:39 -08:00
|
|
|
|
2017-01-11 04:02:38 -08: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 07:51:39 -08:00
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
return true
|
2017-01-09 07:51:39 -08:00
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08: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 07:51:39 -08:00
|
|
|
it := &memSafeIterator{
|
2017-01-11 04:02:38 -08:00
|
|
|
Iterator: c.chunk.Iterator(),
|
2017-01-09 07:51:39 -08:00
|
|
|
i: -1,
|
2017-01-11 04:02:38 -08:00
|
|
|
total: c.samples,
|
|
|
|
buf: s.sampleBuf,
|
2017-01-09 07:51:39 -08:00
|
|
|
}
|
|
|
|
return it
|
|
|
|
}
|
|
|
|
|
2017-01-11 04:02:38 -08: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 07:51:39 -08: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 06:13:01 -08:00
|
|
|
// positionMapper stores a position mapping from unsorted to
|
|
|
|
// sorted indices of a sortable collection.
|
|
|
|
type positionMapper struct {
|
2017-01-06 07:27:50 -08:00
|
|
|
mtx sync.RWMutex
|
2017-01-05 06:13:01 -08:00
|
|
|
sortable sort.Interface
|
|
|
|
iv, fw []int
|
|
|
|
}
|
|
|
|
|
|
|
|
func newPositionMapper(s sort.Interface) *positionMapper {
|
2017-01-06 07:27:50 -08:00
|
|
|
m := &positionMapper{}
|
|
|
|
if s != nil {
|
|
|
|
m.update(s)
|
2017-01-05 06:13:01 -08: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 07:27:50 -08: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
|
|
|
|
}
|
|
|
|
}
|