Consolidate mem index into HeadBlock

This commit is contained in:
Fabian Reinartz 2016-12-22 01:12:28 +01:00
parent 869cccf080
commit 2a825f6c28
3 changed files with 70 additions and 81 deletions

13
db.go
View file

@ -227,13 +227,12 @@ func (s *Shard) appendBatch(samples []hashedSample) error {
s.mtx.Lock() s.mtx.Lock()
defer s.mtx.Unlock() defer s.mtx.Unlock()
var merr MultiError // 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.
err := s.head.appendBatch(samples)
for _, sm := range samples { // TODO(fabxc): randomize over time and use better scoring function.
merr.Add(s.head.append(sm.hash, sm.labels, sm.t, sm.v))
}
// TODO(fabxc): randomize over time
if s.head.stats.SampleCount/(uint64(s.head.stats.ChunkCount)+1) > 400 { if s.head.stats.SampleCount/(uint64(s.head.stats.ChunkCount)+1) > 400 {
select { select {
case s.persistCh <- struct{}{}: case s.persistCh <- struct{}{}:
@ -246,7 +245,7 @@ func (s *Shard) appendBatch(samples []hashedSample) error {
} }
} }
return merr.Err() return err
} }
func intervalOverlap(amin, amax, bmin, bmax int64) bool { func intervalOverlap(amin, amax, bmin, bmax int64) bool {

84
head.go
View file

@ -12,9 +12,18 @@ import (
// HeadBlock handles reads and writes of time series data within a time window. // HeadBlock handles reads and writes of time series data within a time window.
type HeadBlock struct { type HeadBlock struct {
mtx sync.RWMutex mtx sync.RWMutex
descs map[uint64][]*chunkDesc // labels hash to possible chunks descs
index *memIndex // descs holds all chunk descs for the head block. Each chunk implicitly
// is assigned the index as its ID.
descs []*chunkDesc
// hashes contains a collision map of label set hashes of chunks
// to their position in the chunk desc slice.
hashes map[uint64][]int
symbols []string // all seen strings
values map[string]stringset // label names to possible values
postings *memPostings // postings lists for terms
stats BlockStats stats BlockStats
} }
@ -22,8 +31,10 @@ type HeadBlock struct {
// NewHeadBlock creates a new empty head block. // NewHeadBlock creates a new empty head block.
func NewHeadBlock(baseTime int64) *HeadBlock { func NewHeadBlock(baseTime int64) *HeadBlock {
b := &HeadBlock{ b := &HeadBlock{
descs: make(map[uint64][]*chunkDesc, 2048), descs: []*chunkDesc{},
index: newMemIndex(), hashes: map[uint64][]int{},
values: map[string]stringset{},
postings: &memPostings{m: make(map[term][]uint32)},
} }
b.stats.MinTime = baseTime b.stats.MinTime = baseTime
@ -37,11 +48,10 @@ func (h *HeadBlock) Querier(mint, maxt int64) Querier {
// Chunk returns the chunk for the reference number. // Chunk returns the chunk for the reference number.
func (h *HeadBlock) Chunk(ref uint32) (chunks.Chunk, error) { func (h *HeadBlock) Chunk(ref uint32) (chunks.Chunk, error) {
c, ok := h.index.forward[ref] if int(ref) >= len(h.descs) {
if !ok {
return nil, errNotFound return nil, errNotFound
} }
return c.chunk, nil return h.descs[int(ref)].chunk, nil
} }
func (h *HeadBlock) interval() (int64, int64) { func (h *HeadBlock) interval() (int64, int64) {
@ -60,7 +70,7 @@ func (h *HeadBlock) LabelValues(names ...string) (StringTuples, error) {
} }
var sl []string var sl []string
for s := range h.index.values[names[0]] { for s := range h.values[names[0]] {
sl = append(sl, s) sl = append(sl, s)
} }
sort.Strings(sl) sort.Strings(sl)
@ -74,15 +84,16 @@ func (h *HeadBlock) LabelValues(names ...string) (StringTuples, error) {
// Postings returns the postings list iterator for the label pair. // Postings returns the postings list iterator for the label pair.
func (h *HeadBlock) Postings(name, value string) (Postings, error) { func (h *HeadBlock) Postings(name, value string) (Postings, error) {
return h.index.Postings(term{name, value}), nil return h.postings.get(term{name: name, value: value}), nil
} }
// Series returns the series for the given reference. // Series returns the series for the given reference.
func (h *HeadBlock) Series(ref uint32, mint, maxt int64) (Series, error) { func (h *HeadBlock) Series(ref uint32, mint, maxt int64) (Series, error) {
cd, ok := h.index.forward[ref] if int(ref) >= len(h.descs) {
if !ok {
return nil, errNotFound return nil, errNotFound
} }
cd := h.descs[ref]
if !intervalOverlap(cd.firsTimestamp, cd.lastTimestamp, mint, maxt) { if !intervalOverlap(cd.firsTimestamp, cd.lastTimestamp, mint, maxt) {
return nil, nil return nil, nil
} }
@ -101,9 +112,10 @@ func (h *HeadBlock) Series(ref uint32, mint, maxt int64) (Series, error) {
// get retrieves the chunk with the hash and label set and creates // get retrieves the chunk with the hash and label set and creates
// a new one if it doesn't exist yet. // a new one if it doesn't exist yet.
func (h *HeadBlock) get(hash uint64, lset labels.Labels) *chunkDesc { func (h *HeadBlock) get(hash uint64, lset labels.Labels) *chunkDesc {
cds := h.descs[hash] refs := h.hashes[hash]
for _, cd := range cds {
if cd.lset.Equals(lset) { for _, ref := range refs {
if cd := h.descs[ref]; cd.lset.Equals(lset) {
return cd return cd
} }
} }
@ -112,16 +124,44 @@ func (h *HeadBlock) get(hash uint64, lset labels.Labels) *chunkDesc {
lset: lset, lset: lset,
chunk: chunks.NewXORChunk(int(math.MaxInt64)), chunk: chunks.NewXORChunk(int(math.MaxInt64)),
} }
h.index.add(cd) // Index the new chunk.
ref := len(h.descs)
h.descs = append(h.descs, cd)
h.hashes[hash] = append(refs, ref)
// Add each label pair as a term to the inverted index.
terms := make([]term, 0, len(lset))
for _, l := range lset {
terms = append(terms, term{name: l.Name, value: l.Value})
valset, ok := h.values[l.Name]
if !ok {
valset = stringset{}
h.values[l.Name] = valset
}
valset.set(l.Value)
}
h.postings.add(uint32(ref), terms...)
// For the head block there's exactly one chunk per series. // For the head block there's exactly one chunk per series.
h.stats.ChunkCount++ h.stats.ChunkCount++
h.stats.SeriesCount++ h.stats.SeriesCount++
h.descs[hash] = append(cds, cd)
return cd return cd
} }
func (h *HeadBlock) appendBatch(samples []hashedSample) error {
var merr MultiError
for _, s := range samples {
merr.Add(h.append(s.hash, s.labels, s.t, s.v))
}
return merr.Err()
}
// append adds the sample to the headblock. // append adds the sample to the headblock.
func (h *HeadBlock) append(hash uint64, lset labels.Labels, ts int64, v float64) error { func (h *HeadBlock) append(hash uint64, lset labels.Labels, ts int64, v float64) error {
if err := h.get(hash, lset).append(ts, v); err != nil { if err := h.get(hash, lset).append(ts, v); err != nil {
@ -153,8 +193,8 @@ func (h *HeadBlock) persist(p string) (int64, error) {
defer sw.Close() defer sw.Close()
defer iw.Close() defer iw.Close()
for ref, cd := range h.index.forward { for ref, cd := range h.descs {
if err := sw.WriteSeries(ref, cd.lset, []*chunkDesc{cd}); err != nil { if err := sw.WriteSeries(uint32(ref), cd.lset, []*chunkDesc{cd}); err != nil {
return 0, err return 0, err
} }
} }
@ -162,7 +202,7 @@ func (h *HeadBlock) persist(p string) (int64, error) {
if err := iw.WriteStats(h.stats); err != nil { if err := iw.WriteStats(h.stats); err != nil {
return 0, err return 0, err
} }
for n, v := range h.index.values { for n, v := range h.values {
s := make([]string, 0, len(v)) s := make([]string, 0, len(v))
for x := range v { for x := range v {
s = append(s, x) s = append(s, x)
@ -173,8 +213,8 @@ func (h *HeadBlock) persist(p string) (int64, error) {
} }
} }
for t := range h.index.postings.m { for t := range h.postings.m {
if err := iw.WritePostings(t.name, t.value, h.index.postings.get(t)); err != nil { if err := iw.WritePostings(t.name, t.value, h.postings.get(t)); err != nil {
return 0, err return 0, err
} }
} }

View file

@ -5,64 +5,14 @@ import (
"strings" "strings"
) )
type memIndex struct { type memPostings struct {
lastID uint32 m map[term][]uint32
forward map[uint32]*chunkDesc // chunk ID to chunk desc
values map[string]stringset // label names to possible values
postings *memPostings // postings lists for terms
}
// newMemIndex returns a new in-memory index.
func newMemIndex() *memIndex {
return &memIndex{
lastID: 0,
forward: make(map[uint32]*chunkDesc),
values: make(map[string]stringset),
postings: &memPostings{m: make(map[term][]uint32)},
}
}
func (ix *memIndex) numSeries() int {
return len(ix.forward)
}
func (ix *memIndex) Postings(t term) Postings {
return ix.postings.get(t)
} }
type term struct { type term struct {
name, value string name, value string
} }
func (ix *memIndex) add(chkd *chunkDesc) {
// Add each label pair as a term to the inverted index.
terms := make([]term, 0, len(chkd.lset))
for _, l := range chkd.lset {
terms = append(terms, term{name: l.Name, value: l.Value})
// Add to label name to values index.
valset, ok := ix.values[l.Name]
if !ok {
valset = stringset{}
ix.values[l.Name] = valset
}
valset.set(l.Value)
}
ix.lastID++
id := ix.lastID
ix.postings.add(id, terms...)
// Store forward index for the returned ID.
ix.forward[id] = chkd
}
type memPostings struct {
m map[term][]uint32
}
// Postings returns an iterator over the postings list for s. // Postings returns an iterator over the postings list for s.
func (p *memPostings) get(t term) Postings { func (p *memPostings) get(t term) Postings {
return &listPostings{list: p.m[t], idx: -1} return &listPostings{list: p.m[t], idx: -1}