prometheus/head.go

443 lines
9.8 KiB
Go
Raw Normal View History

2016-12-04 04:16:11 -08:00
package tsdb
import (
"errors"
"math"
"sort"
2016-12-04 04:16:11 -08:00
"sync"
"time"
2016-12-04 04:16:11 -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"
"github.com/go-kit/kit/log"
2016-12-04 04:16:11 -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-03 01:09:20 -08:00
d 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.
descs []*chunkDesc
// 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
// to their chunk descs.
hashes map[uint64][]*chunkDesc
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-22 03:05:24 -08:00
wal *WAL
2017-01-07 07:20:32 -08:00
bstats *BlockStats
2016-12-04 04:16:11 -08:00
}
2016-12-22 11:00:24 -08:00
// OpenHeadBlock creates a new empty head block.
func OpenHeadBlock(dir string, l log.Logger) (*HeadBlock, error) {
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
}
b := &HeadBlock{
2017-01-03 01:09:20 -08:00
d: dir,
2016-12-21 16:12:28 -08:00
descs: []*chunkDesc{},
hashes: map[uint64][]*chunkDesc{},
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,
mapper: newPositionMapper(nil),
}
2017-01-07 07:20:32 -08:00
b.bstats = &BlockStats{
MinTime: math.MaxInt64,
MaxTime: math.MinInt64,
}
2016-12-22 06:18:33 -08:00
err = wal.ReadAll(&walHandler{
series: func(lset labels.Labels) {
b.create(lset.Hash(), lset)
},
sample: func(s hashedSample) {
cd := b.descs[s.ref]
// Duplicated from appendBatch TODO(fabxc): deduplicate?
if cd.lastTimestamp == s.t && cd.lastValue != s.v {
return
}
cd.append(s.t, s.v)
if s.t > b.bstats.MaxTime {
b.bstats.MaxTime = s.t
}
if s.t < b.bstats.MinTime {
b.bstats.MinTime = s.t
}
2017-01-03 06:43:26 -08:00
b.bstats.SampleCount++
2016-12-22 06:18:33 -08:00
},
})
if err != nil {
return nil, err
}
b.updateMapping()
2016-12-22 03:05:24 -08:00
return b, nil
}
// Close syncs all data and closes underlying resources of the head block.
func (h *HeadBlock) Close() error {
return h.wal.Close()
}
2017-01-03 01:09:20 -08:00
func (h *HeadBlock) dir() string { return h.d }
2017-01-03 06:43:26 -08:00
func (h *HeadBlock) persisted() bool { return false }
2017-01-03 01:09:20 -08:00
func (h *HeadBlock) index() IndexReader { return h }
func (h *HeadBlock) series() SeriesReader { return h }
2017-01-07 09:02:17 -08:00
func (h *HeadBlock) stats() BlockStats {
h.bstats.mtx.RLock()
defer h.bstats.mtx.RUnlock()
return *h.bstats
}
// Chunk returns the chunk for the reference number.
func (h *HeadBlock) Chunk(ref uint32) (chunks.Chunk, error) {
h.mtx.RLock()
defer h.mtx.RUnlock()
2016-12-21 16:12:28 -08:00
if int(ref) >= len(h.descs) {
return nil, errNotFound
}
2016-12-21 16:12:28 -08:00
return h.descs[int(ref)].chunk, nil
}
func (h *HeadBlock) interval() (int64, int64) {
2017-01-03 06:43:26 -08:00
return h.bstats.MinTime, h.bstats.MaxTime
}
// Stats returns statisitics about the indexed data.
func (h *HeadBlock) Stats() (BlockStats, error) {
2017-01-07 07:20:32 -08:00
return *h.bstats, nil
}
// LabelValues returns the possible label values
func (h *HeadBlock) LabelValues(names ...string) (StringTuples, error) {
h.mtx.RLock()
defer h.mtx.RUnlock()
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]] {
sl = append(sl, s)
}
sort.Strings(sl)
return &stringTuples{l: len(names), s: sl}, nil
}
// Postings returns the postings list iterator for the label pair.
2016-12-14 12:58:29 -08:00
func (h *HeadBlock) Postings(name, value string) (Postings, error) {
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
}
// Series returns the series for the given reference.
func (h *HeadBlock) Series(ref uint32) (labels.Labels, []ChunkMeta, error) {
h.mtx.RLock()
defer h.mtx.RUnlock()
2016-12-21 16:12:28 -08:00
if int(ref) >= len(h.descs) {
return nil, nil, errNotFound
}
2016-12-21 16:12:28 -08:00
cd := h.descs[ref]
2017-01-03 06:43:26 -08:00
meta := ChunkMeta{
MinTime: cd.firstTimestamp,
2017-01-03 06:43:26 -08:00
MaxTime: cd.lastTimestamp,
Ref: ref,
}
return cd.lset, []ChunkMeta{meta}, nil
}
func (h *HeadBlock) LabelIndices() ([][]string, error) {
h.mtx.RLock()
defer h.mtx.RUnlock()
res := [][]string{}
for s := range h.values {
res = append(res, []string{s})
}
return res, 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.
func (h *HeadBlock) get(hash uint64, lset labels.Labels) *chunkDesc {
cds := h.hashes[hash]
2016-12-21 16:12:28 -08:00
for _, cd := range cds {
if cd.lset.Equals(lset) {
return cd
2016-12-04 04:16:11 -08:00
}
}
return nil
2016-12-22 03:05:24 -08:00
}
func (h *HeadBlock) create(hash uint64, lset labels.Labels) *chunkDesc {
2016-12-04 04:16:11 -08:00
cd := &chunkDesc{
lset: lset,
chunk: chunks.NewXORChunk(),
lastTimestamp: math.MinInt64,
2016-12-31 01:10:27 -08:00
}
var err error
2016-12-31 01:10:27 -08:00
cd.app, err = cd.chunk.Appender()
if err != nil {
// Getting an Appender for a new chunk must not panic.
panic(err)
2016-12-04 04:16:11 -08:00
}
2016-12-21 16:12:28 -08:00
// Index the new chunk.
cd.ref = uint32(len(h.descs))
2016-12-21 16:12:28 -08:00
h.descs = append(h.descs, cd)
h.hashes[hash] = append(h.hashes[hash], cd)
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
h.postings.add(cd.ref, term{name: l.Name, value: l.Value})
2016-12-21 16:12:28 -08:00
}
2017-01-03 06:43:26 -08:00
h.postings.add(cd.ref, term{})
2016-12-04 04:16:11 -08:00
2016-12-09 01:00:14 -08:00
return cd
2016-12-04 04:16:11 -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.
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")
)
func (h *HeadBlock) appendBatch(samples []hashedSample) error {
2016-12-22 03:05:24 -08:00
// Find head chunks for all samples and allocate new IDs/refs for
// ones we haven't seen before.
var (
2016-12-26 07:55:32 -08:00
newSeries []labels.Labels
newSamples []*hashedSample
2016-12-26 07:55:32 -08:00
newHashes []uint64
uniqueHashes = map[uint64]uint32{}
2016-12-22 03:05:24 -08:00
)
h.mtx.RLock()
2017-01-07 07:20:32 -08:00
defer h.mtx.RUnlock()
2016-12-21 16:12:28 -08:00
2016-12-22 11:57:00 -08:00
for i := range samples {
s := &samples[i]
cd := h.get(s.hash, s.labels)
2016-12-22 03:05:24 -08:00
if cd != nil {
// Samples must only occur in order.
if s.t < cd.lastTimestamp {
return ErrOutOfOrderSample
}
if cd.lastTimestamp == s.t && cd.lastValue != s.v {
return ErrAmendSample
}
2016-12-22 03:05:24 -08:00
// TODO(fabxc): sample refs are only scoped within a block for
// now and we ignore any previously set value
s.ref = cd.ref
2016-12-22 03:05:24 -08:00
continue
}
2016-12-26 07:55:32 -08:00
// There may be several samples for a new series in a batch.
// We don't want to reserve a new space for each.
if ref, ok := uniqueHashes[s.hash]; ok {
s.ref = ref
continue
}
s.ref = uint32(len(newSeries))
2016-12-26 07:55:32 -08:00
uniqueHashes[s.hash] = s.ref
2016-12-22 11:57:00 -08:00
2016-12-22 03:05:24 -08:00
newSeries = append(newSeries, s.labels)
newHashes = append(newHashes, s.hash)
newSamples = append(newSamples, s)
2016-12-22 03:05:24 -08:00
}
2016-12-21 16:12:28 -08:00
2016-12-22 03:05:24 -08:00
// Write all new series and samples to the WAL and add it to the
// in-mem database on success.
if err := h.wal.Log(newSeries, samples); err != nil {
return err
}
// After the samples were successfully written to the WAL, there may
// be no further failures.
if len(newSeries) > 0 {
2017-01-07 07:20:32 -08:00
h.mtx.RUnlock()
h.mtx.Lock()
base := len(h.descs)
for i, s := range newSeries {
h.create(newHashes[i], s)
}
for _, s := range newSamples {
s.ref = uint32(base) + s.ref
}
h.mtx.Unlock()
h.mtx.RLock()
2016-12-22 03:05:24 -08:00
}
2017-01-07 09:02:17 -08:00
var (
total = uint64(len(samples))
mint = int64(math.MaxInt64)
maxt = int64(math.MinInt64)
)
2016-12-22 03:05:24 -08:00
for _, s := range samples {
cd := h.descs[s.ref]
// Skip duplicate samples.
if cd.lastTimestamp == s.t && cd.lastValue != s.v {
total--
continue
}
cd.append(s.t, s.v)
2016-12-22 03:05:24 -08:00
2017-01-07 09:02:17 -08:00
if mint > s.t {
mint = s.t
2016-12-22 03:05:24 -08:00
}
2017-01-07 09:02:17 -08:00
if maxt < s.t {
maxt = s.t
}
2016-12-09 01:00:14 -08:00
}
2017-01-07 09:02:17 -08:00
h.bstats.mtx.Lock()
defer h.bstats.mtx.Unlock()
h.bstats.SampleCount += total
h.bstats.SeriesCount += uint64(len(newSeries))
h.bstats.ChunkCount += uint64(len(newSeries)) // head block has one chunk/series
if mint < h.bstats.MinTime {
h.bstats.MinTime = mint
}
if maxt > h.bstats.MaxTime {
h.bstats.MaxTime = maxt
}
2016-12-31 01:10:27 -08:00
return nil
}
2017-01-07 09:02:17 -08:00
func (h *HeadBlock) fullness() float64 {
h.bstats.mtx.RLock()
defer h.bstats.mtx.RUnlock()
return float64(h.bstats.SampleCount) / float64(h.bstats.SeriesCount+1) / 250
}
func (h *HeadBlock) updateMapping() {
h.mapper.mtx.Lock()
defer h.mapper.mtx.Unlock()
if h.mapper.sortable != nil && h.mapper.Len() == len(h.descs) {
return
}
cds := make([]*chunkDesc, len(h.descs))
copy(cds, h.descs)
s := slice.SortInterface(cds, func(i, j int) bool {
return labels.Compare(cds[i].lset, cds[j].lset) < 0
})
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.
func (h *HeadBlock) remapPostings(p Postings) Postings {
list, err := expandPostings(p)
if err != nil {
return errPostings{err: err}
}
h.mapper.mtx.RLock()
defer h.mapper.mtx.RUnlock()
h.mapper.Sort(list)
slice.Sort(list, func(i, j int) bool {
return h.mapper.fw[list[i]] < h.mapper.fw[list[j]]
})
return newListPostings(list)
}
// positionMapper stores a position mapping from unsorted to
// sorted indices of a sortable collection.
type positionMapper struct {
mtx sync.RWMutex
sortable sort.Interface
iv, fw []int
}
func newPositionMapper(s sort.Interface) *positionMapper {
m := &positionMapper{}
if s != nil {
m.update(s)
}
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]
}
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
}
}