Browse Source

Add out-of-order sample support to the TSDB (#269)

This implementation is based on this design doc:
https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing

This commit adds support to accept out-of-order ("OOO") sample into the TSDB
up to a configurable time allowance. If OOO is enabled, overlapping querying
are automatically enabled.

Signed-off-by: Ganesh Vernekar <ganeshvern@gmail.com>

Co-authored-by: Jesus Vazquez <jesus.vazquez@grafana.com>
Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com>
Co-authored-by: Dieter Plaetinck <dieter@grafana.com>
owilliams/utf8-02-mimir
Ganesh Vernekar 2 years ago committed by GitHub
parent
commit
df59320886
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 15
      cmd/prometheus/main.go
  2. 2
      cmd/promtool/tsdb.go
  3. 27
      config/config.go
  4. 7
      storage/interface.go
  5. 2
      tsdb/agent/db.go
  6. 2
      tsdb/block.go
  7. 2
      tsdb/block_test.go
  8. 2
      tsdb/blockwriter.go
  9. 21
      tsdb/chunkenc/chunk.go
  10. 80
      tsdb/chunkenc/ooo.go
  11. 84
      tsdb/chunkenc/ooo_test.go
  12. 9
      tsdb/chunkenc/xor.go
  13. 13
      tsdb/chunks/chunks.go
  14. 2
      tsdb/chunks/chunks_test.go
  15. 23
      tsdb/chunks/head_chunks.go
  16. 44
      tsdb/chunks/head_chunks_test.go
  17. 11
      tsdb/chunks/old_head_chunks.go
  18. 40
      tsdb/chunks/old_head_chunks_test.go
  19. 129
      tsdb/compact.go
  20. 2
      tsdb/compact_test.go
  21. 261
      tsdb/db.go
  22. 1691
      tsdb/db_test.go
  23. 436
      tsdb/head.go
  24. 301
      tsdb/head_append.go
  25. 6
      tsdb/head_bench_test.go
  26. 267
      tsdb/head_read.go
  27. 165
      tsdb/head_read_test.go
  28. 310
      tsdb/head_test.go
  29. 290
      tsdb/head_wal.go
  30. 74
      tsdb/ooo_head.go
  31. 409
      tsdb/ooo_head_read.go
  32. 1098
      tsdb/ooo_head_read_test.go
  33. 4
      tsdb/querier.go
  34. 4
      tsdb/querier_bench_test.go
  35. 14
      tsdb/querier_test.go
  36. 50
      tsdb/record/record.go
  37. 47
      tsdb/wal/wal.go
  38. 2
      web/api/v1/api_test.go
  39. 3
      web/ui/assets_vfsdata.go

15
cmd/prometheus/main.go

@ -314,6 +314,12 @@ func main() {
serverOnlyFlag(a, "storage.tsdb.wal-compression", "Compress the tsdb WAL.").
Hidden().Default("true").BoolVar(&cfg.tsdb.WALCompression)
serverOnlyFlag(a, "storage.tsdb.out-of-order-cap-min", "Minimum capacity for out of order chunks (in samples. between 0 and 255.)").
Hidden().Default("4").IntVar(&cfg.tsdb.OutOfOrderCapMin)
serverOnlyFlag(a, "storage.tsdb.out-of-order-cap-max", "Maximum capacity for out of order chunks (in samples. between 1 and 255.)").
Hidden().Default("32").IntVar(&cfg.tsdb.OutOfOrderCapMax)
serverOnlyFlag(a, "storage.tsdb.head-chunks-write-queue-size", "Size of the queue through which head chunks are written to the disk to be m-mapped, 0 disables the queue completely. Experimental.").
Default("0").IntVar(&cfg.tsdb.HeadChunksWriteQueueSize)
@ -456,6 +462,9 @@ func main() {
}
cfg.tsdb.MaxExemplars = int64(cfgFile.StorageConfig.ExemplarsConfig.MaxExemplars)
}
if cfgFile.StorageConfig.TSDBConfig != nil {
cfg.tsdb.OutOfOrderAllowance = cfgFile.StorageConfig.TSDBConfig.OutOfOrderAllowance
}
// Now that the validity of the config is established, set the config
// success metrics accordingly, although the config isn't really loaded
@ -1526,6 +1535,9 @@ type tsdbOptions struct {
StripeSize int
MinBlockDuration model.Duration
MaxBlockDuration model.Duration
OutOfOrderAllowance int64
OutOfOrderCapMin int
OutOfOrderCapMax int
EnableExemplarStorage bool
MaxExemplars int64
EnableMemorySnapshotOnShutdown bool
@ -1548,6 +1560,9 @@ func (opts tsdbOptions) ToTSDBOptions() tsdb.Options {
EnableExemplarStorage: opts.EnableExemplarStorage,
MaxExemplars: opts.MaxExemplars,
EnableMemorySnapshotOnShutdown: opts.EnableMemorySnapshotOnShutdown,
OutOfOrderAllowance: opts.OutOfOrderAllowance,
OutOfOrderCapMin: int64(opts.OutOfOrderCapMin),
OutOfOrderCapMax: int64(opts.OutOfOrderCapMax),
}
}

2
cmd/promtool/tsdb.go

@ -597,7 +597,7 @@ func analyzeCompaction(block tsdb.BlockReader, indexr tsdb.IndexReader) (err err
for _, chk := range chks {
// Load the actual data of the chunk.
chk, err := chunkr.Chunk(chk.Ref)
chk, err := chunkr.Chunk(chk)
if err != nil {
return err
}

27
config/config.go

@ -501,9 +501,36 @@ func (c *ScrapeConfig) MarshalYAML() (interface{}, error) {
// StorageConfig configures runtime reloadable configuration options.
type StorageConfig struct {
TSDBConfig *TSDBConfig `yaml:"tsdb,omitempty"`
ExemplarsConfig *ExemplarsConfig `yaml:"exemplars,omitempty"`
}
// TSDBConfig configures runtime reloadable configuration options.
type TSDBConfig struct {
// OutOfOrderAllowance sets how long back in time an out-of-order sample can be inserted
// into the TSDB. This is the one finally used by the TSDB and should be in the same unit
// as other timestamps in the TSDB.
OutOfOrderAllowance int64
// OutOfOrderAllowanceFlag holds the parsed duration from the config file.
// During unmarshall, this is converted into milliseconds and stored in OutOfOrderAllowance.
// This should not be used directly and must be converted into OutOfOrderAllowance.
OutOfOrderAllowanceFlag model.Duration `yaml:"out_of_order_allowance,omitempty"`
}
// UnmarshalYAML implements the yaml.Unmarshaler interface.
func (t *TSDBConfig) UnmarshalYAML(unmarshal func(interface{}) error) error {
*t = TSDBConfig{}
type plain TSDBConfig
if err := unmarshal((*plain)(t)); err != nil {
return err
}
t.OutOfOrderAllowance = time.Duration(t.OutOfOrderAllowanceFlag).Milliseconds()
return nil
}
type TracingClientType string
const (

7
storage/interface.go

@ -27,9 +27,10 @@ import (
// The errors exposed.
var (
ErrNotFound = errors.New("not found")
ErrOutOfOrderSample = errors.New("out of order sample")
ErrDuplicateSampleForTimestamp = errors.New("duplicate sample for timestamp")
ErrOutOfBounds = errors.New("out of bounds")
ErrOutOfOrderSample = errors.New("out of order sample") // OOO support disabled and sample is OOO
ErrTooOldSample = errors.New("too old sample") // OOO support enabled, but sample outside of tolerance
ErrDuplicateSampleForTimestamp = errors.New("duplicate sample for timestamp") // WARNING: this is only reported if value differs. equal values get silently dropped
ErrOutOfBounds = errors.New("out of bounds") // OOO support disabled and t < minValidTime
ErrOutOfOrderExemplar = errors.New("out of order exemplar")
ErrDuplicateExemplar = errors.New("duplicate exemplar")
ErrExemplarLabelLength = fmt.Errorf("label length for exemplar exceeds maximum of %d UTF-8 characters", exemplar.ExemplarMaxLabelSetLength)

2
tsdb/agent/db.go

@ -566,7 +566,7 @@ func (db *DB) truncate(mint int64) error {
// Start a new segment so low ingestion volume instances don't have more WAL
// than needed.
err = db.wal.NextSegment()
_, err = db.wal.NextSegment()
if err != nil {
return errors.Wrap(err, "next segment")
}

2
tsdb/block.go

@ -127,7 +127,7 @@ type ChunkWriter interface {
// ChunkReader provides reading access of serialized time series data.
type ChunkReader interface {
// Chunk returns the series data chunk with the given reference.
Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error)
Chunk(ref chunks.Meta) (chunkenc.Chunk, error)
// Close releases all underlying resources of the reader.
Close() error

2
tsdb/block_test.go

@ -492,7 +492,7 @@ func createBlockFromHead(tb testing.TB, dir string, head *Head) string {
func createHead(tb testing.TB, w *wal.WAL, series []storage.Series, chunkDir string) *Head {
opts := DefaultHeadOptions()
opts.ChunkDirRoot = chunkDir
head, err := NewHead(nil, nil, w, opts, nil)
head, err := NewHead(nil, nil, w, nil, opts, nil)
require.NoError(tb, err)
app := head.Appender(context.Background())

2
tsdb/blockwriter.go

@ -71,7 +71,7 @@ func (w *BlockWriter) initHead() error {
opts := DefaultHeadOptions()
opts.ChunkRange = w.blockSize
opts.ChunkDirRoot = w.chunkDir
h, err := NewHead(nil, w.logger, nil, opts, NewHeadStats())
h, err := NewHead(nil, w.logger, nil, nil, opts, NewHeadStats())
if err != nil {
return errors.Wrap(err, "tsdb.NewHead")
}

21
tsdb/chunkenc/chunk.go

@ -39,6 +39,21 @@ const (
EncXOR
)
// Chunk encodings for out-of-order chunks.
// These encodings must be only used by the Head block for its internal bookkeeping.
const (
OutOfOrderMask = 0b10000000
EncOOOXOR = EncXOR | OutOfOrderMask
)
func IsOutOfOrderChunk(e Encoding) bool {
return (e & OutOfOrderMask) != 0
}
func IsValidEncoding(e Encoding) bool {
return e == EncXOR || e == EncOOOXOR
}
// Chunk holds a sequence of sample pairs that can be iterated over and appended to.
type Chunk interface {
// Bytes returns the underlying byte slice of the chunk.
@ -155,7 +170,7 @@ func NewPool() Pool {
func (p *pool) Get(e Encoding, b []byte) (Chunk, error) {
switch e {
case EncXOR:
case EncXOR, EncOOOXOR:
c := p.xor.Get().(*XORChunk)
c.b.stream = b
c.b.count = 0
@ -166,7 +181,7 @@ func (p *pool) Get(e Encoding, b []byte) (Chunk, error) {
func (p *pool) Put(c Chunk) error {
switch c.Encoding() {
case EncXOR:
case EncXOR, EncOOOXOR:
xc, ok := c.(*XORChunk)
// This may happen often with wrapped chunks. Nothing we can really do about
// it but returning an error would cause a lot of allocations again. Thus,
@ -188,7 +203,7 @@ func (p *pool) Put(c Chunk) error {
// bytes.
func FromData(e Encoding, d []byte) (Chunk, error) {
switch e {
case EncXOR:
case EncXOR, EncOOOXOR:
return &XORChunk{b: bstream{count: 0, stream: d}}, nil
}
return nil, errors.Errorf("invalid chunk encoding %q", e)

80
tsdb/chunkenc/ooo.go

@ -0,0 +1,80 @@
package chunkenc
import (
"sort"
)
type sample struct {
t int64
v float64
}
// OOOChunk maintains samples in time-ascending order.
// Inserts for timestamps already seen, are dropped.
// Samples are stored uncompressed to allow easy sorting.
// Perhaps we can be more efficient later.
type OOOChunk struct {
samples []sample
}
func NewOOOChunk(capacity int) *OOOChunk {
return &OOOChunk{samples: make([]sample, 0, capacity)}
}
// Insert inserts the sample such that order is maintained.
// Returns false if insert was not possible due to the same timestamp already existing.
func (o *OOOChunk) Insert(t int64, v float64) bool {
// find index of sample we should replace
i := sort.Search(len(o.samples), func(i int) bool { return o.samples[i].t >= t })
if i >= len(o.samples) {
// none found. append it at the end
o.samples = append(o.samples, sample{t, v})
return true
}
if o.samples[i].t == t {
return false
}
// expand length by 1 to make room. use a zero sample, we will overwrite it anyway
o.samples = append(o.samples, sample{})
copy(o.samples[i+1:], o.samples[i:])
o.samples[i] = sample{t, v}
return true
}
func (o *OOOChunk) NumSamples() int {
return len(o.samples)
}
func (o *OOOChunk) ToXor() (*XORChunk, error) {
x := NewXORChunk()
app, err := x.Appender()
if err != nil {
return nil, err
}
for _, s := range o.samples {
app.Append(s.t, s.v)
}
return x, nil
}
func (o *OOOChunk) ToXorBetweenTimestamps(mint, maxt int64) (*XORChunk, error) {
x := NewXORChunk()
app, err := x.Appender()
if err != nil {
return nil, err
}
for _, s := range o.samples {
if s.t < mint {
continue
}
if s.t > maxt {
break
}
app.Append(s.t, s.v)
}
return x, nil
}

84
tsdb/chunkenc/ooo_test.go

@ -0,0 +1,84 @@
package chunkenc
import (
"testing"
"github.com/stretchr/testify/require"
)
const testMaxSize int = 32
// formulas chosen to make testing easy:
func valPre(pos int) int { return pos*2 + 2 } // s[0]=2, s[1]=4, s[2]=6, ..., s[31]=64 // predictable pre-existing values
func valNew(pos int) int { return pos*2 + 1 } // s[0]=1, s[1]=3, s[2]=5, ..., s[31]=63 // new values will interject at chosen position because they sort before the pre-existing vals
func samplify(v int) sample { return sample{int64(v), float64(v)} }
func makePre(n int) []sample {
s := make([]sample, n)
for i := 0; i < n; i++ {
s[i] = samplify(valPre(i))
}
return s
}
// TestOOOInsert tests the following cases:
// number of pre-existing samples anywhere from 0 to testMaxSize-1
// insert new sample before first pre-existing samples, after the last, and anywhere in between
// with a chunk initial capacity of testMaxSize/8 and testMaxSize, which lets us test non-full and full chunks, and chunks that need to expand themselves.
// Note: in all samples used, t always equals v in numeric value. when we talk about 'value' we just refer to a value that will be used for both sample.t and sample.v
func TestOOOInsert(t *testing.T) {
for numPre := 0; numPre <= testMaxSize; numPre++ {
// for example, if we have numPre 2, then:
// chunk.samples indexes filled 0 1
// chunk.samples with these values 2 4 // valPre
// we want to test inserting at index 0 1 2 // insertPos=0..numPre
// we can do this by using values 1, 3 5 // valNew(insertPos)
for insertPos := 0; insertPos <= numPre; insertPos++ {
for capacity := range []int{testMaxSize / 8, testMaxSize} {
chunk := NewOOOChunk(capacity)
chunk.samples = makePre(numPre)
newSample := samplify(valNew(insertPos))
chunk.Insert(newSample.t, newSample.v)
var expSamples []sample
// our expected new samples slice, will be first the original samples...
for i := 0; i < insertPos; i++ {
expSamples = append(expSamples, samplify(valPre(i)))
}
// ... then the new sample ...
expSamples = append(expSamples, newSample)
// ... followed by any original samples that were pushed back by the new one
for i := insertPos; i < numPre; i++ {
expSamples = append(expSamples, samplify(valPre(i)))
}
require.Equal(t, expSamples, chunk.samples, "numPre %d, insertPos %d", numPre, insertPos)
}
}
}
}
// TestOOOInsertDuplicate tests the correct behavior when inserting a sample that is a duplicate of any
// pre-existing samples, with between 1 and testMaxSize pre-existing samples and
// with a chunk initial capacity of testMaxSize/8 and testMaxSize, which lets us test non-full and full chunks, and chunks that need to expand themselves.
func TestOOOInsertDuplicate(t *testing.T) {
for numPre := 1; numPre <= testMaxSize; numPre++ {
for dupPos := 0; dupPos < numPre; dupPos++ {
for capacity := range []int{testMaxSize / 8, testMaxSize} {
chunk := NewOOOChunk(capacity)
chunk.samples = makePre(numPre)
dupSample := chunk.samples[dupPos]
dupSample.v = 0.123 // unmistakeably different from any of the pre-existing values, so we can properly detect the correct value below
ok := chunk.Insert(dupSample.t, dupSample.v)
expSamples := makePre(numPre) // we expect no change
require.False(t, ok)
require.Equal(t, expSamples, chunk.samples, "numPre %d, dupPos %d", numPre, dupPos)
}
}
}
}

9
tsdb/chunkenc/xor.go

@ -457,3 +457,12 @@ func (it *xorIterator) readValue() bool {
it.numRead++
return true
}
// OOOXORChunk holds a XORChunk and overrides the Encoding() method.
type OOOXORChunk struct {
*XORChunk
}
func (c *OOOXORChunk) Encoding() Encoding {
return EncOOOXOR
}

13
tsdb/chunks/chunks.go

@ -120,6 +120,15 @@ type Meta struct {
// Time range the data covers.
// When MaxTime == math.MaxInt64 the chunk is still open and being appended to.
MinTime, MaxTime int64
// OOOLastRef, OOOLastMinTime and OOOLastMaxTime are kept as markers for
// overlapping chunks.
// These fields point to the last created out of order Chunk (the head) that existed
// when Series() was called and was overlapping.
// Series() and Chunk() method responses should be consistent for the same
// query even if new data is added in between the calls.
OOOLastRef ChunkRef
OOOLastMinTime, OOOLastMaxTime int64
}
// Iterator iterates over the chunks of a single time series.
@ -555,8 +564,8 @@ func (s *Reader) Size() int64 {
}
// Chunk returns a chunk from a given reference.
func (s *Reader) Chunk(ref ChunkRef) (chunkenc.Chunk, error) {
sgmIndex, chkStart := BlockChunkRef(ref).Unpack()
func (s *Reader) Chunk(meta Meta) (chunkenc.Chunk, error) {
sgmIndex, chkStart := BlockChunkRef(meta.Ref).Unpack()
if sgmIndex >= len(s.bs) {
return nil, errors.Errorf("segment index %d out of range", sgmIndex)

2
tsdb/chunks/chunks_test.go

@ -23,6 +23,6 @@ func TestReaderWithInvalidBuffer(t *testing.T) {
b := realByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81})
r := &Reader{bs: []ByteSlice{b}}
_, err := r.Chunk(0)
_, err := r.Chunk(Meta{Ref: 0})
require.Error(t, err)
}

23
tsdb/chunks/head_chunks.go

@ -87,6 +87,18 @@ func (ref ChunkDiskMapperRef) Unpack() (seq, offset int) {
return seq, offset
}
func (ref ChunkDiskMapperRef) GreaterThanOrEqualTo(r ChunkDiskMapperRef) bool {
s1, o1 := ref.Unpack()
s2, o2 := r.Unpack()
return s1 > s2 || (s1 == s2 && o1 >= o2)
}
func (ref ChunkDiskMapperRef) GreaterThan(r ChunkDiskMapperRef) bool {
s1, o1 := ref.Unpack()
s2, o2 := r.Unpack()
return s1 > s2 || (s1 == s2 && o1 > o2)
}
// CorruptionErr is an error that's returned when corruption is encountered.
type CorruptionErr struct {
Dir string
@ -859,9 +871,8 @@ func (cdm *ChunkDiskMapper) IterateAllChunks(f func(seriesRef HeadSeriesRef, chu
return nil
}
// Truncate deletes the head chunk files which are strictly below the mint.
// mint should be in milliseconds.
func (cdm *ChunkDiskMapper) Truncate(mint int64) error {
// Truncate deletes the head chunk files whose file number is less than given fileNo.
func (cdm *ChunkDiskMapper) Truncate(fileNo int) error {
if !cdm.fileMaxtSet {
return errors.New("maxt of the files are not set")
}
@ -877,12 +888,10 @@ func (cdm *ChunkDiskMapper) Truncate(mint int64) error {
var removedFiles []int
for _, seq := range chkFileIndices {
if seq == cdm.curFileSequence || cdm.mmappedChunkFiles[seq].maxt >= mint {
if seq == cdm.curFileSequence || seq >= fileNo {
break
}
if cdm.mmappedChunkFiles[seq].maxt < mint {
removedFiles = append(removedFiles, seq)
}
removedFiles = append(removedFiles, seq)
}
cdm.readPathMtx.RUnlock()

44
tsdb/chunks/head_chunks_test.go

@ -58,6 +58,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
mint, maxt int64
numSamples uint16
chunk chunkenc.Chunk
isOOO bool
}
expectedData := []expectedDataType{}
@ -67,7 +68,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
for hrw.curFileSequence < 3 || hrw.chkWriter.Buffered() == 0 {
addChunks := func(numChunks int) {
for i := 0; i < numChunks; i++ {
seriesRef, chkRef, mint, maxt, chunk := createChunk(t, totalChunks, hrw)
seriesRef, chkRef, mint, maxt, chunk, isOOO := createChunk(t, totalChunks, hrw)
totalChunks++
expectedData = append(expectedData, expectedDataType{
seriesRef: seriesRef,
@ -76,6 +77,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
chunkRef: chkRef,
chunk: chunk,
numSamples: uint16(chunk.NumSamples()),
isOOO: isOOO,
})
if hrw.curFileSequence != 1 {
@ -156,6 +158,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
require.Equal(t, expData.mint, mint)
require.Equal(t, expData.maxt, maxt)
require.Equal(t, expData.numSamples, numSamples)
require.Equal(t, expData.isOOO, chunkenc.IsOutOfOrderChunk(encoding))
actChunk, err := hrw.Chunk(expData.chunkRef)
require.NoError(t, err)
@ -216,9 +219,7 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
}()
timeRange := 0
fileTimeStep := 100
var thirdFileMinT, sixthFileMinT int64
addChunk := func() int {
addChunk := func() {
t.Helper()
step := 100
@ -232,8 +233,6 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
<-awaitCb
require.NoError(t, err)
timeRange += step
return mint
}
verifyFiles := func(remainingFiles []int) {
@ -254,17 +253,12 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
// Create segments 1 to 7.
for i := 1; i <= 7; i++ {
require.NoError(t, hrw.CutNewFile())
mint := int64(addChunk())
if i == 3 {
thirdFileMinT = mint
} else if i == 6 {
sixthFileMinT = mint
}
addChunk()
}
verifyFiles([]int{1, 2, 3, 4, 5, 6, 7})
// Truncating files.
require.NoError(t, hrw.Truncate(thirdFileMinT))
require.NoError(t, hrw.Truncate(3))
// Add a chunk to trigger cutting of new file.
addChunk()
@ -283,11 +277,11 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
verifyFiles([]int{3, 4, 5, 6, 7, 8, 9})
// Truncating files after restart.
require.NoError(t, hrw.Truncate(sixthFileMinT))
require.NoError(t, hrw.Truncate(6))
verifyFiles([]int{6, 7, 8, 9})
// Truncating a second time without adding a chunk shouldn't create a new file.
require.NoError(t, hrw.Truncate(sixthFileMinT+1))
require.NoError(t, hrw.Truncate(6))
verifyFiles([]int{6, 7, 8, 9})
// Add a chunk to trigger cutting of new file.
@ -295,8 +289,12 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
verifyFiles([]int{6, 7, 8, 9, 10})
// Truncation by file number.
require.NoError(t, hrw.Truncate(8))
verifyFiles([]int{8, 9, 10})
// Truncating till current time should not delete the current active file.
require.NoError(t, hrw.Truncate(int64(timeRange+(2*fileTimeStep))))
require.NoError(t, hrw.Truncate(10))
// Add a chunk to trigger cutting of new file.
addChunk()
@ -373,8 +371,7 @@ func TestChunkDiskMapper_Truncate_PreservesFileSequence(t *testing.T) {
// Truncating files till 2. It should not delete anything after 3 (inclusive)
// though files 4 and 6 are empty.
file2Maxt := hrw.mmappedChunkFiles[2].maxt
require.NoError(t, hrw.Truncate(file2Maxt+1))
require.NoError(t, hrw.Truncate(3))
verifyFiles([]int{3, 4, 5, 6})
// Add chunk, so file 6 is not empty anymore.
@ -382,8 +379,7 @@ func TestChunkDiskMapper_Truncate_PreservesFileSequence(t *testing.T) {
verifyFiles([]int{3, 4, 5, 6})
// Truncating till file 3 should also delete file 4, because it is empty.
file3Maxt := hrw.mmappedChunkFiles[3].maxt
require.NoError(t, hrw.Truncate(file3Maxt+1))
require.NoError(t, hrw.Truncate(5))
addChunk()
verifyFiles([]int{5, 6, 7})
@ -539,13 +535,17 @@ func randomUnsupportedChunk(t *testing.T) chunkenc.Chunk {
return chunk
}
func createChunk(t *testing.T, idx int, hrw *ChunkDiskMapper) (seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk) {
func createChunk(t *testing.T, idx int, hrw *ChunkDiskMapper) (seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk, isOOO bool) {
var err error
seriesRef = HeadSeriesRef(rand.Int63())
mint = int64((idx)*1000 + 1)
maxt = int64((idx + 1) * 1000)
chunk = randomChunk(t)
awaitCb := make(chan struct{})
if rand.Intn(2) == 0 {
isOOO = true
chunk = &chunkenc.OOOXORChunk{XORChunk: chunk.(*chunkenc.XORChunk)}
}
chunkRef = hrw.WriteChunk(seriesRef, mint, maxt, chunk, func(cbErr error) {
require.NoError(t, err)
close(awaitCb)
@ -577,7 +577,7 @@ func writeUnsupportedChunk(t *testing.T, idx int, hrw *ChunkDiskMapper, hrwOld *
}
const (
UnsupportedMask = 0b10000000
UnsupportedMask = 0b11000000
EncUnsupportedXOR = chunkenc.EncXOR | UnsupportedMask
)

11
tsdb/chunks/old_head_chunks.go

@ -596,9 +596,8 @@ func (cdm *OldChunkDiskMapper) IterateAllChunks(f func(seriesRef HeadSeriesRef,
return nil
}
// Truncate deletes the head chunk files which are strictly below the mint.
// mint should be in milliseconds.
func (cdm *OldChunkDiskMapper) Truncate(mint int64) error {
// Truncate deletes the head chunk files whose file number is less than given fileNo.
func (cdm *OldChunkDiskMapper) Truncate(fileNo int) error {
if !cdm.fileMaxtSet {
return errors.New("maxt of the files are not set")
}
@ -614,12 +613,10 @@ func (cdm *OldChunkDiskMapper) Truncate(mint int64) error {
var removedFiles []int
for _, seq := range chkFileIndices {
if seq == cdm.curFileSequence || cdm.mmappedChunkFiles[seq].maxt >= mint {
if seq == cdm.curFileSequence || seq >= fileNo {
break
}
if cdm.mmappedChunkFiles[seq].maxt < mint {
removedFiles = append(removedFiles, seq)
}
removedFiles = append(removedFiles, seq)
}
cdm.readPathMtx.RUnlock()

40
tsdb/chunks/old_head_chunks_test.go

@ -42,6 +42,7 @@ func TestOldChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
mint, maxt int64
numSamples uint16
chunk chunkenc.Chunk
isOOO bool
}
expectedData := []expectedDataType{}
@ -51,7 +52,7 @@ func TestOldChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
for hrw.curFileSequence < 3 || hrw.chkWriter.Buffered() == 0 {
addChunks := func(numChunks int) {
for i := 0; i < numChunks; i++ {
seriesRef, chkRef, mint, maxt, chunk := createChunkForOld(t, totalChunks, hrw)
seriesRef, chkRef, mint, maxt, chunk, isOOO := createChunkForOld(t, totalChunks, hrw)
totalChunks++
expectedData = append(expectedData, expectedDataType{
seriesRef: seriesRef,
@ -60,6 +61,7 @@ func TestOldChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
chunkRef: chkRef,
chunk: chunk,
numSamples: uint16(chunk.NumSamples()),
isOOO: isOOO,
})
if hrw.curFileSequence != 1 {
@ -141,6 +143,7 @@ func TestOldChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
require.Equal(t, expData.maxt, maxt)
require.Equal(t, expData.maxt, maxt)
require.Equal(t, expData.numSamples, numSamples)
require.Equal(t, expData.isOOO, chunkenc.IsOutOfOrderChunk(encoding))
actChunk, err := hrw.Chunk(expData.chunkRef)
require.NoError(t, err)
@ -204,9 +207,8 @@ func TestOldChunkDiskMapper_Truncate(t *testing.T) {
timeRange := 0
fileTimeStep := 100
var thirdFileMinT, sixthFileMinT int64
addChunk := func() int {
addChunk := func() {
mint := timeRange + 1 // Just after the new file cut.
maxt := timeRange + fileTimeStep - 1 // Just before the next file.
@ -216,8 +218,6 @@ func TestOldChunkDiskMapper_Truncate(t *testing.T) {
})
timeRange += fileTimeStep
return mint
}
verifyFiles := func(remainingFiles []int) {
@ -238,17 +238,12 @@ func TestOldChunkDiskMapper_Truncate(t *testing.T) {
// Create segments 1 to 7.
for i := 1; i <= 7; i++ {
require.NoError(t, hrw.CutNewFile())
mint := int64(addChunk())
if i == 3 {
thirdFileMinT = mint
} else if i == 6 {
sixthFileMinT = mint
}
addChunk()
}
verifyFiles([]int{1, 2, 3, 4, 5, 6, 7})
// Truncating files.
require.NoError(t, hrw.Truncate(thirdFileMinT))
require.NoError(t, hrw.Truncate(3))
verifyFiles([]int{3, 4, 5, 6, 7, 8})
dir := hrw.dir.Name()
@ -271,16 +266,20 @@ func TestOldChunkDiskMapper_Truncate(t *testing.T) {
verifyFiles([]int{3, 4, 5, 6, 7, 8, 9})
// Truncating files after restart.
require.NoError(t, hrw.Truncate(sixthFileMinT))
require.NoError(t, hrw.Truncate(6))
verifyFiles([]int{6, 7, 8, 9, 10})
// As the last file was empty, this creates no new files.
require.NoError(t, hrw.Truncate(sixthFileMinT+1))
require.NoError(t, hrw.Truncate(6))
verifyFiles([]int{6, 7, 8, 9, 10})
require.NoError(t, hrw.Truncate(8))
verifyFiles([]int{8, 9, 10})
addChunk()
// Truncating till current time should not delete the current active file.
require.NoError(t, hrw.Truncate(int64(timeRange+(2*fileTimeStep))))
require.NoError(t, hrw.Truncate(10))
verifyFiles([]int{10, 11}) // One file is the previously active file and one currently created.
}
@ -337,14 +336,13 @@ func TestOldChunkDiskMapper_Truncate_PreservesFileSequence(t *testing.T) {
// Truncating files till 2. It should not delete anything after 3 (inclusive)
// though files 4 and 6 are empty.
file2Maxt := hrw.mmappedChunkFiles[2].maxt
require.NoError(t, hrw.Truncate(file2Maxt+1))
require.NoError(t, hrw.Truncate(3))
// As 6 was empty, it should not create another file.
verifyFiles([]int{3, 4, 5, 6})
addChunk()
// Truncate creates another file as 6 is not empty now.
require.NoError(t, hrw.Truncate(file2Maxt+1))
require.NoError(t, hrw.Truncate(3))
verifyFiles([]int{3, 4, 5, 6, 7})
dir := hrw.dir.Name()
@ -470,11 +468,15 @@ func testOldChunkDiskMapper(t *testing.T) *OldChunkDiskMapper {
return hrw
}
func createChunkForOld(t *testing.T, idx int, hrw *OldChunkDiskMapper) (seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk) {
func createChunkForOld(t *testing.T, idx int, hrw *OldChunkDiskMapper) (seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk, isOOO bool) {
seriesRef = HeadSeriesRef(rand.Int63())
mint = int64((idx)*1000 + 1)
maxt = int64((idx + 1) * 1000)
chunk = randomChunk(t)
if rand.Intn(2) == 0 {
isOOO = true
chunk = &chunkenc.OOOXORChunk{XORChunk: chunk.(*chunkenc.XORChunk)}
}
chunkRef = hrw.WriteChunk(seriesRef, mint, maxt, chunk, func(err error) {
require.NoError(t, err)
})

129
tsdb/compact.go

@ -75,6 +75,10 @@ type Compactor interface {
// * The source dirs are marked Deletable.
// * Returns empty ulid.ULID{}.
Compact(dest string, dirs []string, open []*Block) (ulid.ULID, error)
// CompactOOO creates a new block per possible block range in the compactor's directory from the OOO Head given.
// Each ULID in the result corresponds to a block in a unique time range.
CompactOOO(dest string, oooHead *OOOCompactionHead) (result []ulid.ULID, err error)
}
// LeveledCompactor implements the Compactor interface.
@ -546,6 +550,131 @@ func (c *LeveledCompactor) compact(dest string, dirs []string, open []*Block, sh
return nil, errs.Err()
}
// CompactOOOWithSplitting splits the input OOO Head into shardCount number of output blocks
// per possible block range, and returns slice of block IDs. In result[i][j],
// 'i' corresponds to a single time range of blocks while 'j' corresponds to the shard index.
// If given output block has no series, corresponding block ID will be zero ULID value.
// TODO: write tests for this.
func (c *LeveledCompactor) CompactOOOWithSplitting(dest string, oooHead *OOOCompactionHead, shardCount uint64) (result [][]ulid.ULID, _ error) {
return c.compactOOO(dest, oooHead, shardCount)
}
// CompactOOO creates a new block per possible block range in the compactor's directory from the OOO Head given.
// Each ULID in the result corresponds to a block in a unique time range.
func (c *LeveledCompactor) CompactOOO(dest string, oooHead *OOOCompactionHead) (result []ulid.ULID, err error) {
ulids, err := c.compactOOO(dest, oooHead, 1)
if err != nil {
return nil, err
}
for _, s := range ulids {
if s[0].Compare(ulid.ULID{}) != 0 {
result = append(result, s[0])
}
}
return result, err
}
func (c *LeveledCompactor) compactOOO(dest string, oooHead *OOOCompactionHead, shardCount uint64) (_ [][]ulid.ULID, err error) {
if shardCount == 0 {
shardCount = 1
}
start := time.Now()
if err != nil {
return nil, err
}
// The first dimension of outBlocks determines the time based splitting (i.e. outBlocks[i] has blocks all for the same time range).
// The second dimension of outBlocks determines the label based shard (i.e. outBlocks[i][j] is the (j+1)th shard.
// During ingestion of samples we can identify which ooo blocks will exists so that
// we dont have to prefill symbols and etc for the blocks that will be empty.
// With this, len(outBlocks[x]) will still be the same for all x so that we can pick blocks easily.
// Just that, only some of the outBlocks[x][y] will be valid and populated based on preexisting knowledge of
// which blocks to expect.
// In case we see a sample that is not present in the estimated block ranges, we will create them on flight.
outBlocks := make([][]shardedBlock, 0)
outBlocksTime := ulid.Now() // Make all out blocks share the same timestamp in the ULID.
blockSize := oooHead.ChunkRange()
oooHeadMint, oooHeadMaxt := oooHead.MinTime(), oooHead.MaxTime()
ulids := make([][]ulid.ULID, 0)
for t := blockSize * (oooHeadMint / blockSize); t <= oooHeadMaxt; t = t + blockSize {
mint, maxt := t, t+blockSize
outBlocks = append(outBlocks, make([]shardedBlock, shardCount))
ulids = append(ulids, make([]ulid.ULID, shardCount))
ix := len(outBlocks) - 1
for jx := range outBlocks[ix] {
uid := ulid.MustNew(outBlocksTime, rand.Reader)
meta := &BlockMeta{
ULID: uid,
MinTime: mint,
MaxTime: maxt,
}
meta.Compaction.Level = 1
meta.Compaction.Sources = []ulid.ULID{uid}
outBlocks[ix][jx] = shardedBlock{
meta: meta,
}
ulids[ix][jx] = meta.ULID
}
// Block intervals are half-open: [b.MinTime, b.MaxTime). Block intervals are always +1 than the total samples it includes.
err := c.write(dest, outBlocks[ix], oooHead.CloneForTimeRange(mint, maxt-1))
if err != nil {
// We need to delete all blocks in case there was an error.
for _, obs := range outBlocks {
for _, ob := range obs {
if ob.tmpDir != "" {
if removeErr := os.RemoveAll(ob.tmpDir); removeErr != nil {
level.Error(c.logger).Log("msg", "Failed to remove temp folder after failed compaction", "dir", ob.tmpDir, "err", removeErr.Error())
}
}
if ob.blockDir != "" {
if removeErr := os.RemoveAll(ob.blockDir); removeErr != nil {
level.Error(c.logger).Log("msg", "Failed to remove block folder after failed compaction", "dir", ob.blockDir, "err", removeErr.Error())
}
}
}
}
return nil, err
}
}
noOOOBlock := true
for ix, obs := range outBlocks {
for jx := range obs {
meta := outBlocks[ix][jx].meta
if meta.Stats.NumSamples != 0 {
noOOOBlock = false
level.Info(c.logger).Log(
"msg", "compact ooo head",
"mint", meta.MinTime,
"maxt", meta.MaxTime,
"ulid", meta.ULID,
"duration", time.Since(start),
"shard", fmt.Sprintf("%d_of_%d", jx+1, shardCount),
)
} else {
// This block did not get any data. So clear out the ulid to signal this.
ulids[ix][jx] = ulid.ULID{}
}
}
}
if noOOOBlock {
level.Info(c.logger).Log(
"msg", "compact ooo head resulted in no blocks",
"duration", time.Since(start),
)
return nil, nil
}
return ulids, nil
}
func (c *LeveledCompactor) Write(dest string, b BlockReader, mint, maxt int64, parent *BlockMeta) (ulid.ULID, error) {
start := time.Now()

2
tsdb/compact_test.go

@ -1281,7 +1281,7 @@ func BenchmarkCompactionFromHead(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
for ln := 0; ln < labelNames; ln++ {
app := h.Appender(context.Background())

261
tsdb/db.go

@ -33,6 +33,7 @@ import (
"github.com/oklog/ulid"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/atomic"
"golang.org/x/sync/errgroup"
"github.com/prometheus/prometheus/config"
@ -85,6 +86,8 @@ func DefaultOptions() *Options {
IsolationDisabled: defaultIsolationDisabled,
HeadChunksEndTimeVariance: 0,
HeadChunksWriteQueueSize: chunks.DefaultWriteQueueSize,
OutOfOrderCapMin: DefaultOutOfOrderCapMin,
OutOfOrderCapMax: DefaultOutOfOrderCapMax,
}
}
@ -119,6 +122,7 @@ type Options struct {
// Querying on overlapping blocks are allowed if AllowOverlappingQueries is true.
// Since querying is a required operation for TSDB, if there are going to be
// overlapping blocks, then this should be set to true.
// NOTE: Do not use this directly in DB. Use it via DB.AllowOverlappingQueries().
AllowOverlappingQueries bool
// Compaction of overlapping blocks are allowed if AllowOverlappingCompaction is true.
@ -178,6 +182,19 @@ type Options struct {
// If nil, the cache won't be used.
SeriesHashCache *hashcache.SeriesHashCache
// OutOfOrderAllowance specifies how much out of order is allowed, if any.
// This can change during run-time, so this value from here should only be used
// while initialising.
OutOfOrderAllowance int64
// OutOfOrderCapMin minimum capacity for OOO chunks (in samples).
// If it is <=0, the default value is assumed.
OutOfOrderCapMin int64
// OutOfOrderCapMax is maximum capacity for OOO chunks (in samples).
// If it is <=0, the default value is assumed.
OutOfOrderCapMax int64
// Temporary flag which we use to select whether we want to use the new or the old chunk disk mapper.
NewChunkDiskMapper bool
}
@ -217,6 +234,13 @@ type DB struct {
// Cancel a running compaction when a shutdown is initiated.
compactCancel context.CancelFunc
// oooWasEnabled is true if out of order support was enabled at least one time
// during the time TSDB was up. In which case we need to keep supporting
// out-of-order compaction and vertical queries.
oooWasEnabled atomic.Bool
registerer prometheus.Registerer
}
type dbMetrics struct {
@ -392,9 +416,17 @@ func (db *DBReadOnly) FlushWAL(dir string) (returnErr error) {
if err != nil {
return err
}
var wbl *wal.WAL
wblDir := filepath.Join(db.dir, wal.WblDirName)
if _, err := os.Stat(wblDir); !os.IsNotExist(err) {
wbl, err = wal.Open(db.logger, wblDir)
if err != nil {
return err
}
}
opts := DefaultHeadOptions()
opts.ChunkDirRoot = db.dir
head, err := NewHead(nil, db.logger, w, opts, NewHeadStats())
head, err := NewHead(nil, db.logger, w, wbl, opts, NewHeadStats())
if err != nil {
return err
}
@ -451,7 +483,7 @@ func (db *DBReadOnly) loadDataAsQueryable(maxt int64) (storage.SampleAndChunkQue
opts := DefaultHeadOptions()
opts.ChunkDirRoot = db.dir
head, err := NewHead(nil, db.logger, nil, opts, NewHeadStats())
head, err := NewHead(nil, db.logger, nil, nil, opts, NewHeadStats())
if err != nil {
return nil, err
}
@ -469,9 +501,17 @@ func (db *DBReadOnly) loadDataAsQueryable(maxt int64) (storage.SampleAndChunkQue
if err != nil {
return nil, err
}
var wbl *wal.WAL
wblDir := filepath.Join(db.dir, wal.WblDirName)
if _, err := os.Stat(wblDir); !os.IsNotExist(err) {
wbl, err = wal.Open(db.logger, wblDir)
if err != nil {
return nil, err
}
}
opts := DefaultHeadOptions()
opts.ChunkDirRoot = db.dir
head, err = NewHead(nil, db.logger, w, opts, NewHeadStats())
head, err = NewHead(nil, db.logger, w, wbl, opts, NewHeadStats())
if err != nil {
return nil, err
}
@ -622,6 +662,18 @@ func validateOpts(opts *Options, rngs []int64) (*Options, []int64) {
if opts.MinBlockDuration > opts.MaxBlockDuration {
opts.MaxBlockDuration = opts.MinBlockDuration
}
if opts.OutOfOrderAllowance > 0 {
opts.AllowOverlappingQueries = true
}
if opts.OutOfOrderCapMin <= 0 {
opts.OutOfOrderCapMin = DefaultOutOfOrderCapMin
}
if opts.OutOfOrderCapMax <= 0 {
opts.OutOfOrderCapMax = DefaultOutOfOrderCapMax
}
if opts.OutOfOrderAllowance < 0 {
opts.OutOfOrderAllowance = 0
}
if len(rngs) == 0 {
// Start with smallest block duration and create exponential buckets until the exceed the
@ -658,6 +710,15 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
}
walDir := filepath.Join(dir, "wal")
wblDir := filepath.Join(dir, wal.WblDirName)
// TODO(jesus.vazquez) Remove the block of code below, only necessary until all ooo_wbl dirs in prod have been replaced with wbl
oldWblDir := filepath.Join(dir, "ooo_wbl")
if _, err := os.Stat(oldWblDir); err == nil {
err = fileutil.Rename(oldWblDir, wblDir)
if err != nil {
return nil, errors.Wrap(err, "failed to move old wbl dir to new wbl dir")
}
}
// Migrate old WAL if one exists.
if err := MigrateWAL(l, walDir); err != nil {
@ -680,6 +741,7 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
autoCompact: true,
chunkPool: chunkenc.NewPool(),
blocksToDelete: opts.BlocksToDelete,
registerer: r,
}
defer func() {
// Close files if startup fails somewhere.
@ -718,7 +780,7 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
}
db.compactCancel = cancel
var wlog *wal.WAL
var wlog, wblog *wal.WAL
segmentSize := wal.DefaultSegmentSize
// Wal is enabled.
if opts.WALSegmentSize >= 0 {
@ -730,8 +792,14 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
if err != nil {
return nil, err
}
if opts.OutOfOrderAllowance > 0 {
wblog, err = wal.NewSize(l, r, wblDir, segmentSize, opts.WALCompression)
if err != nil {
return nil, err
}
}
}
db.oooWasEnabled.Store(opts.OutOfOrderAllowance > 0)
headOpts := DefaultHeadOptions()
headOpts.ChunkRange = rngs[0]
headOpts.ChunkDirRoot = dir
@ -744,12 +812,15 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
headOpts.EnableExemplarStorage = opts.EnableExemplarStorage
headOpts.MaxExemplars.Store(opts.MaxExemplars)
headOpts.EnableMemorySnapshotOnShutdown = opts.EnableMemorySnapshotOnShutdown
headOpts.OutOfOrderAllowance.Store(opts.OutOfOrderAllowance)
headOpts.OutOfOrderCapMin.Store(opts.OutOfOrderCapMin)
headOpts.OutOfOrderCapMax.Store(opts.OutOfOrderCapMax)
headOpts.NewChunkDiskMapper = opts.NewChunkDiskMapper
if opts.IsolationDisabled {
// We only override this flag if isolation is disabled at DB level. We use the default otherwise.
headOpts.IsolationDisabled = opts.IsolationDisabled
}
db.head, err = NewHead(r, l, wlog, headOpts, stats.Head)
db.head, err = NewHead(r, l, wlog, wblog, headOpts, stats.Head)
if err != nil {
return nil, err
}
@ -775,10 +846,19 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs
if initErr := db.head.Init(minValidTime); initErr != nil {
db.head.metrics.walCorruptionsTotal.Inc()
level.Warn(db.logger).Log("msg", "Encountered WAL read error, attempting repair", "err", initErr)
if err := wlog.Repair(initErr); err != nil {
return nil, errors.Wrap(err, "repair corrupted WAL")
isOOOErr := isErrLoadOOOWal(initErr)
if isOOOErr {
level.Warn(db.logger).Log("msg", "Encountered OOO WAL read error, attempting repair", "err", initErr)
if err := wblog.Repair(initErr); err != nil {
return nil, errors.Wrap(err, "repair corrupted OOO WAL")
}
} else {
level.Warn(db.logger).Log("msg", "Encountered WAL read error, attempting repair", "err", initErr)
if err := wlog.Repair(initErr); err != nil {
return nil, errors.Wrap(err, "repair corrupted WAL")
}
}
}
go db.run()
@ -872,8 +952,51 @@ func (db *DB) Appender(ctx context.Context) storage.Appender {
return dbAppender{db: db, Appender: db.head.Appender(ctx)}
}
// ApplyConfig applies a new config to the DB.
// Behaviour of 'OutOfOrderAllowance' is as follows:
// OOO enabled = oooAllowance > 0. OOO disabled = oooAllowance is 0.
// 1) Before: OOO disabled, Now: OOO enabled =>
// * A new WBL is created for the head block.
// * OOO compaction is enabled.
// * Overlapping queries are enabled.
// 2) Before: OOO enabled, Now: OOO enabled =>
// * Only the allowance is updated.
// 3) Before: OOO enabled, Now: OOO disabled =>
// * Allowance set to 0. So no new OOO samples will be allowed.
// * OOO WBL will stay and follow the usual cleanup until a restart.
// * OOO Compaction and overlapping queries will remain enabled until a restart.
// 4) Before: OOO disabled, Now: OOO disabled => no-op.
func (db *DB) ApplyConfig(conf *config.Config) error {
return db.head.ApplyConfig(conf)
oooAllowance := int64(0)
if conf.StorageConfig.TSDBConfig != nil {
oooAllowance = conf.StorageConfig.TSDBConfig.OutOfOrderAllowance
}
if oooAllowance < 0 {
oooAllowance = 0
}
// Create WBL if it was not present and if OOO is enabled with WAL enabled.
var wblog *wal.WAL
var err error
if !db.oooWasEnabled.Load() && oooAllowance > 0 && db.opts.WALSegmentSize >= 0 {
segmentSize := wal.DefaultSegmentSize
// Wal is set to a custom size.
if db.opts.WALSegmentSize > 0 {
segmentSize = db.opts.WALSegmentSize
}
oooWalDir := filepath.Join(db.dir, wal.WblDirName)
wblog, err = wal.NewSize(db.logger, db.registerer, oooWalDir, segmentSize, db.opts.WALCompression)
if err != nil {
return err
}
}
db.head.ApplyConfig(conf, wblog)
if !db.oooWasEnabled.Load() {
db.oooWasEnabled.Store(oooAllowance > 0)
}
return nil
}
// dbAppender wraps the DB's head appender and triggers compactions on commit
@ -949,6 +1072,9 @@ func (db *DB) Compact() (returnErr error) {
// so in order to make sure that overlaps are evaluated
// consistently, we explicitly remove the last value
// from the block interval here.
// TODO(jesus.vazquez) Once we have the OOORangeHead we need to update
// TODO(jesus.vazquez) this method to accept a second parameter with an OOORangeHead to
// TODO(jesus.vazquez) compact the OOO Samples.
if err := db.compactHead(NewRangeHead(db.head, mint, maxt-1)); err != nil {
return errors.Wrap(err, "compact head")
}
@ -970,6 +1096,14 @@ func (db *DB) Compact() (returnErr error) {
"block_range", db.head.chunkRange.Load(),
)
}
if lastBlockMaxt != math.MinInt64 {
// The head was compacted, so we compact OOO head as well.
if err := db.compactOOOHead(); err != nil {
return errors.Wrap(err, "compact ooo head")
}
}
return db.compactBlocks()
}
@ -988,6 +1122,47 @@ func (db *DB) CompactHead(head *RangeHead) error {
return nil
}
// CompactOOOHead compacts the OOO Head.
func (db *DB) CompactOOOHead() error {
db.cmtx.Lock()
defer db.cmtx.Unlock()
return db.compactOOOHead()
}
func (db *DB) compactOOOHead() error {
if !db.oooWasEnabled.Load() {
return nil
}
oooHead, err := NewOOOCompactionHead(db.head)
if err != nil {
return errors.Wrap(err, "get ooo compaction head")
}
ulids, err := db.compactor.CompactOOO(db.dir, oooHead)
if err != nil {
return errors.Wrap(err, "compact ooo head")
}
if err := db.reloadBlocks(); err != nil {
errs := tsdb_errors.NewMulti(err)
for _, uid := range ulids {
if errRemoveAll := os.RemoveAll(filepath.Join(db.dir, uid.String())); errRemoveAll != nil {
errs.Add(errRemoveAll)
}
}
return errors.Wrap(errs.Err(), "reloadBlocks blocks after failed compact ooo head")
}
lastWBLFile, minOOOMmapRef := oooHead.LastWBLFile(), oooHead.LastMmapRef()
if lastWBLFile != 0 || minOOOMmapRef != 0 {
if err := db.head.truncateOOO(lastWBLFile, minOOOMmapRef); err != nil {
return errors.Wrap(err, "truncate ooo wbl")
}
}
return nil
}
// compactHead compacts the given RangeHead.
// The compaction mutex should be held before calling this method.
func (db *DB) compactHead(head *RangeHead) error {
@ -1145,7 +1320,7 @@ func (db *DB) reloadBlocks() (err error) {
sort.Slice(toLoad, func(i, j int) bool {
return toLoad[i].Meta().MinTime < toLoad[j].Meta().MinTime
})
if !db.opts.AllowOverlappingQueries {
if !db.AllowOverlappingQueries() {
if err := validateBlockSequence(toLoad); err != nil {
return errors.Wrap(err, "invalid block sequence")
}
@ -1175,6 +1350,10 @@ func (db *DB) reloadBlocks() (err error) {
return nil
}
func (db *DB) AllowOverlappingQueries() bool {
return db.opts.AllowOverlappingQueries || db.oooWasEnabled.Load()
}
func openBlocks(l log.Logger, dir string, loaded []*Block, chunkPool chunkenc.Pool, cache *hashcache.SeriesHashCache) (blocks []*Block, corrupted map[ulid.ULID]error, err error) {
bDirs, err := blockDirs(dir)
if err != nil {
@ -1555,13 +1734,13 @@ func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, err
blocks = append(blocks, b)
}
}
var headQuerier storage.Querier
var inOrderHeadQuerier storage.Querier
if maxt >= db.head.MinTime() {
rh := NewRangeHead(db.head, mint, maxt)
var err error
headQuerier, err = NewBlockQuerier(rh, mint, maxt)
inOrderHeadQuerier, err = NewBlockQuerier(rh, mint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open querier for head %s", rh)
return nil, errors.Wrapf(err, "open block querier for head %s", rh)
}
// Getting the querier above registers itself in the queue that the truncation waits on.
@ -1569,20 +1748,30 @@ func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, err
// won't run into a race later since any truncation that comes after will wait on this querier if it overlaps.
shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(mint, maxt)
if shouldClose {
if err := headQuerier.Close(); err != nil {
return nil, errors.Wrapf(err, "closing head querier %s", rh)
if err := inOrderHeadQuerier.Close(); err != nil {
return nil, errors.Wrapf(err, "closing head block querier %s", rh)
}
headQuerier = nil
inOrderHeadQuerier = nil
}
if getNew {
rh := NewRangeHead(db.head, newMint, maxt)
headQuerier, err = NewBlockQuerier(rh, newMint, maxt)
inOrderHeadQuerier, err = NewBlockQuerier(rh, newMint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open querier for head while getting new querier %s", rh)
return nil, errors.Wrapf(err, "open block querier for head while getting new querier %s", rh)
}
}
}
var outOfOrderHeadQuerier storage.Querier
if overlapsClosedInterval(mint, maxt, db.head.MinOOOTime(), db.head.MaxOOOTime()) {
rh := NewOOORangeHead(db.head, mint, maxt)
var err error
outOfOrderHeadQuerier, err = NewBlockQuerier(rh, mint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open block querier for ooo head %s", rh)
}
}
blockQueriers := make([]storage.Querier, 0, len(blocks))
for _, b := range blocks {
q, err := NewBlockQuerier(b, mint, maxt)
@ -1597,8 +1786,11 @@ func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, err
}
return nil, errors.Wrapf(err, "open querier for block %s", b)
}
if headQuerier != nil {
blockQueriers = append(blockQueriers, headQuerier)
if inOrderHeadQuerier != nil {
blockQueriers = append(blockQueriers, inOrderHeadQuerier)
}
if outOfOrderHeadQuerier != nil {
blockQueriers = append(blockQueriers, outOfOrderHeadQuerier)
}
return storage.NewMergeQuerier(blockQueriers, nil, storage.ChainedSeriesMerge), nil
}
@ -1615,11 +1807,11 @@ func (db *DB) ChunkQuerier(_ context.Context, mint, maxt int64) (storage.ChunkQu
blocks = append(blocks, b)
}
}
var headQuerier storage.ChunkQuerier
var inOrderHeadQuerier storage.ChunkQuerier
if maxt >= db.head.MinTime() {
rh := NewRangeHead(db.head, mint, maxt)
var err error
headQuerier, err = NewBlockChunkQuerier(rh, mint, maxt)
inOrderHeadQuerier, err = NewBlockChunkQuerier(rh, mint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open querier for head %s", rh)
}
@ -1629,20 +1821,30 @@ func (db *DB) ChunkQuerier(_ context.Context, mint, maxt int64) (storage.ChunkQu
// won't run into a race later since any truncation that comes after will wait on this querier if it overlaps.
shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(mint, maxt)
if shouldClose {
if err := headQuerier.Close(); err != nil {
if err := inOrderHeadQuerier.Close(); err != nil {
return nil, errors.Wrapf(err, "closing head querier %s", rh)
}
headQuerier = nil
inOrderHeadQuerier = nil
}
if getNew {
rh := NewRangeHead(db.head, newMint, maxt)
headQuerier, err = NewBlockChunkQuerier(rh, newMint, maxt)
inOrderHeadQuerier, err = NewBlockChunkQuerier(rh, newMint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open querier for head while getting new querier %s", rh)
}
}
}
var outOfOrderHeadQuerier storage.ChunkQuerier
if overlapsClosedInterval(mint, maxt, db.head.MinOOOTime(), db.head.MaxOOOTime()) {
rh := NewOOORangeHead(db.head, mint, maxt)
var err error
outOfOrderHeadQuerier, err = NewBlockChunkQuerier(rh, mint, maxt)
if err != nil {
return nil, errors.Wrapf(err, "open block chunk querier for ooo head %s", rh)
}
}
blockQueriers := make([]storage.ChunkQuerier, 0, len(blocks))
for _, b := range blocks {
q, err := NewBlockChunkQuerier(b, mint, maxt)
@ -1657,8 +1859,11 @@ func (db *DB) ChunkQuerier(_ context.Context, mint, maxt int64) (storage.ChunkQu
}
return nil, errors.Wrapf(err, "open querier for block %s", b)
}
if headQuerier != nil {
blockQueriers = append(blockQueriers, headQuerier)
if inOrderHeadQuerier != nil {
blockQueriers = append(blockQueriers, inOrderHeadQuerier)
}
if outOfOrderHeadQuerier != nil {
blockQueriers = append(blockQueriers, outOfOrderHeadQuerier)
}
return storage.NewMergeChunkQuerier(blockQueriers, nil, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil

1691
tsdb/db_test.go

File diff suppressed because it is too large Load Diff

436
tsdb/head.go

@ -25,9 +25,10 @@ import (
"github.com/go-kit/log/level"
"github.com/oklog/ulid"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/atomic"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/exemplar"
"github.com/prometheus/prometheus/model/labels"
@ -62,7 +63,7 @@ var (
type chunkDiskMapper interface {
CutNewFile() (returnErr error)
IterateAllChunks(f func(seriesRef chunks.HeadSeriesRef, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16, encoding chunkenc.Encoding) error) (err error)
Truncate(mint int64) error
Truncate(fileNo int) error
DeleteCorrupted(originalErr error) error
Size() (int64, error)
Close() error
@ -75,15 +76,19 @@ type chunkDiskMapper interface {
type Head struct {
chunkRange atomic.Int64
numSeries atomic.Uint64
minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head.
minOOOTime, maxOOOTime atomic.Int64 // TODO(jesus) These should be updated after garbage collection
minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. // TODO(jesus.vazquez) Ensure these are properly tracked.
minValidTime atomic.Int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block.
lastWALTruncationTime atomic.Int64
lastMemoryTruncationTime atomic.Int64
lastSeriesID atomic.Uint64
// All the ooo m-map chunks should be after this. This is used to truncate old ooo m-map chunks.
// This should be typecasted to chunks.ChunkDiskMapperRef after loading.
minOOOMmapRef atomic.Uint64
metrics *headMetrics
opts *HeadOptions
wal *wal.WAL
wal, wbl *wal.WAL
exemplarMetrics *ExemplarMetrics
exemplars ExemplarStorage
logger log.Logger
@ -99,6 +104,7 @@ type Head struct {
deletedMtx sync.Mutex
deleted map[chunks.HeadSeriesRef]int // Deleted series, and what WAL segment they must be kept until.
// TODO(ganesh) extend MemPostings to return only OOOPostings, Set OOOStatus, ... Like an additional map of ooo postings.
postings *index.MemPostings // Postings lists for terms.
pfmc *PostingsForMatchersCache
@ -144,6 +150,9 @@ type HeadOptions struct {
ChunkWriteBufferSize int
ChunkEndTimeVariance float64
ChunkWriteQueueSize int
OutOfOrderAllowance atomic.Int64
OutOfOrderCapMin atomic.Int64
OutOfOrderCapMax atomic.Int64
// StripeSize sets the number of entries in the hash map, it must be a power of 2.
// A larger StripeSize will allocate more memory up-front, but will increase performance when handling a large number of series.
@ -160,8 +169,13 @@ type HeadOptions struct {
NewChunkDiskMapper bool
}
const (
DefaultOutOfOrderCapMin int64 = 4
DefaultOutOfOrderCapMax int64 = 32
)
func DefaultHeadOptions() *HeadOptions {
return &HeadOptions{
ho := &HeadOptions{
ChunkRange: DefaultBlockDuration,
ChunkDirRoot: "",
ChunkPool: chunkenc.NewPool(),
@ -173,6 +187,9 @@ func DefaultHeadOptions() *HeadOptions {
IsolationDisabled: defaultIsolationDisabled,
NewChunkDiskMapper: false,
}
ho.OutOfOrderCapMin.Store(DefaultOutOfOrderCapMin)
ho.OutOfOrderCapMax.Store(DefaultOutOfOrderCapMax)
return ho
}
// SeriesLifecycleCallback specifies a list of callbacks that will be called during a lifecycle of a series.
@ -191,11 +208,32 @@ type SeriesLifecycleCallback interface {
}
// NewHead opens the head block in dir.
func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, opts *HeadOptions, stats *HeadStats) (*Head, error) {
func NewHead(r prometheus.Registerer, l log.Logger, wal, wbl *wal.WAL, opts *HeadOptions, stats *HeadStats) (*Head, error) {
var err error
if l == nil {
l = log.NewNopLogger()
}
if opts.OutOfOrderAllowance.Load() < 0 {
opts.OutOfOrderAllowance.Store(0)
}
// Allowance can be set on runtime. So the capMin and capMax should be valid
// even if ooo is not enabled yet.
capMin, capMax := opts.OutOfOrderCapMin.Load(), opts.OutOfOrderCapMax.Load()
if capMin > 255 {
return nil, errors.Errorf("OOOCapMin invalid %d. must be <= 255", capMin)
}
if capMax > 255 {
return nil, errors.Errorf("OOOCapMax invalid %d. must be <= 255", capMin)
}
if capMin < 0 {
return nil, errors.Errorf("OOOCapMin invalid %d. must be >= 0", capMin)
}
if capMax <= 0 || capMax < capMin {
return nil, errors.Errorf("OOOCapMax invalid %d. must be > 0 and >= OOOCapMin", capMax)
}
if opts.ChunkRange < 1 {
return nil, errors.Errorf("invalid chunk range %d", opts.ChunkRange)
}
@ -213,6 +251,7 @@ func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, opts *HeadOpti
h := &Head{
wal: wal,
wbl: wbl,
logger: l,
opts: opts,
memChunkPool: sync.Pool{
@ -284,36 +323,40 @@ func (h *Head) resetInMemoryState() error {
h.chunkRange.Store(h.opts.ChunkRange)
h.minTime.Store(math.MaxInt64)
h.maxTime.Store(math.MinInt64)
h.minOOOTime.Store(math.MaxInt64)
h.maxOOOTime.Store(math.MinInt64)
h.lastWALTruncationTime.Store(math.MinInt64)
h.lastMemoryTruncationTime.Store(math.MinInt64)
return nil
}
type headMetrics struct {
activeAppenders prometheus.Gauge
series prometheus.GaugeFunc
seriesCreated prometheus.Counter
seriesRemoved prometheus.Counter
seriesNotFound prometheus.Counter
chunks prometheus.Gauge
chunksCreated prometheus.Counter
chunksRemoved prometheus.Counter
gcDuration prometheus.Summary
samplesAppended prometheus.Counter
outOfBoundSamples prometheus.Counter
outOfOrderSamples prometheus.Counter
walTruncateDuration prometheus.Summary
walCorruptionsTotal prometheus.Counter
walTotalReplayDuration prometheus.Gauge
headTruncateFail prometheus.Counter
headTruncateTotal prometheus.Counter
checkpointDeleteFail prometheus.Counter
checkpointDeleteTotal prometheus.Counter
checkpointCreationFail prometheus.Counter
checkpointCreationTotal prometheus.Counter
mmapChunkCorruptionTotal prometheus.Counter
snapshotReplayErrorTotal prometheus.Counter // Will be either 0 or 1.
oooHistogram prometheus.Histogram
activeAppenders prometheus.Gauge
series prometheus.GaugeFunc
seriesCreated prometheus.Counter
seriesRemoved prometheus.Counter
seriesNotFound prometheus.Counter
chunks prometheus.Gauge
chunksCreated prometheus.Counter
chunksRemoved prometheus.Counter
gcDuration prometheus.Summary
samplesAppended prometheus.Counter
outOfOrderSamplesAppended prometheus.Counter
outOfBoundSamples prometheus.Counter
outOfOrderSamples prometheus.Counter
tooOldSamples prometheus.Counter
walTruncateDuration prometheus.Summary
walCorruptionsTotal prometheus.Counter
dataTotalReplayDuration prometheus.Gauge
headTruncateFail prometheus.Counter
headTruncateTotal prometheus.Counter
checkpointDeleteFail prometheus.Counter
checkpointDeleteTotal prometheus.Counter
checkpointCreationFail prometheus.Counter
checkpointCreationTotal prometheus.Counter
mmapChunkCorruptionTotal prometheus.Counter
snapshotReplayErrorTotal prometheus.Counter // Will be either 0 or 1.
oooHistogram prometheus.Histogram
}
func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
@ -364,7 +407,7 @@ func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
Name: "prometheus_tsdb_wal_corruptions_total",
Help: "Total number of WAL corruptions.",
}),
walTotalReplayDuration: prometheus.NewGauge(prometheus.GaugeOpts{
dataTotalReplayDuration: prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_tsdb_data_replay_duration_seconds",
Help: "Time taken to replay the data on disk.",
}),
@ -372,13 +415,21 @@ func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
Name: "prometheus_tsdb_head_samples_appended_total",
Help: "Total number of appended samples.",
}),
outOfOrderSamplesAppended: prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_head_out_of_order_samples_appended_total",
Help: "Total number of appended out of order samples.",
}),
outOfBoundSamples: prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_out_of_bound_samples_total",
Help: "Total number of out of bound samples ingestion failed attempts.",
Help: "Total number of out of bound samples ingestion failed attempts with out of order support disabled.",
}),
outOfOrderSamples: prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_out_of_order_samples_total",
Help: "Total number of out of order samples ingestion failed attempts.",
Help: "Total number of out of order samples ingestion failed attempts due to out of order being disabled.",
}),
tooOldSamples: prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_too_old_samples_total",
Help: "Total number of out of order samples ingestion failed attempts with out of support enabled, but sample outside of allowance.",
}),
headTruncateFail: prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_head_truncations_failed_total",
@ -414,7 +465,7 @@ func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
}),
oooHistogram: prometheus.NewHistogram(prometheus.HistogramOpts{
Name: "prometheus_tsdb_sample_ooo_delta",
Help: "Delta in seconds by which a sample is considered out of order.",
Help: "Delta in seconds by which a sample is considered out of order (reported regardless of OOO allowance and whether sample is accepted or not).",
Buckets: []float64{
// Note that mimir distributor only gives us a range of wallclock-12h to wallclock+15min
60 * 10, // 10 min
@ -441,10 +492,12 @@ func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
m.gcDuration,
m.walTruncateDuration,
m.walCorruptionsTotal,
m.walTotalReplayDuration,
m.dataTotalReplayDuration,
m.samplesAppended,
m.outOfOrderSamplesAppended,
m.outOfBoundSamples,
m.outOfOrderSamples,
m.tooOldSamples,
m.headTruncateFail,
m.headTruncateTotal,
m.checkpointDeleteFail,
@ -563,8 +616,9 @@ func (h *Head) Init(minValidTime int64) error {
}
mmapChunkReplayStart := time.Now()
mmappedChunks, err := h.loadMmappedChunks(refSeries)
mmappedChunks, oooMmappedChunks, lastMmapRef, err := h.loadMmappedChunks(refSeries)
if err != nil {
// TODO(codesome): clear out all m-map chunks here for refSeries.
level.Error(h.logger).Log("msg", "Loading on-disk chunks failed", "err", err)
if _, ok := errors.Cause(err).(*chunks.CorruptionErr); ok {
h.metrics.mmapChunkCorruptionTotal.Inc()
@ -575,7 +629,7 @@ func (h *Head) Init(minValidTime int64) error {
// If this fails, data will be recovered from WAL.
// Hence we wont lose any data (given WAL is not corrupt).
mmappedChunks, err = h.removeCorruptedMmappedChunks(err)
mmappedChunks, oooMmappedChunks, lastMmapRef, err = h.removeCorruptedMmappedChunks(err)
if err != nil {
return err
}
@ -618,7 +672,7 @@ func (h *Head) Init(minValidTime int64) error {
// A corrupted checkpoint is a hard error for now and requires user
// intervention. There's likely little data that can be recovered anyway.
if err := h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks); err != nil {
if err := h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks, oooMmappedChunks); err != nil {
return errors.Wrap(err, "backfill checkpoint")
}
h.updateWALReplayStatusRead(startFrom)
@ -651,7 +705,7 @@ func (h *Head) Init(minValidTime int64) error {
if err != nil {
return errors.Wrapf(err, "segment reader (offset=%d)", offset)
}
err = h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks)
err = h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks, oooMmappedChunks)
if err := sr.Close(); err != nil {
level.Warn(h.logger).Log("msg", "Error while closing the wal segments reader", "err", err)
}
@ -661,32 +715,94 @@ func (h *Head) Init(minValidTime int64) error {
level.Info(h.logger).Log("msg", "WAL segment loaded", "segment", i, "maxSegment", endAt)
h.updateWALReplayStatusRead(i)
}
walReplayDuration := time.Since(walReplayStart)
wblReplayStart := time.Now()
if h.wbl != nil {
// Replay OOO WAL.
startFrom, endAt, e = wal.Segments(h.wbl.Dir())
if e != nil {
return errors.Wrap(e, "finding OOO WAL segments")
}
h.startWALReplayStatus(startFrom, endAt)
for i := startFrom; i <= endAt; i++ {
s, err := wal.OpenReadSegment(wal.SegmentName(h.wbl.Dir(), i))
if err != nil {
return errors.Wrap(err, fmt.Sprintf("open WBL segment: %d", i))
}
walReplayDuration := time.Since(start)
h.metrics.walTotalReplayDuration.Set(walReplayDuration.Seconds())
sr := wal.NewSegmentBufReader(s)
err = h.loadWbl(wal.NewReader(sr), multiRef, lastMmapRef)
if err := sr.Close(); err != nil {
level.Warn(h.logger).Log("msg", "Error while closing the wbl segments reader", "err", err)
}
if err != nil {
return err
}
level.Info(h.logger).Log("msg", "WBL segment loaded", "segment", i, "maxSegment", endAt)
h.updateWALReplayStatusRead(i)
}
}
wblReplayDuration := time.Since(wblReplayStart)
totalReplayDuration := time.Since(start)
h.metrics.dataTotalReplayDuration.Set(totalReplayDuration.Seconds())
level.Info(h.logger).Log(
"msg", "WAL replay completed",
"checkpoint_replay_duration", checkpointReplayDuration.String(),
"wal_replay_duration", time.Since(walReplayStart).String(),
"total_replay_duration", walReplayDuration.String(),
"wal_replay_duration", walReplayDuration.String(),
"wbl_replay_duration", wblReplayDuration.String(),
"total_replay_duration", totalReplayDuration.String(),
)
return nil
}
func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) {
func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries) (map[chunks.HeadSeriesRef][]*mmappedChunk, map[chunks.HeadSeriesRef][]*mmappedChunk, chunks.ChunkDiskMapperRef, error) {
mmappedChunks := map[chunks.HeadSeriesRef][]*mmappedChunk{}
oooMmappedChunks := map[chunks.HeadSeriesRef][]*mmappedChunk{}
var lastRef, secondLastRef chunks.ChunkDiskMapperRef
if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef chunks.HeadSeriesRef, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16, encoding chunkenc.Encoding) error {
if maxt < h.minValidTime.Load() {
secondLastRef = lastRef
lastRef = chunkRef
isOOO := chunkenc.IsOutOfOrderChunk(encoding)
if !isOOO && maxt < h.minValidTime.Load() {
return nil
}
// We ignore any chunk that doesnt have a valid encoding
if encoding != chunkenc.EncXOR {
if !chunkenc.IsValidEncoding(encoding) {
return nil
}
ms, ok := refSeries[seriesRef]
if isOOO {
if !ok {
oooMmappedChunks[seriesRef] = append(oooMmappedChunks[seriesRef], &mmappedChunk{
ref: chunkRef,
minTime: mint,
maxTime: maxt,
numSamples: numSamples,
})
return nil
}
h.metrics.chunks.Inc()
h.metrics.chunksCreated.Inc()
ms.oooMmappedChunks = append(ms.oooMmappedChunks, &mmappedChunk{
ref: chunkRef,
minTime: mint,
maxTime: maxt,
numSamples: numSamples,
})
return nil
}
if !ok {
slice := mmappedChunks[seriesRef]
if len(slice) > 0 && slice[len(slice)-1].maxTime >= mint {
@ -727,17 +843,19 @@ func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries)
}
return nil
}); err != nil {
return nil, errors.Wrap(err, "iterate on on-disk chunks")
// secondLastRef because the lastRef caused an error.
return nil, nil, secondLastRef, errors.Wrap(err, "iterate on on-disk chunks")
}
return mmappedChunks, nil
return mmappedChunks, oooMmappedChunks, lastRef, nil
}
// removeCorruptedMmappedChunks attempts to delete the corrupted mmapped chunks and if it fails, it clears all the previously
// loaded mmapped chunks.
func (h *Head) removeCorruptedMmappedChunks(err error) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) {
func (h *Head) removeCorruptedMmappedChunks(err error) (map[chunks.HeadSeriesRef][]*mmappedChunk, map[chunks.HeadSeriesRef][]*mmappedChunk, chunks.ChunkDiskMapperRef, error) {
level.Info(h.logger).Log("msg", "Deleting mmapped chunk files")
// We never want to preserve the in-memory series from snapshots if we are repairing m-map chunks.
if err := h.resetInMemoryState(); err != nil {
return nil, err
return map[chunks.HeadSeriesRef][]*mmappedChunk{}, map[chunks.HeadSeriesRef][]*mmappedChunk{}, 0, err
}
level.Info(h.logger).Log("msg", "Deleting mmapped chunk files")
@ -747,11 +865,11 @@ func (h *Head) removeCorruptedMmappedChunks(err error) (map[chunks.HeadSeriesRef
if err := h.chunkDiskMapper.Truncate(math.MaxInt64); err != nil {
level.Error(h.logger).Log("msg", "Deletion of all mmap chunk files failed", "err", err)
}
return map[chunks.HeadSeriesRef][]*mmappedChunk{}, nil
return map[chunks.HeadSeriesRef][]*mmappedChunk{}, map[chunks.HeadSeriesRef][]*mmappedChunk{}, 0, nil
}
level.Info(h.logger).Log("msg", "Deletion of mmap chunk files successful, reattempting m-mapping the on-disk chunks")
mmappedChunks, err := h.loadMmappedChunks(make(map[chunks.HeadSeriesRef]*memSeries))
mmappedChunks, oooMmappedChunks, lastRef, err := h.loadMmappedChunks(make(map[chunks.HeadSeriesRef]*memSeries))
if err != nil {
level.Error(h.logger).Log("msg", "Loading on-disk chunks failed, discarding chunk files completely", "err", err)
if err := h.chunkDiskMapper.Truncate(math.MaxInt64); err != nil {
@ -760,12 +878,22 @@ func (h *Head) removeCorruptedMmappedChunks(err error) (map[chunks.HeadSeriesRef
mmappedChunks = map[chunks.HeadSeriesRef][]*mmappedChunk{}
}
return mmappedChunks, nil
return mmappedChunks, oooMmappedChunks, lastRef, nil
}
func (h *Head) ApplyConfig(cfg *config.Config) error {
func (h *Head) ApplyConfig(cfg *config.Config, wbl *wal.WAL) {
oooAllowance := int64(0)
if cfg.StorageConfig.TSDBConfig != nil {
oooAllowance = cfg.StorageConfig.TSDBConfig.OutOfOrderAllowance
}
if oooAllowance < 0 {
oooAllowance = 0
}
h.SetOutOfOrderAllowance(oooAllowance, wbl)
if !h.opts.EnableExemplarStorage {
return nil
return
}
// Head uses opts.MaxExemplars in combination with opts.EnableExemplarStorage
@ -776,12 +904,21 @@ func (h *Head) ApplyConfig(cfg *config.Config) error {
newSize := h.opts.MaxExemplars.Load()
if prevSize == newSize {
return nil
return
}
migrated := h.exemplars.(*CircularExemplarStorage).Resize(newSize)
level.Info(h.logger).Log("msg", "Exemplar storage resized", "from", prevSize, "to", newSize, "migrated", migrated)
return nil
}
// SetOutOfOrderAllowance updates the out of order related parameters.
// If the Head already has a WBL set, then the wbl will be ignored.
func (h *Head) SetOutOfOrderAllowance(oooAllowance int64, wbl *wal.WAL) {
if oooAllowance > 0 && h.wbl == nil {
h.wbl = wbl
}
h.opts.OutOfOrderAllowance.Store(oooAllowance)
}
// PostingsCardinalityStats returns top 10 highest cardinality stats By label and value names.
@ -823,6 +960,27 @@ func (h *Head) updateMinMaxTime(mint, maxt int64) {
}
}
func (h *Head) updateMinOOOMaxOOOTime(mint, maxt int64) {
for {
lt := h.MinOOOTime()
if mint >= lt {
break
}
if h.minOOOTime.CAS(lt, mint) {
break
}
}
for {
ht := h.MaxOOOTime()
if maxt <= ht {
break
}
if h.maxOOOTime.CAS(ht, maxt) {
break
}
}
}
// SetMinValidTime sets the minimum timestamp the head can ingest.
func (h *Head) SetMinValidTime(minValidTime int64) {
h.minValidTime.Store(minValidTime)
@ -890,7 +1048,7 @@ func (h *Head) truncateMemory(mint int64) (err error) {
h.metrics.headTruncateTotal.Inc()
start := time.Now()
actualMint := h.gc()
actualMint, minMmapFile := h.gc()
level.Info(h.logger).Log("msg", "Head GC completed", "duration", time.Since(start))
h.metrics.gcDuration.Observe(time.Since(start).Seconds())
if actualMint > h.minTime.Load() {
@ -908,8 +1066,8 @@ func (h *Head) truncateMemory(mint int64) (err error) {
}
// Truncate the chunk m-mapper.
if err := h.chunkDiskMapper.Truncate(mint); err != nil {
return errors.Wrap(err, "truncate chunks.HeadReadWriter")
if err := h.chunkDiskMapper.Truncate(minMmapFile); err != nil {
return errors.Wrap(err, "truncate chunks.HeadReadWriter by file number")
}
return nil
}
@ -1000,7 +1158,7 @@ func (h *Head) truncateWAL(mint int64) error {
}
// Start a new segment, so low ingestion volume TSDB don't have more WAL than
// needed.
if err := h.wal.NextSegment(); err != nil {
if _, err := h.wal.NextSegment(); err != nil {
return errors.Wrap(err, "next segment")
}
last-- // Never consider last segment for checkpoint.
@ -1066,6 +1224,41 @@ func (h *Head) truncateWAL(mint int64) error {
return nil
}
// truncateOOO
// * truncates the OOO WBL files whose index is strictly less than lastWBLFile
// * garbage collects all the m-map chunks from the memory that are less than or equal to minOOOMmapRef
// and then deletes the series that do not have any data anymore.
func (h *Head) truncateOOO(lastWBLFile int, minOOOMmapRef chunks.ChunkDiskMapperRef) error {
curMinOOOMmapRef := chunks.ChunkDiskMapperRef(h.minOOOMmapRef.Load())
if minOOOMmapRef.GreaterThan(curMinOOOMmapRef) {
h.minOOOMmapRef.Store(uint64(minOOOMmapRef))
start := time.Now()
actualMint, minMmapFile := h.gc()
level.Info(h.logger).Log("msg", "Head GC completed in truncateOOO", "duration", time.Since(start))
h.metrics.gcDuration.Observe(time.Since(start).Seconds())
if actualMint > h.minTime.Load() {
// The actual mint of the Head is higher than the one asked to truncate.
appendableMinValidTime := h.appendableMinValidTime()
if actualMint < appendableMinValidTime {
h.minTime.Store(actualMint)
h.minValidTime.Store(actualMint)
} else {
// The actual min time is in the appendable window.
// So we set the mint to the appendableMinValidTime.
h.minTime.Store(appendableMinValidTime)
h.minValidTime.Store(appendableMinValidTime)
}
}
// Truncate the chunk m-mapper.
if err := h.chunkDiskMapper.Truncate(minMmapFile); err != nil {
return errors.Wrap(err, "truncate chunks.HeadReadWriter by file number in truncateOOO")
}
}
return h.wbl.Truncate(lastWBLFile)
}
type Stats struct {
NumSeries uint64
MinTime, MaxTime int64
@ -1195,14 +1388,19 @@ func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error {
}
// gc removes data before the minimum timestamp from the head.
// It returns the actual min times of the chunks present in the Head.
func (h *Head) gc() int64 {
// It returns
// * The actual min times of the chunks present in the Head.
// * Min mmap file number seen in the series (in-order and out-of-order) after gc'ing the series.
func (h *Head) gc() (int64, int) {
// Only data strictly lower than this timestamp must be deleted.
mint := h.MinTime()
// Only ooo m-map chunks strictly lower than or equal to this ref
// must be deleted.
minOOOMmapRef := chunks.ChunkDiskMapperRef(h.minOOOMmapRef.Load())
// Drop old chunks and remember series IDs and hashes if they can be
// deleted entirely.
deleted, chunksRemoved, actualMint := h.series.gc(mint)
deleted, chunksRemoved, actualMint, minMmapFile := h.series.gc(mint, minOOOMmapRef)
seriesRemoved := len(deleted)
h.metrics.seriesRemoved.Add(float64(seriesRemoved))
@ -1232,7 +1430,7 @@ func (h *Head) gc() int64 {
h.deletedMtx.Unlock()
}
return actualMint
return actualMint, minMmapFile
}
// Tombstones returns a new reader over the head's tombstones
@ -1270,6 +1468,18 @@ func (h *Head) MaxTime() int64 {
return h.maxTime.Load()
}
// MinOOOTime returns the lowest time bound on visible data in the out of order
// head.
func (h *Head) MinOOOTime() int64 {
return h.minOOOTime.Load()
}
// MaxOOOTime returns the highest timestamp on visible data in the out of order
// head.
func (h *Head) MaxOOOTime() int64 {
return h.maxOOOTime.Load()
}
// compactable returns whether the head has a compactable range.
// The head has a compactable range when the head time range is 1.5 times the chunk range.
// The 0.5 acts as a buffer of the appendable window.
@ -1287,6 +1497,9 @@ func (h *Head) Close() error {
if h.wal != nil {
errs.Add(h.wal.Close())
}
if h.wbl != nil {
errs.Add(h.wbl.Close())
}
if errs.Err() == nil && h.opts.EnableMemorySnapshotOnShutdown {
errs.Add(h.performChunkSnapshot())
}
@ -1317,7 +1530,9 @@ func (h *Head) getOrCreate(hash uint64, lset labels.Labels) (*memSeries, bool, e
func (h *Head) getOrCreateWithID(id chunks.HeadSeriesRef, hash uint64, lset labels.Labels) (*memSeries, bool, error) {
s, created, err := h.series.getOrSet(hash, lset, func() *memSeries {
return newMemSeries(lset, id, hash, h.chunkRange.Load(), h.opts.ChunkEndTimeVariance, &h.memChunkPool, h.opts.IsolationDisabled)
return newMemSeries(lset, id, hash, h.chunkRange.Load(),
h.opts.OutOfOrderCapMin.Load(), h.opts.OutOfOrderCapMax.Load(),
h.opts.ChunkEndTimeVariance, &h.memChunkPool, h.opts.IsolationDisabled)
})
if err != nil {
return nil, false, err
@ -1379,7 +1594,7 @@ const (
)
// stripeSeries holds series by HeadSeriesRef ("ID") and also by hash of their labels.
// ID-based lookups via (getByID()) are preferred over getByHash() for performance reasons.
// ID-based lookups via getByID() are preferred over getByHash() for performance reasons.
// It locks modulo ranges of IDs and hashes to reduce lock contention.
// The locks are padded to not be on the same cache line. Filling the padded space
// with the maps was profiled to be slower – likely due to the additional pointer
@ -1421,13 +1636,15 @@ func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *st
// note: returning map[chunks.HeadSeriesRef]struct{} would be more accurate,
// but the returned map goes into postings.Delete() which expects a map[storage.SeriesRef]struct
// and there's no easy way to cast maps.
func (s *stripeSeries) gc(mint int64) (map[storage.SeriesRef]struct{}, int, int64) {
// minMmapFile is the min mmap file number seen in the series (in-order and out-of-order) after gc'ing the series.
func (s *stripeSeries) gc(mint int64, minOOOMmapRef chunks.ChunkDiskMapperRef) (_ map[storage.SeriesRef]struct{}, _ int, _ int64, minMmapFile int) {
var (
deleted = map[storage.SeriesRef]struct{}{}
deletedForCallback = []labels.Labels{}
rmChunks = 0
actualMint int64 = math.MaxInt64
)
minMmapFile = math.MaxInt32
// Run through all series and truncate old chunks. Mark those with no
// chunks left as deleted and store their ID.
for i := 0; i < s.size; i++ {
@ -1436,9 +1653,22 @@ func (s *stripeSeries) gc(mint int64) (map[storage.SeriesRef]struct{}, int, int6
for hash, all := range s.hashes[i] {
for _, series := range all {
series.Lock()
rmChunks += series.truncateChunksBefore(mint)
rmChunks += series.truncateChunksBefore(mint, minOOOMmapRef)
if len(series.mmappedChunks) > 0 || series.headChunk != nil || series.pendingCommit {
if len(series.mmappedChunks) > 0 {
seq, _ := series.mmappedChunks[0].ref.Unpack()
if seq < minMmapFile {
minMmapFile = seq
}
}
if len(series.oooMmappedChunks) > 0 {
seq, _ := series.oooMmappedChunks[0].ref.Unpack()
if seq < minMmapFile {
minMmapFile = seq
}
}
if len(series.mmappedChunks) > 0 || len(series.oooMmappedChunks) > 0 ||
series.headChunk != nil || series.oooHeadChunk != nil || series.pendingCommit {
seriesMint := series.minTime()
if seriesMint < actualMint {
actualMint = seriesMint
@ -1481,7 +1711,7 @@ func (s *stripeSeries) gc(mint int64) (map[storage.SeriesRef]struct{}, int, int6
actualMint = mint
}
return deleted, rmChunks, actualMint
return deleted, rmChunks, actualMint, minMmapFile
}
func (s *stripeSeries) getByID(id chunks.HeadSeriesRef) *memSeries {
@ -1574,11 +1804,18 @@ type memSeries struct {
//
// pN is the pointer to the mmappedChunk referered to by HeadChunkID=N
mmappedChunks []*mmappedChunk
mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay.
headChunk *memChunk // Most recent chunk in memory that's still being built.
chunkRange int64
headChunk *memChunk // Most recent chunk in memory that's still being built.
firstChunkID chunks.HeadChunkID // HeadChunkID for mmappedChunks[0]
oooMmappedChunks []*mmappedChunk // Immutable chunks on disk containing OOO samples.
oooHeadChunk *oooHeadChunk // Most recent chunk for ooo samples in memory that's still being built.
firstOOOChunkID chunks.HeadChunkID // HeadOOOChunkID for oooMmappedChunks[0]
mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay.
chunkRange int64
oooCapMin uint8
oooCapMax uint8
// chunkEndTimeVariance is how much variance (between 0 and 1) should be applied to the chunk end time,
// to spread chunks writing across time. Doesn't apply to the last chunk of the chunk range. 0 to disable variance.
chunkEndTimeVariance float64
@ -1602,7 +1839,7 @@ type memSeries struct {
txs *txRing
}
func newMemSeries(lset labels.Labels, id chunks.HeadSeriesRef, hash uint64, chunkRange int64, chunkEndTimeVariance float64, memChunkPool *sync.Pool, isolationDisabled bool) *memSeries {
func newMemSeries(lset labels.Labels, id chunks.HeadSeriesRef, hash uint64, chunkRange, oooCapMin, oooCapMax int64, chunkEndTimeVariance float64, memChunkPool *sync.Pool, isolationDisabled bool) *memSeries {
s := &memSeries{
lset: lset,
hash: hash,
@ -1611,6 +1848,8 @@ func newMemSeries(lset labels.Labels, id chunks.HeadSeriesRef, hash uint64, chun
chunkEndTimeVariance: chunkEndTimeVariance,
nextAt: math.MinInt64,
memChunkPool: memChunkPool,
oooCapMin: uint8(oooCapMin),
oooCapMax: uint8(oooCapMax),
}
if !isolationDisabled {
s.txs = newTxRing(4)
@ -1629,6 +1868,7 @@ func (s *memSeries) minTime() int64 {
}
func (s *memSeries) maxTime() int64 {
// The highest timestamps will always be in the regular (non-OOO) chunks, even if OOO is enabled.
c := s.head()
if c != nil {
return c.maxTime
@ -1642,26 +1882,39 @@ func (s *memSeries) maxTime() int64 {
// truncateChunksBefore removes all chunks from the series that
// have no timestamp at or after mint.
// Chunk IDs remain unchanged.
func (s *memSeries) truncateChunksBefore(mint int64) (removed int) {
func (s *memSeries) truncateChunksBefore(mint int64, minOOOMmapRef chunks.ChunkDiskMapperRef) int {
var removedInOrder int
if s.headChunk != nil && s.headChunk.maxTime < mint {
// If head chunk is truncated, we can truncate all mmapped chunks.
removed = 1 + len(s.mmappedChunks)
s.firstChunkID += chunks.HeadChunkID(removed)
removedInOrder = 1 + len(s.mmappedChunks)
s.firstChunkID += chunks.HeadChunkID(removedInOrder)
s.headChunk = nil
s.mmappedChunks = nil
return removed
}
if len(s.mmappedChunks) > 0 {
for i, c := range s.mmappedChunks {
if c.maxTime >= mint {
break
}
removed = i + 1
removedInOrder = i + 1
}
s.mmappedChunks = append(s.mmappedChunks[:0], s.mmappedChunks[removed:]...)
s.firstChunkID += chunks.HeadChunkID(removed)
s.mmappedChunks = append(s.mmappedChunks[:0], s.mmappedChunks[removedInOrder:]...)
s.firstChunkID += chunks.HeadChunkID(removedInOrder)
}
return removed
var removedOOO int
if len(s.oooMmappedChunks) > 0 {
for i, c := range s.oooMmappedChunks {
if c.ref.GreaterThan(minOOOMmapRef) {
break
}
removedOOO = i + 1
}
s.oooMmappedChunks = append(s.oooMmappedChunks[:0], s.oooMmappedChunks[removedOOO:]...)
s.firstOOOChunkID += chunks.HeadChunkID(removedOOO)
}
return removedInOrder + removedOOO
}
// cleanupAppendIDsBelow cleans up older appendIDs. Has to be called after
@ -1681,6 +1934,16 @@ type memChunk struct {
minTime, maxTime int64
}
type oooHeadChunk struct {
chunk *chunkenc.OOOChunk
minTime, maxTime int64 // can probably be removed and pulled out of the chunk instead
}
// OverlapsClosedInterval returns true if the chunk overlaps [mint, maxt].
func (mc *oooHeadChunk) OverlapsClosedInterval(mint, maxt int64) bool {
return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt)
}
// OverlapsClosedInterval returns true if the chunk overlaps [mint, maxt].
func (mc *memChunk) OverlapsClosedInterval(mint, maxt int64) bool {
return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt)
@ -1709,12 +1972,15 @@ func (noopSeriesLifecycleCallback) PostCreation(labels.Labels) {}
func (noopSeriesLifecycleCallback) PostDeletion(...labels.Labels) {}
func (h *Head) Size() int64 {
var walSize int64
var walSize, wblSize int64
if h.wal != nil {
walSize, _ = h.wal.Size()
}
if h.wbl != nil {
wblSize, _ = h.wbl.Size()
}
cdmSize, _ := h.chunkDiskMapper.Size()
return walSize + cdmSize
return walSize + wblSize + cdmSize
}
func (h *RangeHead) Size() int64 {

301
tsdb/head_append.go

@ -127,6 +127,7 @@ func (h *Head) appender() *headAppender {
minValidTime: h.appendableMinValidTime(),
mint: math.MaxInt64,
maxt: math.MinInt64,
headMaxt: h.MaxTime(),
samples: h.getAppendBuffer(),
sampleSeries: h.getSeriesBuffer(),
exemplars: exemplarsBuf,
@ -238,6 +239,7 @@ type headAppender struct {
head *Head
minValidTime int64 // No samples below this timestamp are allowed.
mint, maxt int64
headMaxt int64 // We track it here to not take the lock for every sample appended.
series []record.RefSeries // New series held by this appender.
samples []record.RefSample // New samples held by this appender.
@ -249,7 +251,10 @@ type headAppender struct {
}
func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) {
if t < a.minValidTime {
// For OOO inserts, this restriction is irrelevant and will be checked later once we confirm the sample is an in-order append.
// If OOO inserts are disabled, we may as well as check this as early as we can and avoid more work.
oooAllowance := a.head.opts.OutOfOrderAllowance.Load()
if oooAllowance == 0 && t < a.minValidTime {
a.head.metrics.outOfBoundSamples.Inc()
return 0, storage.ErrOutOfBounds
}
@ -281,16 +286,25 @@ func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64
}
s.Lock()
if delta, err := s.appendable(t, v); err != nil {
s.Unlock()
// TODO: if we definitely know at this point that the sample is ooo, then optimise
// to skip that sample from the WAL and write only in the WBL.
_, delta, err := s.appendable(t, v, a.headMaxt, a.minValidTime, oooAllowance)
if err == nil {
s.pendingCommit = true
}
s.Unlock()
if delta > 0 {
a.head.metrics.oooHistogram.Observe(float64(delta) / 1000)
}
if err != nil {
if err == storage.ErrOutOfOrderSample {
a.head.metrics.outOfOrderSamples.Inc()
a.head.metrics.oooHistogram.Observe(float64(delta) / 1000)
}
if err == storage.ErrTooOldSample {
a.head.metrics.tooOldSamples.Inc()
}
return 0, err
}
s.pendingCommit = true
s.Unlock()
if t < a.mint {
a.mint = t
@ -308,24 +322,53 @@ func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64
return storage.SeriesRef(s.ref), nil
}
// appendable checks whether the given sample is valid for appending to the series.
func (s *memSeries) appendable(t int64, v float64) (int64, error) {
c := s.head()
if c == nil {
return 0, nil
// appendable checks whether the given sample is valid for appending to the series. (if we return false and no error)
// The sample belongs to the out of order chunk if we return true and no error.
// An error signifies the sample cannot be handled.
func (s *memSeries) appendable(t int64, v float64, headMaxt, minValidTime, oooAllowance int64) (isOutOfOrder bool, delta int64, err error) {
msMaxt := s.maxTime()
if msMaxt == math.MinInt64 {
// The series has no sample and was freshly created.
if t >= minValidTime {
// We can append it in the in-order chunk.
return false, 0, nil
}
// We cannot append it in the in-order head. So we check the oooAllowance
// w.r.t. the head's maxt.
// -1 because for the first sample in the Head, headMaxt will be equal to t.
msMaxt = headMaxt - 1
}
if t > c.maxTime {
return 0, nil
if t > msMaxt {
return false, 0, nil
}
if t < c.maxTime {
return c.maxTime - t, storage.ErrOutOfOrderSample
if t < msMaxt-oooAllowance {
if oooAllowance > 0 {
return true, msMaxt - t, storage.ErrTooOldSample
}
if t < minValidTime {
return false, msMaxt - t, storage.ErrOutOfBounds
}
return false, msMaxt - t, storage.ErrOutOfOrderSample
}
if t != msMaxt || s.head() == nil {
// Sample is ooo and within allowance OR series has no active chunk to check for duplicate sample.
return true, msMaxt - t, nil
}
// 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.
// this only checks against the latest in-order sample.
// the OOO headchunk has its own method to detect these duplicates
if math.Float64bits(s.sampleBuf[3].v) != math.Float64bits(v) {
return 0, storage.ErrDuplicateSampleForTimestamp
return false, 0, storage.ErrDuplicateSampleForTimestamp
}
return 0, nil
// sample is identical (ts + value) with most current (highest ts) sample in sampleBuf
return false, 0, nil
}
// AppendExemplar for headAppender assumes the series ref already exists, and so it doesn't
@ -458,33 +501,201 @@ func (a *headAppender) Commit() (err error) {
defer a.head.putExemplarBuffer(a.exemplars)
defer a.head.iso.closeAppend(a.appendID)
total := len(a.samples)
var series *memSeries
var (
samplesAppended = len(a.samples)
oooAccepted int // number of samples out of order but accepted: with ooo enabled and within allowance
oooRejected int // number of samples rejected due to: out of order but OOO support disabled.
tooOldRejected int // number of samples rejected due to: that are out of order but too old (OOO support enabled, but outside allowance)
oobRejected int // number of samples rejected due to: out of bounds: with t < minValidTime (OOO support disabled)
inOrderMint int64 = math.MaxInt64
inOrderMaxt int64 = math.MinInt64
ooomint int64 = math.MaxInt64
ooomaxt int64 = math.MinInt64
wblSamples []record.RefSample
oooMmapMarkers map[chunks.HeadSeriesRef]chunks.ChunkDiskMapperRef
oooRecords [][]byte
series *memSeries
enc record.Encoder
)
defer func() {
for i := range oooRecords {
a.head.putBytesBuffer(oooRecords[i][:0])
}
}()
collectOOORecords := func() {
if a.head.wbl == nil {
// WBL is not enabled. So no need to collect.
wblSamples = nil
oooMmapMarkers = nil
return
}
// The m-map happens before adding a new sample. So we collect
// the m-map markers first, and then samples.
// WBL Graphically:
// WBL Before this Commit(): [old samples before this commit for chunk 1]
// WBL After this Commit(): [old samples before this commit for chunk 1][new samples in this commit for chunk 1]mmapmarker1[samples for chunk 2]mmapmarker2[samples for chunk 3]
if oooMmapMarkers != nil {
markers := make([]record.RefMmapMarker, 0, len(oooMmapMarkers))
for ref, mmapRef := range oooMmapMarkers {
markers = append(markers, record.RefMmapMarker{
Ref: ref,
MmapRef: mmapRef,
})
}
r := enc.MmapMarkers(markers, a.head.getBytesBuffer())
oooRecords = append(oooRecords, r)
}
if len(wblSamples) > 0 {
r := enc.Samples(wblSamples, a.head.getBytesBuffer())
oooRecords = append(oooRecords, r)
}
wblSamples = nil
oooMmapMarkers = nil
}
oooAllowance := a.head.opts.OutOfOrderAllowance.Load()
for i, s := range a.samples {
series = a.sampleSeries[i]
series.Lock()
delta, ok, chunkCreated := series.append(s.T, s.V, a.appendID, a.head.chunkDiskMapper)
series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow)
series.pendingCommit = false
series.Unlock()
if !ok {
total--
oooSample, delta, err := series.appendable(s.T, s.V, a.headMaxt, a.minValidTime, oooAllowance)
switch err {
case storage.ErrOutOfOrderSample:
samplesAppended--
oooRejected++
case storage.ErrOutOfBounds:
samplesAppended--
oobRejected++
case storage.ErrTooOldSample:
samplesAppended--
tooOldRejected++
case nil:
// Do nothing.
default:
samplesAppended--
}
var ok, chunkCreated bool
if err == nil && oooSample {
// Sample is OOO and OOO handling is enabled
// and the delta is within the OOO tolerance.
var mmapRef chunks.ChunkDiskMapperRef
ok, chunkCreated, mmapRef = series.insert(s.T, s.V, a.head.chunkDiskMapper)
if chunkCreated {
r, ok := oooMmapMarkers[series.ref]
if !ok || r != 0 {
// !ok means there are no markers collected for these samples yet. So we first flush the samples
// before setting this m-map marker.
// r != 0 means we have already m-mapped a chunk for this series in the same Commit().
// Hence, before we m-map again, we should add the samples and m-map markers
// seen till now to the WBL records.
collectOOORecords()
}
if oooMmapMarkers == nil {
oooMmapMarkers = make(map[chunks.HeadSeriesRef]chunks.ChunkDiskMapperRef)
}
oooMmapMarkers[series.ref] = mmapRef
}
if ok {
wblSamples = append(wblSamples, s)
if s.T < ooomint {
ooomint = s.T
}
if s.T > ooomaxt {
ooomaxt = s.T
}
oooAccepted++
} else {
// exact duplicate of last sample.
// the sample was an attempted update.
// note that we can only detect updates if they clash with a sample in the OOOHeadChunk,
// not with samples in already flushed OOO chunks.
// TODO: error reporting? depends on addressing https://github.com/prometheus/prometheus/discussions/10305
samplesAppended--
}
} else if err == nil {
// if we're here, either of these is true:
// - the sample.t is beyond any previously ingested timestamp
// - the sample is an exact duplicate of the 'head sample'
delta, ok, chunkCreated = series.append(s.T, s.V, a.appendID, a.head.chunkDiskMapper)
// TODO: handle overwrite.
// this would be storage.ErrDuplicateSampleForTimestamp, it has no attached counter
// in case of identical timestamp and value, we should drop silently
if ok {
// sample timestamp is beyond any previously ingested timestamp
if s.T < inOrderMint { // TODO(ganesh): dieter thinks this never applies and can be removed because we know we're in order.
inOrderMint = s.T
}
if s.T > inOrderMaxt {
inOrderMaxt = s.T
}
} else {
// ... therefore, in this case, we know the sample is an exact duplicate, and should be silently dropped.
samplesAppended--
}
}
if delta > 0 {
a.head.metrics.oooHistogram.Observe(float64(delta) / 1000)
a.head.metrics.outOfOrderSamples.Inc()
}
if chunkCreated {
a.head.metrics.chunks.Inc()
a.head.metrics.chunksCreated.Inc()
}
}
a.head.metrics.samplesAppended.Add(float64(total))
a.head.updateMinMaxTime(a.mint, a.maxt)
series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow)
series.pendingCommit = false
series.Unlock()
}
a.head.metrics.outOfOrderSamples.Add(float64(oooRejected))
a.head.metrics.outOfBoundSamples.Add(float64(oobRejected))
a.head.metrics.tooOldSamples.Add(float64(tooOldRejected))
a.head.metrics.samplesAppended.Add(float64(samplesAppended))
a.head.metrics.outOfOrderSamplesAppended.Add(float64(oooAccepted))
a.head.updateMinMaxTime(inOrderMint, inOrderMaxt)
a.head.updateMinOOOMaxOOOTime(ooomint, ooomaxt)
// TODO: currently WBL logging of ooo samples is best effort here since we cannot try logging
// until we have found what samples become OOO. We can try having a metric for this failure.
// Returning the error here is not correct because we have already put the samples into the memory,
// hence the append/insert was a success.
collectOOORecords()
if a.head.wbl != nil {
if err := a.head.wbl.Log(oooRecords...); err != nil {
level.Error(a.head.logger).Log("msg", "Failed to log out of order samples into the WAL", "err", err)
}
}
return nil
}
// insert is like append, except it inserts. used for Out Of Order samples.
func (s *memSeries) insert(t int64, v float64, chunkDiskMapper chunkDiskMapper) (inserted, chunkCreated bool, mmapRef chunks.ChunkDiskMapperRef) {
c := s.oooHeadChunk
if c == nil || c.chunk.NumSamples() == int(s.oooCapMax) {
// Note: If no new samples come in then we rely on compaction to clean up stale in-memory OOO chunks.
c, mmapRef = s.cutNewOOOHeadChunk(t, chunkDiskMapper)
chunkCreated = true
}
ok := c.chunk.Insert(t, v)
if ok {
if chunkCreated || t < c.minTime {
c.minTime = t
}
if chunkCreated || t > c.maxTime {
c.maxTime = t
}
}
return ok, chunkCreated, mmapRef
}
// append adds the sample (t, v) to the series. The caller also has to provide
// the appendID for isolation. (The appendID can be zero, which results in no
// isolation for this append.)
@ -502,7 +713,7 @@ func (s *memSeries) append(t int64, v float64, appendID uint64, chunkDiskMapper
// Out of order sample. Sample timestamp is already in the mmapped chunks, so ignore it.
return s.mmappedChunks[len(s.mmappedChunks)-1].maxTime - t, false, false
}
// There is no chunk in this series yet, create the first chunk for the sample.
// There is no head chunk in this series yet, create the first chunk for the sample.
c = s.cutNewHeadChunk(t, chunkDiskMapper)
chunkCreated = true
}
@ -613,6 +824,36 @@ func (s *memSeries) cutNewHeadChunk(mint int64, chunkDiskMapper chunkDiskMapper)
return s.headChunk
}
func (s *memSeries) cutNewOOOHeadChunk(mint int64, chunkDiskMapper chunkDiskMapper) (*oooHeadChunk, chunks.ChunkDiskMapperRef) {
ref := s.mmapCurrentOOOHeadChunk(chunkDiskMapper)
s.oooHeadChunk = &oooHeadChunk{
chunk: chunkenc.NewOOOChunk(int(s.oooCapMin)),
minTime: mint,
maxTime: math.MinInt64,
}
return s.oooHeadChunk, ref
}
func (s *memSeries) mmapCurrentOOOHeadChunk(chunkDiskMapper chunkDiskMapper) chunks.ChunkDiskMapperRef {
if s.oooHeadChunk == nil {
// There is no head chunk, so nothing to m-map here.
return 0
}
xor, _ := s.oooHeadChunk.chunk.ToXor() // encode to XorChunk which is more compact and implements all of the needed functionality to be encoded
oooXor := &chunkenc.OOOXORChunk{XORChunk: xor}
chunkRef := chunkDiskMapper.WriteChunk(s.ref, s.oooHeadChunk.minTime, s.oooHeadChunk.maxTime, oooXor, handleChunkWriteError)
s.oooMmappedChunks = append(s.oooMmappedChunks, &mmappedChunk{
ref: chunkRef,
numSamples: uint16(xor.NumSamples()),
minTime: s.oooHeadChunk.minTime,
maxTime: s.oooHeadChunk.maxTime,
})
s.oooHeadChunk = nil
return chunkRef
}
func (s *memSeries) mmapCurrentHeadChunk(chunkDiskMapper chunkDiskMapper) {
if s.headChunk == nil {
// There is no head chunk, so nothing to m-map here.

6
tsdb/head_bench_test.go

@ -30,7 +30,7 @@ func BenchmarkHeadStripeSeriesCreate(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
defer h.Close()
@ -45,7 +45,7 @@ func BenchmarkHeadStripeSeriesCreateParallel(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
defer h.Close()
@ -69,7 +69,7 @@ func BenchmarkHeadStripeSeriesCreate_PreCreationFailure(b *testing.B) {
// Mock the PreCreation() callback to fail on each series.
opts.SeriesCallback = failingSeriesLifecycleCallback{}
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
defer h.Close()

267
tsdb/head_read.go

@ -212,11 +212,20 @@ func (h *headIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chk
return nil
}
// headChunkID returns the HeadChunkID corresponding to .mmappedChunks[pos]
// headChunkID returns the HeadChunkID referred to by the given position.
// * 0 <= pos < len(s.mmappedChunks) refer to s.mmappedChunks[pos]
// * pos == len(s.mmappedChunks) refers to s.headChunk
func (s *memSeries) headChunkID(pos int) chunks.HeadChunkID {
return chunks.HeadChunkID(pos) + s.firstChunkID
}
// oooHeadChunkID returns the HeadChunkID referred to by the given position.
// * 0 <= pos < len(s.oooMmappedChunks) refer to s.oooMmappedChunks[pos]
// * pos == len(s.oooMmappedChunks) refers to s.oooHeadChunk
func (s *memSeries) oooHeadChunkID(pos int) chunks.HeadChunkID {
return chunks.HeadChunkID(pos) + s.firstOOOChunkID
}
// LabelValueFor returns label value for the given label name in the series referred to by ID.
func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) {
memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id))
@ -287,8 +296,8 @@ func (h *headChunkReader) Close() error {
}
// Chunk returns the chunk for the reference number.
func (h *headChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) {
sid, cid := chunks.HeadChunkRef(ref).Unpack()
func (h *headChunkReader) Chunk(meta chunks.Meta) (chunkenc.Chunk, error) {
sid, cid := chunks.HeadChunkRef(meta.Ref).Unpack()
s := h.head.series.getByID(sid)
// This means that the series has been garbage collected.
@ -358,6 +367,258 @@ func (s *memSeries) chunk(id chunks.HeadChunkID, cdm chunkDiskMapper) (chunk *me
return mc, true, nil
}
// oooMergedChunk returns the requested chunk based on the given chunks.Meta
// reference from memory or by m-mapping it from the disk. The returned chunk
// might be a merge of all the overlapping chunks, if any, amongst all the
// chunks in the OOOHead.
// This function is not thread safe unless the caller holds a lock.
func (s *memSeries) oooMergedChunk(meta chunks.Meta, cdm chunkDiskMapper, mint, maxt int64) (chunk *mergedOOOChunks, err error) {
_, cid := chunks.HeadChunkRef(meta.Ref).Unpack()
// ix represents the index of chunk in the s.mmappedChunks slice. The chunk meta's are
// incremented by 1 when new chunk is created, hence (meta - firstChunkID) gives the slice index.
// The max index for the s.mmappedChunks slice can be len(s.mmappedChunks)-1, hence if the ix
// is len(s.mmappedChunks), it represents the next chunk, which is the head chunk.
ix := int(cid) - int(s.firstOOOChunkID)
if ix < 0 || ix > len(s.oooMmappedChunks) {
return nil, storage.ErrNotFound
}
if ix == len(s.oooMmappedChunks) {
if s.oooHeadChunk == nil {
return nil, errors.New("invalid ooo head chunk")
}
}
// We create a temporary slice of chunk metas to hold the information of all
// possible chunks that may overlap with the requested chunk.
tmpChks := make([]chunkMetaAndChunkDiskMapperRef, 0, len(s.oooMmappedChunks))
oooHeadRef := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(len(s.oooMmappedChunks))))
if s.oooHeadChunk != nil && s.oooHeadChunk.OverlapsClosedInterval(mint, maxt) {
// We only want to append the head chunk if this chunk existed when
// Series() was called. This brings consistency in case new data
// is added in between Series() and Chunk() calls
if oooHeadRef == meta.OOOLastRef {
tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{
meta: chunks.Meta{
MinTime: meta.OOOLastMinTime, // we want to ignore samples that were added before last known min time
MaxTime: meta.OOOLastMaxTime, // we want to ignore samples that were added after last known max time
Ref: oooHeadRef,
},
})
}
}
for i, c := range s.oooMmappedChunks {
chunkRef := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(i)))
// We can skip chunks that came in later than the last known OOOLastRef
if chunkRef > meta.OOOLastRef {
break
}
if chunkRef == meta.OOOLastRef {
tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{
meta: chunks.Meta{
MinTime: meta.OOOLastMinTime,
MaxTime: meta.OOOLastMaxTime,
Ref: chunkRef,
},
ref: c.ref,
origMinT: c.minTime,
origMaxT: c.maxTime,
})
} else if c.OverlapsClosedInterval(mint, maxt) {
tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{
meta: chunks.Meta{
MinTime: c.minTime,
MaxTime: c.maxTime,
Ref: chunkRef,
},
ref: c.ref,
})
}
}
// Next we want to sort all the collected chunks by min time so we can find
// those that overlap and stop when we know the rest don't.
sort.Sort(byMinTimeAndMinRef(tmpChks))
mc := &mergedOOOChunks{}
absoluteMax := int64(math.MinInt64)
for _, c := range tmpChks {
if c.meta.Ref == meta.Ref || len(mc.chunks) > 0 && c.meta.MinTime <= absoluteMax {
if c.meta.Ref == oooHeadRef {
var xor *chunkenc.XORChunk
// If head chunk min and max time match the meta OOO markers
// that means that the chunk has not expanded so we can append
// it as it is.
if s.oooHeadChunk.minTime == meta.OOOLastMinTime && s.oooHeadChunk.maxTime == meta.OOOLastMaxTime {
xor, err = s.oooHeadChunk.chunk.ToXor() // TODO(jesus.vazquez) (This is an optimization idea that has no priority and might not be that useful) See if we could use a copy of the underlying slice. That would leave the more expensive ToXor() function only for the usecase where Bytes() is called.
} else {
// We need to remove samples that are outside of the markers
xor, err = s.oooHeadChunk.chunk.ToXorBetweenTimestamps(meta.OOOLastMinTime, meta.OOOLastMaxTime)
}
if err != nil {
return nil, errors.Wrap(err, "failed to convert ooo head chunk to xor chunk")
}
c.meta.Chunk = xor
} else {
chk, err := cdm.Chunk(c.ref)
if err != nil {
if _, ok := err.(*chunks.CorruptionErr); ok {
return nil, errors.Wrap(err, "invalid ooo mmapped chunk")
}
return nil, err
}
if c.meta.Ref == meta.OOOLastRef &&
(c.origMinT != meta.OOOLastMinTime || c.origMaxT != meta.OOOLastMaxTime) {
// The head expanded and was memory mapped so now we need to
// wrap the chunk within a chunk that doesnt allows us to iterate
// through samples out of the OOOLastMinT and OOOLastMaxT
// markers.
c.meta.Chunk = boundedChunk{chk, meta.OOOLastMinTime, meta.OOOLastMaxTime}
} else {
c.meta.Chunk = chk
}
}
mc.chunks = append(mc.chunks, c.meta)
if c.meta.MaxTime > absoluteMax {
absoluteMax = c.meta.MaxTime
}
}
}
return mc, nil
}
var _ chunkenc.Chunk = &mergedOOOChunks{}
// mergedOOOChunks holds the list of overlapping chunks. This struct satisfies
// chunkenc.Chunk.
type mergedOOOChunks struct {
chunks []chunks.Meta
}
// Bytes is a very expensive method because its calling the iterator of all the
// chunks in the mergedOOOChunk and building a new chunk with the samples.
func (o mergedOOOChunks) Bytes() []byte {
xc := chunkenc.NewXORChunk()
app, err := xc.Appender()
if err != nil {
panic(err)
}
it := o.Iterator(nil)
for it.Next() {
t, v := it.At()
app.Append(t, v)
}
return xc.Bytes()
}
func (o mergedOOOChunks) Encoding() chunkenc.Encoding {
return chunkenc.EncXOR
}
func (o mergedOOOChunks) Appender() (chunkenc.Appender, error) {
return nil, errors.New("can't append to mergedOOOChunks")
}
func (o mergedOOOChunks) Iterator(iterator chunkenc.Iterator) chunkenc.Iterator {
iterators := make([]chunkenc.Iterator, 0, len(o.chunks))
for _, c := range o.chunks {
iterators = append(iterators, c.Chunk.Iterator(nil))
}
return storage.NewChainSampleIterator(iterators)
}
func (o mergedOOOChunks) NumSamples() int {
samples := 0
for _, c := range o.chunks {
samples += c.Chunk.NumSamples()
}
return samples
}
func (o mergedOOOChunks) Compact() {}
var _ chunkenc.Chunk = &boundedChunk{}
// boundedChunk is an implementation of chunkenc.Chunk that uses a
// boundedIterator that only iterates through samples which timestamps are
// >= minT and <= maxT
type boundedChunk struct {
chunkenc.Chunk
minT int64
maxT int64
}
func (b boundedChunk) Bytes() []byte {
xor := chunkenc.NewXORChunk()
a, _ := xor.Appender()
it := b.Iterator(nil)
for it.Next() {
t, v := it.At()
a.Append(t, v)
}
return xor.Bytes()
}
func (b boundedChunk) Iterator(iterator chunkenc.Iterator) chunkenc.Iterator {
it := b.Chunk.Iterator(iterator)
if it == nil {
panic("iterator shouldn't be nil")
}
return boundedIterator{it, b.minT, b.maxT}
}
var _ chunkenc.Iterator = &boundedIterator{}
// boundedIterator is an implementation of Iterator that only iterates through
// samples which timestamps are >= minT and <= maxT
type boundedIterator struct {
chunkenc.Iterator
minT int64
maxT int64
}
// Next the first time its called it will advance as many positions as necessary
// until its able to find a sample within the bounds minT and maxT.
// If there are samples within bounds it will advance one by one amongst them.
// If there are no samples within bounds it will return false.
func (b boundedIterator) Next() bool {
for b.Iterator.Next() {
t, _ := b.Iterator.At()
if t < b.minT {
continue
} else if t > b.maxT {
return false
}
return true
}
return false
}
func (b boundedIterator) Seek(t int64) bool {
if t < b.minT {
// We must seek at least up to b.minT if it is asked for something before that.
ok := b.Iterator.Seek(b.minT)
if !ok {
return false
}
t, _ := b.Iterator.At()
return t <= b.maxT
}
if t > b.maxT {
// We seek anyway so that the subsequent Next() calls will also return false.
b.Iterator.Seek(t)
return false
}
return b.Iterator.Seek(t)
}
// safeChunk makes sure that the chunk can be accessed without a race condition
type safeChunk struct {
chunkenc.Chunk
s *memSeries

165
tsdb/head_read_test.go

@ -0,0 +1,165 @@
package tsdb
import (
"fmt"
"testing"
"github.com/stretchr/testify/require"
"github.com/prometheus/prometheus/tsdb/chunkenc"
)
func TestBoundedChunk(t *testing.T) {
tests := []struct {
name string
inputChunk chunkenc.Chunk
inputMinT int64
inputMaxT int64
initialSeek int64
seekIsASuccess bool
expSamples []sample
}{
{
name: "if there are no samples it returns nothing",
inputChunk: newTestChunk(0),
expSamples: nil,
},
{
name: "bounds represent a single sample",
inputChunk: newTestChunk(10),
expSamples: []sample{
{0, 0},
},
},
{
name: "if there are bounds set only samples within them are returned",
inputChunk: newTestChunk(10),
inputMinT: 1,
inputMaxT: 8,
expSamples: []sample{
{1, 1},
{2, 2},
{3, 3},
{4, 4},
{5, 5},
{6, 6},
{7, 7},
{8, 8},
},
},
{
name: "if bounds set and only maxt is less than actual maxt",
inputChunk: newTestChunk(10),
inputMinT: 0,
inputMaxT: 5,
expSamples: []sample{
{0, 0},
{1, 1},
{2, 2},
{3, 3},
{4, 4},
{5, 5},
},
},
{
name: "if bounds set and only mint is more than actual mint",
inputChunk: newTestChunk(10),
inputMinT: 5,
inputMaxT: 9,
expSamples: []sample{
{5, 5},
{6, 6},
{7, 7},
{8, 8},
{9, 9},
},
},
{
name: "if there are bounds set with seek before mint",
inputChunk: newTestChunk(10),
inputMinT: 3,
inputMaxT: 7,
initialSeek: 1,
seekIsASuccess: true,
expSamples: []sample{
{3, 3},
{4, 4},
{5, 5},
{6, 6},
{7, 7},
},
},
{
name: "if there are bounds set with seek between mint and maxt",
inputChunk: newTestChunk(10),
inputMinT: 3,
inputMaxT: 7,
initialSeek: 5,
seekIsASuccess: true,
expSamples: []sample{
{5, 5},
{6, 6},
{7, 7},
},
},
{
name: "if there are bounds set with seek after maxt",
inputChunk: newTestChunk(10),
inputMinT: 3,
inputMaxT: 7,
initialSeek: 8,
seekIsASuccess: false,
},
}
for _, tc := range tests {
t.Run(fmt.Sprintf("name=%s", tc.name), func(t *testing.T) {
chunk := boundedChunk{tc.inputChunk, tc.inputMinT, tc.inputMaxT}
// Testing Bytes()
expChunk := chunkenc.NewXORChunk()
if tc.inputChunk.NumSamples() > 0 {
app, err := expChunk.Appender()
require.NoError(t, err)
for ts := tc.inputMinT; ts <= tc.inputMaxT; ts++ {
app.Append(ts, float64(ts))
}
}
require.Equal(t, expChunk.Bytes(), chunk.Bytes())
var samples []sample
it := chunk.Iterator(nil)
if tc.initialSeek != 0 {
// Testing Seek()
ok := it.Seek(tc.initialSeek)
require.Equal(t, tc.seekIsASuccess, ok)
if ok {
t, v := it.At()
samples = append(samples, sample{t, v})
}
}
// Testing Next()
for it.Next() {
t, v := it.At()
samples = append(samples, sample{t, v})
}
// it.Next() should keep returning false.
for i := 0; i < 10; i++ {
require.False(t, it.Next())
}
require.Equal(t, tc.expSamples, samples)
})
}
}
func newTestChunk(numSamples int) chunkenc.Chunk {
xor := chunkenc.NewXORChunk()
a, _ := xor.Appender()
for i := 0; i < numSamples; i++ {
a.Append(int64(i), float64(i))
}
return xor
}

310
tsdb/head_test.go

@ -60,7 +60,7 @@ func newTestHead(t testing.TB, chunkRange int64, compressWAL bool) (*Head, *wal.
opts.EnableExemplarStorage = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
h, err := NewHead(nil, nil, wlog, opts, nil)
h, err := NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.chunkDiskMapper.IterateAllChunks(func(_ chunks.HeadSeriesRef, _ chunks.ChunkDiskMapperRef, _, _ int64, _ uint16, _ chunkenc.Encoding) error {
@ -223,7 +223,7 @@ func BenchmarkLoadWAL(b *testing.B) {
for k := 0; k < c.batches*c.seriesPerBatch; k++ {
// Create one mmapped chunk per series, with one sample at the given time.
lbls := labels.Labels{}
s := newMemSeries(lbls, chunks.HeadSeriesRef(k)*101, lbls.Hash(), c.mmappedChunkT, 0, nil, defaultIsolationDisabled)
s := newMemSeries(lbls, chunks.HeadSeriesRef(k)*101, lbls.Hash(), c.mmappedChunkT, 0, 1, 0, nil, defaultIsolationDisabled)
s.append(c.mmappedChunkT, 42, 0, chunkDiskMapper)
s.mmapCurrentHeadChunk(chunkDiskMapper)
}
@ -254,7 +254,7 @@ func BenchmarkLoadWAL(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = w.Dir()
h, err := NewHead(nil, nil, w, opts, nil)
h, err := NewHead(nil, nil, w, nil, opts, nil)
require.NoError(b, err)
h.Init(0)
}
@ -571,7 +571,7 @@ func TestHead_WALMultiRef(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = w.Dir()
head, err = NewHead(nil, nil, w, opts, nil)
head, err = NewHead(nil, nil, w, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
defer func() {
@ -733,7 +733,7 @@ func TestMemSeries_truncateChunks(t *testing.T) {
}
lbls := labels.FromStrings("a", "b")
s := newMemSeries(lbls, 1, lbls.Hash(), 2000, 0, &memChunkPool, defaultIsolationDisabled)
s := newMemSeries(lbls, 1, lbls.Hash(), 2000, 0, 1, 0, &memChunkPool, defaultIsolationDisabled)
for i := 0; i < 4000; i += 5 {
_, ok, _ := s.append(int64(i), float64(i), 0, chunkDiskMapper)
@ -752,7 +752,7 @@ func TestMemSeries_truncateChunks(t *testing.T) {
require.NotNil(t, chk)
require.NoError(t, err)
s.truncateChunksBefore(2000)
s.truncateChunksBefore(2000, 0)
require.Equal(t, int64(2000), s.mmappedChunks[0].minTime)
_, _, err = s.chunk(0, chunkDiskMapper)
@ -857,6 +857,7 @@ func TestHeadDeleteSimple(t *testing.T) {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
for _, c := range cases {
head, w := newTestHead(t, 1000, compress)
require.NoError(t, head.Init(0))
app := head.Appender(context.Background())
for _, smpl := range smplsAll {
@ -886,7 +887,7 @@ func TestHeadDeleteSimple(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = reloadedW.Dir()
reloadedHead, err := NewHead(nil, nil, reloadedW, opts, nil)
reloadedHead, err := NewHead(nil, nil, reloadedW, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, reloadedHead.Init(0))
@ -1268,7 +1269,7 @@ func TestMemSeries_append(t *testing.T) {
}()
lbls := labels.Labels{}
s := newMemSeries(lbls, 1, lbls.Hash(), 500, 0, nil, defaultIsolationDisabled)
s := newMemSeries(lbls, 1, lbls.Hash(), 500, 0, 1, 0, nil, defaultIsolationDisabled)
// Add first two samples at the very end of a chunk range and the next two
// on and after it.
@ -1323,7 +1324,7 @@ func TestMemSeries_append_atVariableRate(t *testing.T) {
})
lbls := labels.Labels{}
s := newMemSeries(lbls, 1, lbls.Hash(), DefaultBlockDuration, 0, nil, defaultIsolationDisabled)
s := newMemSeries(lbls, 1, lbls.Hash(), DefaultBlockDuration, 0, 0, 0, nil, defaultIsolationDisabled)
// At this slow rate, we will fill the chunk in two block durations.
slowRate := (DefaultBlockDuration * 2) / samplesPerChunk
@ -1398,16 +1399,16 @@ func TestGCChunkAccess(t *testing.T) {
cr, err := h.chunksRange(0, 1500, nil)
require.NoError(t, err)
_, err = cr.Chunk(chunks[0].Ref)
_, err = cr.Chunk(chunks[0])
require.NoError(t, err)
_, err = cr.Chunk(chunks[1].Ref)
_, err = cr.Chunk(chunks[1])
require.NoError(t, err)
require.NoError(t, h.Truncate(1500)) // Remove a chunk.
_, err = cr.Chunk(chunks[0].Ref)
_, err = cr.Chunk(chunks[0])
require.Equal(t, storage.ErrNotFound, err)
_, err = cr.Chunk(chunks[1].Ref)
_, err = cr.Chunk(chunks[1])
require.NoError(t, err)
}
@ -1452,18 +1453,18 @@ func TestGCSeriesAccess(t *testing.T) {
cr, err := h.chunksRange(0, 2000, nil)
require.NoError(t, err)
_, err = cr.Chunk(chunks[0].Ref)
_, err = cr.Chunk(chunks[0])
require.NoError(t, err)
_, err = cr.Chunk(chunks[1].Ref)
_, err = cr.Chunk(chunks[1])
require.NoError(t, err)
require.NoError(t, h.Truncate(2000)) // Remove the series.
require.Equal(t, (*memSeries)(nil), h.series.getByID(1))
_, err = cr.Chunk(chunks[0].Ref)
_, err = cr.Chunk(chunks[0])
require.Equal(t, storage.ErrNotFound, err)
_, err = cr.Chunk(chunks[1].Ref)
_, err = cr.Chunk(chunks[1])
require.Equal(t, storage.ErrNotFound, err)
}
@ -1608,7 +1609,7 @@ func TestWalRepair_DecodingError(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1
opts.ChunkDirRoot = w.Dir()
h, err := NewHead(nil, nil, w, opts, nil)
h, err := NewHead(nil, nil, w, nil, opts, nil)
require.NoError(t, err)
require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.walCorruptionsTotal))
initErr := h.Init(math.MinInt64)
@ -1662,7 +1663,8 @@ func TestHeadReadWriterRepair(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = chunkRange
opts.ChunkDirRoot = dir
h, err := NewHead(nil, nil, w, opts, nil)
opts.ChunkWriteQueueSize = 1 // We need to set this option so that we use the async queue. Upstream prometheus uses the queue directly.
h, err := NewHead(nil, nil, w, nil, opts, nil)
require.NoError(t, err)
require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.mmapChunkCorruptionTotal))
require.NoError(t, h.Init(math.MinInt64))
@ -1908,7 +1910,7 @@ func TestMemSeriesIsolation(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = wlog.Dir()
hb, err = NewHead(nil, nil, wlog, opts, nil)
hb, err = NewHead(nil, nil, wlog, nil, opts, nil)
defer func() { require.NoError(t, hb.Close()) }()
require.NoError(t, err)
require.NoError(t, hb.Init(0))
@ -2559,7 +2561,7 @@ func TestMemSafeIteratorSeekIntoBuffer(t *testing.T) {
}()
lbls := labels.Labels{}
s := newMemSeries(lbls, 1, lbls.Hash(), 500, 0, nil, defaultIsolationDisabled)
s := newMemSeries(lbls, 1, lbls.Hash(), 500, 0, 1, 0, nil, defaultIsolationDisabled)
for i := 0; i < 7; i++ {
_, ok, _ := s.append(int64(i), float64(i), 0, chunkDiskMapper)
@ -2909,7 +2911,7 @@ func TestChunkSnapshot(t *testing.T) {
openHeadAndCheckReplay := func() {
w, err := wal.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, head.opts, nil)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
@ -3119,7 +3121,7 @@ func TestSnapshotError(t *testing.T) {
w, err := wal.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
// Testing https://github.com/prometheus/prometheus/issues/9437 with the registry.
head, err = NewHead(prometheus.NewRegistry(), nil, w, head.opts, nil)
head, err = NewHead(prometheus.NewRegistry(), nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
@ -3178,7 +3180,7 @@ func TestChunkSnapshotReplayBug(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.EnableMemorySnapshotOnShutdown = true
head, err := NewHead(nil, nil, wlog, opts, nil)
head, err := NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
defer func() {
@ -3212,7 +3214,7 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.EnableMemorySnapshotOnShutdown = true
head, err := NewHead(nil, nil, wlog, opts, nil)
head, err := NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
@ -3235,6 +3237,175 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) {
require.Greater(t, offset, 0)
}
// TestOOOWalReplay checks the replay at a low level.
// TODO(codesome): Needs test for ooo WAL repair.
func TestOOOWalReplay(t *testing.T) {
dir := t.TempDir()
wlog, err := wal.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wal.NewSize(nil, nil, filepath.Join(dir, wal.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = dir
opts.OutOfOrderAllowance.Store(30 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wlog, oooWlog, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
var expOOOSamples []sample
l := labels.FromStrings("foo", "bar")
appendSample := func(mins int64, isOOO bool) {
app := h.Appender(context.Background())
ts, v := mins*time.Minute.Milliseconds(), float64(mins)
_, err := app.Append(0, l, ts, v)
require.NoError(t, err)
require.NoError(t, app.Commit())
if isOOO {
expOOOSamples = append(expOOOSamples, sample{t: ts, v: v})
}
}
// In-order sample.
appendSample(60, false)
// Out of order samples.
appendSample(40, true)
appendSample(35, true)
appendSample(50, true)
appendSample(55, true)
appendSample(59, true)
appendSample(31, true)
// Check that Head's time ranges are set properly.
require.Equal(t, 60*time.Minute.Milliseconds(), h.MinTime())
require.Equal(t, 60*time.Minute.Milliseconds(), h.MaxTime())
require.Equal(t, 31*time.Minute.Milliseconds(), h.MinOOOTime())
require.Equal(t, 59*time.Minute.Milliseconds(), h.MaxOOOTime())
// Restart head.
require.NoError(t, h.Close())
wlog, err = wal.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err = wal.NewSize(nil, nil, filepath.Join(dir, wal.WblDirName), 32768, true)
require.NoError(t, err)
h, err = NewHead(nil, nil, wlog, oooWlog, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0)) // Replay happens here.
// Get the ooo samples from the Head.
ms, ok, err := h.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.False(t, ok)
require.NotNil(t, ms)
xor, err := ms.oooHeadChunk.chunk.ToXor()
require.NoError(t, err)
it := xor.Iterator(nil)
actOOOSamples := make([]sample, 0, len(expOOOSamples))
for it.Next() {
ts, v := it.At()
actOOOSamples = append(actOOOSamples, sample{t: ts, v: v})
}
// OOO chunk will be sorted. Hence sort the expected samples.
sort.Slice(expOOOSamples, func(i, j int) bool {
return expOOOSamples[i].t < expOOOSamples[j].t
})
require.Equal(t, expOOOSamples, actOOOSamples)
require.NoError(t, h.Close())
}
// TestOOOMmapReplay checks the replay at a low level.
func TestOOOMmapReplay(t *testing.T) {
dir := t.TempDir()
wlog, err := wal.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wal.NewSize(nil, nil, filepath.Join(dir, wal.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = dir
opts.OutOfOrderCapMax.Store(30)
opts.OutOfOrderAllowance.Store(1000 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wlog, oooWlog, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
l := labels.FromStrings("foo", "bar")
appendSample := func(mins int64) {
app := h.Appender(context.Background())
ts, v := mins*time.Minute.Milliseconds(), float64(mins)
_, err := app.Append(0, l, ts, v)
require.NoError(t, err)
require.NoError(t, app.Commit())
}
// In-order sample.
appendSample(200)
// Out of order samples. 92 samples to create 3 m-map chunks.
for mins := int64(100); mins <= 191; mins++ {
appendSample(mins)
}
ms, ok, err := h.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.False(t, ok)
require.NotNil(t, ms)
require.Len(t, ms.oooMmappedChunks, 3)
// Verify that we can access the chunks without error.
for _, m := range ms.oooMmappedChunks {
chk, err := h.chunkDiskMapper.Chunk(m.ref)
require.NoError(t, err)
require.Equal(t, int(m.numSamples), chk.NumSamples())
}
expMmapChunks := make([]*mmappedChunk, 3)
copy(expMmapChunks, ms.oooMmappedChunks)
// Restart head.
require.NoError(t, h.Close())
wlog, err = wal.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err = wal.NewSize(nil, nil, filepath.Join(dir, wal.WblDirName), 32768, true)
require.NoError(t, err)
h, err = NewHead(nil, nil, wlog, oooWlog, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0)) // Replay happens here.
// Get the mmap chunks from the Head.
ms, ok, err = h.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.False(t, ok)
require.NotNil(t, ms)
require.Len(t, ms.oooMmappedChunks, len(expMmapChunks))
// Verify that we can access the chunks without error.
for _, m := range ms.oooMmappedChunks {
chk, err := h.chunkDiskMapper.Chunk(m.ref)
require.NoError(t, err)
require.Equal(t, int(m.numSamples), chk.NumSamples())
}
actMmapChunks := make([]*mmappedChunk, len(expMmapChunks))
copy(actMmapChunks, ms.oooMmappedChunks)
require.Equal(t, expMmapChunks, actMmapChunks)
require.NoError(t, h.Close())
}
func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) {
h, _ := newTestHead(t, 1000, false)
defer func() {
@ -3281,7 +3452,7 @@ func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) {
wlog, err := wal.NewSize(nil, nil, filepath.Join(h.opts.ChunkDirRoot, "wal"), 32768, false)
require.NoError(t, err)
h, err = NewHead(nil, nil, wlog, h.opts, nil)
h, err = NewHead(nil, nil, wlog, nil, h.opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
@ -3323,7 +3494,7 @@ func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) {
opts.EnableExemplarStorage = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
h, err := NewHead(nil, nil, wlog, opts, nil)
h, err := NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
@ -3357,7 +3528,7 @@ func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) {
require.NoError(t, err)
require.NoError(t, f.Close())
h, err = NewHead(nil, nil, wlog, opts, nil)
h, err = NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
@ -3382,7 +3553,7 @@ func TestReplayAfterMmapReplayError(t *testing.T) {
opts.EnableMemorySnapshotOnShutdown = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
h, err = NewHead(nil, nil, wlog, opts, nil)
h, err = NewHead(nil, nil, wlog, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
}
@ -3444,3 +3615,84 @@ func TestReplayAfterMmapReplayError(t *testing.T) {
require.NoError(t, h.Close())
}
func TestOOOAppendWithNoSeries(t *testing.T) {
dir := t.TempDir()
wlog, err := wal.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wal.NewSize(nil, nil, filepath.Join(dir, wal.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.OutOfOrderCapMax.Store(30)
opts.OutOfOrderAllowance.Store(120 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wlog, oooWlog, opts, nil)
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, h.Close())
})
require.NoError(t, h.Init(0))
appendSample := func(lbls labels.Labels, ts int64) {
app := h.Appender(context.Background())
_, err := app.Append(0, lbls, ts*time.Minute.Milliseconds(), float64(ts))
require.NoError(t, err)
require.NoError(t, app.Commit())
}
verifyOOOSamples := func(lbls labels.Labels, expSamples int) {
ms, created, err := h.getOrCreate(lbls.Hash(), lbls)
require.NoError(t, err)
require.False(t, created)
require.NotNil(t, ms)
require.Nil(t, ms.headChunk)
require.NotNil(t, ms.oooHeadChunk)
require.Equal(t, expSamples, ms.oooHeadChunk.chunk.NumSamples())
}
verifyInOrderSamples := func(lbls labels.Labels, expSamples int) {
ms, created, err := h.getOrCreate(lbls.Hash(), lbls)
require.NoError(t, err)
require.False(t, created)
require.NotNil(t, ms)
require.Nil(t, ms.oooHeadChunk)
require.NotNil(t, ms.headChunk)
require.Equal(t, expSamples, ms.headChunk.chunk.NumSamples())
}
newLabels := func(idx int) labels.Labels { return labels.FromStrings("foo", fmt.Sprintf("%d", idx)) }
s1 := newLabels(1)
appendSample(s1, 300) // At 300m.
verifyInOrderSamples(s1, 1)
// At 239m, the sample cannot be appended to in-order chunk since it is
// beyond the minValidTime. So it should go in OOO chunk.
// Series does not exist for s2 yet.
s2 := newLabels(2)
appendSample(s2, 239) // OOO sample.
verifyOOOSamples(s2, 1)
// Similar for 180m.
s3 := newLabels(3)
appendSample(s3, 180) // OOO sample.
verifyOOOSamples(s3, 1)
// Now 179m is too old.
s4 := newLabels(4)
app := h.Appender(context.Background())
_, err = app.Append(0, s4, 179*time.Minute.Milliseconds(), float64(179))
require.Equal(t, storage.ErrTooOldSample, err)
require.NoError(t, app.Rollback())
verifyOOOSamples(s3, 1)
// Samples still go into in-order chunk for samples within
// appendable minValidTime.
s5 := newLabels(5)
appendSample(s5, 240)
verifyInOrderSamples(s5, 1)
}

290
tsdb/head_wal.go

@ -41,7 +41,7 @@ import (
"github.com/prometheus/prometheus/tsdb/wal"
)
func (h *Head) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, mmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (err error) {
func (h *Head) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, mmappedChunks, oooMmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (err error) {
// Track number of samples that referenced a series we don't know about
// for error reporting.
var unknownRefs atomic.Uint64
@ -215,10 +215,11 @@ Outer:
processors[idx].mx.Lock()
mmc := mmappedChunks[walSeries.Ref]
oooMmc := oooMmappedChunks[walSeries.Ref]
if created {
// This is the first WAL series record for this series.
h.resetSeriesWithMMappedChunks(mSeries, mmc)
h.resetSeriesWithMMappedChunks(mSeries, mmc, oooMmc)
processors[idx].mx.Unlock()
continue
}
@ -252,7 +253,7 @@ Outer:
}
// Replacing m-mapped chunks with the new ones (could be empty).
h.resetSeriesWithMMappedChunks(mSeries, mmc)
h.resetSeriesWithMMappedChunks(mSeries, mmc, oooMmc)
processors[idx].mx.Unlock()
}
@ -343,11 +344,12 @@ Outer:
}
// resetSeriesWithMMappedChunks is only used during the WAL replay.
func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc []*mmappedChunk) {
h.metrics.chunksCreated.Add(float64(len(mmc)))
h.metrics.chunksRemoved.Add(float64(len(mSeries.mmappedChunks)))
h.metrics.chunks.Add(float64(len(mmc) - len(mSeries.mmappedChunks)))
func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc, oooMmc []*mmappedChunk) {
h.metrics.chunksCreated.Add(float64(len(mmc) + len(oooMmc)))
h.metrics.chunksRemoved.Add(float64(len(mSeries.mmappedChunks) + len(mSeries.oooMmappedChunks)))
h.metrics.chunks.Add(float64(len(mmc) + len(oooMmc) - len(mSeries.mmappedChunks) - len(mSeries.oooMmappedChunks)))
mSeries.mmappedChunks = mmc
mSeries.oooMmappedChunks = oooMmc
// Cache the last mmapped chunk time, so we can skip calling append() for samples it will reject.
if len(mmc) == 0 {
mSeries.mmMaxTime = math.MinInt64
@ -357,6 +359,8 @@ func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc []*mmappedCh
}
// Any samples replayed till now would already be compacted. Resetting the head chunk.
// We do not reset oooHeadChunk because that is being replayed from a different WAL
// and has not been replayed here.
mSeries.nextAt = 0
mSeries.headChunk = nil
mSeries.app = nil
@ -446,6 +450,278 @@ func (wp *walSubsetProcessor) waitUntilIdle() {
}
}
func (h *Head) loadWbl(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, lastMmapRef chunks.ChunkDiskMapperRef) (err error) {
// Track number of samples that referenced a series we don't know about
// for error reporting.
var unknownRefs atomic.Uint64
lastSeq, lastOff := lastMmapRef.Unpack()
// Start workers that each process samples for a partition of the series ID space.
var (
wg sync.WaitGroup
n = runtime.GOMAXPROCS(0)
processors = make([]wblSubsetProcessor, n)
dec record.Decoder
shards = make([][]record.RefSample, n)
decoded = make(chan interface{}, 10)
decodeErr error
samplesPool = sync.Pool{
New: func() interface{} {
return []record.RefSample{}
},
}
markersPool = sync.Pool{
New: func() interface{} {
return []record.RefMmapMarker{}
},
}
)
defer func() {
// For CorruptionErr ensure to terminate all workers before exiting.
// We also wrap it to identify OOO WBL corruption.
_, ok := err.(*wal.CorruptionErr)
if ok {
err = &errLoadWbl{err: err}
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
wg.Wait()
}
}()
wg.Add(n)
for i := 0; i < n; i++ {
processors[i].setup()
go func(wp *wblSubsetProcessor) {
unknown := wp.processWALSamples(h)
unknownRefs.Add(unknown)
wg.Done()
}(&processors[i])
}
go func() {
defer close(decoded)
for r.Next() {
rec := r.Record()
switch dec.Type(rec) {
case record.Samples:
samples := samplesPool.Get().([]record.RefSample)[:0]
samples, err = dec.Samples(rec, samples)
if err != nil {
decodeErr = &wal.CorruptionErr{
Err: errors.Wrap(err, "decode samples"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- samples
case record.MmapMarkers:
markers := markersPool.Get().([]record.RefMmapMarker)[:0]
markers, err = dec.MmapMarkers(rec, markers)
if err != nil {
decodeErr = &wal.CorruptionErr{
Err: errors.Wrap(err, "decode mmap markers"),
Segment: r.Segment(),
Offset: r.Offset(),
}
return
}
decoded <- markers
default:
// Noop.
}
}
}()
// The records are always replayed from the oldest to the newest.
for d := range decoded {
switch v := d.(type) {
case []record.RefSample:
samples := v
// We split up the samples into parts of 5000 samples or less.
// With O(300 * #cores) in-flight sample batches, large scrapes could otherwise
// cause thousands of very large in flight buffers occupying large amounts
// of unused memory.
for len(samples) > 0 {
m := 5000
if len(samples) < m {
m = len(samples)
}
for i := 0; i < n; i++ {
shards[i] = processors[i].reuseBuf()
}
for _, sam := range samples[:m] {
if r, ok := multiRef[sam.Ref]; ok {
sam.Ref = r
}
mod := uint64(sam.Ref) % uint64(n)
shards[mod] = append(shards[mod], sam)
}
for i := 0; i < n; i++ {
processors[i].input <- shards[i]
}
samples = samples[m:]
}
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
samplesPool.Put(d)
case []record.RefMmapMarker:
markers := v
for _, rm := range markers {
seq, off := rm.MmapRef.Unpack()
if seq > lastSeq || (seq == lastSeq && off > lastOff) {
// This m-map chunk from markers was not present during
// the load of mmapped chunks that happened in the head
// initialization.
continue
}
ms := h.series.getByID(rm.Ref)
if ms == nil {
unknownRefs.Inc()
continue
}
idx := uint64(ms.ref) % uint64(n)
// It is possible that some old sample is being processed in processWALSamples that
// could cause race below. So we wait for the goroutine to empty input the buffer and finish
// processing all old samples after emptying the buffer.
processors[idx].waitUntilIdle()
// Lock the subset so we can modify the series object
processors[idx].mx.Lock()
// All samples till now have been m-mapped. Hence clear out the headChunk.
// In case some samples slipped through and went into m-map chunks because of changed
// chunk size parameters, we are not taking care of that here.
// TODO(codesome): see if there is a way to avoid duplicate m-map chunks if
// the size of ooo chunk was reduced between restart.
ms.oooHeadChunk = nil
processors[idx].mx.Unlock()
}
default:
panic(fmt.Errorf("unexpected decoded type: %T", d))
}
}
if decodeErr != nil {
return decodeErr
}
// Signal termination to each worker and wait for it to close its output channel.
for i := 0; i < n; i++ {
processors[i].closeAndDrain()
}
wg.Wait()
if r.Err() != nil {
return errors.Wrap(r.Err(), "read records")
}
if unknownRefs.Load() > 0 {
level.Warn(h.logger).Log("msg", "Unknown series references for ooo WAL replay", "samples", unknownRefs.Load())
}
return nil
}
type errLoadWbl struct {
err error
}
func (e errLoadWbl) Error() string {
return e.err.Error()
}
// To support errors.Cause().
func (e errLoadWbl) Cause() error {
return e.err
}
// To support errors.Unwrap().
func (e errLoadWbl) Unwrap() error {
return e.err
}
// isErrLoadOOOWal returns a boolean if the error is errLoadWbl.
func isErrLoadOOOWal(err error) bool {
_, ok := err.(*errLoadWbl)
return ok
}
type wblSubsetProcessor struct {
mx sync.Mutex // Take this lock while modifying series in the subset.
input chan []record.RefSample
output chan []record.RefSample
}
func (wp *wblSubsetProcessor) setup() {
wp.output = make(chan []record.RefSample, 300)
wp.input = make(chan []record.RefSample, 300)
}
func (wp *wblSubsetProcessor) closeAndDrain() {
close(wp.input)
for range wp.output {
}
}
// If there is a buffer in the output chan, return it for reuse, otherwise return nil.
func (wp *wblSubsetProcessor) reuseBuf() []record.RefSample {
select {
case buf := <-wp.output:
return buf[:0]
default:
}
return nil
}
// processWALSamples adds the samples it receives to the head and passes
// the buffer received to an output channel for reuse.
// Samples before the minValidTime timestamp are discarded.
func (wp *wblSubsetProcessor) processWALSamples(h *Head) (unknownRefs uint64) {
defer close(wp.output)
// We don't check for minValidTime for ooo samples.
for samples := range wp.input {
wp.mx.Lock()
for _, s := range samples {
ms := h.series.getByID(s.Ref)
if ms == nil {
unknownRefs++
continue
}
if _, chunkCreated, _ := ms.insert(s.T, s.V, h.chunkDiskMapper); chunkCreated {
h.metrics.chunksCreated.Inc()
h.metrics.chunks.Inc()
}
}
wp.mx.Unlock()
wp.output <- samples
}
return unknownRefs
}
func (wp *wblSubsetProcessor) waitUntilIdle() {
select {
case <-wp.output: // Allow output side to drain to avoid deadlock.
default:
}
wp.input <- []record.RefSample{}
for len(wp.input) != 0 {
time.Sleep(10 * time.Microsecond)
select {
case <-wp.output: // Allow output side to drain to avoid deadlock.
default:
}
}
}
const (
chunkSnapshotRecordTypeSeries uint8 = 1
chunkSnapshotRecordTypeTombstones uint8 = 2

74
tsdb/ooo_head.go

@ -0,0 +1,74 @@
package tsdb
import (
"fmt"
"github.com/prometheus/prometheus/tsdb/tombstones"
)
var _ BlockReader = &OOORangeHead{}
// OOORangeHead allows querying Head out of order samples via BlockReader
// interface implementation.
type OOORangeHead struct {
head *Head
// mint and maxt are tracked because when a query is handled we only want
// the timerange of the query and having preexisting pointers to the first
// and last timestamp help with that.
mint, maxt int64
}
func NewOOORangeHead(head *Head, mint, maxt int64) *OOORangeHead {
return &OOORangeHead{
head: head,
mint: mint,
maxt: maxt,
}
}
func (oh *OOORangeHead) Index() (IndexReader, error) {
return NewOOOHeadIndexReader(oh.head, oh.mint, oh.maxt), nil
}
func (oh *OOORangeHead) Chunks() (ChunkReader, error) {
return NewOOOHeadChunkReader(oh.head, oh.mint, oh.maxt), nil
}
func (oh *OOORangeHead) Tombstones() (tombstones.Reader, error) {
// As stated in the design doc https://docs.google.com/document/d/1Kppm7qL9C-BJB1j6yb6-9ObG3AbdZnFUBYPNNWwDBYM/edit?usp=sharing
// Tombstones are not supported for out of order metrics.
return tombstones.NewMemTombstones(), nil
}
func (oh *OOORangeHead) Meta() BlockMeta {
var id [16]byte
copy(id[:], "____ooo_head____")
return BlockMeta{
MinTime: oh.mint,
MaxTime: oh.maxt,
ULID: id,
Stats: BlockStats{
NumSeries: oh.head.NumSeries(),
},
}
}
// Size returns the size taken by the Head block.
func (oh *OOORangeHead) Size() int64 {
return oh.head.Size()
}
// String returns an human readable representation of the out of order range
// head. It's important to keep this function in order to avoid the struct dump
// when the head is stringified in errors or logs.
func (oh *OOORangeHead) String() string {
return fmt.Sprintf("ooo range head (mint: %d, maxt: %d)", oh.MinTime(), oh.MaxTime())
}
func (oh *OOORangeHead) MinTime() int64 {
return oh.mint
}
func (oh *OOORangeHead) MaxTime() int64 {
return oh.maxt
}

409
tsdb/ooo_head_read.go

@ -0,0 +1,409 @@
package tsdb
import (
"errors"
"math"
"sort"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/tombstones"
)
var _ IndexReader = &OOOHeadIndexReader{}
// OOOHeadIndexReader implements IndexReader so ooo samples in the head can be
// accessed.
// It also has a reference to headIndexReader so we can leverage on its
// IndexReader implementation for all the methods that remain the same. We
// decided to do this to avoid code duplication.
// The only methods that change are the ones about getting Series and Postings.
type OOOHeadIndexReader struct {
*headIndexReader // A reference to the headIndexReader so we can reuse as many interface implementation as possible.
}
func NewOOOHeadIndexReader(head *Head, mint, maxt int64) *OOOHeadIndexReader {
hr := &headIndexReader{
head: head,
mint: mint,
maxt: maxt,
}
return &OOOHeadIndexReader{hr}
}
func (oh *OOOHeadIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta) error {
return oh.series(ref, lbls, chks, 0)
}
// The passed lastMmapRef tells upto what max m-map chunk that we can consider.
// If it is 0, it means all chunks need to be considered.
// If it is non-0, then the oooHeadChunk must not be considered.
func (oh *OOOHeadIndexReader) series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta, lastMmapRef chunks.ChunkDiskMapperRef) error {
s := oh.head.series.getByID(chunks.HeadSeriesRef(ref))
if s == nil {
oh.head.metrics.seriesNotFound.Inc()
return storage.ErrNotFound
}
*lbls = append((*lbls)[:0], s.lset...)
if chks == nil {
return nil
}
s.Lock()
defer s.Unlock()
*chks = (*chks)[:0]
tmpChks := make([]chunks.Meta, 0, len(s.oooMmappedChunks))
// We define these markers to track the last chunk reference while we
// fill the chunk meta.
// These markers are useful to give consistent responses to repeated queries
// even if new chunks that might be overlapping or not are added afterwards.
// Also, lastMinT and lastMaxT are initialized to the max int as a sentinel
// value to know they are unset.
var lastChunkRef chunks.ChunkRef
lastMinT, lastMaxT := int64(math.MaxInt64), int64(math.MaxInt64)
addChunk := func(minT, maxT int64, ref chunks.ChunkRef) {
// the first time we get called is for the last included chunk.
// set the markers accordingly
if lastMinT == int64(math.MaxInt64) {
lastChunkRef = ref
lastMinT = minT
lastMaxT = maxT
}
tmpChks = append(tmpChks, chunks.Meta{
MinTime: minT,
MaxTime: maxT,
Ref: ref,
OOOLastRef: lastChunkRef,
OOOLastMinTime: lastMinT,
OOOLastMaxTime: lastMaxT,
})
}
// Collect all chunks that overlap the query range, in order from most recent to most old,
// so we can set the correct markers.
if s.oooHeadChunk != nil {
c := s.oooHeadChunk
if c.OverlapsClosedInterval(oh.mint, oh.maxt) && lastMmapRef == 0 {
ref := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(len(s.oooMmappedChunks))))
addChunk(c.minTime, c.maxTime, ref)
}
}
for i := len(s.oooMmappedChunks) - 1; i >= 0; i-- {
c := s.oooMmappedChunks[i]
if c.OverlapsClosedInterval(oh.mint, oh.maxt) && (lastMmapRef == 0 || lastMmapRef.GreaterThanOrEqualTo(c.ref)) {
ref := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(i)))
addChunk(c.minTime, c.maxTime, ref)
}
}
// There is nothing to do if we did not collect any chunk
if len(tmpChks) == 0 {
return nil
}
// Next we want to sort all the collected chunks by min time so we can find
// those that overlap.
sort.Sort(metaByMinTimeAndMinRef(tmpChks))
// Next we want to iterate the sorted collected chunks and only return the
// chunks Meta the first chunk that overlaps with others.
// Example chunks of a series: 5:(100, 200) 6:(500, 600) 7:(150, 250) 8:(550, 650)
// In the example 5 overlaps with 7 and 6 overlaps with 8 so we only want to
// to return chunk Metas for chunk 5 and chunk 6
*chks = append(*chks, tmpChks[0])
maxTime := tmpChks[0].MaxTime // tracks the maxTime of the previous "to be merged chunk"
for _, c := range tmpChks[1:] {
if c.MinTime > maxTime {
*chks = append(*chks, c)
maxTime = c.MaxTime
} else if c.MaxTime > maxTime {
maxTime = c.MaxTime
(*chks)[len(*chks)-1].MaxTime = c.MaxTime
}
}
return nil
}
type chunkMetaAndChunkDiskMapperRef struct {
meta chunks.Meta
ref chunks.ChunkDiskMapperRef
origMinT int64
origMaxT int64
}
type byMinTimeAndMinRef []chunkMetaAndChunkDiskMapperRef
func (b byMinTimeAndMinRef) Len() int { return len(b) }
func (b byMinTimeAndMinRef) Less(i, j int) bool {
if b[i].meta.MinTime == b[j].meta.MinTime {
return b[i].meta.Ref < b[j].meta.Ref
}
return b[i].meta.MinTime < b[j].meta.MinTime
}
func (b byMinTimeAndMinRef) Swap(i, j int) { b[i], b[j] = b[j], b[i] }
type metaByMinTimeAndMinRef []chunks.Meta
func (b metaByMinTimeAndMinRef) Len() int { return len(b) }
func (b metaByMinTimeAndMinRef) Less(i, j int) bool {
if b[i].MinTime == b[j].MinTime {
return b[i].Ref < b[j].Ref
}
return b[i].MinTime < b[j].MinTime
}
func (b metaByMinTimeAndMinRef) Swap(i, j int) { b[i], b[j] = b[j], b[i] }
func (oh *OOOHeadIndexReader) Postings(name string, values ...string) (index.Postings, error) {
switch len(values) {
case 0:
return index.EmptyPostings(), nil
case 1:
return oh.head.postings.Get(name, values[0]), nil // TODO(ganesh) Also call GetOOOPostings
default:
// TODO(ganesh) We want to only return postings for out of order series.
res := make([]index.Postings, 0, len(values))
for _, value := range values {
res = append(res, oh.head.postings.Get(name, value)) // TODO(ganesh) Also call GetOOOPostings
}
return index.Merge(res...), nil
}
}
type OOOHeadChunkReader struct {
head *Head
mint, maxt int64
}
func NewOOOHeadChunkReader(head *Head, mint, maxt int64) *OOOHeadChunkReader {
return &OOOHeadChunkReader{
head: head,
mint: mint,
maxt: maxt,
}
}
func (cr OOOHeadChunkReader) Chunk(meta chunks.Meta) (chunkenc.Chunk, error) {
sid, _ := chunks.HeadChunkRef(meta.Ref).Unpack()
s := cr.head.series.getByID(sid)
// This means that the series has been garbage collected.
if s == nil {
return nil, storage.ErrNotFound
}
s.Lock()
c, err := s.oooMergedChunk(meta, cr.head.chunkDiskMapper, cr.mint, cr.maxt)
s.Unlock()
if err != nil {
return nil, err
}
// This means that the query range did not overlap with the requested chunk.
if len(c.chunks) == 0 {
return nil, storage.ErrNotFound
}
return c, nil
}
func (cr OOOHeadChunkReader) Close() error {
return nil
}
type OOOCompactionHead struct {
oooIR *OOOHeadIndexReader
lastMmapRef chunks.ChunkDiskMapperRef
lastWBLFile int
postings []storage.SeriesRef
chunkRange int64
mint, maxt int64 // Among all the compactable chunks.
}
// NewOOOCompactionHead does the following:
// 1. M-maps all the in-memory ooo chunks.
// 2. Compute the expected block ranges while iterating through all ooo series and store it.
// 3. Store the list of postings having ooo series.
// 4. Cuts a new WBL file for the OOO WBL.
// All the above together have a bit of CPU and memory overhead, and can have a bit of impact
// on the sample append latency. So call NewOOOCompactionHead only right before compaction.
func NewOOOCompactionHead(head *Head) (*OOOCompactionHead, error) {
newWBLFile, err := head.wbl.NextSegment()
if err != nil {
return nil, err
}
ch := &OOOCompactionHead{
chunkRange: head.chunkRange.Load(),
mint: math.MaxInt64,
maxt: math.MinInt64,
lastWBLFile: newWBLFile,
}
ch.oooIR = NewOOOHeadIndexReader(head, math.MinInt64, math.MaxInt64)
n, v := index.AllPostingsKey()
// TODO: verify this gets only ooo samples.
p, err := ch.oooIR.Postings(n, v)
if err != nil {
return nil, err
}
p = ch.oooIR.SortedPostings(p)
var lastSeq, lastOff int
for p.Next() {
seriesRef := p.At()
ms := head.series.getByID(chunks.HeadSeriesRef(seriesRef))
if ms == nil {
continue
}
// M-map the in-memory chunk and keep track of the last one.
// Also build the block ranges -> series map.
// TODO: consider having a lock specifically for ooo data.
ms.Lock()
mmapRef := ms.mmapCurrentOOOHeadChunk(head.chunkDiskMapper)
if mmapRef == 0 && len(ms.oooMmappedChunks) > 0 {
// Nothing was m-mapped. So take the mmapRef from the existing slice if it exists.
mmapRef = ms.oooMmappedChunks[len(ms.oooMmappedChunks)-1].ref
}
seq, off := mmapRef.Unpack()
if seq > lastSeq || (seq == lastSeq && off > lastOff) {
ch.lastMmapRef, lastSeq, lastOff = mmapRef, seq, off
}
if len(ms.oooMmappedChunks) > 0 {
ch.postings = append(ch.postings, seriesRef)
for _, c := range ms.oooMmappedChunks {
if c.minTime < ch.mint {
ch.mint = c.minTime
}
if c.maxTime > ch.maxt {
ch.maxt = c.maxTime
}
}
}
ms.Unlock()
}
return ch, nil
}
func (ch *OOOCompactionHead) Index() (IndexReader, error) {
return NewOOOCompactionHeadIndexReader(ch), nil
}
func (ch *OOOCompactionHead) Chunks() (ChunkReader, error) {
return NewOOOHeadChunkReader(ch.oooIR.head, ch.oooIR.mint, ch.oooIR.maxt), nil
}
func (ch *OOOCompactionHead) Tombstones() (tombstones.Reader, error) {
return tombstones.NewMemTombstones(), nil
}
func (ch *OOOCompactionHead) Meta() BlockMeta {
var id [16]byte
copy(id[:], "copy(id[:], \"ooo_compact_head\")")
return BlockMeta{
MinTime: ch.mint,
MaxTime: ch.maxt,
ULID: id,
Stats: BlockStats{
NumSeries: uint64(len(ch.postings)),
},
}
}
// CloneForTimeRange clones the OOOCompactionHead such that the IndexReader and ChunkReader
// obtained from this only looks at the m-map chunks within the given time ranges while not looking
// beyond the ch.lastMmapRef.
// Only the method of BlockReader interface are valid for the cloned OOOCompactionHead.
func (ch *OOOCompactionHead) CloneForTimeRange(mint, maxt int64) *OOOCompactionHead {
return &OOOCompactionHead{
oooIR: NewOOOHeadIndexReader(ch.oooIR.head, mint, maxt),
lastMmapRef: ch.lastMmapRef,
postings: ch.postings,
chunkRange: ch.chunkRange,
mint: ch.mint,
maxt: ch.maxt,
}
}
func (ch *OOOCompactionHead) Size() int64 { return 0 }
func (ch *OOOCompactionHead) MinTime() int64 { return ch.mint }
func (ch *OOOCompactionHead) MaxTime() int64 { return ch.maxt }
func (ch *OOOCompactionHead) ChunkRange() int64 { return ch.chunkRange }
func (ch *OOOCompactionHead) LastMmapRef() chunks.ChunkDiskMapperRef { return ch.lastMmapRef }
func (ch *OOOCompactionHead) LastWBLFile() int { return ch.lastWBLFile }
type OOOCompactionHeadIndexReader struct {
ch *OOOCompactionHead
}
func NewOOOCompactionHeadIndexReader(ch *OOOCompactionHead) IndexReader {
return &OOOCompactionHeadIndexReader{ch: ch}
}
func (ir *OOOCompactionHeadIndexReader) Symbols() index.StringIter {
return ir.ch.oooIR.Symbols()
}
func (ir *OOOCompactionHeadIndexReader) Postings(name string, values ...string) (index.Postings, error) {
n, v := index.AllPostingsKey()
if name != n || len(values) != 1 || values[0] != v {
return nil, errors.New("only AllPostingsKey is supported")
}
return index.NewListPostings(ir.ch.postings), nil
}
func (ir *OOOCompactionHeadIndexReader) SortedPostings(p index.Postings) index.Postings {
// This will already be sorted from the Postings() call above.
return p
}
func (ir *OOOCompactionHeadIndexReader) ShardedPostings(p index.Postings, shardIndex, shardCount uint64) index.Postings {
return ir.ch.oooIR.ShardedPostings(p, shardIndex, shardCount)
}
func (ir *OOOCompactionHeadIndexReader) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error {
return ir.ch.oooIR.series(ref, lset, chks, ir.ch.lastMmapRef)
}
func (ir *OOOCompactionHeadIndexReader) SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) {
return nil, errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) {
return nil, errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) PostingsForMatchers(concurrent bool, ms ...*labels.Matcher) (index.Postings, error) {
return nil, errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, error) {
return nil, errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) {
return "", errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) {
return nil, errors.New("not implemented")
}
func (ir *OOOCompactionHeadIndexReader) Close() error {
return ir.ch.oooIR.Close()
}

1098
tsdb/ooo_head_read_test.go

File diff suppressed because it is too large Load Diff

4
tsdb/querier.go

@ -518,7 +518,7 @@ func (p *populateWithDelGenericSeriesIterator) next() bool {
p.i++
p.currChkMeta = p.chks[p.i]
p.currChkMeta.Chunk, p.err = p.chunks.Chunk(p.currChkMeta.Ref)
p.currChkMeta.Chunk, p.err = p.chunks.Chunk(p.currChkMeta)
if p.err != nil {
p.err = errors.Wrapf(p.err, "cannot populate chunk %d", p.currChkMeta.Ref)
return false
@ -847,7 +847,7 @@ func newNopChunkReader() ChunkReader {
}
}
func (cr nopChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) {
func (cr nopChunkReader) Chunk(meta chunks.Meta) (chunkenc.Chunk, error) {
return cr.emptyChunk, nil
}

4
tsdb/querier_bench_test.go

@ -36,7 +36,7 @@ func BenchmarkQuerier(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
defer func() {
require.NoError(b, h.Close())
@ -187,7 +187,7 @@ func BenchmarkQuerierSelect(b *testing.B) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(b, err)
defer h.Close()
app := h.Appender(context.Background())

14
tsdb/querier_test.go

@ -458,7 +458,7 @@ func TestBlockQuerier_AgainstHeadWithOpenChunks(t *testing.T) {
t.Run("", func(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 2 * time.Hour.Milliseconds()
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(t, err)
defer h.Close()
@ -627,10 +627,10 @@ func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksR
return f, chks
}
func (r *fakeChunksReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) {
chk, ok := r.chks[ref]
func (r *fakeChunksReader) Chunk(meta chunks.Meta) (chunkenc.Chunk, error) {
chk, ok := r.chks[meta.Ref]
if !ok {
return nil, errors.Errorf("chunk not found at ref %v", ref)
return nil, errors.Errorf("chunk not found at ref %v", meta.Ref)
}
return chk, nil
}
@ -1016,8 +1016,8 @@ func BenchmarkMergedSeriesSet(b *testing.B) {
type mockChunkReader map[chunks.ChunkRef]chunkenc.Chunk
func (cr mockChunkReader) Chunk(id chunks.ChunkRef) (chunkenc.Chunk, error) {
chk, ok := cr[id]
func (cr mockChunkReader) Chunk(meta chunks.Meta) (chunkenc.Chunk, error) {
chk, ok := cr[meta.Ref]
if ok {
return chk, nil
}
@ -1622,7 +1622,7 @@ func TestPostingsForMatchers(t *testing.T) {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = chunkDir
h, err := NewHead(nil, nil, nil, opts, nil)
h, err := NewHead(nil, nil, nil, nil, opts, nil)
require.NoError(t, err)
defer func() {
require.NoError(t, h.Close())

50
tsdb/record/record.go

@ -42,6 +42,8 @@ const (
Tombstones Type = 3
// Exemplars is used to match WAL records of type Exemplars.
Exemplars Type = 4
// MmapMarkers is used to match OOO WBL records of type MmapMarkers.
MmapMarkers Type = 5
)
// ErrNotFound is returned if a looked up resource was not found. Duplicate ErrNotFound from head.go.
@ -68,6 +70,12 @@ type RefExemplar struct {
Labels labels.Labels
}
// RefMmapMarker marks that the all the samples of the given series until now have been m-mapped to disk.
type RefMmapMarker struct {
Ref chunks.HeadSeriesRef
MmapRef chunks.ChunkDiskMapperRef
}
// Decoder decodes series, sample, and tombstone records.
// The zero value is ready to use.
type Decoder struct{}
@ -79,7 +87,7 @@ func (d *Decoder) Type(rec []byte) Type {
return Unknown
}
switch t := Type(rec[0]); t {
case Series, Samples, Tombstones, Exemplars:
case Series, Samples, Tombstones, Exemplars, MmapMarkers:
return t
}
return Unknown
@ -223,6 +231,34 @@ func (d *Decoder) ExemplarsFromBuffer(dec *encoding.Decbuf, exemplars []RefExemp
return exemplars, nil
}
func (d *Decoder) MmapMarkers(rec []byte, markers []RefMmapMarker) ([]RefMmapMarker, error) {
dec := encoding.Decbuf{B: rec}
t := Type(dec.Byte())
if t != MmapMarkers {
return nil, errors.New("invalid record type")
}
if dec.Len() == 0 {
return markers, nil
}
for len(dec.B) > 0 && dec.Err() == nil {
ref := chunks.HeadSeriesRef(dec.Be64())
mmapRef := chunks.ChunkDiskMapperRef(dec.Be64())
markers = append(markers, RefMmapMarker{
Ref: ref,
MmapRef: mmapRef,
})
}
if dec.Err() != nil {
return nil, errors.Wrapf(dec.Err(), "decode error after %d mmap markers", len(markers))
}
if len(dec.B) > 0 {
return nil, errors.Errorf("unexpected %d bytes left in entry", len(dec.B))
}
return markers, nil
}
// Encoder encodes series, sample, and tombstones records.
// The zero value is ready to use.
type Encoder struct{}
@ -316,3 +352,15 @@ func (e *Encoder) EncodeExemplarsIntoBuffer(exemplars []RefExemplar, buf *encodi
}
}
}
func (e *Encoder) MmapMarkers(markers []RefMmapMarker, b []byte) []byte {
buf := encoding.Encbuf{B: b}
buf.PutByte(byte(MmapMarkers))
for _, s := range markers {
buf.PutBE64(uint64(s.Ref))
buf.PutBE64(uint64(s.MmapRef))
}
return buf.Get()
}

47
tsdb/wal/wal.go

@ -40,6 +40,7 @@ const (
DefaultSegmentSize = 128 * 1024 * 1024 // 128 MB
pageSize = 32 * 1024 // 32KB
recordHeaderSize = 7
WblDirName = "wbl"
)
// The table gets initialized with sync.Once but may still cause a race
@ -200,36 +201,41 @@ type walMetrics struct {
writesFailed prometheus.Counter
}
func newWALMetrics(r prometheus.Registerer) *walMetrics {
func newWALMetrics(r prometheus.Registerer, isOOO bool) *walMetrics {
m := &walMetrics{}
prefix := "prometheus_tsdb_wal"
if isOOO {
prefix = "prometheus_tsdb_out_of_order_wal"
}
m.fsyncDuration = prometheus.NewSummary(prometheus.SummaryOpts{
Name: "prometheus_tsdb_wal_fsync_duration_seconds",
Name: fmt.Sprintf("%s_fsync_duration_seconds", prefix),
Help: "Duration of WAL fsync.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
})
m.pageFlushes = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_wal_page_flushes_total",
Name: fmt.Sprintf("%s_page_flushes_total", prefix),
Help: "Total number of page flushes.",
})
m.pageCompletions = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_wal_completed_pages_total",
Name: fmt.Sprintf("%s_completed_pages_total", prefix),
Help: "Total number of completed pages.",
})
m.truncateFail = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_wal_truncations_failed_total",
Name: fmt.Sprintf("%s_truncations_failed_total", prefix),
Help: "Total number of WAL truncations that failed.",
})
m.truncateTotal = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_wal_truncations_total",
Name: fmt.Sprintf("%s_truncations_total", prefix),
Help: "Total number of WAL truncations attempted.",
})
m.currentSegment = prometheus.NewGauge(prometheus.GaugeOpts{
Name: "prometheus_tsdb_wal_segment_current",
Name: fmt.Sprintf("%s_segment_current", prefix),
Help: "WAL segment index that TSDB is currently writing to.",
})
m.writesFailed = prometheus.NewCounter(prometheus.CounterOpts{
Name: "prometheus_tsdb_wal_writes_failed_total",
Name: fmt.Sprintf("%s_writes_failed_total", prefix),
Help: "Total number of WAL writes that failed.",
})
@ -274,7 +280,12 @@ func NewSize(logger log.Logger, reg prometheus.Registerer, dir string, segmentSi
stopc: make(chan chan struct{}),
compress: compress,
}
w.metrics = newWALMetrics(reg)
isOOO := false
if filepath.Base(dir) == WblDirName {
// TODO(codesome): have a less hacky way to do it.
isOOO = true
}
w.metrics = newWALMetrics(reg, isOOO)
_, last, err := Segments(w.Dir())
if err != nil {
@ -460,31 +471,33 @@ func SegmentName(dir string, i int) string {
}
// NextSegment creates the next segment and closes the previous one.
func (w *WAL) NextSegment() error {
// It returns the file number of the new file.
func (w *WAL) NextSegment() (int, error) {
w.mtx.Lock()
defer w.mtx.Unlock()
return w.nextSegment()
}
// nextSegment creates the next segment and closes the previous one.
func (w *WAL) nextSegment() error {
// It returns the file number of the new file.
func (w *WAL) nextSegment() (int, error) {
if w.closed {
return errors.New("wal is closed")
return 0, errors.New("wal is closed")
}
// Only flush the current page if it actually holds data.
if w.page.alloc > 0 {
if err := w.flushPage(true); err != nil {
return err
return 0, err
}
}
next, err := CreateSegment(w.Dir(), w.segment.Index()+1)
if err != nil {
return errors.Wrap(err, "create new segment file")
return 0, errors.Wrap(err, "create new segment file")
}
prev := w.segment
if err := w.setSegment(next); err != nil {
return err
return 0, err
}
// Don't block further writes by fsyncing the last segment.
@ -496,7 +509,7 @@ func (w *WAL) nextSegment() error {
level.Error(w.logger).Log("msg", "close previous segment", "err", err)
}
}
return nil
return next.Index(), nil
}
func (w *WAL) setSegment(segment *Segment) error {
@ -638,7 +651,7 @@ func (w *WAL) log(rec []byte, final bool) error {
left += (pageSize - recordHeaderSize) * (w.pagesPerSegment() - w.donePages - 1) // Free pages in the active segment.
if len(rec) > left {
if err := w.nextSegment(); err != nil {
if _, err := w.nextSegment(); err != nil {
return err
}
}

2
web/api/v1/api_test.go

@ -2300,7 +2300,7 @@ func (f *fakeDB) Stats(statsByLabelName string) (_ *tsdb.Stats, retErr error) {
}()
opts := tsdb.DefaultHeadOptions()
opts.ChunkRange = 1000
h, _ := tsdb.NewHead(nil, nil, nil, opts, nil)
h, _ := tsdb.NewHead(nil, nil, nil, nil, opts, nil)
return h.Stats(statsByLabelName), nil
}

3
web/ui/assets_vfsdata.go

@ -10,7 +10,6 @@ import (
"compress/gzip"
"fmt"
"io"
"io/ioutil"
"net/http"
"os"
pathpkg "path"
@ -1082,7 +1081,7 @@ func (f *vfsgen۰CompressedFile) Read(p []byte) (n int, err error) {
}
if f.grPos < f.seekPos {
// Fast-forward.
_, err = io.CopyN(ioutil.Discard, f.gr, f.seekPos-f.grPos)
_, err = io.CopyN(io.Discard, f.gr, f.seekPos-f.grPos)
if err != nil {
return 0, err
}

Loading…
Cancel
Save