mirror of
https://github.com/prometheus/prometheus.git
synced 2024-11-09 23:24:05 -08:00
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>
This commit is contained in:
parent
f2aba4af80
commit
df59320886
|
@ -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),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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
Normal file
80
tsdb/chunkenc/ooo.go
Normal file
|
@ -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
Normal file
84
tsdb/chunkenc/ooo_test.go
Normal file
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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
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()
|
||||
|
||||
|
|
|
@ -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
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
1691
tsdb/db_test.go
File diff suppressed because it is too large
Load diff
436
tsdb/head.go
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)
|
||||
|
||||
walReplayDuration := time.Since(start)
|
||||
h.metrics.walTotalReplayDuration.Set(walReplayDuration.Seconds())
|
||||
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))
|
||||
}
|
||||
|
||||
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 {
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
||||
series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow)
|
||||
series.pendingCommit = false
|
||||
series.Unlock()
|
||||
}
|
||||
|
||||
a.head.metrics.samplesAppended.Add(float64(total))
|
||||
a.head.updateMinMaxTime(a.mint, a.maxt)
|
||||
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.
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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
Normal file
165
tsdb/head_read_test.go
Normal file
|
@ -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
|
||||
}
|
|
@ -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
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
Normal file
74
tsdb/ooo_head.go
Normal file
|
@ -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
Normal file
409
tsdb/ooo_head_read.go
Normal file
|
@ -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
Normal file
1098
tsdb/ooo_head_read_test.go
Normal file
File diff suppressed because it is too large
Load diff
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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…
Reference in a new issue