mirror of
https://github.com/prometheus/prometheus.git
synced 2025-01-12 06:17:27 -08:00
Remove multiple heads
This changes the structure to a single WAL backed by a single head block. Parts of the head block can be compacted. This relieves us from any head amangement and greatly simplifies any consistency and isolation concerns by just having a single head.
This commit is contained in:
parent
0fe67df9f2
commit
3901b6e70b
48
block.go
48
block.go
|
@ -26,14 +26,21 @@ import (
|
||||||
"github.com/prometheus/tsdb/labels"
|
"github.com/prometheus/tsdb/labels"
|
||||||
)
|
)
|
||||||
|
|
||||||
// DiskBlock handles reads against a Block of time series data.
|
|
||||||
type DiskBlock interface {
|
type DiskBlock interface {
|
||||||
|
BlockReader
|
||||||
|
|
||||||
// Directory where block data is stored.
|
// Directory where block data is stored.
|
||||||
Dir() string
|
Dir() string
|
||||||
|
|
||||||
// Stats returns statistics about the block.
|
// Stats returns statistics about the block.
|
||||||
Meta() BlockMeta
|
Meta() BlockMeta
|
||||||
|
|
||||||
|
Delete(mint, maxt int64, m ...labels.Matcher) error
|
||||||
|
|
||||||
|
Close() error
|
||||||
|
}
|
||||||
|
|
||||||
|
type BlockReader interface {
|
||||||
// Index returns an IndexReader over the block's data.
|
// Index returns an IndexReader over the block's data.
|
||||||
Index() IndexReader
|
Index() IndexReader
|
||||||
|
|
||||||
|
@ -42,31 +49,14 @@ type DiskBlock interface {
|
||||||
|
|
||||||
// Tombstones returns a TombstoneReader over the block's deleted data.
|
// Tombstones returns a TombstoneReader over the block's deleted data.
|
||||||
Tombstones() TombstoneReader
|
Tombstones() TombstoneReader
|
||||||
|
|
||||||
// Delete deletes data from the block.
|
|
||||||
Delete(mint, maxt int64, ms ...labels.Matcher) error
|
|
||||||
|
|
||||||
// Close releases all underlying resources of the block.
|
|
||||||
Close() error
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Block is an interface to a DiskBlock that can also be queried.
|
// // Block is an interface to a DiskBlock that can also be queried.
|
||||||
type Block interface {
|
// type Block interface {
|
||||||
DiskBlock
|
// DiskBlock
|
||||||
Queryable
|
// Queryable
|
||||||
Snapshottable
|
// Snapshottable
|
||||||
}
|
// }
|
||||||
|
|
||||||
// headBlock is a regular block that can still be appended to.
|
|
||||||
type headBlock interface {
|
|
||||||
Block
|
|
||||||
Appendable
|
|
||||||
|
|
||||||
// ActiveWriters returns the number of currently active appenders.
|
|
||||||
ActiveWriters() int
|
|
||||||
// HighTimestamp returns the highest currently inserted timestamp.
|
|
||||||
HighTimestamp() int64
|
|
||||||
}
|
|
||||||
|
|
||||||
// Snapshottable defines an entity that can be backedup online.
|
// Snapshottable defines an entity that can be backedup online.
|
||||||
type Snapshottable interface {
|
type Snapshottable interface {
|
||||||
|
@ -225,16 +215,6 @@ func (pb *persistedBlock) String() string {
|
||||||
return pb.meta.ULID.String()
|
return pb.meta.ULID.String()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (pb *persistedBlock) Querier(mint, maxt int64) Querier {
|
|
||||||
return &blockQuerier{
|
|
||||||
mint: mint,
|
|
||||||
maxt: maxt,
|
|
||||||
index: pb.Index(),
|
|
||||||
chunks: pb.Chunks(),
|
|
||||||
tombstones: pb.Tombstones(),
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (pb *persistedBlock) Dir() string { return pb.dir }
|
func (pb *persistedBlock) Dir() string { return pb.dir }
|
||||||
func (pb *persistedBlock) Index() IndexReader { return pb.indexr }
|
func (pb *persistedBlock) Index() IndexReader { return pb.indexr }
|
||||||
func (pb *persistedBlock) Chunks() ChunkReader { return pb.chunkr }
|
func (pb *persistedBlock) Chunks() ChunkReader { return pb.chunkr }
|
||||||
|
|
95
compact.go
95
compact.go
|
@ -14,7 +14,6 @@
|
||||||
package tsdb
|
package tsdb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
|
@ -51,7 +50,7 @@ type Compactor interface {
|
||||||
Plan(dir string) ([]string, error)
|
Plan(dir string) ([]string, error)
|
||||||
|
|
||||||
// Write persists a Block into a directory.
|
// Write persists a Block into a directory.
|
||||||
Write(dest string, b Block) error
|
Write(dest string, b BlockReader, mint, maxt int64) error
|
||||||
|
|
||||||
// Compact runs compaction against the provided directories. Must
|
// Compact runs compaction against the provided directories. Must
|
||||||
// only be called concurrently with results of Plan().
|
// only be called concurrently with results of Plan().
|
||||||
|
@ -124,8 +123,6 @@ type compactionInfo struct {
|
||||||
mint, maxt int64
|
mint, maxt int64
|
||||||
}
|
}
|
||||||
|
|
||||||
const compactionBlocksLen = 3
|
|
||||||
|
|
||||||
type dirMeta struct {
|
type dirMeta struct {
|
||||||
dir string
|
dir string
|
||||||
meta *BlockMeta
|
meta *BlockMeta
|
||||||
|
@ -258,9 +255,12 @@ func splitByRange(ds []dirMeta, tr int64) [][]dirMeta {
|
||||||
return splitDirs
|
return splitDirs
|
||||||
}
|
}
|
||||||
|
|
||||||
func compactBlockMetas(blocks ...BlockMeta) (res BlockMeta) {
|
func compactBlockMetas(uid ulid.ULID, blocks ...*BlockMeta) *BlockMeta {
|
||||||
res.MinTime = blocks[0].MinTime
|
res := &BlockMeta{
|
||||||
res.MaxTime = blocks[len(blocks)-1].MaxTime
|
ULID: uid,
|
||||||
|
MinTime: blocks[0].MinTime,
|
||||||
|
MaxTime: blocks[len(blocks)-1].MaxTime,
|
||||||
|
}
|
||||||
|
|
||||||
sources := map[ulid.ULID]struct{}{}
|
sources := map[ulid.ULID]struct{}{}
|
||||||
|
|
||||||
|
@ -271,10 +271,6 @@ func compactBlockMetas(blocks ...BlockMeta) (res BlockMeta) {
|
||||||
for _, s := range b.Compaction.Sources {
|
for _, s := range b.Compaction.Sources {
|
||||||
sources[s] = struct{}{}
|
sources[s] = struct{}{}
|
||||||
}
|
}
|
||||||
// If it's an in memory block, its ULID goes into the sources.
|
|
||||||
if b.Compaction.Level == 0 {
|
|
||||||
sources[b.ULID] = struct{}{}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
res.Compaction.Level++
|
res.Compaction.Level++
|
||||||
|
|
||||||
|
@ -291,7 +287,8 @@ func compactBlockMetas(blocks ...BlockMeta) (res BlockMeta) {
|
||||||
// Compact creates a new block in the compactor's directory from the blocks in the
|
// Compact creates a new block in the compactor's directory from the blocks in the
|
||||||
// provided directories.
|
// provided directories.
|
||||||
func (c *LeveledCompactor) Compact(dest string, dirs ...string) (err error) {
|
func (c *LeveledCompactor) Compact(dest string, dirs ...string) (err error) {
|
||||||
var blocks []Block
|
var blocks []BlockReader
|
||||||
|
var metas []*BlockMeta
|
||||||
|
|
||||||
for _, d := range dirs {
|
for _, d := range dirs {
|
||||||
b, err := newPersistedBlock(d, c.opts.chunkPool)
|
b, err := newPersistedBlock(d, c.opts.chunkPool)
|
||||||
|
@ -300,31 +297,40 @@ func (c *LeveledCompactor) Compact(dest string, dirs ...string) (err error) {
|
||||||
}
|
}
|
||||||
defer b.Close()
|
defer b.Close()
|
||||||
|
|
||||||
|
meta, err := readMetaFile(d)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
metas = append(metas, meta)
|
||||||
blocks = append(blocks, b)
|
blocks = append(blocks, b)
|
||||||
}
|
}
|
||||||
|
|
||||||
entropy := rand.New(rand.NewSource(time.Now().UnixNano()))
|
entropy := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||||
uid := ulid.MustNew(ulid.Now(), entropy)
|
uid := ulid.MustNew(ulid.Now(), entropy)
|
||||||
|
|
||||||
return c.write(dest, uid, blocks...)
|
return c.write(dest, compactBlockMetas(uid, metas...), blocks...)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *LeveledCompactor) Write(dest string, b Block) error {
|
func (c *LeveledCompactor) Write(dest string, b BlockReader, mint, maxt int64) error {
|
||||||
// Buffering blocks might have been created that often have no data.
|
|
||||||
if b.Meta().Stats.NumSeries == 0 {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
entropy := rand.New(rand.NewSource(time.Now().UnixNano()))
|
entropy := rand.New(rand.NewSource(time.Now().UnixNano()))
|
||||||
uid := ulid.MustNew(ulid.Now(), entropy)
|
uid := ulid.MustNew(ulid.Now(), entropy)
|
||||||
|
|
||||||
return c.write(dest, uid, b)
|
meta := &BlockMeta{
|
||||||
|
ULID: uid,
|
||||||
|
MinTime: mint,
|
||||||
|
MaxTime: maxt,
|
||||||
|
}
|
||||||
|
meta.Compaction.Level = 1
|
||||||
|
meta.Compaction.Sources = []ulid.ULID{uid}
|
||||||
|
|
||||||
|
return c.write(dest, meta, b)
|
||||||
}
|
}
|
||||||
|
|
||||||
// write creates a new block that is the union of the provided blocks into dir.
|
// write creates a new block that is the union of the provided blocks into dir.
|
||||||
// It cleans up all files of the old blocks after completing successfully.
|
// It cleans up all files of the old blocks after completing successfully.
|
||||||
func (c *LeveledCompactor) write(dest string, uid ulid.ULID, blocks ...Block) (err error) {
|
func (c *LeveledCompactor) write(dest string, meta *BlockMeta, blocks ...BlockReader) (err error) {
|
||||||
c.logger.Log("msg", "compact blocks", "blocks", fmt.Sprintf("%v", blocks))
|
c.logger.Log("msg", "compact blocks", "count", len(blocks), "mint", meta.MinTime, "maxt", meta.MaxTime)
|
||||||
|
|
||||||
defer func(t time.Time) {
|
defer func(t time.Time) {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -334,7 +340,7 @@ func (c *LeveledCompactor) write(dest string, uid ulid.ULID, blocks ...Block) (e
|
||||||
c.metrics.duration.Observe(time.Since(t).Seconds())
|
c.metrics.duration.Observe(time.Since(t).Seconds())
|
||||||
}(time.Now())
|
}(time.Now())
|
||||||
|
|
||||||
dir := filepath.Join(dest, uid.String())
|
dir := filepath.Join(dest, meta.ULID.String())
|
||||||
tmp := dir + ".tmp"
|
tmp := dir + ".tmp"
|
||||||
|
|
||||||
if err = os.RemoveAll(tmp); err != nil {
|
if err = os.RemoveAll(tmp); err != nil {
|
||||||
|
@ -356,11 +362,9 @@ func (c *LeveledCompactor) write(dest string, uid ulid.ULID, blocks ...Block) (e
|
||||||
return errors.Wrap(err, "open index writer")
|
return errors.Wrap(err, "open index writer")
|
||||||
}
|
}
|
||||||
|
|
||||||
meta, err := c.populateBlock(blocks, indexw, chunkw)
|
if err := c.populateBlock(blocks, meta, indexw, chunkw); err != nil {
|
||||||
if err != nil {
|
|
||||||
return errors.Wrap(err, "write compaction")
|
return errors.Wrap(err, "write compaction")
|
||||||
}
|
}
|
||||||
meta.ULID = uid
|
|
||||||
|
|
||||||
if err = writeMetaFile(tmp, meta); err != nil {
|
if err = writeMetaFile(tmp, meta); err != nil {
|
||||||
return errors.Wrap(err, "write merged meta")
|
return errors.Wrap(err, "write merged meta")
|
||||||
|
@ -398,18 +402,16 @@ func (c *LeveledCompactor) write(dest string, uid ulid.ULID, blocks ...Block) (e
|
||||||
|
|
||||||
// populateBlock fills the index and chunk writers with new data gathered as the union
|
// populateBlock fills the index and chunk writers with new data gathered as the union
|
||||||
// of the provided blocks. It returns meta information for the new block.
|
// of the provided blocks. It returns meta information for the new block.
|
||||||
func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chunkw ChunkWriter) (*BlockMeta, error) {
|
func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, indexw IndexWriter, chunkw ChunkWriter) error {
|
||||||
var (
|
var (
|
||||||
set compactionSet
|
set compactionSet
|
||||||
metas []BlockMeta
|
|
||||||
allSymbols = make(map[string]struct{}, 1<<16)
|
allSymbols = make(map[string]struct{}, 1<<16)
|
||||||
)
|
)
|
||||||
for i, b := range blocks {
|
for i, b := range blocks {
|
||||||
metas = append(metas, b.Meta())
|
|
||||||
|
|
||||||
symbols, err := b.Index().Symbols()
|
symbols, err := b.Index().Symbols()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Wrap(err, "read symbols")
|
return errors.Wrap(err, "read symbols")
|
||||||
}
|
}
|
||||||
for s := range symbols {
|
for s := range symbols {
|
||||||
allSymbols[s] = struct{}{}
|
allSymbols[s] = struct{}{}
|
||||||
|
@ -419,7 +421,7 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
|
|
||||||
all, err := indexr.Postings("", "")
|
all, err := indexr.Postings("", "")
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return err
|
||||||
}
|
}
|
||||||
all = indexr.SortedPostings(all)
|
all = indexr.SortedPostings(all)
|
||||||
|
|
||||||
|
@ -431,7 +433,7 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
}
|
}
|
||||||
set, err = newCompactionMerger(set, s)
|
set, err = newCompactionMerger(set, s)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -440,11 +442,10 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
postings = &memPostings{m: make(map[term][]uint32, 512)}
|
postings = &memPostings{m: make(map[term][]uint32, 512)}
|
||||||
values = map[string]stringset{}
|
values = map[string]stringset{}
|
||||||
i = uint32(0)
|
i = uint32(0)
|
||||||
meta = compactBlockMetas(metas...)
|
|
||||||
)
|
)
|
||||||
|
|
||||||
if err := indexw.AddSymbols(allSymbols); err != nil {
|
if err := indexw.AddSymbols(allSymbols); err != nil {
|
||||||
return nil, errors.Wrap(err, "add symbols")
|
return errors.Wrap(err, "add symbols")
|
||||||
}
|
}
|
||||||
|
|
||||||
for set.Next() {
|
for set.Next() {
|
||||||
|
@ -462,7 +463,7 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
newChunk := chunks.NewXORChunk()
|
newChunk := chunks.NewXORChunk()
|
||||||
app, err := newChunk.Appender()
|
app, err := newChunk.Appender()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
it := &deletedIterator{it: chk.Chunk.Iterator(), intervals: dranges}
|
it := &deletedIterator{it: chk.Chunk.Iterator(), intervals: dranges}
|
||||||
|
@ -476,11 +477,11 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if err := chunkw.WriteChunks(chks...); err != nil {
|
if err := chunkw.WriteChunks(chks...); err != nil {
|
||||||
return nil, err
|
return errors.Wrap(err, "write chunks")
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := indexw.AddSeries(i, lset, chks...); err != nil {
|
if err := indexw.AddSeries(i, lset, chks...); err != nil {
|
||||||
return nil, errors.Wrapf(err, "add series")
|
return errors.Wrap(err, "add series")
|
||||||
}
|
}
|
||||||
|
|
||||||
meta.Stats.NumChunks += uint64(len(chks))
|
meta.Stats.NumChunks += uint64(len(chks))
|
||||||
|
@ -508,7 +509,7 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
i++
|
i++
|
||||||
}
|
}
|
||||||
if set.Err() != nil {
|
if set.Err() != nil {
|
||||||
return nil, set.Err()
|
return errors.Wrap(set.Err(), "iterate compaction set")
|
||||||
}
|
}
|
||||||
|
|
||||||
s := make([]string, 0, 256)
|
s := make([]string, 0, 256)
|
||||||
|
@ -519,13 +520,13 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
s = append(s, x)
|
s = append(s, x)
|
||||||
}
|
}
|
||||||
if err := indexw.WriteLabelIndex([]string{n}, s); err != nil {
|
if err := indexw.WriteLabelIndex([]string{n}, s); err != nil {
|
||||||
return nil, err
|
return errors.Wrap(err, "write label index")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for t := range postings.m {
|
for t := range postings.m {
|
||||||
if err := indexw.WritePostings(t.name, t.value, postings.get(t)); err != nil {
|
if err := indexw.WritePostings(t.name, t.value, postings.get(t)); err != nil {
|
||||||
return nil, err
|
return errors.Wrap(err, "write postings")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Write a postings list containing all series.
|
// Write a postings list containing all series.
|
||||||
|
@ -534,10 +535,10 @@ func (c *LeveledCompactor) populateBlock(blocks []Block, indexw IndexWriter, chu
|
||||||
all[i] = uint32(i)
|
all[i] = uint32(i)
|
||||||
}
|
}
|
||||||
if err := indexw.WritePostings("", "", newListPostings(all)); err != nil {
|
if err := indexw.WritePostings("", "", newListPostings(all)); err != nil {
|
||||||
return nil, err
|
return errors.Wrap(err, "write 'all' postings")
|
||||||
}
|
}
|
||||||
|
|
||||||
return &meta, nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
type compactionSet interface {
|
type compactionSet interface {
|
||||||
|
@ -572,9 +573,12 @@ func (c *compactionSeriesSet) Next() bool {
|
||||||
if !c.p.Next() {
|
if !c.p.Next() {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
var err error
|
||||||
|
|
||||||
c.intervals = c.tombstones.Get(c.p.At())
|
c.intervals = c.tombstones.Get(c.p.At())
|
||||||
|
|
||||||
if c.err = c.index.Series(c.p.At(), &c.l, &c.c); c.err != nil {
|
if err = c.index.Series(c.p.At(), &c.l, &c.c); err != nil {
|
||||||
|
c.err = errors.Wrapf(err, "get series %d", c.p.At())
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -593,8 +597,9 @@ func (c *compactionSeriesSet) Next() bool {
|
||||||
for i := range c.c {
|
for i := range c.c {
|
||||||
chk := &c.c[i]
|
chk := &c.c[i]
|
||||||
|
|
||||||
chk.Chunk, c.err = c.chunks.Chunk(chk.Ref)
|
chk.Chunk, err = c.chunks.Chunk(chk.Ref)
|
||||||
if c.err != nil {
|
if err != nil {
|
||||||
|
c.err = errors.Wrapf(err, "chunk %d not found", chk.Ref)
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
743
db.go
743
db.go
|
@ -16,9 +16,11 @@ package tsdb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"encoding/binary"
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
|
"math"
|
||||||
"os"
|
"os"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"runtime"
|
"runtime"
|
||||||
|
@ -26,6 +28,7 @@ import (
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
"sync"
|
"sync"
|
||||||
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
|
@ -100,18 +103,15 @@ type DB struct {
|
||||||
metrics *dbMetrics
|
metrics *dbMetrics
|
||||||
opts *Options
|
opts *Options
|
||||||
chunkPool chunks.Pool
|
chunkPool chunks.Pool
|
||||||
|
appendPool sync.Pool
|
||||||
|
compactor Compactor
|
||||||
|
wal WAL
|
||||||
|
|
||||||
// Mutex for that must be held when modifying the general block layout.
|
// Mutex for that must be held when modifying the general block layout.
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
blocks []Block
|
blocks []DiskBlock
|
||||||
|
|
||||||
// Mutex that must be held when modifying just the head blocks
|
head *Head
|
||||||
// or the general layout.
|
|
||||||
// mtx must be held before acquiring.
|
|
||||||
headmtx sync.RWMutex
|
|
||||||
heads []headBlock
|
|
||||||
|
|
||||||
compactor Compactor
|
|
||||||
|
|
||||||
compactc chan struct{}
|
compactc chan struct{}
|
||||||
donec chan struct{}
|
donec chan struct{}
|
||||||
|
@ -187,20 +187,24 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db
|
||||||
if err := os.MkdirAll(dir, 0777); err != nil {
|
if err := os.MkdirAll(dir, 0777); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if l == nil {
|
if l == nil {
|
||||||
l = log.NewLogfmtLogger(os.Stdout)
|
l = log.NewLogfmtLogger(os.Stdout)
|
||||||
l = log.With(l, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
|
l = log.With(l, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
|
||||||
}
|
}
|
||||||
|
|
||||||
if opts == nil {
|
if opts == nil {
|
||||||
opts = DefaultOptions
|
opts = DefaultOptions
|
||||||
}
|
}
|
||||||
|
|
||||||
|
wal, err := OpenSegmentWAL(filepath.Join(dir, "wal"), l, 10*time.Second)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
db = &DB{
|
db = &DB{
|
||||||
dir: dir,
|
dir: dir,
|
||||||
logger: l,
|
logger: l,
|
||||||
opts: opts,
|
opts: opts,
|
||||||
|
wal: wal,
|
||||||
compactc: make(chan struct{}, 1),
|
compactc: make(chan struct{}, 1),
|
||||||
donec: make(chan struct{}),
|
donec: make(chan struct{}),
|
||||||
stopc: make(chan struct{}),
|
stopc: make(chan struct{}),
|
||||||
|
@ -237,16 +241,20 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db
|
||||||
if len(copts.blockRanges) == 1 {
|
if len(copts.blockRanges) == 1 {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
|
||||||
// Max overflow is restricted to 20%.
|
// Max overflow is restricted to 20%.
|
||||||
copts.blockRanges = copts.blockRanges[:len(copts.blockRanges)-1]
|
copts.blockRanges = copts.blockRanges[:len(copts.blockRanges)-1]
|
||||||
}
|
}
|
||||||
|
|
||||||
db.compactor = NewLeveledCompactor(r, l, copts)
|
db.compactor = NewLeveledCompactor(r, l, copts)
|
||||||
|
|
||||||
|
db.head, err = NewHead(l, db.wal.Reader(), copts.blockRanges[0])
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
if err := db.reloadBlocks(); err != nil {
|
if err := db.reloadBlocks(); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
go db.run()
|
go db.run()
|
||||||
|
|
||||||
return db, nil
|
return db, nil
|
||||||
|
@ -260,12 +268,16 @@ func (db *DB) Dir() string {
|
||||||
func (db *DB) run() {
|
func (db *DB) run() {
|
||||||
defer close(db.donec)
|
defer close(db.donec)
|
||||||
|
|
||||||
tick := time.NewTicker(30 * time.Second)
|
backoff := time.Duration(0)
|
||||||
defer tick.Stop()
|
|
||||||
|
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-tick.C:
|
case <-db.stopc:
|
||||||
|
case <-time.After(backoff):
|
||||||
|
}
|
||||||
|
|
||||||
|
select {
|
||||||
|
case <-time.After(1 * time.Minute):
|
||||||
select {
|
select {
|
||||||
case db.compactc <- struct{}{}:
|
case db.compactc <- struct{}{}:
|
||||||
default:
|
default:
|
||||||
|
@ -273,20 +285,18 @@ func (db *DB) run() {
|
||||||
case <-db.compactc:
|
case <-db.compactc:
|
||||||
db.metrics.compactionsTriggered.Inc()
|
db.metrics.compactionsTriggered.Inc()
|
||||||
|
|
||||||
changes1, err := db.retentionCutoff()
|
_, err1 := db.retentionCutoff()
|
||||||
if err != nil {
|
if err1 != nil {
|
||||||
db.logger.Log("msg", "retention cutoff failed", "err", err)
|
db.logger.Log("msg", "retention cutoff failed", "err", err1)
|
||||||
}
|
}
|
||||||
|
|
||||||
changes2, err := db.compact()
|
_, err2 := db.compact()
|
||||||
if err != nil {
|
if err2 != nil {
|
||||||
db.logger.Log("msg", "compaction failed", "err", err)
|
db.logger.Log("msg", "compaction failed", "err", err2)
|
||||||
}
|
}
|
||||||
|
|
||||||
if changes1 || changes2 {
|
if err1 != nil || err2 != nil {
|
||||||
if err := db.reloadBlocks(); err != nil {
|
exponential(backoff, 1*time.Second, 1*time.Minute)
|
||||||
db.logger.Log("msg", "reloading blocks failed", "err", err)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
case <-db.stopc:
|
case <-db.stopc:
|
||||||
|
@ -303,76 +313,16 @@ func (db *DB) retentionCutoff() (bool, error) {
|
||||||
db.mtx.RLock()
|
db.mtx.RLock()
|
||||||
defer db.mtx.RUnlock()
|
defer db.mtx.RUnlock()
|
||||||
|
|
||||||
// We only consider the already persisted blocks. Head blocks generally
|
if len(db.blocks) == 0 {
|
||||||
// only account for a fraction of the total data.
|
|
||||||
db.headmtx.RLock()
|
|
||||||
lenp := len(db.blocks) - len(db.heads)
|
|
||||||
db.headmtx.RUnlock()
|
|
||||||
|
|
||||||
if lenp == 0 {
|
|
||||||
return false, nil
|
return false, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
last := db.blocks[lenp-1]
|
last := db.blocks[len(db.blocks)-1]
|
||||||
mint := last.Meta().MaxTime - int64(db.opts.RetentionDuration)
|
mint := last.Meta().MaxTime - int64(db.opts.RetentionDuration)
|
||||||
|
|
||||||
return retentionCutoff(db.dir, mint)
|
return retentionCutoff(db.dir, mint)
|
||||||
}
|
}
|
||||||
|
|
||||||
// headFullness returns up to which fraction of a blocks time range samples
|
|
||||||
// were already inserted.
|
|
||||||
func headFullness(h headBlock) float64 {
|
|
||||||
m := h.Meta()
|
|
||||||
a := float64(h.HighTimestamp() - m.MinTime)
|
|
||||||
b := float64(m.MaxTime - m.MinTime)
|
|
||||||
return a / b
|
|
||||||
}
|
|
||||||
|
|
||||||
// appendableHeads returns a copy of a slice of HeadBlocks that can still be appended to.
|
|
||||||
func (db *DB) appendableHeads() (r []headBlock) {
|
|
||||||
switch l := len(db.heads); l {
|
|
||||||
case 0:
|
|
||||||
case 1:
|
|
||||||
r = append(r, db.heads[0])
|
|
||||||
default:
|
|
||||||
if headFullness(db.heads[l-1]) < 0.5 {
|
|
||||||
r = append(r, db.heads[l-2])
|
|
||||||
}
|
|
||||||
r = append(r, db.heads[l-1])
|
|
||||||
}
|
|
||||||
return r
|
|
||||||
}
|
|
||||||
|
|
||||||
func (db *DB) completedHeads() (r []headBlock) {
|
|
||||||
db.mtx.RLock()
|
|
||||||
defer db.mtx.RUnlock()
|
|
||||||
|
|
||||||
db.headmtx.RLock()
|
|
||||||
defer db.headmtx.RUnlock()
|
|
||||||
|
|
||||||
if len(db.heads) < 2 {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Select all old heads unless they still have pending appenders.
|
|
||||||
for _, h := range db.heads[:len(db.heads)-2] {
|
|
||||||
if h.ActiveWriters() > 0 {
|
|
||||||
return r
|
|
||||||
}
|
|
||||||
r = append(r, h)
|
|
||||||
}
|
|
||||||
// Add the 2nd last head if the last head is more than 50% filled.
|
|
||||||
// Compacting it early allows us to free its memory before allocating
|
|
||||||
// more for the next block and thus reduces spikes.
|
|
||||||
h0 := db.heads[len(db.heads)-1]
|
|
||||||
h1 := db.heads[len(db.heads)-2]
|
|
||||||
|
|
||||||
if headFullness(h0) >= 0.5 && h1.ActiveWriters() == 0 {
|
|
||||||
r = append(r, h1)
|
|
||||||
}
|
|
||||||
return r
|
|
||||||
}
|
|
||||||
|
|
||||||
func (db *DB) compact() (changes bool, err error) {
|
func (db *DB) compact() (changes bool, err error) {
|
||||||
db.cmtx.Lock()
|
db.cmtx.Lock()
|
||||||
defer db.cmtx.Unlock()
|
defer db.cmtx.Unlock()
|
||||||
|
@ -383,20 +333,32 @@ func (db *DB) compact() (changes bool, err error) {
|
||||||
|
|
||||||
// Check whether we have pending head blocks that are ready to be persisted.
|
// Check whether we have pending head blocks that are ready to be persisted.
|
||||||
// They have the highest priority.
|
// They have the highest priority.
|
||||||
for _, h := range db.completedHeads() {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-db.stopc:
|
case <-db.stopc:
|
||||||
return changes, nil
|
return changes, nil
|
||||||
default:
|
default:
|
||||||
}
|
}
|
||||||
|
// The head has a compactable range if 1.5 level 0 ranges are between the oldest
|
||||||
|
// and newest timestamp. The 0.5 acts as a buffer of the appendable window.
|
||||||
|
if db.head.MaxTime()-db.head.MinTime() <= db.opts.BlockRanges[0]/2*3 {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
mint, maxt := rangeForTimestamp(db.head.MinTime(), db.opts.BlockRanges[0])
|
||||||
|
|
||||||
if err = db.compactor.Write(db.dir, h); err != nil {
|
// Wrap head into a range that bounds all reads to it.
|
||||||
|
head := &rangeHead{
|
||||||
|
head: db.head,
|
||||||
|
mint: mint,
|
||||||
|
maxt: maxt,
|
||||||
|
}
|
||||||
|
if err = db.compactor.Write(db.dir, head, mint, maxt); err != nil {
|
||||||
return changes, errors.Wrap(err, "persist head block")
|
return changes, errors.Wrap(err, "persist head block")
|
||||||
}
|
}
|
||||||
changes = true
|
changes = true
|
||||||
|
|
||||||
if err := os.RemoveAll(h.Dir()); err != nil {
|
if err := db.reloadBlocks(); err != nil {
|
||||||
return changes, errors.Wrap(err, "delete compacted head block")
|
return changes, errors.Wrap(err, "reload blocks")
|
||||||
}
|
}
|
||||||
runtime.GC()
|
runtime.GC()
|
||||||
}
|
}
|
||||||
|
@ -427,6 +389,10 @@ func (db *DB) compact() (changes bool, err error) {
|
||||||
return changes, errors.Wrap(err, "delete compacted block")
|
return changes, errors.Wrap(err, "delete compacted block")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if err := db.reloadBlocks(); err != nil {
|
||||||
|
return changes, errors.Wrap(err, "reload blocks")
|
||||||
|
}
|
||||||
runtime.GC()
|
runtime.GC()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -469,7 +435,7 @@ func retentionCutoff(dir string, mint int64) (bool, error) {
|
||||||
return changes, fileutil.Fsync(df)
|
return changes, fileutil.Fsync(df)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (db *DB) getBlock(id ulid.ULID) (Block, bool) {
|
func (db *DB) getBlock(id ulid.ULID) (DiskBlock, bool) {
|
||||||
for _, b := range db.blocks {
|
for _, b := range db.blocks {
|
||||||
if b.Meta().ULID == id {
|
if b.Meta().ULID == id {
|
||||||
return b, true
|
return b, true
|
||||||
|
@ -490,18 +456,12 @@ func (db *DB) reloadBlocks() (err error) {
|
||||||
var cs []io.Closer
|
var cs []io.Closer
|
||||||
defer func() { closeAll(cs...) }()
|
defer func() { closeAll(cs...) }()
|
||||||
|
|
||||||
db.mtx.Lock()
|
|
||||||
defer db.mtx.Unlock()
|
|
||||||
|
|
||||||
db.headmtx.Lock()
|
|
||||||
defer db.headmtx.Unlock()
|
|
||||||
|
|
||||||
dirs, err := blockDirs(db.dir)
|
dirs, err := blockDirs(db.dir)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Wrap(err, "find blocks")
|
return errors.Wrap(err, "find blocks")
|
||||||
}
|
}
|
||||||
var (
|
var (
|
||||||
blocks []Block
|
blocks []DiskBlock
|
||||||
exist = map[ulid.ULID]struct{}{}
|
exist = map[ulid.ULID]struct{}{}
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -513,11 +473,7 @@ func (db *DB) reloadBlocks() (err error) {
|
||||||
|
|
||||||
b, ok := db.getBlock(meta.ULID)
|
b, ok := db.getBlock(meta.ULID)
|
||||||
if !ok {
|
if !ok {
|
||||||
if meta.Compaction.Level == 0 {
|
|
||||||
b, err = db.openHeadBlock(dir)
|
|
||||||
} else {
|
|
||||||
b, err = newPersistedBlock(dir, db.chunkPool)
|
b, err = newPersistedBlock(dir, db.chunkPool)
|
||||||
}
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return errors.Wrapf(err, "open block %s", dir)
|
return errors.Wrapf(err, "open block %s", dir)
|
||||||
}
|
}
|
||||||
|
@ -532,25 +488,40 @@ func (db *DB) reloadBlocks() (err error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Close all opened blocks that no longer exist after we returned all locks.
|
// Close all opened blocks that no longer exist after we returned all locks.
|
||||||
|
// TODO(fabxc: probably races with querier still reading from them. Can
|
||||||
|
// we just abandon them and have the open FDs be GC'd automatically eventually?
|
||||||
for _, b := range db.blocks {
|
for _, b := range db.blocks {
|
||||||
if _, ok := exist[b.Meta().ULID]; !ok {
|
if _, ok := exist[b.Meta().ULID]; !ok {
|
||||||
cs = append(cs, b)
|
cs = append(cs, b)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
db.mtx.Lock()
|
||||||
db.blocks = blocks
|
db.blocks = blocks
|
||||||
db.heads = nil
|
db.mtx.Unlock()
|
||||||
|
|
||||||
for _, b := range blocks {
|
// Garbage collect data in the head if the most recent persisted block
|
||||||
if b.Meta().Compaction.Level == 0 {
|
// covers data of its current time range.
|
||||||
db.heads = append(db.heads, b.(*HeadBlock))
|
if len(blocks) == 0 {
|
||||||
|
return
|
||||||
}
|
}
|
||||||
|
maxt := blocks[len(db.blocks)-1].Meta().MaxTime
|
||||||
|
if maxt <= db.head.MinTime() {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
start := time.Now()
|
||||||
|
atomic.StoreInt64(&db.head.minTime, maxt)
|
||||||
|
db.head.gc()
|
||||||
|
db.logger.Log("msg", "head GC completed", "duration", time.Since(start))
|
||||||
|
|
||||||
|
if err := db.wal.Truncate(maxt); err != nil {
|
||||||
|
return errors.Wrapf(err, "truncate WAL at %d", maxt)
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func validateBlockSequence(bs []Block) error {
|
func validateBlockSequence(bs []DiskBlock) error {
|
||||||
if len(bs) == 0 {
|
if len(bs) == 0 {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
@ -584,10 +555,10 @@ func (db *DB) Close() error {
|
||||||
var merr MultiError
|
var merr MultiError
|
||||||
|
|
||||||
merr.Add(g.Wait())
|
merr.Add(g.Wait())
|
||||||
|
|
||||||
if db.lockf != nil {
|
if db.lockf != nil {
|
||||||
merr.Add(db.lockf.Unlock())
|
merr.Add(db.lockf.Unlock())
|
||||||
}
|
}
|
||||||
|
|
||||||
return merr.Err()
|
return merr.Err()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -611,128 +582,348 @@ func (db *DB) EnableCompactions() {
|
||||||
|
|
||||||
// Snapshot writes the current data to the directory.
|
// Snapshot writes the current data to the directory.
|
||||||
func (db *DB) Snapshot(dir string) error {
|
func (db *DB) Snapshot(dir string) error {
|
||||||
if dir == db.dir {
|
// if dir == db.dir {
|
||||||
return errors.Errorf("cannot snapshot into base directory")
|
// return errors.Errorf("cannot snapshot into base directory")
|
||||||
}
|
// }
|
||||||
db.cmtx.Lock()
|
// db.cmtx.Lock()
|
||||||
defer db.cmtx.Unlock()
|
// defer db.cmtx.Unlock()
|
||||||
|
|
||||||
db.mtx.Lock() // To block any appenders.
|
// db.mtx.Lock() // To block any appenders.
|
||||||
defer db.mtx.Unlock()
|
// defer db.mtx.Unlock()
|
||||||
|
|
||||||
blocks := db.blocks[:]
|
// blocks := db.blocks[:]
|
||||||
for _, b := range blocks {
|
// for _, b := range blocks {
|
||||||
db.logger.Log("msg", "snapshotting block", "block", b)
|
// db.logger.Log("msg", "snapshotting block", "block", b)
|
||||||
if err := b.Snapshot(dir); err != nil {
|
// if err := b.Snapshot(dir); err != nil {
|
||||||
return errors.Wrap(err, "error snapshotting headblock")
|
// return errors.Wrap(err, "error snapshotting headblock")
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Querier returns a new querier over the data partition for the given time range.
|
||||||
|
// A goroutine must not handle more than one open Querier.
|
||||||
|
func (db *DB) Querier(mint, maxt int64) Querier {
|
||||||
|
db.mtx.RLock()
|
||||||
|
|
||||||
|
blocks := db.blocksForInterval(mint, maxt)
|
||||||
|
|
||||||
|
sq := &querier{
|
||||||
|
blocks: make([]Querier, 0, len(blocks)),
|
||||||
|
db: db,
|
||||||
|
}
|
||||||
|
for _, b := range blocks {
|
||||||
|
sq.blocks = append(sq.blocks, &blockQuerier{
|
||||||
|
mint: mint,
|
||||||
|
maxt: maxt,
|
||||||
|
index: b.Index(),
|
||||||
|
chunks: b.Chunks(),
|
||||||
|
tombstones: b.Tombstones(),
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
return sq
|
||||||
|
}
|
||||||
|
|
||||||
|
// initAppender is a helper to initialize the time bounds of a the head
|
||||||
|
// upon the first sample it receives.
|
||||||
|
type initAppender struct {
|
||||||
|
app Appender
|
||||||
|
db *DB
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *initAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
||||||
|
if a.app != nil {
|
||||||
|
return a.app.Add(lset, t, v)
|
||||||
|
}
|
||||||
|
for {
|
||||||
|
// In the init state, the head has a high timestamp of math.MinInt64.
|
||||||
|
ht := a.db.head.MaxTime()
|
||||||
|
if ht != math.MinInt64 {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
cr := a.db.opts.BlockRanges[0]
|
||||||
|
mint, _ := rangeForTimestamp(t, cr)
|
||||||
|
|
||||||
|
atomic.CompareAndSwapInt64(&a.db.head.maxTime, ht, t)
|
||||||
|
atomic.StoreInt64(&a.db.head.minTime, mint-cr)
|
||||||
|
}
|
||||||
|
a.app = a.db.appender()
|
||||||
|
|
||||||
|
return a.app.Add(lset, t, v)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *initAppender) AddFast(ref string, t int64, v float64) error {
|
||||||
|
if a.app == nil {
|
||||||
|
return ErrNotFound
|
||||||
|
}
|
||||||
|
return a.app.AddFast(ref, t, v)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *initAppender) Commit() error {
|
||||||
|
if a.app == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return a.app.Commit()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a *initAppender) Rollback() error {
|
||||||
|
if a.app == nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
return a.app.Rollback()
|
||||||
|
}
|
||||||
|
|
||||||
// Appender returns a new Appender on the database.
|
// Appender returns a new Appender on the database.
|
||||||
func (db *DB) Appender() Appender {
|
func (db *DB) Appender() Appender {
|
||||||
db.metrics.activeAppenders.Inc()
|
db.metrics.activeAppenders.Inc()
|
||||||
|
|
||||||
db.mtx.RLock()
|
// The head cache might not have a starting point yet. The init appender
|
||||||
return &dbAppender{db: db}
|
// picks up the first appended timestamp as the base.
|
||||||
|
if db.head.MaxTime() == math.MinInt64 {
|
||||||
|
return &initAppender{db: db}
|
||||||
|
}
|
||||||
|
return db.appender()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (db *DB) appender() *dbAppender {
|
||||||
|
db.head.mtx.RLock()
|
||||||
|
|
||||||
|
return &dbAppender{
|
||||||
|
db: db,
|
||||||
|
head: db.head,
|
||||||
|
wal: db.wal,
|
||||||
|
mint: db.head.MaxTime() - db.opts.BlockRanges[0]/2,
|
||||||
|
samples: db.getAppendBuffer(),
|
||||||
|
highTimestamp: math.MinInt64,
|
||||||
|
lowTimestamp: math.MaxInt64,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (db *DB) getAppendBuffer() []RefSample {
|
||||||
|
b := db.appendPool.Get()
|
||||||
|
if b == nil {
|
||||||
|
return make([]RefSample, 0, 512)
|
||||||
|
}
|
||||||
|
return b.([]RefSample)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (db *DB) putAppendBuffer(b []RefSample) {
|
||||||
|
db.appendPool.Put(b[:0])
|
||||||
}
|
}
|
||||||
|
|
||||||
type dbAppender struct {
|
type dbAppender struct {
|
||||||
db *DB
|
db *DB
|
||||||
heads []*metaAppender
|
head *Head
|
||||||
|
wal WAL
|
||||||
|
mint int64
|
||||||
|
|
||||||
samples int
|
newSeries []*hashedLabels
|
||||||
|
newLabels []labels.Labels
|
||||||
|
newHashes map[uint64]uint64
|
||||||
|
|
||||||
|
samples []RefSample
|
||||||
|
highTimestamp int64
|
||||||
|
lowTimestamp int64
|
||||||
}
|
}
|
||||||
|
|
||||||
type metaAppender struct {
|
type hashedLabels struct {
|
||||||
meta BlockMeta
|
ref uint64
|
||||||
app Appender
|
hash uint64
|
||||||
|
labels labels.Labels
|
||||||
}
|
}
|
||||||
|
|
||||||
func (a *dbAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
func (a *dbAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
||||||
h, err := a.appenderAt(t)
|
if t < a.mint {
|
||||||
if err != nil {
|
return "", ErrOutOfBounds
|
||||||
return "", err
|
|
||||||
}
|
}
|
||||||
ref, err := h.app.Add(lset, t, v)
|
|
||||||
if err != nil {
|
|
||||||
return "", err
|
|
||||||
}
|
|
||||||
a.samples++
|
|
||||||
|
|
||||||
if ref == "" {
|
hash := lset.Hash()
|
||||||
return "", nil
|
refb := make([]byte, 8)
|
||||||
|
|
||||||
|
// Series exists already in the block.
|
||||||
|
if ms := a.head.get(hash, lset); ms != nil {
|
||||||
|
binary.BigEndian.PutUint64(refb, uint64(ms.ref))
|
||||||
|
return string(refb), a.AddFast(string(refb), t, v)
|
||||||
}
|
}
|
||||||
return string(append(h.meta.ULID[:], ref...)), nil
|
// Series was added in this transaction previously.
|
||||||
|
if ref, ok := a.newHashes[hash]; ok {
|
||||||
|
binary.BigEndian.PutUint64(refb, ref)
|
||||||
|
// XXX(fabxc): there's no fast path for multiple samples for the same new series
|
||||||
|
// in the same transaction. We always return the invalid empty ref. It's has not
|
||||||
|
// been a relevant use case so far and is not worth the trouble.
|
||||||
|
return "", a.AddFast(string(refb), t, v)
|
||||||
|
}
|
||||||
|
|
||||||
|
// The series is completely new.
|
||||||
|
if a.newSeries == nil {
|
||||||
|
a.newHashes = map[uint64]uint64{}
|
||||||
|
}
|
||||||
|
// First sample for new series.
|
||||||
|
ref := uint64(len(a.newSeries))
|
||||||
|
|
||||||
|
a.newSeries = append(a.newSeries, &hashedLabels{
|
||||||
|
ref: ref,
|
||||||
|
hash: hash,
|
||||||
|
labels: lset,
|
||||||
|
})
|
||||||
|
// First bit indicates its a series created in this transaction.
|
||||||
|
ref |= (1 << 63)
|
||||||
|
|
||||||
|
a.newHashes[hash] = ref
|
||||||
|
binary.BigEndian.PutUint64(refb, ref)
|
||||||
|
|
||||||
|
return "", a.AddFast(string(refb), t, v)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (a *dbAppender) AddFast(ref string, t int64, v float64) error {
|
func (a *dbAppender) AddFast(ref string, t int64, v float64) error {
|
||||||
if len(ref) < 16 {
|
if len(ref) != 8 {
|
||||||
return errors.Wrap(ErrNotFound, "invalid ref length")
|
return errors.Wrap(ErrNotFound, "invalid ref length")
|
||||||
}
|
}
|
||||||
// The first 16 bytes a ref hold the ULID of the head block.
|
var (
|
||||||
h, err := a.appenderAt(t)
|
refn = binary.BigEndian.Uint64(yoloBytes(ref))
|
||||||
if err != nil {
|
id = uint32(refn)
|
||||||
|
inTx = refn&(1<<63) != 0
|
||||||
|
)
|
||||||
|
// Distinguish between existing series and series created in
|
||||||
|
// this transaction.
|
||||||
|
if inTx {
|
||||||
|
if id > uint32(len(a.newSeries)-1) {
|
||||||
|
return errors.Wrap(ErrNotFound, "transaction series ID too high")
|
||||||
|
}
|
||||||
|
// TODO(fabxc): we also have to validate here that the
|
||||||
|
// sample sequence is valid.
|
||||||
|
// We also have to revalidate it as we switch locks and create
|
||||||
|
// the new series.
|
||||||
|
} else {
|
||||||
|
ms, ok := a.head.series[id]
|
||||||
|
if !ok {
|
||||||
|
return errors.Wrap(ErrNotFound, "unknown series")
|
||||||
|
}
|
||||||
|
if err := ms.appendable(t, v); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
// Validate the ref points to the same block we got for t.
|
|
||||||
if string(h.meta.ULID[:]) != ref[:16] {
|
|
||||||
return ErrNotFound
|
|
||||||
}
|
}
|
||||||
if err := h.app.AddFast(ref[16:], t, v); err != nil {
|
if t < a.mint {
|
||||||
// The block the ref points to might fit the given timestamp.
|
return ErrOutOfBounds
|
||||||
// We mask the error to stick with our contract.
|
|
||||||
if errors.Cause(err) == ErrOutOfBounds {
|
|
||||||
err = ErrNotFound
|
|
||||||
}
|
|
||||||
return err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
a.samples++
|
if t > a.highTimestamp {
|
||||||
|
a.highTimestamp = t
|
||||||
|
}
|
||||||
|
// if t < a.lowTimestamp {
|
||||||
|
// a.lowTimestamp = t
|
||||||
|
// }
|
||||||
|
|
||||||
|
a.samples = append(a.samples, RefSample{
|
||||||
|
Ref: refn,
|
||||||
|
T: t,
|
||||||
|
V: v,
|
||||||
|
})
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// appenderFor gets the appender for the head containing timestamp t.
|
func (a *dbAppender) createSeries() error {
|
||||||
// If the head block doesn't exist yet, it gets created.
|
if len(a.newSeries) == 0 {
|
||||||
func (a *dbAppender) appenderAt(t int64) (*metaAppender, error) {
|
return nil
|
||||||
for _, h := range a.heads {
|
}
|
||||||
if intervalContains(h.meta.MinTime, h.meta.MaxTime-1, t) {
|
a.newLabels = make([]labels.Labels, 0, len(a.newSeries))
|
||||||
return h, nil
|
base0 := len(a.head.series)
|
||||||
|
|
||||||
|
a.head.mtx.RUnlock()
|
||||||
|
defer a.head.mtx.RLock()
|
||||||
|
a.head.mtx.Lock()
|
||||||
|
defer a.head.mtx.Unlock()
|
||||||
|
|
||||||
|
base1 := len(a.head.series)
|
||||||
|
|
||||||
|
for _, 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 base1 > base0 {
|
||||||
|
if ms := a.head.get(l.hash, l.labels); ms != nil {
|
||||||
|
l.ref = uint64(ms.ref)
|
||||||
|
continue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Currently opened appenders do not cover t. Ensure the head block is
|
// Series is still new.
|
||||||
// created and add missing appenders.
|
a.newLabels = append(a.newLabels, l.labels)
|
||||||
a.db.headmtx.Lock()
|
|
||||||
|
|
||||||
if err := a.db.ensureHead(t); err != nil {
|
s := a.head.create(l.hash, l.labels)
|
||||||
a.db.headmtx.Unlock()
|
l.ref = uint64(s.ref)
|
||||||
return nil, err
|
|
||||||
}
|
}
|
||||||
|
|
||||||
var hb headBlock
|
// Write all new series to the WAL.
|
||||||
for _, h := range a.db.appendableHeads() {
|
if err := a.wal.LogSeries(a.newLabels); err != nil {
|
||||||
m := h.Meta()
|
return errors.Wrap(err, "WAL log series")
|
||||||
|
}
|
||||||
|
|
||||||
if intervalContains(m.MinTime, m.MaxTime-1, t) {
|
return nil
|
||||||
hb = h
|
}
|
||||||
|
|
||||||
|
func (a *dbAppender) Commit() error {
|
||||||
|
defer a.head.mtx.RUnlock()
|
||||||
|
|
||||||
|
defer a.db.metrics.activeAppenders.Dec()
|
||||||
|
defer a.db.putAppendBuffer(a.samples)
|
||||||
|
|
||||||
|
if err := a.createSeries(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
// We have to update the refs of samples for series we just created.
|
||||||
|
for i := range a.samples {
|
||||||
|
s := &a.samples[i]
|
||||||
|
if s.Ref&(1<<63) != 0 {
|
||||||
|
s.Ref = a.newSeries[(s.Ref<<1)>>1].ref
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Write all new samples to the WAL and add them to the
|
||||||
|
// in-mem database on success.
|
||||||
|
if err := a.wal.LogSamples(a.samples); err != nil {
|
||||||
|
return errors.Wrap(err, "WAL log samples")
|
||||||
|
}
|
||||||
|
|
||||||
|
total := uint64(len(a.samples))
|
||||||
|
|
||||||
|
for _, s := range a.samples {
|
||||||
|
series, ok := a.head.series[uint32(s.Ref)]
|
||||||
|
if !ok {
|
||||||
|
return errors.Errorf("series with ID %d not found", s.Ref)
|
||||||
|
}
|
||||||
|
if !series.append(s.T, s.V) {
|
||||||
|
total--
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
ht := a.head.MaxTime()
|
||||||
|
if a.highTimestamp <= ht {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
if a.highTimestamp-a.head.MinTime() > a.head.chunkRange/2*3 {
|
||||||
|
select {
|
||||||
|
case a.db.compactc <- struct{}{}:
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if atomic.CompareAndSwapInt64(&a.head.maxTime, ht, a.highTimestamp) {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
a.db.headmtx.Unlock()
|
|
||||||
|
|
||||||
if hb == nil {
|
return nil
|
||||||
return nil, ErrOutOfBounds
|
}
|
||||||
}
|
|
||||||
// Instantiate appender after returning headmtx!
|
|
||||||
app := &metaAppender{
|
|
||||||
meta: hb.Meta(),
|
|
||||||
app: hb.Appender(),
|
|
||||||
}
|
|
||||||
a.heads = append(a.heads, app)
|
|
||||||
|
|
||||||
return app, nil
|
func (a *dbAppender) Rollback() error {
|
||||||
|
a.head.mtx.RUnlock()
|
||||||
|
|
||||||
|
a.db.metrics.activeAppenders.Dec()
|
||||||
|
a.db.putAppendBuffer(a.samples)
|
||||||
|
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func rangeForTimestamp(t int64, width int64) (mint, maxt int64) {
|
func rangeForTimestamp(t int64, width int64) (mint, maxt int64) {
|
||||||
|
@ -740,87 +931,7 @@ func rangeForTimestamp(t int64, width int64) (mint, maxt int64) {
|
||||||
return mint, mint + width
|
return mint, mint + width
|
||||||
}
|
}
|
||||||
|
|
||||||
// ensureHead makes sure that there is a head block for the timestamp t if
|
// Delete implements deletion of metrics. It only has atomicity guarantees on a per-block basis.
|
||||||
// it is within or after the currently appendable window.
|
|
||||||
func (db *DB) ensureHead(t int64) error {
|
|
||||||
var (
|
|
||||||
mint, maxt = rangeForTimestamp(t, int64(db.opts.BlockRanges[0]))
|
|
||||||
addBuffer = len(db.blocks) == 0
|
|
||||||
last BlockMeta
|
|
||||||
)
|
|
||||||
|
|
||||||
if !addBuffer {
|
|
||||||
last = db.blocks[len(db.blocks)-1].Meta()
|
|
||||||
addBuffer = last.MaxTime <= mint-int64(db.opts.BlockRanges[0])
|
|
||||||
}
|
|
||||||
// Create another block of buffer in front if the DB is initialized or retrieving
|
|
||||||
// new data after a long gap.
|
|
||||||
// This ensures we always have a full block width of append window.
|
|
||||||
if addBuffer {
|
|
||||||
if _, err := db.createHeadBlock(mint-int64(db.opts.BlockRanges[0]), mint); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
// If the previous block reaches into our new window, make it smaller.
|
|
||||||
} else if mt := last.MaxTime; mt > mint {
|
|
||||||
mint = mt
|
|
||||||
}
|
|
||||||
if mint >= maxt {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
// Error if the requested time for a head is before the appendable window.
|
|
||||||
if len(db.heads) > 0 && t < db.heads[0].Meta().MinTime {
|
|
||||||
return ErrOutOfBounds
|
|
||||||
}
|
|
||||||
|
|
||||||
_, err := db.createHeadBlock(mint, maxt)
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *dbAppender) Commit() error {
|
|
||||||
defer a.db.metrics.activeAppenders.Dec()
|
|
||||||
defer a.db.mtx.RUnlock()
|
|
||||||
|
|
||||||
// Commits to partial appenders must be concurrent as concurrent appenders
|
|
||||||
// may have conflicting locks on head appenders.
|
|
||||||
// For high-throughput use cases the errgroup causes significant blocking. Typically,
|
|
||||||
// we just deal with a single appender and special case it.
|
|
||||||
var err error
|
|
||||||
|
|
||||||
switch len(a.heads) {
|
|
||||||
case 1:
|
|
||||||
err = a.heads[0].app.Commit()
|
|
||||||
default:
|
|
||||||
var g errgroup.Group
|
|
||||||
for _, h := range a.heads {
|
|
||||||
g.Go(h.app.Commit)
|
|
||||||
}
|
|
||||||
err = g.Wait()
|
|
||||||
}
|
|
||||||
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
// XXX(fabxc): Push the metric down into head block to account properly
|
|
||||||
// for partial appends?
|
|
||||||
a.db.metrics.samplesAppended.Add(float64(a.samples))
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *dbAppender) Rollback() error {
|
|
||||||
defer a.db.metrics.activeAppenders.Dec()
|
|
||||||
defer a.db.mtx.RUnlock()
|
|
||||||
|
|
||||||
var g errgroup.Group
|
|
||||||
|
|
||||||
for _, h := range a.heads {
|
|
||||||
g.Go(h.app.Rollback)
|
|
||||||
}
|
|
||||||
|
|
||||||
return g.Wait()
|
|
||||||
}
|
|
||||||
|
|
||||||
// Delete implements deletion of metrics.
|
|
||||||
func (db *DB) Delete(mint, maxt int64, ms ...labels.Matcher) error {
|
func (db *DB) Delete(mint, maxt int64, ms ...labels.Matcher) error {
|
||||||
db.cmtx.Lock()
|
db.cmtx.Lock()
|
||||||
defer db.cmtx.Unlock()
|
defer db.cmtx.Unlock()
|
||||||
|
@ -828,15 +939,51 @@ func (db *DB) Delete(mint, maxt int64, ms ...labels.Matcher) error {
|
||||||
db.mtx.Lock()
|
db.mtx.Lock()
|
||||||
defer db.mtx.Unlock()
|
defer db.mtx.Unlock()
|
||||||
|
|
||||||
blocks := db.blocksForInterval(mint, maxt)
|
|
||||||
|
|
||||||
var g errgroup.Group
|
var g errgroup.Group
|
||||||
|
|
||||||
for _, b := range blocks {
|
for _, b := range db.blocks {
|
||||||
g.Go(func(b Block) func() error {
|
m := b.Meta()
|
||||||
|
if intervalOverlap(mint, maxt, m.MinTime, m.MaxTime) {
|
||||||
|
g.Go(func(b DiskBlock) func() error {
|
||||||
return func() error { return b.Delete(mint, maxt, ms...) }
|
return func() error { return b.Delete(mint, maxt, ms...) }
|
||||||
}(b))
|
}(b))
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
if err := g.Wait(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
ir := db.head.Index()
|
||||||
|
|
||||||
|
pr := newPostingsReader(ir)
|
||||||
|
p, absent := pr.Select(ms...)
|
||||||
|
|
||||||
|
var stones []Stone
|
||||||
|
|
||||||
|
Outer:
|
||||||
|
for p.Next() {
|
||||||
|
series := db.head.series[p.At()]
|
||||||
|
|
||||||
|
for _, abs := range absent {
|
||||||
|
if series.lset.Get(abs) != "" {
|
||||||
|
continue Outer
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Delete only until the current values and not beyond.
|
||||||
|
t0, t1 := clampInterval(mint, maxt, series.minTime(), series.maxTime())
|
||||||
|
stones = append(stones, Stone{p.At(), Intervals{{t0, t1}}})
|
||||||
|
}
|
||||||
|
|
||||||
|
if p.Err() != nil {
|
||||||
|
return p.Err()
|
||||||
|
}
|
||||||
|
if err := db.wal.LogDeletes(stones); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
for _, s := range stones {
|
||||||
|
db.head.tombstones.add(s.ref, s.intervals[0])
|
||||||
|
}
|
||||||
|
|
||||||
if err := g.Wait(); err != nil {
|
if err := g.Wait(); err != nil {
|
||||||
return err
|
return err
|
||||||
|
@ -856,8 +1003,8 @@ func intervalContains(min, max, t int64) bool {
|
||||||
|
|
||||||
// blocksForInterval returns all blocks within the partition that may contain
|
// blocksForInterval returns all blocks within the partition that may contain
|
||||||
// data for the given time range.
|
// data for the given time range.
|
||||||
func (db *DB) blocksForInterval(mint, maxt int64) []Block {
|
func (db *DB) blocksForInterval(mint, maxt int64) []BlockReader {
|
||||||
var bs []Block
|
var bs []BlockReader
|
||||||
|
|
||||||
for _, b := range db.blocks {
|
for _, b := range db.blocks {
|
||||||
m := b.Meta()
|
m := b.Meta()
|
||||||
|
@ -865,52 +1012,13 @@ func (db *DB) blocksForInterval(mint, maxt int64) []Block {
|
||||||
bs = append(bs, b)
|
bs = append(bs, b)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if maxt >= db.head.MinTime() {
|
||||||
|
bs = append(bs, db.head)
|
||||||
|
}
|
||||||
|
|
||||||
return bs
|
return bs
|
||||||
}
|
}
|
||||||
|
|
||||||
// openHeadBlock opens the head block at dir.
|
|
||||||
func (db *DB) openHeadBlock(dir string) (*HeadBlock, error) {
|
|
||||||
var (
|
|
||||||
wdir = walDir(dir)
|
|
||||||
l = log.With(db.logger, "wal", wdir)
|
|
||||||
)
|
|
||||||
wal, err := OpenSegmentWAL(wdir, l, 5*time.Second)
|
|
||||||
if err != nil {
|
|
||||||
return nil, errors.Wrapf(err, "open WAL %s", dir)
|
|
||||||
}
|
|
||||||
|
|
||||||
h, err := OpenHeadBlock(dir, log.With(db.logger, "block", dir), wal, db.compactor)
|
|
||||||
if err != nil {
|
|
||||||
return nil, errors.Wrapf(err, "open head block %s", dir)
|
|
||||||
}
|
|
||||||
return h, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// createHeadBlock starts a new head block to append to.
|
|
||||||
func (db *DB) createHeadBlock(mint, maxt int64) (headBlock, error) {
|
|
||||||
dir, err := TouchHeadBlock(db.dir, mint, maxt)
|
|
||||||
if err != nil {
|
|
||||||
return nil, errors.Wrapf(err, "touch head block %s", dir)
|
|
||||||
}
|
|
||||||
newHead, err := db.openHeadBlock(dir)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
db.logger.Log("msg", "created head block", "ulid", newHead.meta.ULID, "mint", mint, "maxt", maxt)
|
|
||||||
|
|
||||||
db.blocks = append(db.blocks, newHead) // TODO(fabxc): this is a race!
|
|
||||||
db.heads = append(db.heads, newHead)
|
|
||||||
|
|
||||||
select {
|
|
||||||
case db.compactc <- struct{}{}:
|
|
||||||
default:
|
|
||||||
}
|
|
||||||
|
|
||||||
return newHead, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func isBlockDir(fi os.FileInfo) bool {
|
func isBlockDir(fi os.FileInfo) bool {
|
||||||
if !fi.IsDir() {
|
if !fi.IsDir() {
|
||||||
return false
|
return false
|
||||||
|
@ -1032,3 +1140,14 @@ func closeAll(cs ...io.Closer) error {
|
||||||
}
|
}
|
||||||
return merr.Err()
|
return merr.Err()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func exponential(d, min, max time.Duration) time.Duration {
|
||||||
|
d *= 2
|
||||||
|
if d < min {
|
||||||
|
d = min
|
||||||
|
}
|
||||||
|
if d > max {
|
||||||
|
d = max
|
||||||
|
}
|
||||||
|
return d
|
||||||
|
}
|
||||||
|
|
347
db_test.go
347
db_test.go
|
@ -15,8 +15,10 @@ package tsdb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
|
"math"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"os"
|
"os"
|
||||||
|
"sort"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
@ -24,8 +26,20 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
func openTestDB(t testing.TB, opts *Options) (db *DB, close func()) {
|
||||||
|
tmpdir, _ := ioutil.TempDir("", "test")
|
||||||
|
|
||||||
|
db, err := Open(tmpdir, nil, nil, opts)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
// Do not close the test database by default as it will deadlock on test failures.
|
||||||
|
return db, func() {
|
||||||
|
os.RemoveAll(tmpdir)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Convert a SeriesSet into a form useable with reflect.DeepEqual.
|
// Convert a SeriesSet into a form useable with reflect.DeepEqual.
|
||||||
func readSeriesSet(ss SeriesSet) (map[string][]sample, error) {
|
func readSeriesSet(t testing.TB, ss SeriesSet) map[string][]sample {
|
||||||
result := map[string][]sample{}
|
result := map[string][]sample{}
|
||||||
|
|
||||||
for ss.Next() {
|
for ss.Next() {
|
||||||
|
@ -37,31 +51,28 @@ func readSeriesSet(ss SeriesSet) (map[string][]sample, error) {
|
||||||
t, v := it.At()
|
t, v := it.At()
|
||||||
samples = append(samples, sample{t: t, v: v})
|
samples = append(samples, sample{t: t, v: v})
|
||||||
}
|
}
|
||||||
|
require.NoError(t, it.Err())
|
||||||
|
|
||||||
name := series.Labels().String()
|
name := series.Labels().String()
|
||||||
result[name] = samples
|
result[name] = samples
|
||||||
if err := ss.Err(); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
}
|
||||||
}
|
require.NoError(t, ss.Err())
|
||||||
return result, nil
|
|
||||||
|
return result
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestDataAvailableOnlyAfterCommit(t *testing.T) {
|
func TestDataAvailableOnlyAfterCommit(t *testing.T) {
|
||||||
tmpdir, _ := ioutil.TempDir("", "test")
|
db, close := openTestDB(t, nil)
|
||||||
defer os.RemoveAll(tmpdir)
|
defer close()
|
||||||
|
|
||||||
db, err := Open(tmpdir, nil, nil, nil)
|
|
||||||
require.NoError(t, err)
|
|
||||||
defer db.Close()
|
|
||||||
|
|
||||||
app := db.Appender()
|
app := db.Appender()
|
||||||
_, err = app.Add(labels.FromStrings("foo", "bar"), 0, 0)
|
|
||||||
|
_, err := app.Add(labels.FromStrings("foo", "bar"), 0, 0)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
querier := db.Querier(0, 1)
|
querier := db.Querier(0, 1)
|
||||||
seriesSet, err := readSeriesSet(querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
seriesSet := readSeriesSet(t, querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
||||||
require.NoError(t, err)
|
|
||||||
require.Equal(t, seriesSet, map[string][]sample{})
|
require.Equal(t, seriesSet, map[string][]sample{})
|
||||||
require.NoError(t, querier.Close())
|
require.NoError(t, querier.Close())
|
||||||
|
|
||||||
|
@ -71,23 +82,17 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) {
|
||||||
querier = db.Querier(0, 1)
|
querier = db.Querier(0, 1)
|
||||||
defer querier.Close()
|
defer querier.Close()
|
||||||
|
|
||||||
seriesSet, err = readSeriesSet(querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
seriesSet = readSeriesSet(t, querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
||||||
require.NoError(t, err)
|
|
||||||
require.Equal(t, seriesSet, map[string][]sample{`{foo="bar"}`: []sample{{t: 0, v: 0}}})
|
require.Equal(t, seriesSet, map[string][]sample{`{foo="bar"}`: []sample{{t: 0, v: 0}}})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestDataNotAvailableAfterRollback(t *testing.T) {
|
func TestDataNotAvailableAfterRollback(t *testing.T) {
|
||||||
tmpdir, _ := ioutil.TempDir("", "test")
|
db, close := openTestDB(t, nil)
|
||||||
defer os.RemoveAll(tmpdir)
|
defer close()
|
||||||
|
|
||||||
db, err := Open(tmpdir, nil, nil, nil)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("Error opening database: %q", err)
|
|
||||||
}
|
|
||||||
defer db.Close()
|
|
||||||
|
|
||||||
app := db.Appender()
|
app := db.Appender()
|
||||||
_, err = app.Add(labels.FromStrings("foo", "bar"), 0, 0)
|
_, err := app.Add(labels.FromStrings("foo", "bar"), 0, 0)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
err = app.Rollback()
|
err = app.Rollback()
|
||||||
|
@ -96,22 +101,18 @@ func TestDataNotAvailableAfterRollback(t *testing.T) {
|
||||||
querier := db.Querier(0, 1)
|
querier := db.Querier(0, 1)
|
||||||
defer querier.Close()
|
defer querier.Close()
|
||||||
|
|
||||||
seriesSet, err := readSeriesSet(querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
seriesSet := readSeriesSet(t, querier.Select(labels.NewEqualMatcher("foo", "bar")))
|
||||||
require.NoError(t, err)
|
|
||||||
require.Equal(t, seriesSet, map[string][]sample{})
|
require.Equal(t, seriesSet, map[string][]sample{})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestDBAppenderAddRef(t *testing.T) {
|
func TestDBAppenderAddRef(t *testing.T) {
|
||||||
tmpdir, _ := ioutil.TempDir("", "test")
|
db, close := openTestDB(t, nil)
|
||||||
defer os.RemoveAll(tmpdir)
|
defer close()
|
||||||
|
|
||||||
db, err := Open(tmpdir, nil, nil, nil)
|
|
||||||
require.NoError(t, err)
|
|
||||||
defer db.Close()
|
|
||||||
|
|
||||||
app1 := db.Appender()
|
app1 := db.Appender()
|
||||||
|
|
||||||
ref, err := app1.Add(labels.FromStrings("a", "b"), 0, 0)
|
ref, err := app1.Add(labels.FromStrings("a", "b"), 123, 0)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
// When a series is first created, refs don't work within that transaction.
|
// When a series is first created, refs don't work within that transaction.
|
||||||
|
@ -122,35 +123,40 @@ func TestDBAppenderAddRef(t *testing.T) {
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
app2 := db.Appender()
|
app2 := db.Appender()
|
||||||
defer app2.Rollback()
|
ref, err = app2.Add(labels.FromStrings("a", "b"), 133, 1)
|
||||||
|
|
||||||
ref, err = app2.Add(labels.FromStrings("a", "b"), 1, 1)
|
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
// Ref must be prefixed with block ULID of the block we wrote to.
|
|
||||||
id := db.blocks[len(db.blocks)-1].Meta().ULID
|
|
||||||
require.Equal(t, string(id[:]), ref[:16])
|
|
||||||
|
|
||||||
// Reference must be valid to add another sample.
|
// Reference must be valid to add another sample.
|
||||||
err = app2.AddFast(ref, 2, 2)
|
err = app2.AddFast(ref, 143, 2)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
// AddFast for the same timestamp must fail if the generation in the reference
|
// AddFast for the same timestamp must fail if the generation in the reference
|
||||||
// doesn't add up.
|
// doesn't add up.
|
||||||
refb := []byte(ref)
|
err = app2.AddFast("abc_invalid_xyz", 1, 1)
|
||||||
refb[15] ^= refb[15]
|
|
||||||
err = app2.AddFast(string(refb), 1, 1)
|
|
||||||
require.EqualError(t, errors.Cause(err), ErrNotFound.Error())
|
require.EqualError(t, errors.Cause(err), ErrNotFound.Error())
|
||||||
|
|
||||||
|
require.NoError(t, app2.Commit())
|
||||||
|
|
||||||
|
q := db.Querier(0, 200)
|
||||||
|
res := readSeriesSet(t, q.Select(labels.NewEqualMatcher("a", "b")))
|
||||||
|
|
||||||
|
require.Equal(t, map[string][]sample{
|
||||||
|
labels.FromStrings("a", "b").String(): []sample{
|
||||||
|
{t: 123, v: 0},
|
||||||
|
{t: 133, v: 1},
|
||||||
|
{t: 143, v: 2},
|
||||||
|
},
|
||||||
|
}, res)
|
||||||
|
|
||||||
|
require.NoError(t, q.Close())
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestDeleteSimple(t *testing.T) {
|
func TestDeleteSimple(t *testing.T) {
|
||||||
numSamples := int64(10)
|
numSamples := int64(10)
|
||||||
|
|
||||||
tmpdir, _ := ioutil.TempDir("", "test")
|
db, close := openTestDB(t, nil)
|
||||||
defer os.RemoveAll(tmpdir)
|
defer close()
|
||||||
|
|
||||||
db, err := Open(tmpdir, nil, nil, nil)
|
|
||||||
require.NoError(t, err)
|
|
||||||
app := db.Appender()
|
app := db.Appender()
|
||||||
|
|
||||||
smpls := make([]float64, numSamples)
|
smpls := make([]float64, numSamples)
|
||||||
|
@ -216,3 +222,246 @@ Outer:
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestAmendDatapointCausesError(t *testing.T) {
|
||||||
|
db, close := openTestDB(t, nil)
|
||||||
|
defer close()
|
||||||
|
|
||||||
|
app := db.Appender()
|
||||||
|
_, err := app.Add(labels.Labels{}, 0, 0)
|
||||||
|
require.NoError(t, err, "Failed to add sample")
|
||||||
|
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
||||||
|
|
||||||
|
app = db.Appender()
|
||||||
|
_, err = app.Add(labels.Labels{}, 0, 1)
|
||||||
|
require.Equal(t, ErrAmendSample, err)
|
||||||
|
require.NoError(t, app.Rollback(), "Unexpected error rolling back appender")
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDuplicateNaNDatapointNoAmendError(t *testing.T) {
|
||||||
|
db, close := openTestDB(t, nil)
|
||||||
|
defer close()
|
||||||
|
|
||||||
|
app := db.Appender()
|
||||||
|
_, err := app.Add(labels.Labels{}, 0, math.NaN())
|
||||||
|
require.NoError(t, err, "Failed to add sample")
|
||||||
|
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
||||||
|
|
||||||
|
app = db.Appender()
|
||||||
|
_, err = app.Add(labels.Labels{}, 0, math.NaN())
|
||||||
|
require.NoError(t, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) {
|
||||||
|
db, close := openTestDB(t, nil)
|
||||||
|
defer close()
|
||||||
|
|
||||||
|
app := db.Appender()
|
||||||
|
_, err := app.Add(labels.Labels{}, 0, math.Float64frombits(0x7ff0000000000001))
|
||||||
|
require.NoError(t, err, "Failed to add sample")
|
||||||
|
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
||||||
|
|
||||||
|
app = db.Appender()
|
||||||
|
_, err = app.Add(labels.Labels{}, 0, math.Float64frombits(0x7ff0000000000002))
|
||||||
|
require.Equal(t, ErrAmendSample, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestSkippingInvalidValuesInSameTxn(t *testing.T) {
|
||||||
|
db, close := openTestDB(t, nil)
|
||||||
|
defer close()
|
||||||
|
|
||||||
|
// Append AmendedValue.
|
||||||
|
app := db.Appender()
|
||||||
|
_, err := app.Add(labels.Labels{{"a", "b"}}, 0, 1)
|
||||||
|
require.NoError(t, err)
|
||||||
|
_, err = app.Add(labels.Labels{{"a", "b"}}, 0, 2)
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.NoError(t, app.Commit())
|
||||||
|
|
||||||
|
// Make sure the right value is stored.
|
||||||
|
q := db.Querier(0, 10)
|
||||||
|
ss := q.Select(labels.NewEqualMatcher("a", "b"))
|
||||||
|
ssMap := readSeriesSet(t, ss)
|
||||||
|
|
||||||
|
require.Equal(t, map[string][]sample{
|
||||||
|
labels.New(labels.Label{"a", "b"}).String(): []sample{{0, 1}},
|
||||||
|
}, ssMap)
|
||||||
|
|
||||||
|
require.NoError(t, q.Close())
|
||||||
|
|
||||||
|
// Append Out of Order Value.
|
||||||
|
app = db.Appender()
|
||||||
|
_, err = app.Add(labels.Labels{{"a", "b"}}, 10, 3)
|
||||||
|
require.NoError(t, err)
|
||||||
|
_, err = app.Add(labels.Labels{{"a", "b"}}, 7, 5)
|
||||||
|
require.NoError(t, err)
|
||||||
|
require.NoError(t, app.Commit())
|
||||||
|
|
||||||
|
q = db.Querier(0, 10)
|
||||||
|
ss = q.Select(labels.NewEqualMatcher("a", "b"))
|
||||||
|
ssMap = readSeriesSet(t, ss)
|
||||||
|
|
||||||
|
require.Equal(t, map[string][]sample{
|
||||||
|
labels.New(labels.Label{"a", "b"}).String(): []sample{{0, 1}, {10, 3}},
|
||||||
|
}, ssMap)
|
||||||
|
require.NoError(t, q.Close())
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestDB_e2e(t *testing.T) {
|
||||||
|
const (
|
||||||
|
numDatapoints = 1000
|
||||||
|
numRanges = 1000
|
||||||
|
timeInterval = int64(3)
|
||||||
|
maxTime = int64(2 * 1000)
|
||||||
|
minTime = int64(200)
|
||||||
|
)
|
||||||
|
// Create 8 series with 1000 data-points of different ranges and run queries.
|
||||||
|
lbls := [][]labels.Label{
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"instance", "localhost:9090"},
|
||||||
|
{"job", "prometheus"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"instance", "127.0.0.1:9090"},
|
||||||
|
{"job", "prometheus"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"instance", "127.0.0.1:9090"},
|
||||||
|
{"job", "prom-k8s"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "b"},
|
||||||
|
{"instance", "localhost:9090"},
|
||||||
|
{"job", "prom-k8s"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "c"},
|
||||||
|
{"instance", "localhost:9090"},
|
||||||
|
{"job", "prometheus"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "c"},
|
||||||
|
{"instance", "127.0.0.1:9090"},
|
||||||
|
{"job", "prometheus"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "c"},
|
||||||
|
{"instance", "127.0.0.1:9090"},
|
||||||
|
{"job", "prom-k8s"},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
{"a", "c"},
|
||||||
|
{"instance", "localhost:9090"},
|
||||||
|
{"job", "prom-k8s"},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
seriesMap := map[string][]sample{}
|
||||||
|
for _, l := range lbls {
|
||||||
|
seriesMap[labels.New(l...).String()] = []sample{}
|
||||||
|
}
|
||||||
|
|
||||||
|
db, close := openTestDB(t, nil)
|
||||||
|
defer close()
|
||||||
|
|
||||||
|
app := db.Appender()
|
||||||
|
|
||||||
|
for _, l := range lbls {
|
||||||
|
lset := labels.New(l...)
|
||||||
|
series := []sample{}
|
||||||
|
|
||||||
|
ts := rand.Int63n(300)
|
||||||
|
for i := 0; i < numDatapoints; i++ {
|
||||||
|
v := rand.Float64()
|
||||||
|
|
||||||
|
series = append(series, sample{ts, v})
|
||||||
|
|
||||||
|
_, err := app.Add(lset, ts, v)
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
ts += rand.Int63n(timeInterval) + 1
|
||||||
|
}
|
||||||
|
|
||||||
|
seriesMap[lset.String()] = series
|
||||||
|
}
|
||||||
|
|
||||||
|
require.NoError(t, app.Commit())
|
||||||
|
|
||||||
|
// Query each selector on 1000 random time-ranges.
|
||||||
|
queries := []struct {
|
||||||
|
ms []labels.Matcher
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
ms: []labels.Matcher{
|
||||||
|
labels.NewEqualMatcher("a", "b"),
|
||||||
|
labels.NewEqualMatcher("job", "prom-k8s"),
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
ms: []labels.Matcher{
|
||||||
|
labels.NewEqualMatcher("a", "c"),
|
||||||
|
labels.NewEqualMatcher("instance", "localhost:9090"),
|
||||||
|
labels.NewEqualMatcher("job", "prometheus"),
|
||||||
|
},
|
||||||
|
},
|
||||||
|
// TODO: Add Regexp Matchers.
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, qry := range queries {
|
||||||
|
matched := labels.Slice{}
|
||||||
|
for _, ls := range lbls {
|
||||||
|
s := labels.Selector(qry.ms)
|
||||||
|
if s.Matches(ls) {
|
||||||
|
matched = append(matched, ls)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
sort.Sort(matched)
|
||||||
|
|
||||||
|
for i := 0; i < numRanges; i++ {
|
||||||
|
mint := rand.Int63n(300)
|
||||||
|
maxt := mint + rand.Int63n(timeInterval*int64(numDatapoints))
|
||||||
|
|
||||||
|
t.Logf("run query %s, [%d, %d]", qry.ms, mint, maxt)
|
||||||
|
|
||||||
|
expected := map[string][]sample{}
|
||||||
|
|
||||||
|
// Build the mockSeriesSet.
|
||||||
|
for _, m := range matched {
|
||||||
|
smpls := boundedSamples(seriesMap[m.String()], mint, maxt)
|
||||||
|
if len(smpls) > 0 {
|
||||||
|
expected[m.String()] = smpls
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
q := db.Querier(mint, maxt)
|
||||||
|
ss := q.Select(qry.ms...)
|
||||||
|
|
||||||
|
result := map[string][]sample{}
|
||||||
|
|
||||||
|
for ss.Next() {
|
||||||
|
x := ss.At()
|
||||||
|
|
||||||
|
smpls, err := expandSeriesIterator(x.Iterator())
|
||||||
|
require.NoError(t, err)
|
||||||
|
|
||||||
|
if len(smpls) > 0 {
|
||||||
|
result[x.Labels().String()] = smpls
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
require.NoError(t, ss.Err())
|
||||||
|
require.Equal(t, expected, result)
|
||||||
|
|
||||||
|
q.Close()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
760
head.go
760
head.go
|
@ -16,18 +16,11 @@ package tsdb
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
"math/rand"
|
|
||||||
"os"
|
|
||||||
"path/filepath"
|
|
||||||
"sort"
|
"sort"
|
||||||
"sync"
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
|
||||||
|
|
||||||
"encoding/binary"
|
|
||||||
|
|
||||||
"github.com/go-kit/kit/log"
|
"github.com/go-kit/kit/log"
|
||||||
"github.com/oklog/ulid"
|
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
"github.com/prometheus/tsdb/chunks"
|
"github.com/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/tsdb/labels"
|
"github.com/prometheus/tsdb/labels"
|
||||||
|
@ -50,20 +43,17 @@ var (
|
||||||
ErrOutOfBounds = errors.New("out of bounds")
|
ErrOutOfBounds = errors.New("out of bounds")
|
||||||
)
|
)
|
||||||
|
|
||||||
// HeadBlock handles reads and writes of time series data within a time window.
|
// Head handles reads and writes of time series data within a time window.
|
||||||
type HeadBlock struct {
|
type Head struct {
|
||||||
|
chunkRange int64
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
dir string
|
|
||||||
wal WAL
|
|
||||||
compactor Compactor
|
|
||||||
|
|
||||||
activeWriters uint64
|
minTime, maxTime int64
|
||||||
highTimestamp int64
|
lastSeriesID uint32
|
||||||
closed bool
|
|
||||||
|
|
||||||
// descs holds all chunk descs for the head block. Each chunk implicitly
|
// descs holds all chunk descs for the head block. Each chunk implicitly
|
||||||
// is assigned the index as its ID.
|
// is assigned the index as its ID.
|
||||||
series []*memSeries
|
series map[uint32]*memSeries
|
||||||
// hashes contains a collision map of label set hashes of chunks
|
// hashes contains a collision map of label set hashes of chunks
|
||||||
// to their chunk descs.
|
// to their chunk descs.
|
||||||
hashes map[uint64][]*memSeries
|
hashes map[uint64][]*memSeries
|
||||||
|
@ -73,70 +63,37 @@ type HeadBlock struct {
|
||||||
postings *memPostings // postings lists for terms
|
postings *memPostings // postings lists for terms
|
||||||
|
|
||||||
tombstones tombstoneReader
|
tombstones tombstoneReader
|
||||||
|
|
||||||
meta BlockMeta
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// TouchHeadBlock atomically touches a new head block in dir for
|
// NewHead opens the head block in dir.
|
||||||
// samples in the range [mint,maxt).
|
func NewHead(l log.Logger, wal WALReader, chunkRange int64) (*Head, error) {
|
||||||
func TouchHeadBlock(dir string, mint, maxt int64) (string, error) {
|
h := &Head{
|
||||||
entropy := rand.New(rand.NewSource(time.Now().UnixNano()))
|
chunkRange: chunkRange,
|
||||||
|
minTime: math.MaxInt64,
|
||||||
ulid, err := ulid.New(ulid.Now(), entropy)
|
maxTime: math.MinInt64,
|
||||||
if err != nil {
|
series: map[uint32]*memSeries{},
|
||||||
return "", err
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make head block creation appear atomic.
|
|
||||||
dir = filepath.Join(dir, ulid.String())
|
|
||||||
tmp := dir + ".tmp"
|
|
||||||
|
|
||||||
if err := os.MkdirAll(tmp, 0777); err != nil {
|
|
||||||
return "", err
|
|
||||||
}
|
|
||||||
|
|
||||||
if err := writeMetaFile(tmp, &BlockMeta{
|
|
||||||
ULID: ulid,
|
|
||||||
MinTime: mint,
|
|
||||||
MaxTime: maxt,
|
|
||||||
}); err != nil {
|
|
||||||
return "", err
|
|
||||||
}
|
|
||||||
|
|
||||||
return dir, renameFile(tmp, dir)
|
|
||||||
}
|
|
||||||
|
|
||||||
// OpenHeadBlock opens the head block in dir.
|
|
||||||
func OpenHeadBlock(dir string, l log.Logger, wal WAL, c Compactor) (*HeadBlock, error) {
|
|
||||||
meta, err := readMetaFile(dir)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
h := &HeadBlock{
|
|
||||||
dir: dir,
|
|
||||||
wal: wal,
|
|
||||||
compactor: c,
|
|
||||||
series: []*memSeries{nil}, // 0 is not a valid posting, filled with nil.
|
|
||||||
hashes: map[uint64][]*memSeries{},
|
hashes: map[uint64][]*memSeries{},
|
||||||
values: map[string]stringset{},
|
values: map[string]stringset{},
|
||||||
symbols: map[string]struct{}{},
|
symbols: map[string]struct{}{},
|
||||||
postings: &memPostings{m: make(map[term][]uint32)},
|
postings: &memPostings{m: make(map[term][]uint32)},
|
||||||
meta: *meta,
|
|
||||||
tombstones: newEmptyTombstoneReader(),
|
tombstones: newEmptyTombstoneReader(),
|
||||||
}
|
}
|
||||||
return h, h.init()
|
if wal == nil {
|
||||||
|
wal = NopWAL{}
|
||||||
|
}
|
||||||
|
return h, h.init(wal)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *HeadBlock) init() error {
|
func (h *Head) String() string {
|
||||||
r := h.wal.Reader()
|
return "<head>"
|
||||||
|
}
|
||||||
|
|
||||||
|
func (h *Head) init(r WALReader) error {
|
||||||
|
|
||||||
seriesFunc := func(series []labels.Labels) error {
|
seriesFunc := func(series []labels.Labels) error {
|
||||||
for _, lset := range series {
|
for _, lset := range series {
|
||||||
h.create(lset.Hash(), lset)
|
h.create(lset.Hash(), lset)
|
||||||
h.meta.Stats.NumSeries++
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
samplesFunc := func(samples []RefSample) error {
|
samplesFunc := func(samples []RefSample) error {
|
||||||
|
@ -145,12 +102,7 @@ func (h *HeadBlock) init() error {
|
||||||
return errors.Errorf("unknown series reference %d (max %d); abort WAL restore",
|
return errors.Errorf("unknown series reference %d (max %d); abort WAL restore",
|
||||||
s.Ref, len(h.series))
|
s.Ref, len(h.series))
|
||||||
}
|
}
|
||||||
h.series[s.Ref].append(s.T, s.V)
|
h.series[uint32(s.Ref)].append(s.T, s.V)
|
||||||
|
|
||||||
if !h.inBounds(s.T) {
|
|
||||||
return errors.Wrap(ErrOutOfBounds, "consume WAL")
|
|
||||||
}
|
|
||||||
h.meta.Stats.NumSamples++
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
|
@ -172,444 +124,234 @@ func (h *HeadBlock) init() error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// inBounds returns true if the given timestamp is within the valid
|
// gc removes data before the minimum timestmap from the head.
|
||||||
// time bounds of the block.
|
func (h *Head) gc() {
|
||||||
func (h *HeadBlock) inBounds(t int64) bool {
|
// Only data strictly lower than this timestamp must be deleted.
|
||||||
return t >= h.meta.MinTime && t <= h.meta.MaxTime
|
mint := h.MinTime()
|
||||||
}
|
|
||||||
|
|
||||||
func (h *HeadBlock) String() string {
|
deletedHashes := map[uint64][]uint32{}
|
||||||
return h.meta.ULID.String()
|
|
||||||
}
|
h.mtx.RLock()
|
||||||
|
|
||||||
|
for hash, ss := range h.hashes {
|
||||||
|
for _, s := range ss {
|
||||||
|
s.mtx.Lock()
|
||||||
|
s.truncateChunksBefore(mint)
|
||||||
|
|
||||||
|
if len(s.chunks) == 0 {
|
||||||
|
deletedHashes[hash] = append(deletedHashes[hash], s.ref)
|
||||||
|
}
|
||||||
|
|
||||||
|
s.mtx.Unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
deletedIDs := make(map[uint32]struct{}, len(deletedHashes))
|
||||||
|
|
||||||
|
h.mtx.RUnlock()
|
||||||
|
|
||||||
// Close syncs all data and closes underlying resources of the head block.
|
|
||||||
func (h *HeadBlock) Close() error {
|
|
||||||
h.mtx.Lock()
|
h.mtx.Lock()
|
||||||
defer h.mtx.Unlock()
|
defer h.mtx.Unlock()
|
||||||
|
|
||||||
if err := h.wal.Close(); err != nil {
|
for hash, ids := range deletedHashes {
|
||||||
return errors.Wrapf(err, "close WAL for head %s", h.dir)
|
|
||||||
|
inIDs := func(id uint32) bool {
|
||||||
|
for _, o := range ids {
|
||||||
|
if o == id {
|
||||||
|
return true
|
||||||
}
|
}
|
||||||
// Check whether the head block still exists in the underlying dir
|
|
||||||
// or has already been replaced with a compacted version or removed.
|
|
||||||
meta, err := readMetaFile(h.dir)
|
|
||||||
if os.IsNotExist(err) {
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
if err != nil {
|
return false
|
||||||
return err
|
}
|
||||||
|
var rem []*memSeries
|
||||||
|
|
||||||
|
for _, s := range h.hashes[hash] {
|
||||||
|
if !inIDs(s.ref) {
|
||||||
|
rem = append(rem, s)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
deletedIDs[s.ref] = struct{}{}
|
||||||
|
// We switched locks and the series might have received new samples by now,
|
||||||
|
// check again.
|
||||||
|
s.mtx.Lock()
|
||||||
|
chkCount := len(s.chunks)
|
||||||
|
s.mtx.Unlock()
|
||||||
|
|
||||||
|
if chkCount > 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
delete(h.series, s.ref)
|
||||||
|
}
|
||||||
|
if len(rem) > 0 {
|
||||||
|
h.hashes[hash] = rem
|
||||||
|
} else {
|
||||||
|
delete(h.hashes, hash)
|
||||||
}
|
}
|
||||||
if meta.ULID == h.meta.ULID {
|
|
||||||
return writeMetaFile(h.dir, &h.meta)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
h.closed = true
|
for t, p := range h.postings.m {
|
||||||
return nil
|
repl := make([]uint32, 0, len(p))
|
||||||
|
|
||||||
|
for _, id := range p {
|
||||||
|
if _, ok := deletedIDs[id]; !ok {
|
||||||
|
repl = append(repl, id)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(repl) == 0 {
|
||||||
|
delete(h.postings.m, t)
|
||||||
|
} else {
|
||||||
|
h.postings.m[t] = repl
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
symbols := make(map[string]struct{}, len(h.symbols))
|
||||||
|
values := make(map[string]stringset, len(h.values))
|
||||||
|
|
||||||
|
for t := range h.postings.m {
|
||||||
|
symbols[t.name] = struct{}{}
|
||||||
|
symbols[t.value] = struct{}{}
|
||||||
|
|
||||||
|
ss, ok := values[t.name]
|
||||||
|
if !ok {
|
||||||
|
ss = stringset{}
|
||||||
|
values[t.name] = ss
|
||||||
|
}
|
||||||
|
ss.set(t.value)
|
||||||
|
}
|
||||||
|
|
||||||
|
h.symbols = symbols
|
||||||
|
h.values = values
|
||||||
}
|
}
|
||||||
|
|
||||||
// Meta returns a BlockMeta for the head block.
|
func (h *Head) Tombstones() TombstoneReader {
|
||||||
func (h *HeadBlock) Meta() BlockMeta {
|
|
||||||
m := BlockMeta{
|
|
||||||
ULID: h.meta.ULID,
|
|
||||||
MinTime: h.meta.MinTime,
|
|
||||||
MaxTime: h.meta.MaxTime,
|
|
||||||
Compaction: h.meta.Compaction,
|
|
||||||
}
|
|
||||||
|
|
||||||
m.Stats.NumChunks = atomic.LoadUint64(&h.meta.Stats.NumChunks)
|
|
||||||
m.Stats.NumSeries = atomic.LoadUint64(&h.meta.Stats.NumSeries)
|
|
||||||
m.Stats.NumSamples = atomic.LoadUint64(&h.meta.Stats.NumSamples)
|
|
||||||
|
|
||||||
return m
|
|
||||||
}
|
|
||||||
|
|
||||||
// Tombstones returns the TombstoneReader against the block.
|
|
||||||
func (h *HeadBlock) Tombstones() TombstoneReader {
|
|
||||||
return h.tombstones
|
return h.tombstones
|
||||||
}
|
}
|
||||||
|
|
||||||
// Delete implements headBlock.
|
|
||||||
func (h *HeadBlock) Delete(mint int64, maxt int64, ms ...labels.Matcher) error {
|
|
||||||
ir := h.Index()
|
|
||||||
|
|
||||||
pr := newPostingsReader(ir)
|
|
||||||
p, absent := pr.Select(ms...)
|
|
||||||
|
|
||||||
var stones []Stone
|
|
||||||
|
|
||||||
Outer:
|
|
||||||
for p.Next() {
|
|
||||||
ref := p.At()
|
|
||||||
lset := h.series[ref].lset
|
|
||||||
for _, abs := range absent {
|
|
||||||
if lset.Get(abs) != "" {
|
|
||||||
continue Outer
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Delete only until the current values and not beyond.
|
|
||||||
tmin, tmax := clampInterval(mint, maxt, h.series[ref].chunks[0].minTime, h.series[ref].head().maxTime)
|
|
||||||
stones = append(stones, Stone{ref, Intervals{{tmin, tmax}}})
|
|
||||||
}
|
|
||||||
|
|
||||||
if p.Err() != nil {
|
|
||||||
return p.Err()
|
|
||||||
}
|
|
||||||
if err := h.wal.LogDeletes(stones); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, s := range stones {
|
|
||||||
h.tombstones.add(s.ref, s.intervals[0])
|
|
||||||
}
|
|
||||||
|
|
||||||
h.meta.Stats.NumTombstones = uint64(len(h.tombstones))
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// Snapshot persists the current state of the headblock to the given directory.
|
|
||||||
// Callers must ensure that there are no active appenders against the block.
|
|
||||||
// DB does this by acquiring its own write lock.
|
|
||||||
func (h *HeadBlock) Snapshot(snapshotDir string) error {
|
|
||||||
if h.meta.Stats.NumSeries == 0 {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
return h.compactor.Write(snapshotDir, h)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Dir returns the directory of the block.
|
|
||||||
func (h *HeadBlock) Dir() string { return h.dir }
|
|
||||||
|
|
||||||
// Index returns an IndexReader against the block.
|
// Index returns an IndexReader against the block.
|
||||||
func (h *HeadBlock) Index() IndexReader {
|
func (h *Head) Index() IndexReader {
|
||||||
h.mtx.RLock()
|
return h.indexRange(math.MinInt64, math.MaxInt64)
|
||||||
defer h.mtx.RUnlock()
|
}
|
||||||
|
|
||||||
return &headIndexReader{HeadBlock: h, maxSeries: uint32(len(h.series) - 1)}
|
func (h *Head) indexRange(mint, maxt int64) *headIndexReader {
|
||||||
|
if hmin := h.MinTime(); hmin > mint {
|
||||||
|
mint = hmin
|
||||||
|
}
|
||||||
|
return &headIndexReader{head: h, mint: mint, maxt: maxt}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Chunks returns a ChunkReader against the block.
|
// Chunks returns a ChunkReader against the block.
|
||||||
func (h *HeadBlock) Chunks() ChunkReader { return &headChunkReader{h} }
|
func (h *Head) Chunks() ChunkReader {
|
||||||
|
return h.chunksRange(math.MinInt64, math.MaxInt64)
|
||||||
// Querier returns a new Querier against the block for the range [mint, maxt].
|
|
||||||
func (h *HeadBlock) Querier(mint, maxt int64) Querier {
|
|
||||||
h.mtx.RLock()
|
|
||||||
if h.closed {
|
|
||||||
panic(fmt.Sprintf("block %s already closed", h.dir))
|
|
||||||
}
|
|
||||||
h.mtx.RUnlock()
|
|
||||||
|
|
||||||
return &blockQuerier{
|
|
||||||
mint: mint,
|
|
||||||
maxt: maxt,
|
|
||||||
index: h.Index(),
|
|
||||||
chunks: h.Chunks(),
|
|
||||||
tombstones: h.Tombstones(),
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Appender returns a new Appender against the head block.
|
func (h *Head) chunksRange(mint, maxt int64) *headChunkReader {
|
||||||
func (h *HeadBlock) Appender() Appender {
|
if hmin := h.MinTime(); hmin > mint {
|
||||||
atomic.AddUint64(&h.activeWriters, 1)
|
mint = hmin
|
||||||
|
|
||||||
h.mtx.RLock()
|
|
||||||
|
|
||||||
if h.closed {
|
|
||||||
panic(fmt.Sprintf("block %s already closed", h.dir))
|
|
||||||
}
|
}
|
||||||
return &headAppender{HeadBlock: h, samples: getHeadAppendBuffer()}
|
return &headChunkReader{head: h, mint: mint, maxt: maxt}
|
||||||
}
|
}
|
||||||
|
|
||||||
// ActiveWriters returns true if the block has open write transactions.
|
// MinTime returns the lowest time bound on visible data in the head.
|
||||||
func (h *HeadBlock) ActiveWriters() int {
|
func (h *Head) MinTime() int64 {
|
||||||
return int(atomic.LoadUint64(&h.activeWriters))
|
return atomic.LoadInt64(&h.minTime)
|
||||||
}
|
}
|
||||||
|
|
||||||
// HighTimestamp returns the highest inserted sample timestamp.
|
// MaxTime returns the highest timestamp seen in data of the head.
|
||||||
func (h *HeadBlock) HighTimestamp() int64 {
|
func (h *Head) MaxTime() int64 {
|
||||||
return atomic.LoadInt64(&h.highTimestamp)
|
return atomic.LoadInt64(&h.maxTime)
|
||||||
}
|
|
||||||
|
|
||||||
var headPool = sync.Pool{}
|
|
||||||
|
|
||||||
func getHeadAppendBuffer() []RefSample {
|
|
||||||
b := headPool.Get()
|
|
||||||
if b == nil {
|
|
||||||
return make([]RefSample, 0, 512)
|
|
||||||
}
|
|
||||||
return b.([]RefSample)
|
|
||||||
}
|
|
||||||
|
|
||||||
func putHeadAppendBuffer(b []RefSample) {
|
|
||||||
headPool.Put(b[:0])
|
|
||||||
}
|
|
||||||
|
|
||||||
type headAppender struct {
|
|
||||||
*HeadBlock
|
|
||||||
|
|
||||||
newSeries []*hashedLabels
|
|
||||||
newLabels []labels.Labels
|
|
||||||
newHashes map[uint64]uint64
|
|
||||||
|
|
||||||
samples []RefSample
|
|
||||||
highTimestamp int64
|
|
||||||
}
|
|
||||||
|
|
||||||
type hashedLabels struct {
|
|
||||||
ref uint64
|
|
||||||
hash uint64
|
|
||||||
labels labels.Labels
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
|
||||||
if !a.inBounds(t) {
|
|
||||||
return "", ErrOutOfBounds
|
|
||||||
}
|
|
||||||
|
|
||||||
hash := lset.Hash()
|
|
||||||
refb := make([]byte, 8)
|
|
||||||
|
|
||||||
// Series exists already in the block.
|
|
||||||
if ms := a.get(hash, lset); ms != nil {
|
|
||||||
binary.BigEndian.PutUint64(refb, uint64(ms.ref))
|
|
||||||
return string(refb), a.AddFast(string(refb), t, v)
|
|
||||||
}
|
|
||||||
// Series was added in this transaction previously.
|
|
||||||
if ref, ok := a.newHashes[hash]; ok {
|
|
||||||
binary.BigEndian.PutUint64(refb, ref)
|
|
||||||
// XXX(fabxc): there's no fast path for multiple samples for the same new series
|
|
||||||
// in the same transaction. We always return the invalid empty ref. It's has not
|
|
||||||
// been a relevant use case so far and is not worth the trouble.
|
|
||||||
return "", a.AddFast(string(refb), t, v)
|
|
||||||
}
|
|
||||||
|
|
||||||
// The series is completely new.
|
|
||||||
if a.newSeries == nil {
|
|
||||||
a.newHashes = map[uint64]uint64{}
|
|
||||||
}
|
|
||||||
// First sample for new series.
|
|
||||||
ref := uint64(len(a.newSeries))
|
|
||||||
|
|
||||||
a.newSeries = append(a.newSeries, &hashedLabels{
|
|
||||||
ref: ref,
|
|
||||||
hash: hash,
|
|
||||||
labels: lset,
|
|
||||||
})
|
|
||||||
// First bit indicates its a series created in this transaction.
|
|
||||||
ref |= (1 << 63)
|
|
||||||
|
|
||||||
a.newHashes[hash] = ref
|
|
||||||
binary.BigEndian.PutUint64(refb, ref)
|
|
||||||
|
|
||||||
return "", a.AddFast(string(refb), t, v)
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *headAppender) AddFast(ref string, t int64, v float64) error {
|
|
||||||
if len(ref) != 8 {
|
|
||||||
return errors.Wrap(ErrNotFound, "invalid ref length")
|
|
||||||
}
|
|
||||||
var (
|
|
||||||
refn = binary.BigEndian.Uint64(yoloBytes(ref))
|
|
||||||
id = (refn << 1) >> 1
|
|
||||||
inTx = refn&(1<<63) != 0
|
|
||||||
)
|
|
||||||
// Distinguish between existing series and series created in
|
|
||||||
// this transaction.
|
|
||||||
if inTx {
|
|
||||||
if id > uint64(len(a.newSeries)-1) {
|
|
||||||
return errors.Wrap(ErrNotFound, "transaction series ID too high")
|
|
||||||
}
|
|
||||||
// TODO(fabxc): we also have to validate here that the
|
|
||||||
// sample sequence is valid.
|
|
||||||
// We also have to revalidate it as we switch locks and create
|
|
||||||
// the new series.
|
|
||||||
} else if id > uint64(len(a.series)) {
|
|
||||||
return errors.Wrap(ErrNotFound, "transaction series ID too high")
|
|
||||||
} else {
|
|
||||||
ms := a.series[id]
|
|
||||||
if ms == nil {
|
|
||||||
return errors.Wrap(ErrNotFound, "nil series")
|
|
||||||
}
|
|
||||||
// TODO(fabxc): memory series should be locked here already.
|
|
||||||
// Only problem is release of locks in case of a rollback.
|
|
||||||
c := ms.head()
|
|
||||||
|
|
||||||
if !a.inBounds(t) {
|
|
||||||
return ErrOutOfBounds
|
|
||||||
}
|
|
||||||
if t < c.maxTime {
|
|
||||||
return ErrOutOfOrderSample
|
|
||||||
}
|
|
||||||
|
|
||||||
// We are allowing exact duplicates as we can encounter them in valid cases
|
|
||||||
// like federation and erroring out at that time would be extremely noisy.
|
|
||||||
if c.maxTime == t && math.Float64bits(ms.lastValue) != math.Float64bits(v) {
|
|
||||||
return ErrAmendSample
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if t > a.highTimestamp {
|
|
||||||
a.highTimestamp = t
|
|
||||||
}
|
|
||||||
|
|
||||||
a.samples = append(a.samples, RefSample{
|
|
||||||
Ref: refn,
|
|
||||||
T: t,
|
|
||||||
V: v,
|
|
||||||
})
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *headAppender) createSeries() error {
|
|
||||||
if len(a.newSeries) == 0 {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
a.newLabels = make([]labels.Labels, 0, len(a.newSeries))
|
|
||||||
base0 := len(a.series)
|
|
||||||
|
|
||||||
a.mtx.RUnlock()
|
|
||||||
defer a.mtx.RLock()
|
|
||||||
a.mtx.Lock()
|
|
||||||
defer a.mtx.Unlock()
|
|
||||||
|
|
||||||
base1 := len(a.series)
|
|
||||||
|
|
||||||
for _, 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 base1 > base0 {
|
|
||||||
if ms := a.get(l.hash, l.labels); ms != nil {
|
|
||||||
l.ref = uint64(ms.ref)
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Series is still new.
|
|
||||||
a.newLabels = append(a.newLabels, l.labels)
|
|
||||||
l.ref = uint64(len(a.series))
|
|
||||||
|
|
||||||
a.create(l.hash, l.labels)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Write all new series to the WAL.
|
|
||||||
if err := a.wal.LogSeries(a.newLabels); err != nil {
|
|
||||||
return errors.Wrap(err, "WAL log series")
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *headAppender) Commit() error {
|
|
||||||
defer atomic.AddUint64(&a.activeWriters, ^uint64(0))
|
|
||||||
defer putHeadAppendBuffer(a.samples)
|
|
||||||
defer a.mtx.RUnlock()
|
|
||||||
|
|
||||||
if err := a.createSeries(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
// We have to update the refs of samples for series we just created.
|
|
||||||
for i := range a.samples {
|
|
||||||
s := &a.samples[i]
|
|
||||||
if s.Ref&(1<<63) != 0 {
|
|
||||||
s.Ref = a.newSeries[(s.Ref<<1)>>1].ref
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Write all new samples to the WAL and add them to the
|
|
||||||
// in-mem database on success.
|
|
||||||
if err := a.wal.LogSamples(a.samples); err != nil {
|
|
||||||
return errors.Wrap(err, "WAL log samples")
|
|
||||||
}
|
|
||||||
|
|
||||||
total := uint64(len(a.samples))
|
|
||||||
|
|
||||||
for _, s := range a.samples {
|
|
||||||
if !a.series[s.Ref].append(s.T, s.V) {
|
|
||||||
total--
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
atomic.AddUint64(&a.meta.Stats.NumSamples, total)
|
|
||||||
atomic.AddUint64(&a.meta.Stats.NumSeries, uint64(len(a.newSeries)))
|
|
||||||
|
|
||||||
for {
|
|
||||||
ht := a.HeadBlock.HighTimestamp()
|
|
||||||
if a.highTimestamp <= ht {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
if atomic.CompareAndSwapInt64(&a.HeadBlock.highTimestamp, ht, a.highTimestamp) {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (a *headAppender) Rollback() error {
|
|
||||||
a.mtx.RUnlock()
|
|
||||||
atomic.AddUint64(&a.activeWriters, ^uint64(0))
|
|
||||||
putHeadAppendBuffer(a.samples)
|
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
type headChunkReader struct {
|
type headChunkReader struct {
|
||||||
*HeadBlock
|
head *Head
|
||||||
|
mint, maxt int64
|
||||||
|
}
|
||||||
|
|
||||||
|
func (h *headChunkReader) Close() error {
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Chunk returns the chunk for the reference number.
|
// Chunk returns the chunk for the reference number.
|
||||||
func (h *headChunkReader) Chunk(ref uint64) (chunks.Chunk, error) {
|
func (h *headChunkReader) Chunk(ref uint64) (chunks.Chunk, error) {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
si := ref >> 32
|
s := h.head.series[uint32(ref>>32)]
|
||||||
ci := (ref << 32) >> 32
|
|
||||||
|
|
||||||
c := &safeChunk{
|
s.mtx.RLock()
|
||||||
Chunk: h.series[si].chunks[ci].chunk,
|
cid := int((ref << 32) >> 32)
|
||||||
s: h.series[si],
|
c := s.chunk(cid)
|
||||||
i: int(ci),
|
s.mtx.RUnlock()
|
||||||
|
|
||||||
|
// Do not expose chunks that are outside of the specified range.
|
||||||
|
if !intervalOverlap(c.minTime, c.maxTime, h.mint, h.maxt) {
|
||||||
|
return nil, ErrNotFound
|
||||||
}
|
}
|
||||||
return c, nil
|
|
||||||
|
return &safeChunk{
|
||||||
|
Chunk: c.chunk,
|
||||||
|
s: s,
|
||||||
|
cid: cid,
|
||||||
|
}, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
type safeChunk struct {
|
type safeChunk struct {
|
||||||
chunks.Chunk
|
chunks.Chunk
|
||||||
s *memSeries
|
s *memSeries
|
||||||
i int
|
cid int
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *safeChunk) Iterator() chunks.Iterator {
|
func (c *safeChunk) Iterator() chunks.Iterator {
|
||||||
c.s.mtx.RLock()
|
c.s.mtx.RLock()
|
||||||
defer c.s.mtx.RUnlock()
|
defer c.s.mtx.RUnlock()
|
||||||
return c.s.iterator(c.i)
|
return c.s.iterator(c.cid)
|
||||||
}
|
}
|
||||||
|
|
||||||
// func (c *safeChunk) Appender() (chunks.Appender, error) { panic("illegal") }
|
// func (c *safeChunk) Appender() (chunks.Appender, error) { panic("illegal") }
|
||||||
// func (c *safeChunk) Bytes() []byte { panic("illegal") }
|
// func (c *safeChunk) Bytes() []byte { panic("illegal") }
|
||||||
// func (c *safeChunk) Encoding() chunks.Encoding { panic("illegal") }
|
// func (c *safeChunk) Encoding() chunks.Encoding { panic("illegal") }
|
||||||
|
|
||||||
|
type rangeHead struct {
|
||||||
|
head *Head
|
||||||
|
mint, maxt int64
|
||||||
|
}
|
||||||
|
|
||||||
|
func (h *rangeHead) Index() IndexReader {
|
||||||
|
return h.head.indexRange(h.mint, h.maxt)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (h *rangeHead) Chunks() ChunkReader {
|
||||||
|
return h.head.chunksRange(h.mint, h.maxt)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (h *rangeHead) Tombstones() TombstoneReader {
|
||||||
|
return newEmptyTombstoneReader()
|
||||||
|
}
|
||||||
|
|
||||||
type headIndexReader struct {
|
type headIndexReader struct {
|
||||||
*HeadBlock
|
head *Head
|
||||||
// Highest series that existed when the index reader was instantiated.
|
mint, maxt int64
|
||||||
maxSeries uint32
|
}
|
||||||
|
|
||||||
|
func (h *headIndexReader) Close() error {
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *headIndexReader) Symbols() (map[string]struct{}, error) {
|
func (h *headIndexReader) Symbols() (map[string]struct{}, error) {
|
||||||
return h.symbols, nil
|
return h.head.symbols, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// LabelValues returns the possible label values
|
// LabelValues returns the possible label values
|
||||||
func (h *headIndexReader) LabelValues(names ...string) (StringTuples, error) {
|
func (h *headIndexReader) LabelValues(names ...string) (StringTuples, error) {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
if len(names) != 1 {
|
if len(names) != 1 {
|
||||||
return nil, errInvalidSize
|
return nil, errInvalidSize
|
||||||
}
|
}
|
||||||
var sl []string
|
var sl []string
|
||||||
|
|
||||||
for s := range h.values[names[0]] {
|
for s := range h.head.values[names[0]] {
|
||||||
sl = append(sl, s)
|
sl = append(sl, s)
|
||||||
}
|
}
|
||||||
sort.Strings(sl)
|
sort.Strings(sl)
|
||||||
|
@ -619,46 +361,51 @@ func (h *headIndexReader) 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 *headIndexReader) Postings(name, value string) (Postings, error) {
|
func (h *headIndexReader) Postings(name, value string) (Postings, error) {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
return h.postings.get(term{name: name, value: value}), nil
|
return h.head.postings.get(term{name: name, value: value}), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *headIndexReader) SortedPostings(p Postings) Postings {
|
func (h *headIndexReader) SortedPostings(p Postings) Postings {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
ep := make([]uint32, 0, 1024)
|
ep := make([]uint32, 0, 1024)
|
||||||
|
|
||||||
for p.Next() {
|
for p.Next() {
|
||||||
// Skip posting entries that include series added after we
|
|
||||||
// instantiated the index reader.
|
|
||||||
if p.At() > h.maxSeries {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
ep = append(ep, p.At())
|
ep = append(ep, p.At())
|
||||||
}
|
}
|
||||||
if err := p.Err(); err != nil {
|
if err := p.Err(); err != nil {
|
||||||
return errPostings{err: errors.Wrap(err, "expand postings")}
|
return errPostings{err: errors.Wrap(err, "expand postings")}
|
||||||
}
|
}
|
||||||
|
var err error
|
||||||
|
|
||||||
sort.Slice(ep, func(i, j int) bool {
|
sort.Slice(ep, func(i, j int) bool {
|
||||||
return labels.Compare(h.series[ep[i]].lset, h.series[ep[j]].lset) < 0
|
if err != nil {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
a, ok1 := h.head.series[ep[i]]
|
||||||
|
b, ok2 := h.head.series[ep[j]]
|
||||||
|
|
||||||
|
if !ok1 || !ok2 {
|
||||||
|
err = errors.Errorf("series not found")
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return labels.Compare(a.lset, b.lset) < 0
|
||||||
})
|
})
|
||||||
|
if err != nil {
|
||||||
|
return errPostings{err: err}
|
||||||
|
}
|
||||||
return newListPostings(ep)
|
return newListPostings(ep)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Series returns the series for the given reference.
|
// Series returns the series for the given reference.
|
||||||
func (h *headIndexReader) Series(ref uint32, lbls *labels.Labels, chks *[]ChunkMeta) error {
|
func (h *headIndexReader) Series(ref uint32, lbls *labels.Labels, chks *[]ChunkMeta) error {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
if ref > h.maxSeries {
|
s := h.head.series[ref]
|
||||||
return ErrNotFound
|
|
||||||
}
|
|
||||||
|
|
||||||
s := h.series[ref]
|
|
||||||
if s == nil {
|
if s == nil {
|
||||||
return ErrNotFound
|
return ErrNotFound
|
||||||
}
|
}
|
||||||
|
@ -670,10 +417,14 @@ func (h *headIndexReader) Series(ref uint32, lbls *labels.Labels, chks *[]ChunkM
|
||||||
*chks = (*chks)[:0]
|
*chks = (*chks)[:0]
|
||||||
|
|
||||||
for i, c := range s.chunks {
|
for i, c := range s.chunks {
|
||||||
|
// Do not expose chunks that are outside of the specified range.
|
||||||
|
if !intervalOverlap(c.minTime, c.maxTime, h.mint, h.maxt) {
|
||||||
|
continue
|
||||||
|
}
|
||||||
*chks = append(*chks, ChunkMeta{
|
*chks = append(*chks, ChunkMeta{
|
||||||
MinTime: c.minTime,
|
MinTime: c.minTime,
|
||||||
MaxTime: c.maxTime,
|
MaxTime: c.maxTime,
|
||||||
Ref: (uint64(ref) << 32) | uint64(i),
|
Ref: (uint64(ref) << 32) | uint64(s.chunkID(i)),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -681,12 +432,12 @@ func (h *headIndexReader) Series(ref uint32, lbls *labels.Labels, chks *[]ChunkM
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *headIndexReader) LabelIndices() ([][]string, error) {
|
func (h *headIndexReader) LabelIndices() ([][]string, error) {
|
||||||
h.mtx.RLock()
|
h.head.mtx.RLock()
|
||||||
defer h.mtx.RUnlock()
|
defer h.head.mtx.RUnlock()
|
||||||
|
|
||||||
res := [][]string{}
|
res := [][]string{}
|
||||||
|
|
||||||
for s := range h.values {
|
for s := range h.head.values {
|
||||||
res = append(res, []string{s})
|
res = append(res, []string{s})
|
||||||
}
|
}
|
||||||
return res, nil
|
return res, nil
|
||||||
|
@ -694,7 +445,7 @@ func (h *headIndexReader) LabelIndices() ([][]string, 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) *memSeries {
|
func (h *Head) get(hash uint64, lset labels.Labels) *memSeries {
|
||||||
series := h.hashes[hash]
|
series := h.hashes[hash]
|
||||||
|
|
||||||
for _, s := range series {
|
for _, s := range series {
|
||||||
|
@ -705,11 +456,11 @@ func (h *HeadBlock) get(hash uint64, lset labels.Labels) *memSeries {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *HeadBlock) create(hash uint64, lset labels.Labels) *memSeries {
|
func (h *Head) create(hash uint64, lset labels.Labels) *memSeries {
|
||||||
s := newMemSeries(lset, uint32(len(h.series)), h.meta.MaxTime)
|
id := atomic.AddUint32(&h.lastSeriesID, 1)
|
||||||
|
|
||||||
// Allocate empty space until we can insert at the given index.
|
s := newMemSeries(lset, id, h.chunkRange)
|
||||||
h.series = append(h.series, s)
|
h.series[id] = s
|
||||||
|
|
||||||
h.hashes[hash] = append(h.hashes[hash], s)
|
h.hashes[hash] = append(h.hashes[hash], s)
|
||||||
|
|
||||||
|
@ -727,7 +478,7 @@ func (h *HeadBlock) create(hash uint64, lset labels.Labels) *memSeries {
|
||||||
h.symbols[l.Value] = struct{}{}
|
h.symbols[l.Value] = struct{}{}
|
||||||
}
|
}
|
||||||
|
|
||||||
h.postings.add(s.ref, term{})
|
h.postings.add(id, term{})
|
||||||
|
|
||||||
return s
|
return s
|
||||||
}
|
}
|
||||||
|
@ -743,15 +494,24 @@ type memSeries struct {
|
||||||
ref uint32
|
ref uint32
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
chunks []*memChunk
|
chunks []*memChunk
|
||||||
|
chunkRange int64
|
||||||
|
firstChunkID int
|
||||||
|
|
||||||
nextAt int64 // timestamp at which to cut the next chunk.
|
nextAt int64 // timestamp at which to cut the next chunk.
|
||||||
maxt int64 // maximum timestamp for the series.
|
|
||||||
lastValue float64
|
lastValue float64
|
||||||
sampleBuf [4]sample
|
sampleBuf [4]sample
|
||||||
|
|
||||||
app chunks.Appender // Current appender for the chunk.
|
app chunks.Appender // Current appender for the chunk.
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *memSeries) minTime() int64 {
|
||||||
|
return s.chunks[0].minTime
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *memSeries) maxTime() int64 {
|
||||||
|
return s.head().maxTime
|
||||||
|
}
|
||||||
|
|
||||||
func (s *memSeries) cut(mint int64) *memChunk {
|
func (s *memSeries) cut(mint int64) *memChunk {
|
||||||
c := &memChunk{
|
c := &memChunk{
|
||||||
chunk: chunks.NewXORChunk(),
|
chunk: chunks.NewXORChunk(),
|
||||||
|
@ -768,16 +528,65 @@ func (s *memSeries) cut(mint int64) *memChunk {
|
||||||
return c
|
return c
|
||||||
}
|
}
|
||||||
|
|
||||||
func newMemSeries(lset labels.Labels, id uint32, maxt int64) *memSeries {
|
func newMemSeries(lset labels.Labels, id uint32, chunkRange int64) *memSeries {
|
||||||
s := &memSeries{
|
s := &memSeries{
|
||||||
lset: lset,
|
lset: lset,
|
||||||
ref: id,
|
ref: id,
|
||||||
maxt: maxt,
|
chunkRange: chunkRange,
|
||||||
nextAt: math.MinInt64,
|
nextAt: math.MinInt64,
|
||||||
}
|
}
|
||||||
return s
|
return s
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// appendable checks whether the given sample is valid for appending to the series.
|
||||||
|
func (s *memSeries) appendable(t int64, v float64) error {
|
||||||
|
if len(s.chunks) == 0 {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
c := s.head()
|
||||||
|
|
||||||
|
if t > c.maxTime {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
if t < c.maxTime {
|
||||||
|
return ErrOutOfOrderSample
|
||||||
|
}
|
||||||
|
// We are allowing exact duplicates as we can encounter them in valid cases
|
||||||
|
// like federation and erroring out at that time would be extremely noisy.
|
||||||
|
if math.Float64bits(s.lastValue) != math.Float64bits(v) {
|
||||||
|
return ErrAmendSample
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *memSeries) chunk(id int) *memChunk {
|
||||||
|
ix := id - s.firstChunkID
|
||||||
|
if ix >= len(s.chunks) || ix < 0 {
|
||||||
|
fmt.Println("get chunk", id, len(s.chunks), s.firstChunkID)
|
||||||
|
}
|
||||||
|
|
||||||
|
return s.chunks[ix]
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *memSeries) chunkID(pos int) int {
|
||||||
|
return pos + s.firstChunkID
|
||||||
|
}
|
||||||
|
|
||||||
|
// truncateChunksBefore removes all chunks from the series that have not timestamp
|
||||||
|
// at or after mint. Chunk IDs remain unchanged.
|
||||||
|
func (s *memSeries) truncateChunksBefore(mint int64) {
|
||||||
|
var k int
|
||||||
|
for i, c := range s.chunks {
|
||||||
|
if c.maxTime >= mint {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
k = i + 1
|
||||||
|
}
|
||||||
|
s.chunks = append(s.chunks[:0], s.chunks[k:]...)
|
||||||
|
s.firstChunkID += k
|
||||||
|
}
|
||||||
|
|
||||||
|
// append adds the sample (t, v) to the series.
|
||||||
func (s *memSeries) append(t int64, v float64) bool {
|
func (s *memSeries) append(t int64, v float64) bool {
|
||||||
const samplesPerChunk = 120
|
const samplesPerChunk = 120
|
||||||
|
|
||||||
|
@ -802,7 +611,8 @@ func (s *memSeries) append(t int64, v float64) bool {
|
||||||
c.samples++
|
c.samples++
|
||||||
|
|
||||||
if c.samples == samplesPerChunk/4 {
|
if c.samples == samplesPerChunk/4 {
|
||||||
s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, s.maxt)
|
_, maxt := rangeForTimestamp(c.minTime, s.chunkRange)
|
||||||
|
s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, maxt)
|
||||||
}
|
}
|
||||||
|
|
||||||
s.lastValue = v
|
s.lastValue = v
|
||||||
|
@ -827,7 +637,7 @@ func computeChunkEndTime(start, cur, max int64) int64 {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *memSeries) iterator(i int) chunks.Iterator {
|
func (s *memSeries) iterator(i int) chunks.Iterator {
|
||||||
c := s.chunks[i]
|
c := s.chunk(i)
|
||||||
|
|
||||||
if i < len(s.chunks)-1 {
|
if i < len(s.chunks)-1 {
|
||||||
return c.chunk.Iterator()
|
return c.chunk.Iterator()
|
||||||
|
|
934
head_test.go
934
head_test.go
|
@ -15,12 +15,8 @@ package tsdb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"io/ioutil"
|
"io/ioutil"
|
||||||
"math"
|
|
||||||
"math/rand"
|
|
||||||
"os"
|
"os"
|
||||||
"sort"
|
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
@ -31,23 +27,6 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
)
|
)
|
||||||
|
|
||||||
// createTestHeadBlock creates a new head block with a SegmentWAL.
|
|
||||||
func createTestHeadBlock(t testing.TB, dir string, mint, maxt int64) *HeadBlock {
|
|
||||||
dir, err := TouchHeadBlock(dir, mint, maxt)
|
|
||||||
require.NoError(t, err)
|
|
||||||
|
|
||||||
return openTestHeadBlock(t, dir)
|
|
||||||
}
|
|
||||||
|
|
||||||
func openTestHeadBlock(t testing.TB, dir string) *HeadBlock {
|
|
||||||
wal, err := OpenSegmentWAL(dir, nil, 5*time.Second)
|
|
||||||
require.NoError(t, err)
|
|
||||||
|
|
||||||
h, err := OpenHeadBlock(dir, nil, wal, nil)
|
|
||||||
require.NoError(t, err)
|
|
||||||
return h
|
|
||||||
}
|
|
||||||
|
|
||||||
func BenchmarkCreateSeries(b *testing.B) {
|
func BenchmarkCreateSeries(b *testing.B) {
|
||||||
lbls, err := readPrometheusLabels("cmd/tsdb/testdata.1m", 1e6)
|
lbls, err := readPrometheusLabels("cmd/tsdb/testdata.1m", 1e6)
|
||||||
require.NoError(b, err)
|
require.NoError(b, err)
|
||||||
|
@ -57,7 +36,10 @@ func BenchmarkCreateSeries(b *testing.B) {
|
||||||
require.NoError(b, err)
|
require.NoError(b, err)
|
||||||
defer os.RemoveAll(dir)
|
defer os.RemoveAll(dir)
|
||||||
|
|
||||||
h := createTestHeadBlock(b, dir, 0, 1)
|
h, err := NewHead(nil, nil, 10000)
|
||||||
|
if err != nil {
|
||||||
|
require.NoError(b, err)
|
||||||
|
}
|
||||||
|
|
||||||
b.ReportAllocs()
|
b.ReportAllocs()
|
||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
|
@ -106,598 +88,321 @@ func readPrometheusLabels(fn string, n int) ([]labels.Labels, error) {
|
||||||
return mets, nil
|
return mets, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestAmendDatapointCausesError(t *testing.T) {
|
// func TestHBDeleteSimple(t *testing.T) {
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// numSamples := int64(10)
|
||||||
defer os.RemoveAll(dir)
|
|
||||||
|
// hb, close := openTestDB(t, nil)
|
||||||
hb := createTestHeadBlock(t, dir, 0, 1000)
|
// defer close()
|
||||||
|
|
||||||
app := hb.Appender()
|
// app := hb.Appender()
|
||||||
_, err := app.Add(labels.Labels{}, 0, 0)
|
|
||||||
require.NoError(t, err, "Failed to add sample")
|
// smpls := make([]float64, numSamples)
|
||||||
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
// for i := int64(0); i < numSamples; i++ {
|
||||||
|
// smpls[i] = rand.Float64()
|
||||||
app = hb.Appender()
|
// app.Add(labels.Labels{{"a", "b"}}, i, smpls[i])
|
||||||
_, err = app.Add(labels.Labels{}, 0, 1)
|
// }
|
||||||
require.Equal(t, ErrAmendSample, err)
|
|
||||||
}
|
// require.NoError(t, app.Commit())
|
||||||
|
// cases := []struct {
|
||||||
func TestDuplicateNaNDatapointNoAmendError(t *testing.T) {
|
// intervals Intervals
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// remaint []int64
|
||||||
defer os.RemoveAll(dir)
|
// }{
|
||||||
|
// {
|
||||||
hb := createTestHeadBlock(t, dir, 0, 1000)
|
// intervals: Intervals{{0, 3}},
|
||||||
|
// remaint: []int64{4, 5, 6, 7, 8, 9},
|
||||||
app := hb.Appender()
|
// },
|
||||||
_, err := app.Add(labels.Labels{}, 0, math.NaN())
|
// {
|
||||||
require.NoError(t, err, "Failed to add sample")
|
// intervals: Intervals{{1, 3}},
|
||||||
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
// remaint: []int64{0, 4, 5, 6, 7, 8, 9},
|
||||||
|
// },
|
||||||
app = hb.Appender()
|
// {
|
||||||
_, err = app.Add(labels.Labels{}, 0, math.NaN())
|
// intervals: Intervals{{1, 3}, {4, 7}},
|
||||||
require.NoError(t, err)
|
// remaint: []int64{0, 8, 9},
|
||||||
}
|
// },
|
||||||
|
// {
|
||||||
func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) {
|
// intervals: Intervals{{1, 3}, {4, 700}},
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// remaint: []int64{0},
|
||||||
defer os.RemoveAll(dir)
|
// },
|
||||||
|
// {
|
||||||
hb := createTestHeadBlock(t, dir, 0, 1000)
|
// intervals: Intervals{{0, 9}},
|
||||||
|
// remaint: []int64{},
|
||||||
app := hb.Appender()
|
// },
|
||||||
_, err := app.Add(labels.Labels{}, 0, math.Float64frombits(0x7ff0000000000001))
|
// }
|
||||||
require.NoError(t, err, "Failed to add sample")
|
|
||||||
require.NoError(t, app.Commit(), "Unexpected error committing appender")
|
// Outer:
|
||||||
|
// for _, c := range cases {
|
||||||
app = hb.Appender()
|
// // Reset the tombstones.
|
||||||
_, err = app.Add(labels.Labels{}, 0, math.Float64frombits(0x7ff0000000000002))
|
// hb.tombstones = newEmptyTombstoneReader()
|
||||||
require.Equal(t, ErrAmendSample, err)
|
|
||||||
}
|
// // Delete the ranges.
|
||||||
|
// for _, r := range c.intervals {
|
||||||
func TestSkippingInvalidValuesInSameTxn(t *testing.T) {
|
// require.NoError(t, hb.Delete(r.Mint, r.Maxt, labels.NewEqualMatcher("a", "b")))
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// }
|
||||||
defer os.RemoveAll(dir)
|
|
||||||
|
// // Compare the result.
|
||||||
hb := createTestHeadBlock(t, dir, 0, 1000)
|
// q := hb.Querier(0, numSamples)
|
||||||
|
// res := q.Select(labels.NewEqualMatcher("a", "b"))
|
||||||
// Append AmendedValue.
|
|
||||||
app := hb.Appender()
|
// expSamples := make([]sample, 0, len(c.remaint))
|
||||||
_, err := app.Add(labels.Labels{{"a", "b"}}, 0, 1)
|
// for _, ts := range c.remaint {
|
||||||
require.NoError(t, err)
|
// expSamples = append(expSamples, sample{ts, smpls[ts]})
|
||||||
_, err = app.Add(labels.Labels{{"a", "b"}}, 0, 2)
|
// }
|
||||||
require.NoError(t, err)
|
|
||||||
require.NoError(t, app.Commit())
|
// expss := newListSeriesSet([]Series{
|
||||||
require.Equal(t, uint64(1), hb.Meta().Stats.NumSamples)
|
// newSeries(map[string]string{"a": "b"}, expSamples),
|
||||||
|
// })
|
||||||
// Make sure the right value is stored.
|
|
||||||
q := hb.Querier(0, 10)
|
// if len(expSamples) == 0 {
|
||||||
ss := q.Select(labels.NewEqualMatcher("a", "b"))
|
// require.False(t, res.Next())
|
||||||
ssMap, err := readSeriesSet(ss)
|
// continue
|
||||||
require.NoError(t, err)
|
// }
|
||||||
|
|
||||||
require.Equal(t, map[string][]sample{
|
// for {
|
||||||
labels.New(labels.Label{"a", "b"}).String(): []sample{{0, 1}},
|
// eok, rok := expss.Next(), res.Next()
|
||||||
}, ssMap)
|
// require.Equal(t, eok, rok, "next")
|
||||||
|
|
||||||
require.NoError(t, q.Close())
|
// if !eok {
|
||||||
|
// continue Outer
|
||||||
// Append Out of Order Value.
|
// }
|
||||||
app = hb.Appender()
|
// sexp := expss.At()
|
||||||
_, err = app.Add(labels.Labels{{"a", "b"}}, 10, 3)
|
// sres := res.At()
|
||||||
require.NoError(t, err)
|
|
||||||
_, err = app.Add(labels.Labels{{"a", "b"}}, 7, 5)
|
// require.Equal(t, sexp.Labels(), sres.Labels(), "labels")
|
||||||
require.NoError(t, err)
|
|
||||||
require.NoError(t, app.Commit())
|
// smplExp, errExp := expandSeriesIterator(sexp.Iterator())
|
||||||
require.Equal(t, uint64(2), hb.Meta().Stats.NumSamples)
|
// smplRes, errRes := expandSeriesIterator(sres.Iterator())
|
||||||
|
|
||||||
q = hb.Querier(0, 10)
|
// require.Equal(t, errExp, errRes, "samples error")
|
||||||
ss = q.Select(labels.NewEqualMatcher("a", "b"))
|
// require.Equal(t, smplExp, smplRes, "samples")
|
||||||
ssMap, err = readSeriesSet(ss)
|
// }
|
||||||
require.NoError(t, err)
|
// }
|
||||||
|
// }
|
||||||
require.Equal(t, map[string][]sample{
|
|
||||||
labels.New(labels.Label{"a", "b"}).String(): []sample{{0, 1}, {10, 3}},
|
// func TestDeleteUntilCurMax(t *testing.T) {
|
||||||
}, ssMap)
|
// numSamples := int64(10)
|
||||||
require.NoError(t, q.Close())
|
|
||||||
}
|
// dir, _ := ioutil.TempDir("", "test")
|
||||||
|
// defer os.RemoveAll(dir)
|
||||||
func TestHeadBlock_e2e(t *testing.T) {
|
|
||||||
numDatapoints := 1000
|
// hb := createTestHead(t, dir, 0, 2*numSamples)
|
||||||
numRanges := 1000
|
// app := hb.Appender()
|
||||||
timeInterval := int64(3)
|
|
||||||
maxTime := int64(2 * 1000)
|
// smpls := make([]float64, numSamples)
|
||||||
minTime := int64(200)
|
// for i := int64(0); i < numSamples; i++ {
|
||||||
// Create 8 series with 1000 data-points of different ranges and run queries.
|
// smpls[i] = rand.Float64()
|
||||||
lbls := [][]labels.Label{
|
// app.Add(labels.Labels{{"a", "b"}}, i, smpls[i])
|
||||||
{
|
// }
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "localhost:9090"},
|
// require.NoError(t, app.Commit())
|
||||||
{"job", "prometheus"},
|
// require.NoError(t, hb.Delete(0, 10000, labels.NewEqualMatcher("a", "b")))
|
||||||
},
|
// app = hb.Appender()
|
||||||
{
|
// _, err := app.Add(labels.Labels{{"a", "b"}}, 11, 1)
|
||||||
{"a", "b"},
|
// require.NoError(t, err)
|
||||||
{"instance", "127.0.0.1:9090"},
|
// require.NoError(t, app.Commit())
|
||||||
{"job", "prometheus"},
|
|
||||||
},
|
// q := hb.Querier(0, 100000)
|
||||||
{
|
// res := q.Select(labels.NewEqualMatcher("a", "b"))
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "127.0.0.1:9090"},
|
// require.True(t, res.Next())
|
||||||
{"job", "prom-k8s"},
|
// exps := res.At()
|
||||||
},
|
// it := exps.Iterator()
|
||||||
{
|
// ressmpls, err := expandSeriesIterator(it)
|
||||||
{"a", "b"},
|
// require.NoError(t, err)
|
||||||
{"instance", "localhost:9090"},
|
// require.Equal(t, []sample{{11, 1}}, ressmpls)
|
||||||
{"job", "prom-k8s"},
|
// }
|
||||||
},
|
|
||||||
{
|
// func TestDelete_e2e(t *testing.T) {
|
||||||
{"a", "c"},
|
// numDatapoints := 1000
|
||||||
{"instance", "localhost:9090"},
|
// numRanges := 1000
|
||||||
{"job", "prometheus"},
|
// timeInterval := int64(2)
|
||||||
},
|
// maxTime := int64(2 * 1000)
|
||||||
{
|
// minTime := int64(200)
|
||||||
{"a", "c"},
|
// // Create 8 series with 1000 data-points of different ranges, delete and run queries.
|
||||||
{"instance", "127.0.0.1:9090"},
|
// lbls := [][]labels.Label{
|
||||||
{"job", "prometheus"},
|
// {
|
||||||
},
|
// {"a", "b"},
|
||||||
{
|
// {"instance", "localhost:9090"},
|
||||||
{"a", "c"},
|
// {"job", "prometheus"},
|
||||||
{"instance", "127.0.0.1:9090"},
|
// },
|
||||||
{"job", "prom-k8s"},
|
// {
|
||||||
},
|
// {"a", "b"},
|
||||||
{
|
// {"instance", "127.0.0.1:9090"},
|
||||||
{"a", "c"},
|
// {"job", "prometheus"},
|
||||||
{"instance", "localhost:9090"},
|
// },
|
||||||
{"job", "prom-k8s"},
|
// {
|
||||||
},
|
// {"a", "b"},
|
||||||
}
|
// {"instance", "127.0.0.1:9090"},
|
||||||
|
// {"job", "prom-k8s"},
|
||||||
seriesMap := map[string][]sample{}
|
// },
|
||||||
for _, l := range lbls {
|
// {
|
||||||
seriesMap[labels.New(l...).String()] = []sample{}
|
// {"a", "b"},
|
||||||
}
|
// {"instance", "localhost:9090"},
|
||||||
|
// {"job", "prom-k8s"},
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// },
|
||||||
defer os.RemoveAll(dir)
|
// {
|
||||||
|
// {"a", "c"},
|
||||||
hb := createTestHeadBlock(t, dir, minTime, maxTime)
|
// {"instance", "localhost:9090"},
|
||||||
app := hb.Appender()
|
// {"job", "prometheus"},
|
||||||
|
// },
|
||||||
for _, l := range lbls {
|
// {
|
||||||
ls := labels.New(l...)
|
// {"a", "c"},
|
||||||
series := []sample{}
|
// {"instance", "127.0.0.1:9090"},
|
||||||
|
// {"job", "prometheus"},
|
||||||
ts := rand.Int63n(300)
|
// },
|
||||||
for i := 0; i < numDatapoints; i++ {
|
// {
|
||||||
v := rand.Float64()
|
// {"a", "c"},
|
||||||
if ts >= minTime && ts <= maxTime {
|
// {"instance", "127.0.0.1:9090"},
|
||||||
series = append(series, sample{ts, v})
|
// {"job", "prom-k8s"},
|
||||||
}
|
// },
|
||||||
|
// {
|
||||||
_, err := app.Add(ls, ts, v)
|
// {"a", "c"},
|
||||||
if ts >= minTime && ts <= maxTime {
|
// {"instance", "localhost:9090"},
|
||||||
require.NoError(t, err)
|
// {"job", "prom-k8s"},
|
||||||
} else {
|
// },
|
||||||
require.EqualError(t, err, ErrOutOfBounds.Error())
|
// }
|
||||||
}
|
|
||||||
|
// seriesMap := map[string][]sample{}
|
||||||
ts += rand.Int63n(timeInterval) + 1
|
// for _, l := range lbls {
|
||||||
}
|
// seriesMap[labels.New(l...).String()] = []sample{}
|
||||||
|
// }
|
||||||
seriesMap[labels.New(l...).String()] = series
|
|
||||||
}
|
// dir, _ := ioutil.TempDir("", "test")
|
||||||
|
// defer os.RemoveAll(dir)
|
||||||
require.NoError(t, app.Commit())
|
|
||||||
|
// hb := createTestHead(t, dir, minTime, maxTime)
|
||||||
// Query each selector on 1000 random time-ranges.
|
// app := hb.Appender()
|
||||||
queries := []struct {
|
|
||||||
ms []labels.Matcher
|
// for _, l := range lbls {
|
||||||
}{
|
// ls := labels.New(l...)
|
||||||
{
|
// series := []sample{}
|
||||||
ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")},
|
|
||||||
},
|
// ts := rand.Int63n(300)
|
||||||
{
|
// for i := 0; i < numDatapoints; i++ {
|
||||||
ms: []labels.Matcher{
|
// v := rand.Float64()
|
||||||
labels.NewEqualMatcher("a", "b"),
|
// if ts >= minTime && ts <= maxTime {
|
||||||
labels.NewEqualMatcher("job", "prom-k8s"),
|
// series = append(series, sample{ts, v})
|
||||||
},
|
// }
|
||||||
},
|
|
||||||
{
|
// _, err := app.Add(ls, ts, v)
|
||||||
ms: []labels.Matcher{
|
// if ts >= minTime && ts <= maxTime {
|
||||||
labels.NewEqualMatcher("a", "c"),
|
// require.NoError(t, err)
|
||||||
labels.NewEqualMatcher("instance", "localhost:9090"),
|
// } else {
|
||||||
labels.NewEqualMatcher("job", "prometheus"),
|
// require.EqualError(t, err, ErrOutOfBounds.Error())
|
||||||
},
|
// }
|
||||||
},
|
|
||||||
// TODO: Add Regexp Matchers.
|
// ts += rand.Int63n(timeInterval) + 1
|
||||||
}
|
// }
|
||||||
|
|
||||||
for _, qry := range queries {
|
// seriesMap[labels.New(l...).String()] = series
|
||||||
matched := labels.Slice{}
|
// }
|
||||||
for _, ls := range lbls {
|
|
||||||
s := labels.Selector(qry.ms)
|
// require.NoError(t, app.Commit())
|
||||||
if s.Matches(ls) {
|
|
||||||
matched = append(matched, ls)
|
// // Delete a time-range from each-selector.
|
||||||
}
|
// dels := []struct {
|
||||||
}
|
// ms []labels.Matcher
|
||||||
|
// drange Intervals
|
||||||
sort.Sort(matched)
|
// }{
|
||||||
|
// {
|
||||||
for i := 0; i < numRanges; i++ {
|
// ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")},
|
||||||
mint := rand.Int63n(300)
|
// drange: Intervals{{300, 500}, {600, 670}},
|
||||||
maxt := mint + rand.Int63n(timeInterval*int64(numDatapoints))
|
// },
|
||||||
|
// {
|
||||||
q := hb.Querier(mint, maxt)
|
// ms: []labels.Matcher{
|
||||||
ss := q.Select(qry.ms...)
|
// labels.NewEqualMatcher("a", "b"),
|
||||||
|
// labels.NewEqualMatcher("job", "prom-k8s"),
|
||||||
// Build the mockSeriesSet.
|
// },
|
||||||
matchedSeries := make([]Series, 0, len(matched))
|
// drange: Intervals{{300, 500}, {100, 670}},
|
||||||
for _, m := range matched {
|
// },
|
||||||
smpls := boundedSamples(seriesMap[m.String()], mint, maxt)
|
// {
|
||||||
|
// ms: []labels.Matcher{
|
||||||
// Only append those series for which samples exist as mockSeriesSet
|
// labels.NewEqualMatcher("a", "c"),
|
||||||
// doesn't skip series with no samples.
|
// labels.NewEqualMatcher("instance", "localhost:9090"),
|
||||||
// TODO: But sometimes SeriesSet returns an empty SeriesIterator
|
// labels.NewEqualMatcher("job", "prometheus"),
|
||||||
if len(smpls) > 0 {
|
// },
|
||||||
matchedSeries = append(matchedSeries, newSeries(
|
// drange: Intervals{{300, 400}, {100, 6700}},
|
||||||
m.Map(),
|
// },
|
||||||
smpls,
|
// // TODO: Add Regexp Matchers.
|
||||||
))
|
// }
|
||||||
}
|
|
||||||
}
|
// for _, del := range dels {
|
||||||
expSs := newListSeriesSet(matchedSeries)
|
// // Reset the deletes everytime.
|
||||||
|
// writeTombstoneFile(hb.dir, newEmptyTombstoneReader())
|
||||||
// Compare both SeriesSets.
|
// hb.tombstones = newEmptyTombstoneReader()
|
||||||
for {
|
|
||||||
eok, rok := expSs.Next(), ss.Next()
|
// for _, r := range del.drange {
|
||||||
|
// require.NoError(t, hb.Delete(r.Mint, r.Maxt, del.ms...))
|
||||||
// Skip a series if iterator is empty.
|
// }
|
||||||
if rok {
|
|
||||||
for !ss.At().Iterator().Next() {
|
// matched := labels.Slice{}
|
||||||
rok = ss.Next()
|
// for _, ls := range lbls {
|
||||||
if !rok {
|
// s := labels.Selector(del.ms)
|
||||||
break
|
// if s.Matches(ls) {
|
||||||
}
|
// matched = append(matched, ls)
|
||||||
}
|
// }
|
||||||
}
|
// }
|
||||||
|
|
||||||
require.Equal(t, eok, rok, "next")
|
// sort.Sort(matched)
|
||||||
|
|
||||||
if !eok {
|
// for i := 0; i < numRanges; i++ {
|
||||||
break
|
// mint := rand.Int63n(200)
|
||||||
}
|
// maxt := mint + rand.Int63n(timeInterval*int64(numDatapoints))
|
||||||
sexp := expSs.At()
|
|
||||||
sres := ss.At()
|
// q := hb.Querier(mint, maxt)
|
||||||
|
// ss := q.Select(del.ms...)
|
||||||
require.Equal(t, sexp.Labels(), sres.Labels(), "labels")
|
|
||||||
|
// // Build the mockSeriesSet.
|
||||||
smplExp, errExp := expandSeriesIterator(sexp.Iterator())
|
// matchedSeries := make([]Series, 0, len(matched))
|
||||||
smplRes, errRes := expandSeriesIterator(sres.Iterator())
|
// for _, m := range matched {
|
||||||
|
// smpls := boundedSamples(seriesMap[m.String()], mint, maxt)
|
||||||
require.Equal(t, errExp, errRes, "samples error")
|
// smpls = deletedSamples(smpls, del.drange)
|
||||||
require.Equal(t, smplExp, smplRes, "samples")
|
|
||||||
}
|
// // Only append those series for which samples exist as mockSeriesSet
|
||||||
}
|
// // doesn't skip series with no samples.
|
||||||
}
|
// // TODO: But sometimes SeriesSet returns an empty SeriesIterator
|
||||||
|
// if len(smpls) > 0 {
|
||||||
return
|
// matchedSeries = append(matchedSeries, newSeries(
|
||||||
}
|
// m.Map(),
|
||||||
|
// smpls,
|
||||||
func TestHBDeleteSimple(t *testing.T) {
|
// ))
|
||||||
numSamples := int64(10)
|
// }
|
||||||
|
// }
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
// expSs := newListSeriesSet(matchedSeries)
|
||||||
defer os.RemoveAll(dir)
|
|
||||||
|
// // Compare both SeriesSets.
|
||||||
hb := createTestHeadBlock(t, dir, 0, numSamples)
|
// for {
|
||||||
app := hb.Appender()
|
// eok, rok := expSs.Next(), ss.Next()
|
||||||
|
|
||||||
smpls := make([]float64, numSamples)
|
// // Skip a series if iterator is empty.
|
||||||
for i := int64(0); i < numSamples; i++ {
|
// if rok {
|
||||||
smpls[i] = rand.Float64()
|
// for !ss.At().Iterator().Next() {
|
||||||
app.Add(labels.Labels{{"a", "b"}}, i, smpls[i])
|
// rok = ss.Next()
|
||||||
}
|
// if !rok {
|
||||||
|
// break
|
||||||
require.NoError(t, app.Commit())
|
// }
|
||||||
cases := []struct {
|
// }
|
||||||
intervals Intervals
|
// }
|
||||||
remaint []int64
|
// require.Equal(t, eok, rok, "next")
|
||||||
}{
|
|
||||||
{
|
// if !eok {
|
||||||
intervals: Intervals{{0, 3}},
|
// break
|
||||||
remaint: []int64{4, 5, 6, 7, 8, 9},
|
// }
|
||||||
},
|
// sexp := expSs.At()
|
||||||
{
|
// sres := ss.At()
|
||||||
intervals: Intervals{{1, 3}},
|
|
||||||
remaint: []int64{0, 4, 5, 6, 7, 8, 9},
|
// require.Equal(t, sexp.Labels(), sres.Labels(), "labels")
|
||||||
},
|
|
||||||
{
|
// smplExp, errExp := expandSeriesIterator(sexp.Iterator())
|
||||||
intervals: Intervals{{1, 3}, {4, 7}},
|
// smplRes, errRes := expandSeriesIterator(sres.Iterator())
|
||||||
remaint: []int64{0, 8, 9},
|
|
||||||
},
|
// require.Equal(t, errExp, errRes, "samples error")
|
||||||
{
|
// require.Equal(t, smplExp, smplRes, "samples")
|
||||||
intervals: Intervals{{1, 3}, {4, 700}},
|
// }
|
||||||
remaint: []int64{0},
|
// }
|
||||||
},
|
// }
|
||||||
{
|
|
||||||
intervals: Intervals{{0, 9}},
|
// return
|
||||||
remaint: []int64{},
|
// }
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
Outer:
|
|
||||||
for _, c := range cases {
|
|
||||||
// Reset the tombstones.
|
|
||||||
hb.tombstones = newEmptyTombstoneReader()
|
|
||||||
|
|
||||||
// Delete the ranges.
|
|
||||||
for _, r := range c.intervals {
|
|
||||||
require.NoError(t, hb.Delete(r.Mint, r.Maxt, labels.NewEqualMatcher("a", "b")))
|
|
||||||
}
|
|
||||||
|
|
||||||
// Compare the result.
|
|
||||||
q := hb.Querier(0, numSamples)
|
|
||||||
res := q.Select(labels.NewEqualMatcher("a", "b"))
|
|
||||||
|
|
||||||
expSamples := make([]sample, 0, len(c.remaint))
|
|
||||||
for _, ts := range c.remaint {
|
|
||||||
expSamples = append(expSamples, sample{ts, smpls[ts]})
|
|
||||||
}
|
|
||||||
|
|
||||||
expss := newListSeriesSet([]Series{
|
|
||||||
newSeries(map[string]string{"a": "b"}, expSamples),
|
|
||||||
})
|
|
||||||
|
|
||||||
if len(expSamples) == 0 {
|
|
||||||
require.False(t, res.Next())
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
for {
|
|
||||||
eok, rok := expss.Next(), res.Next()
|
|
||||||
require.Equal(t, eok, rok, "next")
|
|
||||||
|
|
||||||
if !eok {
|
|
||||||
continue Outer
|
|
||||||
}
|
|
||||||
sexp := expss.At()
|
|
||||||
sres := res.At()
|
|
||||||
|
|
||||||
require.Equal(t, sexp.Labels(), sres.Labels(), "labels")
|
|
||||||
|
|
||||||
smplExp, errExp := expandSeriesIterator(sexp.Iterator())
|
|
||||||
smplRes, errRes := expandSeriesIterator(sres.Iterator())
|
|
||||||
|
|
||||||
require.Equal(t, errExp, errRes, "samples error")
|
|
||||||
require.Equal(t, smplExp, smplRes, "samples")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestDeleteUntilCurMax(t *testing.T) {
|
|
||||||
numSamples := int64(10)
|
|
||||||
|
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
|
||||||
defer os.RemoveAll(dir)
|
|
||||||
|
|
||||||
hb := createTestHeadBlock(t, dir, 0, 2*numSamples)
|
|
||||||
app := hb.Appender()
|
|
||||||
|
|
||||||
smpls := make([]float64, numSamples)
|
|
||||||
for i := int64(0); i < numSamples; i++ {
|
|
||||||
smpls[i] = rand.Float64()
|
|
||||||
app.Add(labels.Labels{{"a", "b"}}, i, smpls[i])
|
|
||||||
}
|
|
||||||
|
|
||||||
require.NoError(t, app.Commit())
|
|
||||||
require.NoError(t, hb.Delete(0, 10000, labels.NewEqualMatcher("a", "b")))
|
|
||||||
app = hb.Appender()
|
|
||||||
_, err := app.Add(labels.Labels{{"a", "b"}}, 11, 1)
|
|
||||||
require.NoError(t, err)
|
|
||||||
require.NoError(t, app.Commit())
|
|
||||||
|
|
||||||
q := hb.Querier(0, 100000)
|
|
||||||
res := q.Select(labels.NewEqualMatcher("a", "b"))
|
|
||||||
|
|
||||||
require.True(t, res.Next())
|
|
||||||
exps := res.At()
|
|
||||||
it := exps.Iterator()
|
|
||||||
ressmpls, err := expandSeriesIterator(it)
|
|
||||||
require.NoError(t, err)
|
|
||||||
require.Equal(t, []sample{{11, 1}}, ressmpls)
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestDelete_e2e(t *testing.T) {
|
|
||||||
numDatapoints := 1000
|
|
||||||
numRanges := 1000
|
|
||||||
timeInterval := int64(2)
|
|
||||||
maxTime := int64(2 * 1000)
|
|
||||||
minTime := int64(200)
|
|
||||||
// Create 8 series with 1000 data-points of different ranges, delete and run queries.
|
|
||||||
lbls := [][]labels.Label{
|
|
||||||
{
|
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "localhost:9090"},
|
|
||||||
{"job", "prometheus"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "127.0.0.1:9090"},
|
|
||||||
{"job", "prometheus"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "127.0.0.1:9090"},
|
|
||||||
{"job", "prom-k8s"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "b"},
|
|
||||||
{"instance", "localhost:9090"},
|
|
||||||
{"job", "prom-k8s"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "c"},
|
|
||||||
{"instance", "localhost:9090"},
|
|
||||||
{"job", "prometheus"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "c"},
|
|
||||||
{"instance", "127.0.0.1:9090"},
|
|
||||||
{"job", "prometheus"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "c"},
|
|
||||||
{"instance", "127.0.0.1:9090"},
|
|
||||||
{"job", "prom-k8s"},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
{"a", "c"},
|
|
||||||
{"instance", "localhost:9090"},
|
|
||||||
{"job", "prom-k8s"},
|
|
||||||
},
|
|
||||||
}
|
|
||||||
|
|
||||||
seriesMap := map[string][]sample{}
|
|
||||||
for _, l := range lbls {
|
|
||||||
seriesMap[labels.New(l...).String()] = []sample{}
|
|
||||||
}
|
|
||||||
|
|
||||||
dir, _ := ioutil.TempDir("", "test")
|
|
||||||
defer os.RemoveAll(dir)
|
|
||||||
|
|
||||||
hb := createTestHeadBlock(t, dir, minTime, maxTime)
|
|
||||||
app := hb.Appender()
|
|
||||||
|
|
||||||
for _, l := range lbls {
|
|
||||||
ls := labels.New(l...)
|
|
||||||
series := []sample{}
|
|
||||||
|
|
||||||
ts := rand.Int63n(300)
|
|
||||||
for i := 0; i < numDatapoints; i++ {
|
|
||||||
v := rand.Float64()
|
|
||||||
if ts >= minTime && ts <= maxTime {
|
|
||||||
series = append(series, sample{ts, v})
|
|
||||||
}
|
|
||||||
|
|
||||||
_, err := app.Add(ls, ts, v)
|
|
||||||
if ts >= minTime && ts <= maxTime {
|
|
||||||
require.NoError(t, err)
|
|
||||||
} else {
|
|
||||||
require.EqualError(t, err, ErrOutOfBounds.Error())
|
|
||||||
}
|
|
||||||
|
|
||||||
ts += rand.Int63n(timeInterval) + 1
|
|
||||||
}
|
|
||||||
|
|
||||||
seriesMap[labels.New(l...).String()] = series
|
|
||||||
}
|
|
||||||
|
|
||||||
require.NoError(t, app.Commit())
|
|
||||||
|
|
||||||
// Delete a time-range from each-selector.
|
|
||||||
dels := []struct {
|
|
||||||
ms []labels.Matcher
|
|
||||||
drange Intervals
|
|
||||||
}{
|
|
||||||
{
|
|
||||||
ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")},
|
|
||||||
drange: Intervals{{300, 500}, {600, 670}},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
ms: []labels.Matcher{
|
|
||||||
labels.NewEqualMatcher("a", "b"),
|
|
||||||
labels.NewEqualMatcher("job", "prom-k8s"),
|
|
||||||
},
|
|
||||||
drange: Intervals{{300, 500}, {100, 670}},
|
|
||||||
},
|
|
||||||
{
|
|
||||||
ms: []labels.Matcher{
|
|
||||||
labels.NewEqualMatcher("a", "c"),
|
|
||||||
labels.NewEqualMatcher("instance", "localhost:9090"),
|
|
||||||
labels.NewEqualMatcher("job", "prometheus"),
|
|
||||||
},
|
|
||||||
drange: Intervals{{300, 400}, {100, 6700}},
|
|
||||||
},
|
|
||||||
// TODO: Add Regexp Matchers.
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, del := range dels {
|
|
||||||
// Reset the deletes everytime.
|
|
||||||
writeTombstoneFile(hb.dir, newEmptyTombstoneReader())
|
|
||||||
hb.tombstones = newEmptyTombstoneReader()
|
|
||||||
|
|
||||||
for _, r := range del.drange {
|
|
||||||
require.NoError(t, hb.Delete(r.Mint, r.Maxt, del.ms...))
|
|
||||||
}
|
|
||||||
|
|
||||||
matched := labels.Slice{}
|
|
||||||
for _, ls := range lbls {
|
|
||||||
s := labels.Selector(del.ms)
|
|
||||||
if s.Matches(ls) {
|
|
||||||
matched = append(matched, ls)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
sort.Sort(matched)
|
|
||||||
|
|
||||||
for i := 0; i < numRanges; i++ {
|
|
||||||
mint := rand.Int63n(200)
|
|
||||||
maxt := mint + rand.Int63n(timeInterval*int64(numDatapoints))
|
|
||||||
|
|
||||||
q := hb.Querier(mint, maxt)
|
|
||||||
ss := q.Select(del.ms...)
|
|
||||||
|
|
||||||
// Build the mockSeriesSet.
|
|
||||||
matchedSeries := make([]Series, 0, len(matched))
|
|
||||||
for _, m := range matched {
|
|
||||||
smpls := boundedSamples(seriesMap[m.String()], mint, maxt)
|
|
||||||
smpls = deletedSamples(smpls, del.drange)
|
|
||||||
|
|
||||||
// Only append those series for which samples exist as mockSeriesSet
|
|
||||||
// doesn't skip series with no samples.
|
|
||||||
// TODO: But sometimes SeriesSet returns an empty SeriesIterator
|
|
||||||
if len(smpls) > 0 {
|
|
||||||
matchedSeries = append(matchedSeries, newSeries(
|
|
||||||
m.Map(),
|
|
||||||
smpls,
|
|
||||||
))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
expSs := newListSeriesSet(matchedSeries)
|
|
||||||
|
|
||||||
// Compare both SeriesSets.
|
|
||||||
for {
|
|
||||||
eok, rok := expSs.Next(), ss.Next()
|
|
||||||
|
|
||||||
// Skip a series if iterator is empty.
|
|
||||||
if rok {
|
|
||||||
for !ss.At().Iterator().Next() {
|
|
||||||
rok = ss.Next()
|
|
||||||
if !rok {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
require.Equal(t, eok, rok, "next")
|
|
||||||
|
|
||||||
if !eok {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
sexp := expSs.At()
|
|
||||||
sres := ss.At()
|
|
||||||
|
|
||||||
require.Equal(t, sexp.Labels(), sres.Labels(), "labels")
|
|
||||||
|
|
||||||
smplExp, errExp := expandSeriesIterator(sexp.Iterator())
|
|
||||||
smplRes, errRes := expandSeriesIterator(sres.Iterator())
|
|
||||||
|
|
||||||
require.Equal(t, errExp, errRes, "samples error")
|
|
||||||
require.Equal(t, smplExp, smplRes, "samples")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
func boundedSamples(full []sample, mint, maxt int64) []sample {
|
func boundedSamples(full []sample, mint, maxt int64) []sample {
|
||||||
for len(full) > 0 {
|
for len(full) > 0 {
|
||||||
|
@ -725,7 +430,6 @@ Outer:
|
||||||
continue Outer
|
continue Outer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ds = append(ds, s)
|
ds = append(ds, s)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
36
querier.go
36
querier.go
|
@ -54,26 +54,6 @@ type querier struct {
|
||||||
blocks []Querier
|
blocks []Querier
|
||||||
}
|
}
|
||||||
|
|
||||||
// Querier returns a new querier over the data partition for the given time range.
|
|
||||||
// A goroutine must not handle more than one open Querier.
|
|
||||||
func (s *DB) Querier(mint, maxt int64) Querier {
|
|
||||||
s.mtx.RLock()
|
|
||||||
|
|
||||||
s.headmtx.RLock()
|
|
||||||
blocks := s.blocksForInterval(mint, maxt)
|
|
||||||
s.headmtx.RUnlock()
|
|
||||||
|
|
||||||
sq := &querier{
|
|
||||||
blocks: make([]Querier, 0, len(blocks)),
|
|
||||||
db: s,
|
|
||||||
}
|
|
||||||
for _, b := range blocks {
|
|
||||||
sq.blocks = append(sq.blocks, b.Querier(mint, maxt))
|
|
||||||
}
|
|
||||||
|
|
||||||
return sq
|
|
||||||
}
|
|
||||||
|
|
||||||
func (q *querier) LabelValues(n string) ([]string, error) {
|
func (q *querier) LabelValues(n string) ([]string, error) {
|
||||||
return q.lvals(q.blocks, n)
|
return q.lvals(q.blocks, n)
|
||||||
}
|
}
|
||||||
|
@ -700,6 +680,7 @@ type chunkSeriesIterator struct {
|
||||||
|
|
||||||
func newChunkSeriesIterator(cs []ChunkMeta, dranges Intervals, mint, maxt int64) *chunkSeriesIterator {
|
func newChunkSeriesIterator(cs []ChunkMeta, dranges Intervals, mint, maxt int64) *chunkSeriesIterator {
|
||||||
it := cs[0].Chunk.Iterator()
|
it := cs[0].Chunk.Iterator()
|
||||||
|
|
||||||
if len(dranges) > 0 {
|
if len(dranges) > 0 {
|
||||||
it = &deletedIterator{it: it, intervals: dranges}
|
it = &deletedIterator{it: it, intervals: dranges}
|
||||||
}
|
}
|
||||||
|
@ -750,19 +731,22 @@ func (it *chunkSeriesIterator) At() (t int64, v float64) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *chunkSeriesIterator) Next() bool {
|
func (it *chunkSeriesIterator) Next() bool {
|
||||||
for it.cur.Next() {
|
if it.cur.Next() {
|
||||||
t, _ := it.cur.At()
|
t, _ := it.cur.At()
|
||||||
if t < it.mint {
|
|
||||||
return it.Seek(it.mint)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
if t < it.mint {
|
||||||
|
if !it.Seek(it.mint) {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
t, _ = it.At()
|
||||||
|
|
||||||
|
return t <= it.maxt
|
||||||
|
}
|
||||||
if t > it.maxt {
|
if t > it.maxt {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := it.cur.Err(); err != nil {
|
if err := it.cur.Err(); err != nil {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
|
@ -1138,6 +1138,17 @@ func TestChunkSeriesIterator_SeekInCurrentChunk(t *testing.T) {
|
||||||
require.Equal(t, float64(6), v)
|
require.Equal(t, float64(6), v)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Regression when calling Next() with a time bounded to fit within two samples.
|
||||||
|
// Seek gets called and advances beyond the max time, which was just accepted as a valid sample.
|
||||||
|
func TestChunkSeriesIterator_NextWithMinTime(t *testing.T) {
|
||||||
|
metas := []ChunkMeta{
|
||||||
|
chunkFromSamples([]sample{{1, 6}, {5, 6}, {7, 8}}),
|
||||||
|
}
|
||||||
|
|
||||||
|
it := newChunkSeriesIterator(metas, nil, 2, 4)
|
||||||
|
require.False(t, it.Next())
|
||||||
|
}
|
||||||
|
|
||||||
func TestPopulatedCSReturnsValidChunkSlice(t *testing.T) {
|
func TestPopulatedCSReturnsValidChunkSlice(t *testing.T) {
|
||||||
lbls := []labels.Labels{labels.New(labels.Label{"a", "b"})}
|
lbls := []labels.Labels{labels.New(labels.Label{"a", "b"})}
|
||||||
chunkMetas := [][]ChunkMeta{
|
chunkMetas := [][]ChunkMeta{
|
||||||
|
|
17
wal.go
17
wal.go
|
@ -84,9 +84,19 @@ type WAL interface {
|
||||||
LogSeries([]labels.Labels) error
|
LogSeries([]labels.Labels) error
|
||||||
LogSamples([]RefSample) error
|
LogSamples([]RefSample) error
|
||||||
LogDeletes([]Stone) error
|
LogDeletes([]Stone) error
|
||||||
|
Truncate(maxt int64) error
|
||||||
Close() error
|
Close() error
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type NopWAL struct{}
|
||||||
|
|
||||||
|
func (NopWAL) Read(SeriesCB, SamplesCB, DeletesCB) error { return nil }
|
||||||
|
func (w NopWAL) Reader() WALReader { return w }
|
||||||
|
func (NopWAL) LogSeries([]labels.Labels) error { return nil }
|
||||||
|
func (NopWAL) LogSamples([]RefSample) error { return nil }
|
||||||
|
func (NopWAL) LogDeletes([]Stone) error { return nil }
|
||||||
|
func (NopWAL) Close() error { return nil }
|
||||||
|
|
||||||
// WALReader reads entries from a WAL.
|
// WALReader reads entries from a WAL.
|
||||||
type WALReader interface {
|
type WALReader interface {
|
||||||
Read(SeriesCB, SamplesCB, DeletesCB) error
|
Read(SeriesCB, SamplesCB, DeletesCB) error
|
||||||
|
@ -319,6 +329,10 @@ func (w *SegmentWAL) Sync() error {
|
||||||
return fileutil.Fdatasync(tail)
|
return fileutil.Fdatasync(tail)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (w *SegmentWAL) Truncate(maxt int64) error {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
func (w *SegmentWAL) sync() error {
|
func (w *SegmentWAL) sync() error {
|
||||||
if err := w.flush(); err != nil {
|
if err := w.flush(); err != nil {
|
||||||
return err
|
return err
|
||||||
|
@ -360,9 +374,8 @@ func (w *SegmentWAL) Close() error {
|
||||||
close(w.stopc)
|
close(w.stopc)
|
||||||
<-w.donec
|
<-w.donec
|
||||||
|
|
||||||
// Lock mutex and leave it locked so we panic if there's a bug causing
|
|
||||||
// the block to be used afterwards.
|
|
||||||
w.mtx.Lock()
|
w.mtx.Lock()
|
||||||
|
defer w.mtx.Unlock()
|
||||||
|
|
||||||
if err := w.sync(); err != nil {
|
if err := w.sync(); err != nil {
|
||||||
return err
|
return err
|
||||||
|
|
Loading…
Reference in a new issue