prometheus/tsdb/head_test.go
Björn Rabenstein 37fe9b89dc
Merge pull request #12055 from leizor/leizor/prometheus/issues/12009
Adjust samplesPerChunk from 120 to 220
2023-05-10 14:45:12 +02:00

4890 lines
154 KiB
Go

// Copyright 2017 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// nolint:revive // Many legitimately empty blocks in this file.
package tsdb
import (
"context"
"fmt"
"io"
"math"
"math/rand"
"os"
"path"
"path/filepath"
"sort"
"strconv"
"strings"
"sync"
"testing"
"time"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
prom_testutil "github.com/prometheus/client_golang/prometheus/testutil"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
"golang.org/x/sync/errgroup"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/exemplar"
"github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/value"
"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/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/tsdbutil"
"github.com/prometheus/prometheus/tsdb/wlog"
)
func newTestHead(t testing.TB, chunkRange int64, compressWAL, oooEnabled bool) (*Head, *wlog.WL) {
dir := t.TempDir()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compressWAL)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = chunkRange
opts.ChunkDirRoot = dir
opts.EnableExemplarStorage = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
opts.EnableNativeHistograms.Store(true)
if oooEnabled {
opts.OutOfOrderTimeWindow.Store(10 * time.Minute.Milliseconds())
}
h, err := NewHead(nil, nil, wal, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.chunkDiskMapper.IterateAllChunks(func(_ chunks.HeadSeriesRef, _ chunks.ChunkDiskMapperRef, _, _ int64, _ uint16, _ chunkenc.Encoding, _ bool) error {
return nil
}))
return h, wal
}
func BenchmarkCreateSeries(b *testing.B) {
series := genSeries(b.N, 10, 0, 0)
h, _ := newTestHead(b, 10000, false, false)
b.Cleanup(func() {
require.NoError(b, h.Close())
})
b.ReportAllocs()
b.ResetTimer()
for _, s := range series {
h.getOrCreate(s.Labels().Hash(), s.Labels())
}
}
func BenchmarkHeadAppender_Append_Commit_ExistingSeries(b *testing.B) {
seriesCounts := []int{100, 1000, 10000}
series := genSeries(10000, 10, 0, 0)
for _, seriesCount := range seriesCounts {
b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) {
for _, samplesPerAppend := range []int64{1, 2, 5, 100} {
b.Run(fmt.Sprintf("%d samples per append", samplesPerAppend), func(b *testing.B) {
h, _ := newTestHead(b, 10000, false, false)
b.Cleanup(func() { require.NoError(b, h.Close()) })
ts := int64(1000)
appendSamples := func() error {
var err error
app := h.Appender(context.Background())
for _, s := range series[:seriesCount] {
var ref storage.SeriesRef
for sampleIndex := int64(0); sampleIndex < samplesPerAppend; sampleIndex++ {
ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex))
if err != nil {
return err
}
}
}
ts += 1000 // should increment more than highest samplesPerAppend
return app.Commit()
}
// Init series, that's not what we're benchmarking here.
require.NoError(b, appendSamples())
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
require.NoError(b, appendSamples())
}
})
}
})
}
}
func populateTestWAL(t testing.TB, w *wlog.WL, recs []interface{}) {
var enc record.Encoder
for _, r := range recs {
switch v := r.(type) {
case []record.RefSeries:
require.NoError(t, w.Log(enc.Series(v, nil)))
case []record.RefSample:
require.NoError(t, w.Log(enc.Samples(v, nil)))
case []tombstones.Stone:
require.NoError(t, w.Log(enc.Tombstones(v, nil)))
case []record.RefExemplar:
require.NoError(t, w.Log(enc.Exemplars(v, nil)))
}
}
}
func readTestWAL(t testing.TB, dir string) (recs []interface{}) {
sr, err := wlog.NewSegmentsReader(dir)
require.NoError(t, err)
defer func() {
require.NoError(t, sr.Close())
}()
var dec record.Decoder
r := wlog.NewReader(sr)
for r.Next() {
rec := r.Record()
switch dec.Type(rec) {
case record.Series:
series, err := dec.Series(rec, nil)
require.NoError(t, err)
recs = append(recs, series)
case record.Samples:
samples, err := dec.Samples(rec, nil)
require.NoError(t, err)
recs = append(recs, samples)
case record.HistogramSamples:
samples, err := dec.HistogramSamples(rec, nil)
require.NoError(t, err)
recs = append(recs, samples)
case record.FloatHistogramSamples:
samples, err := dec.FloatHistogramSamples(rec, nil)
require.NoError(t, err)
recs = append(recs, samples)
case record.Tombstones:
tstones, err := dec.Tombstones(rec, nil)
require.NoError(t, err)
recs = append(recs, tstones)
case record.Metadata:
meta, err := dec.Metadata(rec, nil)
require.NoError(t, err)
recs = append(recs, meta)
default:
t.Fatalf("unknown record type")
}
}
require.NoError(t, r.Err())
return recs
}
func BenchmarkLoadWAL(b *testing.B) {
cases := []struct {
// Total series is (batches*seriesPerBatch).
batches int
seriesPerBatch int
samplesPerSeries int
mmappedChunkT int64
}{
{ // Less series and more samples. 2 hour WAL with 1 second scrape interval.
batches: 10,
seriesPerBatch: 100,
samplesPerSeries: 7200,
},
{ // More series and less samples.
batches: 10,
seriesPerBatch: 10000,
samplesPerSeries: 50,
},
{ // In between.
batches: 10,
seriesPerBatch: 1000,
samplesPerSeries: 480,
},
{ // 2 hour WAL with 15 second scrape interval, and mmapped chunks up to last 100 samples.
batches: 100,
seriesPerBatch: 1000,
samplesPerSeries: 480,
mmappedChunkT: 3800,
},
}
labelsPerSeries := 5
// Rough estimates of most common % of samples that have an exemplar for each scrape.
exemplarsPercentages := []float64{0, 0.5, 1, 5}
lastExemplarsPerSeries := -1
for _, c := range cases {
for _, p := range exemplarsPercentages {
exemplarsPerSeries := int(math.RoundToEven(float64(c.samplesPerSeries) * p / 100))
// For tests with low samplesPerSeries we could end up testing with 0 exemplarsPerSeries
// multiple times without this check.
if exemplarsPerSeries == lastExemplarsPerSeries {
continue
}
lastExemplarsPerSeries = exemplarsPerSeries
b.Run(fmt.Sprintf("batches=%d,seriesPerBatch=%d,samplesPerSeries=%d,exemplarsPerSeries=%d,mmappedChunkT=%d", c.batches, c.seriesPerBatch, c.samplesPerSeries, exemplarsPerSeries, c.mmappedChunkT),
func(b *testing.B) {
dir := b.TempDir()
w, err := wlog.New(nil, nil, dir, false)
require.NoError(b, err)
// Write series.
refSeries := make([]record.RefSeries, 0, c.seriesPerBatch)
for k := 0; k < c.batches; k++ {
refSeries = refSeries[:0]
for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ {
lbls := make(map[string]string, labelsPerSeries)
lbls[defaultLabelName] = strconv.Itoa(i)
for j := 1; len(lbls) < labelsPerSeries; j++ {
lbls[defaultLabelName+strconv.Itoa(j)] = defaultLabelValue + strconv.Itoa(j)
}
refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: labels.FromMap(lbls)})
}
populateTestWAL(b, w, []interface{}{refSeries})
}
// Write samples.
refSamples := make([]record.RefSample, 0, c.seriesPerBatch)
for i := 0; i < c.samplesPerSeries; i++ {
for j := 0; j < c.batches; j++ {
refSamples = refSamples[:0]
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
refSamples = append(refSamples, record.RefSample{
Ref: chunks.HeadSeriesRef(k) * 101,
T: int64(i) * 10,
V: float64(i) * 100,
})
}
populateTestWAL(b, w, []interface{}{refSamples})
}
}
// Write mmapped chunks.
if c.mmappedChunkT != 0 {
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, mmappedChunksDir(dir), chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(b, err)
for k := 0; k < c.batches*c.seriesPerBatch; k++ {
// Create one mmapped chunk per series, with one sample at the given time.
s := newMemSeries(labels.Labels{}, chunks.HeadSeriesRef(k)*101, defaultIsolationDisabled, DefaultSamplesPerChunk)
s.append(c.mmappedChunkT, 42, 0, chunkDiskMapper, c.mmappedChunkT)
s.mmapCurrentHeadChunk(chunkDiskMapper)
}
require.NoError(b, chunkDiskMapper.Close())
}
// Write exemplars.
refExemplars := make([]record.RefExemplar, 0, c.seriesPerBatch)
for i := 0; i < exemplarsPerSeries; i++ {
for j := 0; j < c.batches; j++ {
refExemplars = refExemplars[:0]
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
refExemplars = append(refExemplars, record.RefExemplar{
Ref: chunks.HeadSeriesRef(k) * 101,
T: int64(i) * 10,
V: float64(i) * 100,
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", i)),
})
}
populateTestWAL(b, w, []interface{}{refExemplars})
}
}
b.ResetTimer()
// Load the WAL.
for i := 0; i < b.N; i++ {
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = w.Dir()
h, err := NewHead(nil, nil, w, nil, opts, nil)
require.NoError(b, err)
h.Init(0)
}
b.StopTimer()
w.Close()
})
}
}
}
// TestHead_HighConcurrencyReadAndWrite generates 1000 series with a step of 15s and fills a whole block with samples,
// this means in total it generates 4000 chunks because with a step of 15s there are 4 chunks per block per series.
// While appending the samples to the head it concurrently queries them from multiple go routines and verifies that the
// returned results are correct.
func TestHead_HighConcurrencyReadAndWrite(t *testing.T) {
head, _ := newTestHead(t, DefaultBlockDuration, false, false)
defer func() {
require.NoError(t, head.Close())
}()
seriesCnt := 1000
readConcurrency := 2
writeConcurrency := 10
startTs := uint64(DefaultBlockDuration) // start at the second block relative to the unix epoch.
qryRange := uint64(5 * time.Minute.Milliseconds())
step := uint64(15 * time.Second / time.Millisecond)
endTs := startTs + uint64(DefaultBlockDuration)
labelSets := make([]labels.Labels, seriesCnt)
for i := 0; i < seriesCnt; i++ {
labelSets[i] = labels.FromStrings("seriesId", strconv.Itoa(i))
}
head.Init(0)
g, ctx := errgroup.WithContext(context.Background())
whileNotCanceled := func(f func() (bool, error)) error {
for ctx.Err() == nil {
cont, err := f()
if err != nil {
return err
}
if !cont {
return nil
}
}
return nil
}
// Create one channel for each write worker, the channels will be used by the coordinator
// go routine to coordinate which timestamps each write worker has to write.
writerTsCh := make([]chan uint64, writeConcurrency)
for writerTsChIdx := range writerTsCh {
writerTsCh[writerTsChIdx] = make(chan uint64)
}
// workerReadyWg is used to synchronize the start of the test,
// we only start the test once all workers signal that they're ready.
var workerReadyWg sync.WaitGroup
workerReadyWg.Add(writeConcurrency + readConcurrency)
// Start the write workers.
for wid := 0; wid < writeConcurrency; wid++ {
// Create copy of workerID to be used by worker routine.
workerID := wid
g.Go(func() error {
// The label sets which this worker will write.
workerLabelSets := labelSets[(seriesCnt/writeConcurrency)*workerID : (seriesCnt/writeConcurrency)*(workerID+1)]
// Signal that this worker is ready.
workerReadyWg.Done()
return whileNotCanceled(func() (bool, error) {
ts, ok := <-writerTsCh[workerID]
if !ok {
return false, nil
}
app := head.Appender(ctx)
for i := 0; i < len(workerLabelSets); i++ {
// We also use the timestamp as the sample value.
_, err := app.Append(0, workerLabelSets[i], int64(ts), float64(ts))
if err != nil {
return false, fmt.Errorf("Error when appending to head: %w", err)
}
}
return true, app.Commit()
})
})
}
// queryHead is a helper to query the head for a given time range and labelset.
queryHead := func(mint, maxt uint64, label labels.Label) (map[string][]tsdbutil.Sample, error) {
q, err := NewBlockQuerier(head, int64(mint), int64(maxt))
if err != nil {
return nil, err
}
return query(t, q, labels.MustNewMatcher(labels.MatchEqual, label.Name, label.Value)), nil
}
// readerTsCh will be used by the coordinator go routine to coordinate which timestamps the reader should read.
readerTsCh := make(chan uint64)
// Start the read workers.
for wid := 0; wid < readConcurrency; wid++ {
// Create copy of threadID to be used by worker routine.
workerID := wid
g.Go(func() error {
querySeriesRef := (seriesCnt / readConcurrency) * workerID
// Signal that this worker is ready.
workerReadyWg.Done()
return whileNotCanceled(func() (bool, error) {
ts, ok := <-readerTsCh
if !ok {
return false, nil
}
querySeriesRef = (querySeriesRef + 1) % seriesCnt
lbls := labelSets[querySeriesRef]
// lbls has a single entry; extract it so we can run a query.
var lbl labels.Label
lbls.Range(func(l labels.Label) {
lbl = l
})
samples, err := queryHead(ts-qryRange, ts, lbl)
if err != nil {
return false, err
}
if len(samples) != 1 {
return false, fmt.Errorf("expected 1 series, got %d", len(samples))
}
series := lbls.String()
expectSampleCnt := qryRange/step + 1
if expectSampleCnt != uint64(len(samples[series])) {
return false, fmt.Errorf("expected %d samples, got %d", expectSampleCnt, len(samples[series]))
}
for sampleIdx, sample := range samples[series] {
expectedValue := ts - qryRange + (uint64(sampleIdx) * step)
if sample.T() != int64(expectedValue) {
return false, fmt.Errorf("expected sample %d to have ts %d, got %d", sampleIdx, expectedValue, sample.T())
}
if sample.F() != float64(expectedValue) {
return false, fmt.Errorf("expected sample %d to have value %d, got %f", sampleIdx, expectedValue, sample.F())
}
}
return true, nil
})
})
}
// Start the coordinator go routine.
g.Go(func() error {
currTs := startTs
defer func() {
// End of the test, close all channels to stop the workers.
for _, ch := range writerTsCh {
close(ch)
}
close(readerTsCh)
}()
// Wait until all workers are ready to start the test.
workerReadyWg.Wait()
return whileNotCanceled(func() (bool, error) {
// Send the current timestamp to each of the writers.
for _, ch := range writerTsCh {
select {
case ch <- currTs:
case <-ctx.Done():
return false, nil
}
}
// Once data for at least <qryRange> has been ingested, send the current timestamp to the readers.
if currTs > startTs+qryRange {
select {
case readerTsCh <- currTs - step:
case <-ctx.Done():
return false, nil
}
}
currTs += step
if currTs > endTs {
return false, nil
}
return true, nil
})
})
require.NoError(t, g.Wait())
}
func TestHead_ReadWAL(t *testing.T) {
for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
entries := []interface{}{
[]record.RefSeries{
{Ref: 10, Labels: labels.FromStrings("a", "1")},
{Ref: 11, Labels: labels.FromStrings("a", "2")},
{Ref: 100, Labels: labels.FromStrings("a", "3")},
},
[]record.RefSample{
{Ref: 0, T: 99, V: 1},
{Ref: 10, T: 100, V: 2},
{Ref: 100, T: 100, V: 3},
},
[]record.RefSeries{
{Ref: 50, Labels: labels.FromStrings("a", "4")},
// This series has two refs pointing to it.
{Ref: 101, Labels: labels.FromStrings("a", "3")},
},
[]record.RefSample{
{Ref: 10, T: 101, V: 5},
{Ref: 50, T: 101, V: 6},
{Ref: 101, T: 101, V: 7},
},
[]tombstones.Stone{
{Ref: 0, Intervals: []tombstones.Interval{{Mint: 99, Maxt: 101}}},
},
[]record.RefExemplar{
{Ref: 10, T: 100, V: 1, Labels: labels.FromStrings("traceID", "asdf")},
},
}
head, w := newTestHead(t, 1000, compress, false)
defer func() {
require.NoError(t, head.Close())
}()
populateTestWAL(t, w, entries)
require.NoError(t, head.Init(math.MinInt64))
require.Equal(t, uint64(101), head.lastSeriesID.Load())
s10 := head.series.getByID(10)
s11 := head.series.getByID(11)
s50 := head.series.getByID(50)
s100 := head.series.getByID(100)
require.Equal(t, labels.FromStrings("a", "1"), s10.lset)
require.Equal(t, (*memSeries)(nil), s11) // Series without samples should be garbage collected at head.Init().
require.Equal(t, labels.FromStrings("a", "4"), s50.lset)
require.Equal(t, labels.FromStrings("a", "3"), s100.lset)
expandChunk := func(c chunkenc.Iterator) (x []sample) {
for c.Next() == chunkenc.ValFloat {
t, v := c.At()
x = append(x, sample{t: t, f: v})
}
require.NoError(t, c.Err())
return x
}
c, _, err := s10.chunk(0, head.chunkDiskMapper, &head.memChunkPool)
require.NoError(t, err)
require.Equal(t, []sample{{100, 2, nil, nil}, {101, 5, nil, nil}}, expandChunk(c.chunk.Iterator(nil)))
c, _, err = s50.chunk(0, head.chunkDiskMapper, &head.memChunkPool)
require.NoError(t, err)
require.Equal(t, []sample{{101, 6, nil, nil}}, expandChunk(c.chunk.Iterator(nil)))
// The samples before the new series record should be discarded since a duplicate record
// is only possible when old samples were compacted.
c, _, err = s100.chunk(0, head.chunkDiskMapper, &head.memChunkPool)
require.NoError(t, err)
require.Equal(t, []sample{{101, 7, nil, nil}}, expandChunk(c.chunk.Iterator(nil)))
q, err := head.ExemplarQuerier(context.Background())
require.NoError(t, err)
e, err := q.Select(0, 1000, []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "1")})
require.NoError(t, err)
require.Equal(t, e[0].Exemplars[0], exemplar.Exemplar{Ts: 100, Value: 1, Labels: labels.FromStrings("traceID", "asdf")})
})
}
}
func TestHead_WALMultiRef(t *testing.T) {
head, w := newTestHead(t, 1000, false, false)
require.NoError(t, head.Init(0))
app := head.Appender(context.Background())
ref1, err := app.Append(0, labels.FromStrings("foo", "bar"), 100, 1)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.chunksCreated))
// Add another sample outside chunk range to mmap a chunk.
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 1500, 2)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 2.0, prom_testutil.ToFloat64(head.metrics.chunksCreated))
require.NoError(t, head.Truncate(1600))
app = head.Appender(context.Background())
ref2, err := app.Append(0, labels.FromStrings("foo", "bar"), 1700, 3)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 3.0, prom_testutil.ToFloat64(head.metrics.chunksCreated))
// Add another sample outside chunk range to mmap a chunk.
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 2000, 4)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 4.0, prom_testutil.ToFloat64(head.metrics.chunksCreated))
require.NotEqual(t, ref1, ref2, "Refs are the same")
require.NoError(t, head.Close())
w, err = wlog.New(nil, nil, w.Dir(), false)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = w.Dir()
head, err = NewHead(nil, nil, w, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
defer func() {
require.NoError(t, head.Close())
}()
q, err := NewBlockQuerier(head, 0, 2100)
require.NoError(t, err)
series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"))
// The samples before the new ref should be discarded since Head truncation
// happens only after compacting the Head.
require.Equal(t, map[string][]tsdbutil.Sample{`{foo="bar"}`: {
sample{1700, 3, nil, nil},
sample{2000, 4, nil, nil},
}}, series)
}
func TestHead_ActiveAppenders(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
defer head.Close()
require.NoError(t, head.Init(0))
// First rollback with no samples.
app := head.Appender(context.Background())
require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
require.NoError(t, app.Rollback())
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
// Then commit with no samples.
app = head.Appender(context.Background())
require.NoError(t, app.Commit())
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
// Now rollback with one sample.
app = head.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("foo", "bar"), 100, 1)
require.NoError(t, err)
require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
require.NoError(t, app.Rollback())
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
// Now commit with one sample.
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 100, 1)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders))
}
func TestHead_UnknownWALRecord(t *testing.T) {
head, w := newTestHead(t, 1000, false, false)
w.Log([]byte{255, 42})
require.NoError(t, head.Init(0))
require.NoError(t, head.Close())
}
func TestHead_Truncate(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
s1, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1", "b", "1"))
s2, _, _ := h.getOrCreate(2, labels.FromStrings("a", "2", "b", "1"))
s3, _, _ := h.getOrCreate(3, labels.FromStrings("a", "1", "b", "2"))
s4, _, _ := h.getOrCreate(4, labels.FromStrings("a", "2", "b", "2", "c", "1"))
s1.mmappedChunks = []*mmappedChunk{
{minTime: 0, maxTime: 999},
{minTime: 1000, maxTime: 1999},
{minTime: 2000, maxTime: 2999},
}
s2.mmappedChunks = []*mmappedChunk{
{minTime: 1000, maxTime: 1999},
{minTime: 2000, maxTime: 2999},
{minTime: 3000, maxTime: 3999},
}
s3.mmappedChunks = []*mmappedChunk{
{minTime: 0, maxTime: 999},
{minTime: 1000, maxTime: 1999},
}
s4.mmappedChunks = []*mmappedChunk{}
// Truncation need not be aligned.
require.NoError(t, h.Truncate(1))
require.NoError(t, h.Truncate(2000))
require.Equal(t, []*mmappedChunk{
{minTime: 2000, maxTime: 2999},
}, h.series.getByID(s1.ref).mmappedChunks)
require.Equal(t, []*mmappedChunk{
{minTime: 2000, maxTime: 2999},
{minTime: 3000, maxTime: 3999},
}, h.series.getByID(s2.ref).mmappedChunks)
require.Nil(t, h.series.getByID(s3.ref))
require.Nil(t, h.series.getByID(s4.ref))
postingsA1, _ := index.ExpandPostings(h.postings.Get("a", "1"))
postingsA2, _ := index.ExpandPostings(h.postings.Get("a", "2"))
postingsB1, _ := index.ExpandPostings(h.postings.Get("b", "1"))
postingsB2, _ := index.ExpandPostings(h.postings.Get("b", "2"))
postingsC1, _ := index.ExpandPostings(h.postings.Get("c", "1"))
postingsAll, _ := index.ExpandPostings(h.postings.Get("", ""))
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref)}, postingsA1)
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s2.ref)}, postingsA2)
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsB1)
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsAll)
require.Nil(t, postingsB2)
require.Nil(t, postingsC1)
iter := h.postings.Symbols()
symbols := []string{}
for iter.Next() {
symbols = append(symbols, iter.At())
}
require.Equal(t,
[]string{"" /* from 'all' postings list */, "1", "2", "a", "b"},
symbols)
values := map[string]map[string]struct{}{}
for _, name := range h.postings.LabelNames() {
ss, ok := values[name]
if !ok {
ss = map[string]struct{}{}
values[name] = ss
}
for _, value := range h.postings.LabelValues(name) {
ss[value] = struct{}{}
}
}
require.Equal(t, map[string]map[string]struct{}{
"a": {"1": struct{}{}, "2": struct{}{}},
"b": {"1": struct{}{}},
}, values)
}
// Validate various behaviors brought on by firstChunkID accounting for
// garbage collected chunks.
func TestMemSeries_truncateChunks(t *testing.T) {
dir := t.TempDir()
// This is usually taken from the Head, but passing manually here.
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(t, err)
defer func() {
require.NoError(t, chunkDiskMapper.Close())
}()
const chunkRange = 2000
memChunkPool := sync.Pool{
New: func() interface{} {
return &memChunk{}
},
}
s := newMemSeries(labels.FromStrings("a", "b"), 1, defaultIsolationDisabled, DefaultSamplesPerChunk)
for i := 0; i < 4000; i += 5 {
ok, _ := s.append(int64(i), float64(i), 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "sample append failed")
}
// Check that truncate removes half of the chunks and afterwards
// that the ID of the last chunk still gives us the same chunk afterwards.
countBefore := len(s.mmappedChunks) + 1 // +1 for the head chunk.
lastID := s.headChunkID(countBefore - 1)
lastChunk, _, err := s.chunk(lastID, chunkDiskMapper, &memChunkPool)
require.NoError(t, err)
require.NotNil(t, lastChunk)
chk, _, err := s.chunk(0, chunkDiskMapper, &memChunkPool)
require.NotNil(t, chk)
require.NoError(t, err)
s.truncateChunksBefore(2000, 0)
require.Equal(t, int64(2000), s.mmappedChunks[0].minTime)
_, _, err = s.chunk(0, chunkDiskMapper, &memChunkPool)
require.Equal(t, storage.ErrNotFound, err, "first chunks not gone")
require.Equal(t, countBefore/2, len(s.mmappedChunks)+1) // +1 for the head chunk.
chk, _, err = s.chunk(lastID, chunkDiskMapper, &memChunkPool)
require.NoError(t, err)
require.Equal(t, lastChunk, chk)
}
func TestHeadDeleteSeriesWithoutSamples(t *testing.T) {
for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
entries := []interface{}{
[]record.RefSeries{
{Ref: 10, Labels: labels.FromStrings("a", "1")},
},
[]record.RefSample{},
[]record.RefSeries{
{Ref: 50, Labels: labels.FromStrings("a", "2")},
},
[]record.RefSample{
{Ref: 50, T: 80, V: 1},
{Ref: 50, T: 90, V: 1},
},
}
head, w := newTestHead(t, 1000, compress, false)
defer func() {
require.NoError(t, head.Close())
}()
populateTestWAL(t, w, entries)
require.NoError(t, head.Init(math.MinInt64))
require.NoError(t, head.Delete(0, 100, labels.MustNewMatcher(labels.MatchEqual, "a", "1")))
})
}
}
func TestHeadDeleteSimple(t *testing.T) {
buildSmpls := func(s []int64) []sample {
ss := make([]sample, 0, len(s))
for _, t := range s {
ss = append(ss, sample{t: t, f: float64(t)})
}
return ss
}
smplsAll := buildSmpls([]int64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})
lblDefault := labels.Label{Name: "a", Value: "b"}
lblsDefault := labels.FromStrings("a", "b")
cases := []struct {
dranges tombstones.Intervals
addSamples []sample // Samples to add after delete.
smplsExp []sample
}{
{
dranges: tombstones.Intervals{{Mint: 0, Maxt: 3}},
smplsExp: buildSmpls([]int64{4, 5, 6, 7, 8, 9}),
},
{
dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}},
smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9}),
},
{
dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}},
smplsExp: buildSmpls([]int64{0, 8, 9}),
},
{
dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 700}},
smplsExp: buildSmpls([]int64{0}),
},
{ // This case is to ensure that labels and symbols are deleted.
dranges: tombstones.Intervals{{Mint: 0, Maxt: 9}},
smplsExp: buildSmpls([]int64{}),
},
{
dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}},
addSamples: buildSmpls([]int64{11, 13, 15}),
smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9, 11, 13, 15}),
},
{
// After delete, the appended samples in the deleted range should be visible
// as the tombstones are clamped to head min/max time.
dranges: tombstones.Intervals{{Mint: 7, Maxt: 20}},
addSamples: buildSmpls([]int64{11, 13, 15}),
smplsExp: buildSmpls([]int64{0, 1, 2, 3, 4, 5, 6, 11, 13, 15}),
},
}
for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
for _, c := range cases {
head, w := newTestHead(t, 1000, compress, false)
require.NoError(t, head.Init(0))
app := head.Appender(context.Background())
for _, smpl := range smplsAll {
_, err := app.Append(0, lblsDefault, smpl.t, smpl.f)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
// Delete the ranges.
for _, r := range c.dranges {
require.NoError(t, head.Delete(r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value)))
}
// Add more samples.
app = head.Appender(context.Background())
for _, smpl := range c.addSamples {
_, err := app.Append(0, lblsDefault, smpl.t, smpl.f)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
// Compare the samples for both heads - before and after the reloadBlocks.
reloadedW, err := wlog.New(nil, nil, w.Dir(), compress) // Use a new wal to ensure deleted samples are gone even after a reloadBlocks.
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = reloadedW.Dir()
reloadedHead, err := NewHead(nil, nil, reloadedW, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, reloadedHead.Init(0))
// Compare the query results for both heads - before and after the reloadBlocks.
Outer:
for _, h := range []*Head{head, reloadedHead} {
q, err := NewBlockQuerier(h, h.MinTime(), h.MaxTime())
require.NoError(t, err)
actSeriesSet := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value))
require.NoError(t, q.Close())
expSeriesSet := newMockSeriesSet([]storage.Series{
storage.NewListSeries(lblsDefault, func() []tsdbutil.Sample {
ss := make([]tsdbutil.Sample, 0, len(c.smplsExp))
for _, s := range c.smplsExp {
ss = append(ss, s)
}
return ss
}(),
),
})
for {
eok, rok := expSeriesSet.Next(), actSeriesSet.Next()
require.Equal(t, eok, rok)
if !eok {
require.NoError(t, h.Close())
require.NoError(t, actSeriesSet.Err())
require.Equal(t, 0, len(actSeriesSet.Warnings()))
continue Outer
}
expSeries := expSeriesSet.At()
actSeries := actSeriesSet.At()
require.Equal(t, expSeries.Labels(), actSeries.Labels())
smplExp, errExp := storage.ExpandSamples(expSeries.Iterator(nil), nil)
smplRes, errRes := storage.ExpandSamples(actSeries.Iterator(nil), nil)
require.Equal(t, errExp, errRes)
require.Equal(t, smplExp, smplRes)
}
}
}
})
}
}
func TestDeleteUntilCurMax(t *testing.T) {
hb, _ := newTestHead(t, 1000000, false, false)
defer func() {
require.NoError(t, hb.Close())
}()
numSamples := int64(10)
app := hb.Appender(context.Background())
smpls := make([]float64, numSamples)
for i := int64(0); i < numSamples; i++ {
smpls[i] = rand.Float64()
_, err := app.Append(0, labels.FromStrings("a", "b"), i, smpls[i])
require.NoError(t, err)
}
require.NoError(t, app.Commit())
require.NoError(t, hb.Delete(0, 10000, labels.MustNewMatcher(labels.MatchEqual, "a", "b")))
// Test the series returns no samples. The series is cleared only after compaction.
q, err := NewBlockQuerier(hb, 0, 100000)
require.NoError(t, err)
res := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.True(t, res.Next(), "series is not present")
s := res.At()
it := s.Iterator(nil)
require.Equal(t, chunkenc.ValNone, it.Next(), "expected no samples")
for res.Next() {
}
require.NoError(t, res.Err())
require.Equal(t, 0, len(res.Warnings()))
// Add again and test for presence.
app = hb.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("a", "b"), 11, 1)
require.NoError(t, err)
require.NoError(t, app.Commit())
q, err = NewBlockQuerier(hb, 0, 100000)
require.NoError(t, err)
res = q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.True(t, res.Next(), "series don't exist")
exps := res.At()
it = exps.Iterator(nil)
resSamples, err := storage.ExpandSamples(it, newSample)
require.NoError(t, err)
require.Equal(t, []tsdbutil.Sample{sample{11, 1, nil, nil}}, resSamples)
for res.Next() {
}
require.NoError(t, res.Err())
require.Equal(t, 0, len(res.Warnings()))
}
func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) {
numSamples := 10000
// Enough samples to cause a checkpoint.
hb, w := newTestHead(t, int64(numSamples)*10, false, false)
for i := 0; i < numSamples; i++ {
app := hb.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("a", "b"), int64(i), 0)
require.NoError(t, err)
require.NoError(t, app.Commit())
}
require.NoError(t, hb.Delete(0, int64(numSamples), labels.MustNewMatcher(labels.MatchEqual, "a", "b")))
require.NoError(t, hb.Truncate(1))
require.NoError(t, hb.Close())
// Confirm there's been a checkpoint.
cdir, _, err := wlog.LastCheckpoint(w.Dir())
require.NoError(t, err)
// Read in checkpoint and WAL.
recs := readTestWAL(t, cdir)
recs = append(recs, readTestWAL(t, w.Dir())...)
var series, samples, stones, metadata int
for _, rec := range recs {
switch rec.(type) {
case []record.RefSeries:
series++
case []record.RefSample:
samples++
case []tombstones.Stone:
stones++
case []record.RefMetadata:
metadata++
default:
t.Fatalf("unknown record type")
}
}
require.Equal(t, 1, series)
require.Equal(t, 9999, samples)
require.Equal(t, 1, stones)
require.Equal(t, 0, metadata)
}
func TestDelete_e2e(t *testing.T) {
numDatapoints := 1000
numRanges := 1000
timeInterval := int64(2)
// Create 8 series with 1000 data-points of different ranges, delete and run queries.
lbls := [][]labels.Label{
{
{Name: "a", Value: "b"},
{Name: "instance", Value: "localhost:9090"},
{Name: "job", Value: "prometheus"},
},
{
{Name: "a", Value: "b"},
{Name: "instance", Value: "127.0.0.1:9090"},
{Name: "job", Value: "prometheus"},
},
{
{Name: "a", Value: "b"},
{Name: "instance", Value: "127.0.0.1:9090"},
{Name: "job", Value: "prom-k8s"},
},
{
{Name: "a", Value: "b"},
{Name: "instance", Value: "localhost:9090"},
{Name: "job", Value: "prom-k8s"},
},
{
{Name: "a", Value: "c"},
{Name: "instance", Value: "localhost:9090"},
{Name: "job", Value: "prometheus"},
},
{
{Name: "a", Value: "c"},
{Name: "instance", Value: "127.0.0.1:9090"},
{Name: "job", Value: "prometheus"},
},
{
{Name: "a", Value: "c"},
{Name: "instance", Value: "127.0.0.1:9090"},
{Name: "job", Value: "prom-k8s"},
},
{
{Name: "a", Value: "c"},
{Name: "instance", Value: "localhost:9090"},
{Name: "job", Value: "prom-k8s"},
},
}
seriesMap := map[string][]tsdbutil.Sample{}
for _, l := range lbls {
seriesMap[labels.New(l...).String()] = []tsdbutil.Sample{}
}
hb, _ := newTestHead(t, 100000, false, false)
defer func() {
require.NoError(t, hb.Close())
}()
app := hb.Appender(context.Background())
for _, l := range lbls {
ls := labels.New(l...)
series := []tsdbutil.Sample{}
ts := rand.Int63n(300)
for i := 0; i < numDatapoints; i++ {
v := rand.Float64()
_, err := app.Append(0, ls, ts, v)
require.NoError(t, err)
series = append(series, sample{ts, v, nil, nil})
ts += rand.Int63n(timeInterval) + 1
}
seriesMap[labels.New(l...).String()] = series
}
require.NoError(t, app.Commit())
// Delete a time-range from each-selector.
dels := []struct {
ms []*labels.Matcher
drange tombstones.Intervals
}{
{
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "b")},
drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 600, Maxt: 670}},
},
{
ms: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "a", "b"),
labels.MustNewMatcher(labels.MatchEqual, "job", "prom-k8s"),
},
drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 100, Maxt: 670}},
},
{
ms: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "a", "c"),
labels.MustNewMatcher(labels.MatchEqual, "instance", "localhost:9090"),
labels.MustNewMatcher(labels.MatchEqual, "job", "prometheus"),
},
drange: tombstones.Intervals{{Mint: 300, Maxt: 400}, {Mint: 100, Maxt: 6700}},
},
// TODO: Add Regexp Matchers.
}
for _, del := range dels {
for _, r := range del.drange {
require.NoError(t, hb.Delete(r.Mint, r.Maxt, del.ms...))
}
matched := labels.Slice{}
for _, l := range lbls {
s := labels.Selector(del.ms)
ls := labels.New(l...)
if s.Matches(ls) {
matched = append(matched, ls)
}
}
sort.Sort(matched)
for i := 0; i < numRanges; i++ {
q, err := NewBlockQuerier(hb, 0, 100000)
require.NoError(t, err)
defer q.Close()
ss := q.Select(true, nil, del.ms...)
// Build the mockSeriesSet.
matchedSeries := make([]storage.Series, 0, len(matched))
for _, m := range matched {
smpls := seriesMap[m.String()]
smpls = deletedSamples(smpls, del.drange)
// Only append those series for which samples exist as mockSeriesSet
// doesn't skip series with no samples.
// TODO: But sometimes SeriesSet returns an empty chunkenc.Iterator
if len(smpls) > 0 {
matchedSeries = append(matchedSeries, storage.NewListSeries(m, smpls))
}
}
expSs := newMockSeriesSet(matchedSeries)
// Compare both SeriesSets.
for {
eok, rok := expSs.Next(), ss.Next()
// Skip a series if iterator is empty.
if rok {
for ss.At().Iterator(nil).Next() == chunkenc.ValNone {
rok = ss.Next()
if !rok {
break
}
}
}
require.Equal(t, eok, rok)
if !eok {
break
}
sexp := expSs.At()
sres := ss.At()
require.Equal(t, sexp.Labels(), sres.Labels())
smplExp, errExp := storage.ExpandSamples(sexp.Iterator(nil), nil)
smplRes, errRes := storage.ExpandSamples(sres.Iterator(nil), nil)
require.Equal(t, errExp, errRes)
require.Equal(t, smplExp, smplRes)
}
require.NoError(t, ss.Err())
require.Equal(t, 0, len(ss.Warnings()))
}
}
}
func boundedSamples(full []tsdbutil.Sample, mint, maxt int64) []tsdbutil.Sample {
for len(full) > 0 {
if full[0].T() >= mint {
break
}
full = full[1:]
}
for i, s := range full {
// labels.Labelinate on the first sample larger than maxt.
if s.T() > maxt {
return full[:i]
}
}
// maxt is after highest sample.
return full
}
func deletedSamples(full []tsdbutil.Sample, dranges tombstones.Intervals) []tsdbutil.Sample {
ds := make([]tsdbutil.Sample, 0, len(full))
Outer:
for _, s := range full {
for _, r := range dranges {
if r.InBounds(s.T()) {
continue Outer
}
}
ds = append(ds, s)
}
return ds
}
func TestComputeChunkEndTime(t *testing.T) {
cases := []struct {
start, cur, max int64
res int64
}{
{
start: 0,
cur: 250,
max: 1000,
res: 1000,
},
{
start: 100,
cur: 200,
max: 1000,
res: 550,
},
// Case where we fit floored 0 chunks. Must catch division by 0
// and default to maximum time.
{
start: 0,
cur: 500,
max: 1000,
res: 1000,
},
// Catch division by zero for cur == start. Strictly not a possible case.
{
start: 100,
cur: 100,
max: 1000,
res: 104,
},
}
for _, c := range cases {
got := computeChunkEndTime(c.start, c.cur, c.max)
if got != c.res {
t.Errorf("expected %d for (start: %d, cur: %d, max: %d), got %d", c.res, c.start, c.cur, c.max, got)
}
}
}
func TestMemSeries_append(t *testing.T) {
dir := t.TempDir()
// This is usually taken from the Head, but passing manually here.
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(t, err)
defer func() {
require.NoError(t, chunkDiskMapper.Close())
}()
const chunkRange = 500
s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled, DefaultSamplesPerChunk)
// Add first two samples at the very end of a chunk range and the next two
// on and after it.
// New chunk must correctly be cut at 1000.
ok, chunkCreated := s.append(998, 1, 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.True(t, chunkCreated, "first sample created chunk")
ok, chunkCreated = s.append(999, 2, 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.False(t, chunkCreated, "second sample should use same chunk")
ok, chunkCreated = s.append(1000, 3, 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.True(t, chunkCreated, "expected new chunk on boundary")
ok, chunkCreated = s.append(1001, 4, 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.False(t, chunkCreated, "second sample should use same chunk")
require.Equal(t, 1, len(s.mmappedChunks), "there should be only 1 mmapped chunk")
require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range")
require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range")
require.Equal(t, int64(1000), s.headChunk.minTime, "wrong chunk range")
require.Equal(t, int64(1001), s.headChunk.maxTime, "wrong chunk range")
// Fill the range [1000,2000) with many samples. Intermediate chunks should be cut
// at approximately 120 samples per chunk.
for i := 1; i < 1000; i++ {
ok, _ := s.append(1001+int64(i), float64(i), 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
}
require.Greater(t, len(s.mmappedChunks)+1, 7, "expected intermediate chunks")
// All chunks but the first and last should now be moderately full.
for i, c := range s.mmappedChunks[1:] {
chk, err := chunkDiskMapper.Chunk(c.ref)
require.NoError(t, err)
require.Greater(t, chk.NumSamples(), 100, "unexpected small chunk %d of length %d", i, chk.NumSamples())
}
}
func TestMemSeries_appendHistogram(t *testing.T) {
dir := t.TempDir()
// This is usually taken from the Head, but passing manually here.
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(t, err)
defer func() {
require.NoError(t, chunkDiskMapper.Close())
}()
chunkRange := int64(1000)
s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled, DefaultSamplesPerChunk)
histograms := tsdbutil.GenerateTestHistograms(4)
histogramWithOneMoreBucket := histograms[3].Copy()
histogramWithOneMoreBucket.Count++
histogramWithOneMoreBucket.Sum += 1.23
histogramWithOneMoreBucket.PositiveSpans[1].Length = 3
histogramWithOneMoreBucket.PositiveBuckets = append(histogramWithOneMoreBucket.PositiveBuckets, 1)
// Add first two samples at the very end of a chunk range and the next two
// on and after it.
// New chunk must correctly be cut at 1000.
ok, chunkCreated := s.appendHistogram(998, histograms[0], 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.True(t, chunkCreated, "first sample created chunk")
ok, chunkCreated = s.appendHistogram(999, histograms[1], 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.False(t, chunkCreated, "second sample should use same chunk")
ok, chunkCreated = s.appendHistogram(1000, histograms[2], 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.True(t, chunkCreated, "expected new chunk on boundary")
ok, chunkCreated = s.appendHistogram(1001, histograms[3], 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.False(t, chunkCreated, "second sample should use same chunk")
require.Equal(t, 1, len(s.mmappedChunks), "there should be only 1 mmapped chunk")
require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range")
require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range")
require.Equal(t, int64(1000), s.headChunk.minTime, "wrong chunk range")
require.Equal(t, int64(1001), s.headChunk.maxTime, "wrong chunk range")
ok, chunkCreated = s.appendHistogram(1002, histogramWithOneMoreBucket, 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "append failed")
require.False(t, chunkCreated, "third sample should trigger a re-encoded chunk")
require.Equal(t, 1, len(s.mmappedChunks), "there should be only 1 mmapped chunk")
require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range")
require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range")
require.Equal(t, int64(1000), s.headChunk.minTime, "wrong chunk range")
require.Equal(t, int64(1002), s.headChunk.maxTime, "wrong chunk range")
}
func TestMemSeries_append_atVariableRate(t *testing.T) {
const samplesPerChunk = 120
dir := t.TempDir()
// This is usually taken from the Head, but passing manually here.
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, chunkDiskMapper.Close())
})
chunkRange := DefaultBlockDuration
s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled, DefaultSamplesPerChunk)
// At this slow rate, we will fill the chunk in two block durations.
slowRate := (DefaultBlockDuration * 2) / samplesPerChunk
var nextTs int64
var totalAppendedSamples int
for i := 0; i < samplesPerChunk/4; i++ {
ok, _ := s.append(nextTs, float64(i), 0, chunkDiskMapper, chunkRange)
require.Truef(t, ok, "slow sample %d was not appended", i)
nextTs += slowRate
totalAppendedSamples++
}
require.Equal(t, DefaultBlockDuration, s.nextAt, "after appending a samplesPerChunk/4 samples at a slow rate, we should aim to cut a new block at the default block duration %d, but it's set to %d", DefaultBlockDuration, s.nextAt)
// Suddenly, the rate increases and we receive a sample every millisecond.
for i := 0; i < math.MaxUint16; i++ {
ok, _ := s.append(nextTs, float64(i), 0, chunkDiskMapper, chunkRange)
require.Truef(t, ok, "quick sample %d was not appended", i)
nextTs++
totalAppendedSamples++
}
ok, chunkCreated := s.append(DefaultBlockDuration, float64(0), 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "new chunk sample was not appended")
require.True(t, chunkCreated, "sample at block duration timestamp should create a new chunk")
var totalSamplesInChunks int
for i, c := range s.mmappedChunks {
totalSamplesInChunks += int(c.numSamples)
require.LessOrEqualf(t, c.numSamples, uint16(2*samplesPerChunk), "mmapped chunk %d has more than %d samples", i, 2*samplesPerChunk)
}
require.Equal(t, totalAppendedSamples, totalSamplesInChunks, "wrong number of samples in %d mmapped chunks", len(s.mmappedChunks))
}
func TestGCChunkAccess(t *testing.T) {
// Put a chunk, select it. GC it and then access it.
const chunkRange = 1000
h, _ := newTestHead(t, chunkRange, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"))
// Appending 2 samples for the first chunk.
ok, chunkCreated := s.append(0, 0, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.True(t, chunkCreated, "chunks was not created")
ok, chunkCreated = s.append(999, 999, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.False(t, chunkCreated, "chunks was created")
// A new chunks should be created here as it's beyond the chunk range.
ok, chunkCreated = s.append(1000, 1000, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.True(t, chunkCreated, "chunks was not created")
ok, chunkCreated = s.append(1999, 1999, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.False(t, chunkCreated, "chunks was created")
idx := h.indexRange(0, 1500)
var (
chunks []chunks.Meta
builder labels.ScratchBuilder
)
require.NoError(t, idx.Series(1, &builder, &chunks))
require.Equal(t, labels.FromStrings("a", "1"), builder.Labels())
require.Equal(t, 2, len(chunks))
cr, err := h.chunksRange(0, 1500, nil)
require.NoError(t, err)
_, err = cr.Chunk(chunks[0])
require.NoError(t, err)
_, err = cr.Chunk(chunks[1])
require.NoError(t, err)
require.NoError(t, h.Truncate(1500)) // Remove a chunk.
_, err = cr.Chunk(chunks[0])
require.Equal(t, storage.ErrNotFound, err)
_, err = cr.Chunk(chunks[1])
require.NoError(t, err)
}
func TestGCSeriesAccess(t *testing.T) {
// Put a series, select it. GC it and then access it.
const chunkRange = 1000
h, _ := newTestHead(t, chunkRange, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"))
// Appending 2 samples for the first chunk.
ok, chunkCreated := s.append(0, 0, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.True(t, chunkCreated, "chunks was not created")
ok, chunkCreated = s.append(999, 999, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.False(t, chunkCreated, "chunks was created")
// A new chunks should be created here as it's beyond the chunk range.
ok, chunkCreated = s.append(1000, 1000, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.True(t, chunkCreated, "chunks was not created")
ok, chunkCreated = s.append(1999, 1999, 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.False(t, chunkCreated, "chunks was created")
idx := h.indexRange(0, 2000)
var (
chunks []chunks.Meta
builder labels.ScratchBuilder
)
require.NoError(t, idx.Series(1, &builder, &chunks))
require.Equal(t, labels.FromStrings("a", "1"), builder.Labels())
require.Equal(t, 2, len(chunks))
cr, err := h.chunksRange(0, 2000, nil)
require.NoError(t, err)
_, err = cr.Chunk(chunks[0])
require.NoError(t, err)
_, 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])
require.Equal(t, storage.ErrNotFound, err)
_, err = cr.Chunk(chunks[1])
require.Equal(t, storage.ErrNotFound, err)
}
func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
app := h.appender()
lset := labels.FromStrings("a", "1")
_, err := app.Append(0, lset, 2100, 1)
require.NoError(t, err)
require.NoError(t, h.Truncate(2000))
require.NotNil(t, h.series.getByHash(lset.Hash(), lset), "series should not have been garbage collected")
require.NoError(t, app.Commit())
q, err := NewBlockQuerier(h, 1500, 2500)
require.NoError(t, err)
defer q.Close()
ss := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1"))
require.Equal(t, true, ss.Next())
for ss.Next() {
}
require.NoError(t, ss.Err())
require.Equal(t, 0, len(ss.Warnings()))
}
func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
app := h.appender()
lset := labels.FromStrings("a", "1")
_, err := app.Append(0, lset, 2100, 1)
require.NoError(t, err)
require.NoError(t, h.Truncate(2000))
require.NotNil(t, h.series.getByHash(lset.Hash(), lset), "series should not have been garbage collected")
require.NoError(t, app.Rollback())
q, err := NewBlockQuerier(h, 1500, 2500)
require.NoError(t, err)
ss := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1"))
require.Equal(t, false, ss.Next())
require.Equal(t, 0, len(ss.Warnings()))
require.NoError(t, q.Close())
// Truncate again, this time the series should be deleted
require.NoError(t, h.Truncate(2050))
require.Equal(t, (*memSeries)(nil), h.series.getByHash(lset.Hash(), lset))
}
func TestHead_LogRollback(t *testing.T) {
for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
h, w := newTestHead(t, 1000, compress, false)
defer func() {
require.NoError(t, h.Close())
}()
app := h.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("a", "b"), 1, 2)
require.NoError(t, err)
require.NoError(t, app.Rollback())
recs := readTestWAL(t, w.Dir())
require.Equal(t, 1, len(recs))
series, ok := recs[0].([]record.RefSeries)
require.True(t, ok, "expected series record but got %+v", recs[0])
require.Equal(t, []record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, series)
})
}
}
// TestWalRepair_DecodingError ensures that a repair is run for an error
// when decoding a record.
func TestWalRepair_DecodingError(t *testing.T) {
var enc record.Encoder
for name, test := range map[string]struct {
corrFunc func(rec []byte) []byte // Func that applies the corruption to a record.
rec []byte
totalRecs int
expRecs int
}{
"decode_series": {
func(rec []byte) []byte {
return rec[:3]
},
enc.Series([]record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}),
9,
5,
},
"decode_samples": {
func(rec []byte) []byte {
return rec[:3]
},
enc.Samples([]record.RefSample{{Ref: 0, T: 99, V: 1}}, []byte{}),
9,
5,
},
"decode_tombstone": {
func(rec []byte) []byte {
return rec[:3]
},
enc.Tombstones([]tombstones.Stone{{Ref: 1, Intervals: tombstones.Intervals{}}}, []byte{}),
9,
5,
},
} {
for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("%s,compress=%t", name, compress), func(t *testing.T) {
dir := t.TempDir()
// Fill the wal and corrupt it.
{
w, err := wlog.New(nil, nil, filepath.Join(dir, "wal"), compress)
require.NoError(t, err)
for i := 1; i <= test.totalRecs; i++ {
// At this point insert a corrupted record.
if i-1 == test.expRecs {
require.NoError(t, w.Log(test.corrFunc(test.rec)))
continue
}
require.NoError(t, w.Log(test.rec))
}
opts := DefaultHeadOptions()
opts.ChunkRange = 1
opts.ChunkDirRoot = w.Dir()
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)
err = errors.Cause(initErr) // So that we can pick up errors even if wrapped.
_, corrErr := err.(*wlog.CorruptionErr)
require.True(t, corrErr, "reading the wal didn't return corruption error")
require.NoError(t, h.Close()) // Head will close the wal as well.
}
// Open the db to trigger a repair.
{
db, err := Open(dir, nil, nil, DefaultOptions(), nil)
require.NoError(t, err)
defer func() {
require.NoError(t, db.Close())
}()
require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal))
}
// Read the wal content after the repair.
{
sr, err := wlog.NewSegmentsReader(filepath.Join(dir, "wal"))
require.NoError(t, err)
defer sr.Close()
r := wlog.NewReader(sr)
var actRec int
for r.Next() {
actRec++
}
require.NoError(t, r.Err())
require.Equal(t, test.expRecs, actRec, "Wrong number of intact records")
}
})
}
}
}
func TestHeadReadWriterRepair(t *testing.T) {
dir := t.TempDir()
const chunkRange = 1000
walDir := filepath.Join(dir, "wal")
// Fill the chunk segments and corrupt it.
{
w, err := wlog.New(nil, nil, walDir, false)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = chunkRange
opts.ChunkDirRoot = dir
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))
s, created, _ := h.getOrCreate(1, labels.FromStrings("a", "1"))
require.True(t, created, "series was not created")
for i := 0; i < 7; i++ {
ok, chunkCreated := s.append(int64(i*chunkRange), float64(i*chunkRange), 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.True(t, chunkCreated, "chunk was not created")
ok, chunkCreated = s.append(int64(i*chunkRange)+chunkRange-1, float64(i*chunkRange), 0, h.chunkDiskMapper, chunkRange)
require.True(t, ok, "series append failed")
require.False(t, chunkCreated, "chunk was created")
h.chunkDiskMapper.CutNewFile()
}
require.NoError(t, h.Close())
// Verify that there are 6 segment files.
// It should only be 6 because the last call to .CutNewFile() won't
// take effect without another chunk being written.
files, err := os.ReadDir(mmappedChunksDir(dir))
require.NoError(t, err)
require.Equal(t, 6, len(files))
// Corrupt the 4th file by writing a random byte to series ref.
f, err := os.OpenFile(filepath.Join(mmappedChunksDir(dir), files[3].Name()), os.O_WRONLY, 0o666)
require.NoError(t, err)
n, err := f.WriteAt([]byte{67, 88}, chunks.HeadChunkFileHeaderSize+2)
require.NoError(t, err)
require.Equal(t, 2, n)
require.NoError(t, f.Close())
}
// Open the db to trigger a repair.
{
db, err := Open(dir, nil, nil, DefaultOptions(), nil)
require.NoError(t, err)
defer func() {
require.NoError(t, db.Close())
}()
require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.mmapChunkCorruptionTotal))
}
// Verify that there are 3 segment files after the repair.
// The segments from the corrupt segment should be removed.
{
files, err := os.ReadDir(mmappedChunksDir(dir))
require.NoError(t, err)
require.Equal(t, 3, len(files))
}
}
func TestNewWalSegmentOnTruncate(t *testing.T) {
h, wal := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
add := func(ts int64) {
app := h.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("a", "b"), ts, 0)
require.NoError(t, err)
require.NoError(t, app.Commit())
}
add(0)
_, last, err := wlog.Segments(wal.Dir())
require.NoError(t, err)
require.Equal(t, 0, last)
add(1)
require.NoError(t, h.Truncate(1))
_, last, err = wlog.Segments(wal.Dir())
require.NoError(t, err)
require.Equal(t, 1, last)
add(2)
require.NoError(t, h.Truncate(2))
_, last, err = wlog.Segments(wal.Dir())
require.NoError(t, err)
require.Equal(t, 2, last)
}
func TestAddDuplicateLabelName(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
add := func(labels labels.Labels, labelName string) {
app := h.Appender(context.Background())
_, err := app.Append(0, labels, 0, 0)
require.Error(t, err)
require.Equal(t, fmt.Sprintf(`label name "%s" is not unique: invalid sample`, labelName), err.Error())
}
add(labels.FromStrings("a", "c", "a", "b"), "a")
add(labels.FromStrings("a", "c", "a", "c"), "a")
add(labels.FromStrings("__name__", "up", "job", "prometheus", "le", "500", "le", "400", "unit", "s"), "le")
}
func TestMemSeriesIsolation(t *testing.T) {
if defaultIsolationDisabled {
t.Skip("skipping test since tsdb isolation is disabled")
}
// Put a series, select it. GC it and then access it.
lastValue := func(h *Head, maxAppendID uint64) int {
idx, err := h.Index()
require.NoError(t, err)
iso := h.iso.State(math.MinInt64, math.MaxInt64)
iso.maxAppendID = maxAppendID
chunks, err := h.chunksRange(math.MinInt64, math.MaxInt64, iso)
require.NoError(t, err)
// Hm.. here direct block chunk querier might be required?
querier := blockQuerier{
blockBaseQuerier: &blockBaseQuerier{
index: idx,
chunks: chunks,
tombstones: tombstones.NewMemTombstones(),
mint: 0,
maxt: 10000,
},
}
require.NoError(t, err)
defer querier.Close()
ss := querier.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"))
_, seriesSet, ws, err := expandSeriesSet(ss)
require.NoError(t, err)
require.Equal(t, 0, len(ws))
for _, series := range seriesSet {
return int(series[len(series)-1].f)
}
return -1
}
addSamples := func(h *Head) int {
i := 1
for ; i <= 1000; i++ {
var app storage.Appender
// To initialize bounds.
if h.MinTime() == math.MaxInt64 {
app = &initAppender{head: h}
} else {
a := h.appender()
a.cleanupAppendIDsBelow = 0
app = a
}
_, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
require.NoError(t, err)
require.NoError(t, app.Commit())
}
return i
}
testIsolation := func(h *Head, i int) {
}
// Test isolation without restart of Head.
hb, _ := newTestHead(t, 1000, false, false)
i := addSamples(hb)
testIsolation(hb, i)
// Test simple cases in different chunks when no appendID cleanup has been performed.
require.Equal(t, 10, lastValue(hb, 10))
require.Equal(t, 130, lastValue(hb, 130))
require.Equal(t, 160, lastValue(hb, 160))
require.Equal(t, 240, lastValue(hb, 240))
require.Equal(t, 500, lastValue(hb, 500))
require.Equal(t, 750, lastValue(hb, 750))
require.Equal(t, 995, lastValue(hb, 995))
require.Equal(t, 999, lastValue(hb, 999))
// Cleanup appendIDs below 500.
app := hb.appender()
app.cleanupAppendIDsBelow = 500
_, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
require.NoError(t, err)
require.NoError(t, app.Commit())
i++
// We should not get queries with a maxAppendID below 500 after the cleanup,
// but they only take the remaining appendIDs into account.
require.Equal(t, 499, lastValue(hb, 10))
require.Equal(t, 499, lastValue(hb, 130))
require.Equal(t, 499, lastValue(hb, 160))
require.Equal(t, 499, lastValue(hb, 240))
require.Equal(t, 500, lastValue(hb, 500))
require.Equal(t, 995, lastValue(hb, 995))
require.Equal(t, 999, lastValue(hb, 999))
// Cleanup appendIDs below 1000, which means the sample buffer is
// the only thing with appendIDs.
app = hb.appender()
app.cleanupAppendIDsBelow = 1000
_, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 999, lastValue(hb, 998))
require.Equal(t, 999, lastValue(hb, 999))
require.Equal(t, 1000, lastValue(hb, 1000))
require.Equal(t, 1001, lastValue(hb, 1001))
require.Equal(t, 1002, lastValue(hb, 1002))
require.Equal(t, 1002, lastValue(hb, 1003))
i++
// Cleanup appendIDs below 1001, but with a rollback.
app = hb.appender()
app.cleanupAppendIDsBelow = 1001
_, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
require.NoError(t, err)
require.NoError(t, app.Rollback())
require.Equal(t, 1000, lastValue(hb, 999))
require.Equal(t, 1000, lastValue(hb, 1000))
require.Equal(t, 1001, lastValue(hb, 1001))
require.Equal(t, 1002, lastValue(hb, 1002))
require.Equal(t, 1002, lastValue(hb, 1003))
require.NoError(t, hb.Close())
// Test isolation with restart of Head. This is to verify the num samples of chunks after m-map chunk replay.
hb, w := newTestHead(t, 1000, false, false)
i = addSamples(hb)
require.NoError(t, hb.Close())
wal, err := wlog.NewSize(nil, nil, w.Dir(), 32768, false)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = wal.Dir()
hb, err = NewHead(nil, nil, wal, nil, opts, nil)
defer func() { require.NoError(t, hb.Close()) }()
require.NoError(t, err)
require.NoError(t, hb.Init(0))
// No appends after restarting. Hence all should return the last value.
require.Equal(t, 1000, lastValue(hb, 10))
require.Equal(t, 1000, lastValue(hb, 130))
require.Equal(t, 1000, lastValue(hb, 160))
require.Equal(t, 1000, lastValue(hb, 240))
require.Equal(t, 1000, lastValue(hb, 500))
// Cleanup appendIDs below 1000, which means the sample buffer is
// the only thing with appendIDs.
app = hb.appender()
_, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
i++
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, 1001, lastValue(hb, 998))
require.Equal(t, 1001, lastValue(hb, 999))
require.Equal(t, 1001, lastValue(hb, 1000))
require.Equal(t, 1001, lastValue(hb, 1001))
require.Equal(t, 1001, lastValue(hb, 1002))
require.Equal(t, 1001, lastValue(hb, 1003))
// Cleanup appendIDs below 1002, but with a rollback.
app = hb.appender()
_, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i))
require.NoError(t, err)
require.NoError(t, app.Rollback())
require.Equal(t, 1001, lastValue(hb, 999))
require.Equal(t, 1001, lastValue(hb, 1000))
require.Equal(t, 1001, lastValue(hb, 1001))
require.Equal(t, 1001, lastValue(hb, 1002))
require.Equal(t, 1001, lastValue(hb, 1003))
}
func TestIsolationRollback(t *testing.T) {
if defaultIsolationDisabled {
t.Skip("skipping test since tsdb isolation is disabled")
}
// Rollback after a failed append and test if the low watermark has progressed anyway.
hb, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, hb.Close())
}()
app := hb.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, uint64(1), hb.iso.lowWatermark())
app = hb.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 1, 1)
require.NoError(t, err)
_, err = app.Append(0, labels.FromStrings("foo", "bar", "foo", "baz"), 2, 2)
require.Error(t, err)
require.NoError(t, app.Rollback())
require.Equal(t, uint64(2), hb.iso.lowWatermark())
app = hb.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 3, 3)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, uint64(3), hb.iso.lowWatermark(), "Low watermark should proceed to 3 even if append #2 was rolled back.")
}
func TestIsolationLowWatermarkMonotonous(t *testing.T) {
if defaultIsolationDisabled {
t.Skip("skipping test since tsdb isolation is disabled")
}
hb, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, hb.Close())
}()
app1 := hb.Appender(context.Background())
_, err := app1.Append(0, labels.FromStrings("foo", "bar"), 0, 0)
require.NoError(t, err)
require.NoError(t, app1.Commit())
require.Equal(t, uint64(1), hb.iso.lowWatermark(), "Low watermark should by 1 after 1st append.")
app1 = hb.Appender(context.Background())
_, err = app1.Append(0, labels.FromStrings("foo", "bar"), 1, 1)
require.NoError(t, err)
require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Low watermark should be two, even if append is not committed yet.")
app2 := hb.Appender(context.Background())
_, err = app2.Append(0, labels.FromStrings("foo", "baz"), 1, 1)
require.NoError(t, err)
require.NoError(t, app2.Commit())
require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Low watermark should stay two because app1 is not committed yet.")
is := hb.iso.State(math.MinInt64, math.MaxInt64)
require.Equal(t, uint64(2), hb.iso.lowWatermark(), "After simulated read (iso state retrieved), low watermark should stay at 2.")
require.NoError(t, app1.Commit())
require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Even after app1 is committed, low watermark should stay at 2 because read is still ongoing.")
is.Close()
require.Equal(t, uint64(3), hb.iso.lowWatermark(), "After read has finished (iso state closed), low watermark should jump to three.")
}
func TestIsolationAppendIDZeroIsNoop(t *testing.T) {
if defaultIsolationDisabled {
t.Skip("skipping test since tsdb isolation is disabled")
}
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
h.initTime(0)
s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"))
ok, _ := s.append(0, 0, 0, h.chunkDiskMapper, h.chunkRange.Load())
require.True(t, ok, "Series append failed.")
require.Equal(t, 0, s.txs.txIDCount, "Series should not have an appendID after append with appendID=0.")
}
func TestHeadSeriesChunkRace(t *testing.T) {
for i := 0; i < 1000; i++ {
testHeadSeriesChunkRace(t)
}
}
func TestIsolationWithoutAdd(t *testing.T) {
if defaultIsolationDisabled {
t.Skip("skipping test since tsdb isolation is disabled")
}
hb, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, hb.Close())
}()
app := hb.Appender(context.Background())
require.NoError(t, app.Commit())
app = hb.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("foo", "baz"), 1, 1)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, hb.iso.lastAppendID(), hb.iso.lowWatermark(), "High watermark should be equal to the low watermark")
}
func TestOutOfOrderSamplesMetric(t *testing.T) {
dir := t.TempDir()
db, err := Open(dir, nil, nil, DefaultOptions(), nil)
require.NoError(t, err)
defer func() {
require.NoError(t, db.Close())
}()
db.DisableCompactions()
ctx := context.Background()
app := db.Appender(ctx)
for i := 1; i <= 5; i++ {
_, err = app.Append(0, labels.FromStrings("a", "b"), int64(i), 99)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
// Test out of order metric.
require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
app = db.Appender(ctx)
_, err = app.Append(0, labels.FromStrings("a", "b"), 2, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
_, err = app.Append(0, labels.FromStrings("a", "b"), 3, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 2.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
_, err = app.Append(0, labels.FromStrings("a", "b"), 4, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 3.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
require.NoError(t, app.Commit())
// Compact Head to test out of bound metric.
app = db.Appender(ctx)
_, err = app.Append(0, labels.FromStrings("a", "b"), DefaultBlockDuration*2, 99)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Equal(t, int64(math.MinInt64), db.head.minValidTime.Load())
require.NoError(t, db.Compact())
require.Greater(t, db.head.minValidTime.Load(), int64(0))
app = db.Appender(ctx)
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()-2, 99)
require.Equal(t, storage.ErrOutOfBounds, err)
require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat)))
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()-1, 99)
require.Equal(t, storage.ErrOutOfBounds, err)
require.Equal(t, 2.0, prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat)))
require.NoError(t, app.Commit())
// Some more valid samples for out of order.
app = db.Appender(ctx)
for i := 1; i <= 5; i++ {
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()+DefaultBlockDuration+int64(i), 99)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
// Test out of order metric.
app = db.Appender(ctx)
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()+DefaultBlockDuration+2, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 4.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()+DefaultBlockDuration+3, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 5.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
_, err = app.Append(0, labels.FromStrings("a", "b"), db.head.minValidTime.Load()+DefaultBlockDuration+4, 99)
require.Equal(t, storage.ErrOutOfOrderSample, err)
require.Equal(t, 6.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat)))
require.NoError(t, app.Commit())
}
func testHeadSeriesChunkRace(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
require.NoError(t, h.Init(0))
app := h.Appender(context.Background())
s2, err := app.Append(0, labels.FromStrings("foo2", "bar"), 5, 0)
require.NoError(t, err)
for ts := int64(6); ts < 11; ts++ {
_, err = app.Append(s2, labels.EmptyLabels(), ts, 0)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
var wg sync.WaitGroup
matcher := labels.MustNewMatcher(labels.MatchEqual, "", "")
q, err := NewBlockQuerier(h, 18, 22)
require.NoError(t, err)
defer q.Close()
wg.Add(1)
go func() {
h.updateMinMaxTime(20, 25)
h.gc()
wg.Done()
}()
ss := q.Select(false, nil, matcher)
for ss.Next() {
}
require.NoError(t, ss.Err())
wg.Wait()
}
func TestHeadLabelNamesValuesWithMinMaxRange(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, head.Close())
}()
const (
firstSeriesTimestamp int64 = 100
secondSeriesTimestamp int64 = 200
lastSeriesTimestamp int64 = 300
)
var (
seriesTimestamps = []int64{
firstSeriesTimestamp,
secondSeriesTimestamp,
lastSeriesTimestamp,
}
expectedLabelNames = []string{"a", "b", "c"}
expectedLabelValues = []string{"d", "e", "f"}
)
app := head.Appender(context.Background())
for i, name := range expectedLabelNames {
_, err := app.Append(0, labels.FromStrings(name, expectedLabelValues[i]), seriesTimestamps[i], 0)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
require.Equal(t, head.MinTime(), firstSeriesTimestamp)
require.Equal(t, head.MaxTime(), lastSeriesTimestamp)
testCases := []struct {
name string
mint int64
maxt int64
expectedNames []string
expectedValues []string
}{
{"maxt less than head min", head.MaxTime() - 10, head.MinTime() - 10, []string{}, []string{}},
{"mint less than head max", head.MaxTime() + 10, head.MinTime() + 10, []string{}, []string{}},
{"mint and maxt outside head", head.MaxTime() + 10, head.MinTime() - 10, []string{}, []string{}},
{"mint and maxt within head", head.MaxTime() - 10, head.MinTime() + 10, expectedLabelNames, expectedLabelValues},
}
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
headIdxReader := head.indexRange(tt.mint, tt.maxt)
actualLabelNames, err := headIdxReader.LabelNames()
require.NoError(t, err)
require.Equal(t, tt.expectedNames, actualLabelNames)
if len(tt.expectedValues) > 0 {
for i, name := range expectedLabelNames {
actualLabelValue, err := headIdxReader.SortedLabelValues(name)
require.NoError(t, err)
require.Equal(t, []string{tt.expectedValues[i]}, actualLabelValue)
}
}
})
}
}
func TestHeadLabelValuesWithMatchers(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() { require.NoError(t, head.Close()) })
app := head.Appender(context.Background())
for i := 0; i < 100; i++ {
_, err := app.Append(0, labels.FromStrings(
"tens", fmt.Sprintf("value%d", i/10),
"unique", fmt.Sprintf("value%d", i),
), 100, 0)
require.NoError(t, err)
}
require.NoError(t, app.Commit())
testCases := []struct {
name string
labelName string
matchers []*labels.Matcher
expectedValues []string
}{
{
name: "get tens based on unique id",
labelName: "tens",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "unique", "value35")},
expectedValues: []string{"value3"},
}, {
name: "get unique ids based on a ten",
labelName: "unique",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "tens", "value1")},
expectedValues: []string{"value10", "value11", "value12", "value13", "value14", "value15", "value16", "value17", "value18", "value19"},
}, {
name: "get tens by pattern matching on unique id",
labelName: "tens",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "unique", "value[5-7]5")},
expectedValues: []string{"value5", "value6", "value7"},
}, {
name: "get tens by matching for absence of unique label",
labelName: "tens",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "unique", "")},
expectedValues: []string{"value0", "value1", "value2", "value3", "value4", "value5", "value6", "value7", "value8", "value9"},
},
}
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
headIdxReader := head.indexRange(0, 200)
actualValues, err := headIdxReader.SortedLabelValues(tt.labelName, tt.matchers...)
require.NoError(t, err)
require.Equal(t, tt.expectedValues, actualValues)
actualValues, err = headIdxReader.LabelValues(tt.labelName, tt.matchers...)
sort.Strings(actualValues)
require.NoError(t, err)
require.Equal(t, tt.expectedValues, actualValues)
})
}
}
func TestHeadLabelNamesWithMatchers(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, head.Close())
}()
app := head.Appender(context.Background())
for i := 0; i < 100; i++ {
_, err := app.Append(0, labels.FromStrings(
"unique", fmt.Sprintf("value%d", i),
), 100, 0)
require.NoError(t, err)
if i%10 == 0 {
_, err := app.Append(0, labels.FromStrings(
"tens", fmt.Sprintf("value%d", i/10),
"unique", fmt.Sprintf("value%d", i),
), 100, 0)
require.NoError(t, err)
}
if i%20 == 0 {
_, err := app.Append(0, labels.FromStrings(
"tens", fmt.Sprintf("value%d", i/10),
"twenties", fmt.Sprintf("value%d", i/20),
"unique", fmt.Sprintf("value%d", i),
), 100, 0)
require.NoError(t, err)
}
}
require.NoError(t, app.Commit())
testCases := []struct {
name string
labelName string
matchers []*labels.Matcher
expectedNames []string
}{
{
name: "get with non-empty unique: all",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "unique", "")},
expectedNames: []string{"tens", "twenties", "unique"},
}, {
name: "get with unique ending in 1: only unique",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "unique", "value.*1")},
expectedNames: []string{"unique"},
}, {
name: "get with unique = value20: all",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "unique", "value20")},
expectedNames: []string{"tens", "twenties", "unique"},
}, {
name: "get tens = 1: unique & tens",
matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "tens", "value1")},
expectedNames: []string{"tens", "unique"},
},
}
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
headIdxReader := head.indexRange(0, 200)
actualNames, err := headIdxReader.LabelNames(tt.matchers...)
require.NoError(t, err)
require.Equal(t, tt.expectedNames, actualNames)
})
}
}
func TestErrReuseAppender(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, head.Close())
}()
app := head.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("test", "test"), 0, 0)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Error(t, app.Commit())
require.Error(t, app.Rollback())
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("test", "test"), 1, 0)
require.NoError(t, err)
require.NoError(t, app.Rollback())
require.Error(t, app.Rollback())
require.Error(t, app.Commit())
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("test", "test"), 2, 0)
require.NoError(t, err)
require.NoError(t, app.Commit())
require.Error(t, app.Rollback())
require.Error(t, app.Commit())
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("test", "test"), 3, 0)
require.NoError(t, err)
require.NoError(t, app.Rollback())
require.Error(t, app.Commit())
require.Error(t, app.Rollback())
}
func TestHeadMintAfterTruncation(t *testing.T) {
chunkRange := int64(2000)
head, _ := newTestHead(t, chunkRange, false, false)
app := head.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("a", "b"), 100, 100)
require.NoError(t, err)
_, err = app.Append(0, labels.FromStrings("a", "b"), 4000, 200)
require.NoError(t, err)
_, err = app.Append(0, labels.FromStrings("a", "b"), 8000, 300)
require.NoError(t, err)
require.NoError(t, app.Commit())
// Truncating outside the appendable window and actual mint being outside
// appendable window should leave mint at the actual mint.
require.NoError(t, head.Truncate(3500))
require.Equal(t, int64(4000), head.MinTime())
require.Equal(t, int64(4000), head.minValidTime.Load())
// After truncation outside the appendable window if the actual min time
// is in the appendable window then we should leave mint at the start of appendable window.
require.NoError(t, head.Truncate(5000))
require.Equal(t, head.appendableMinValidTime(), head.MinTime())
require.Equal(t, head.appendableMinValidTime(), head.minValidTime.Load())
// If the truncation time is inside the appendable window, then the min time
// should be the truncation time.
require.NoError(t, head.Truncate(7500))
require.Equal(t, int64(7500), head.MinTime())
require.Equal(t, int64(7500), head.minValidTime.Load())
require.NoError(t, head.Close())
}
func TestHeadExemplars(t *testing.T) {
chunkRange := int64(2000)
head, _ := newTestHead(t, chunkRange, false, false)
app := head.Appender(context.Background())
l := labels.FromStrings("traceId", "123")
// It is perfectly valid to add Exemplars before the current start time -
// histogram buckets that haven't been update in a while could still be
// exported exemplars from an hour ago.
ref, err := app.Append(0, labels.FromStrings("a", "b"), 100, 100)
require.NoError(t, err)
_, err = app.AppendExemplar(ref, l, exemplar.Exemplar{
Labels: l,
HasTs: true,
Ts: -1000,
Value: 1,
})
require.NoError(t, err)
require.NoError(t, app.Commit())
require.NoError(t, head.Close())
}
func BenchmarkHeadLabelValuesWithMatchers(b *testing.B) {
chunkRange := int64(2000)
head, _ := newTestHead(b, chunkRange, false, false)
b.Cleanup(func() { require.NoError(b, head.Close()) })
app := head.Appender(context.Background())
metricCount := 1000000
for i := 0; i < metricCount; i++ {
_, err := app.Append(0, labels.FromStrings(
"a_unique", fmt.Sprintf("value%d", i),
"b_tens", fmt.Sprintf("value%d", i/(metricCount/10)),
"c_ninety", fmt.Sprintf("value%d", i/(metricCount/10)/9), // "0" for the first 90%, then "1"
), 100, 0)
require.NoError(b, err)
}
require.NoError(b, app.Commit())
headIdxReader := head.indexRange(0, 200)
matchers := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "c_ninety", "value0")}
b.ResetTimer()
b.ReportAllocs()
for benchIdx := 0; benchIdx < b.N; benchIdx++ {
actualValues, err := headIdxReader.LabelValues("b_tens", matchers...)
require.NoError(b, err)
require.Equal(b, 9, len(actualValues))
}
}
func TestIteratorSeekIntoBuffer(t *testing.T) {
dir := t.TempDir()
// This is usually taken from the Head, but passing manually here.
chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
require.NoError(t, err)
defer func() {
require.NoError(t, chunkDiskMapper.Close())
}()
const chunkRange = 500
s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled, DefaultSamplesPerChunk)
for i := 0; i < 7; i++ {
ok, _ := s.append(int64(i), float64(i), 0, chunkDiskMapper, chunkRange)
require.True(t, ok, "sample append failed")
}
c, _, err := s.chunk(0, chunkDiskMapper, &sync.Pool{
New: func() interface{} {
return &memChunk{}
},
})
require.NoError(t, err)
it := c.chunk.Iterator(nil)
// First point.
require.Equal(t, chunkenc.ValFloat, it.Seek(0))
ts, val := it.At()
require.Equal(t, int64(0), ts)
require.Equal(t, float64(0), val)
// Advance one point.
require.Equal(t, chunkenc.ValFloat, it.Next())
ts, val = it.At()
require.Equal(t, int64(1), ts)
require.Equal(t, float64(1), val)
// Seeking an older timestamp shouldn't cause the iterator to go backwards.
require.Equal(t, chunkenc.ValFloat, it.Seek(0))
ts, val = it.At()
require.Equal(t, int64(1), ts)
require.Equal(t, float64(1), val)
// Seek into the buffer.
require.Equal(t, chunkenc.ValFloat, it.Seek(3))
ts, val = it.At()
require.Equal(t, int64(3), ts)
require.Equal(t, float64(3), val)
// Iterate through the rest of the buffer.
for i := 4; i < 7; i++ {
require.Equal(t, chunkenc.ValFloat, it.Next())
ts, val = it.At()
require.Equal(t, int64(i), ts)
require.Equal(t, float64(i), val)
}
// Run out of elements in the iterator.
require.Equal(t, chunkenc.ValNone, it.Next())
require.Equal(t, chunkenc.ValNone, it.Seek(7))
}
// Tests https://github.com/prometheus/prometheus/issues/8221.
func TestChunkNotFoundHeadGCRace(t *testing.T) {
db := newTestDB(t)
db.DisableCompactions()
var (
app = db.Appender(context.Background())
ref = storage.SeriesRef(0)
mint, maxt = int64(0), int64(0)
err error
)
// Appends samples to span over 1.5 block ranges.
// 7 chunks with 15s scrape interval.
for i := int64(0); i <= 120*7; i++ {
ts := i * DefaultBlockDuration / (4 * 120)
ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i))
require.NoError(t, err)
maxt = ts
}
require.NoError(t, app.Commit())
// Get a querier before compaction (or when compaction is about to begin).
q, err := db.Querier(context.Background(), mint, maxt)
require.NoError(t, err)
// Query the compacted range and get the first series before compaction.
ss := q.Select(true, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.True(t, ss.Next())
s := ss.At()
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
// Compacting head while the querier spans the compaction time.
require.NoError(t, db.Compact())
require.Greater(t, len(db.Blocks()), 0)
}()
// Give enough time for compaction to finish.
// We expect it to be blocked until querier is closed.
<-time.After(3 * time.Second)
// Now consume after compaction when it's gone.
it := s.Iterator(nil)
for it.Next() == chunkenc.ValFloat {
_, _ = it.At()
}
// It should error here without any fix for the mentioned issue.
require.NoError(t, it.Err())
for ss.Next() {
s = ss.At()
it = s.Iterator(it)
for it.Next() == chunkenc.ValFloat {
_, _ = it.At()
}
require.NoError(t, it.Err())
}
require.NoError(t, ss.Err())
require.NoError(t, q.Close())
wg.Wait()
}
// Tests https://github.com/prometheus/prometheus/issues/9079.
func TestDataMissingOnQueryDuringCompaction(t *testing.T) {
db := newTestDB(t)
db.DisableCompactions()
var (
app = db.Appender(context.Background())
ref = storage.SeriesRef(0)
mint, maxt = int64(0), int64(0)
err error
)
// Appends samples to span over 1.5 block ranges.
expSamples := make([]tsdbutil.Sample, 0)
// 7 chunks with 15s scrape interval.
for i := int64(0); i <= 120*7; i++ {
ts := i * DefaultBlockDuration / (4 * 120)
ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i))
require.NoError(t, err)
maxt = ts
expSamples = append(expSamples, sample{ts, float64(i), nil, nil})
}
require.NoError(t, app.Commit())
// Get a querier before compaction (or when compaction is about to begin).
q, err := db.Querier(context.Background(), mint, maxt)
require.NoError(t, err)
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
// Compacting head while the querier spans the compaction time.
require.NoError(t, db.Compact())
require.Greater(t, len(db.Blocks()), 0)
}()
// Give enough time for compaction to finish.
// We expect it to be blocked until querier is closed.
<-time.After(3 * time.Second)
// Querying the querier that was got before compaction.
series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.Equal(t, map[string][]tsdbutil.Sample{`{a="b"}`: expSamples}, series)
wg.Wait()
}
func TestIsQuerierCollidingWithTruncation(t *testing.T) {
db := newTestDB(t)
db.DisableCompactions()
var (
app = db.Appender(context.Background())
ref = storage.SeriesRef(0)
err error
)
for i := int64(0); i <= 3000; i++ {
ref, err = app.Append(ref, labels.FromStrings("a", "b"), i, float64(i))
require.NoError(t, err)
}
require.NoError(t, app.Commit())
// This mocks truncation.
db.head.memTruncationInProcess.Store(true)
db.head.lastMemoryTruncationTime.Store(2000)
// Test that IsQuerierValid suggests correct querier ranges.
cases := []struct {
mint, maxt int64 // For the querier.
expShouldClose, expGetNew bool
expNewMint int64
}{
{-200, -100, true, false, 0},
{-200, 300, true, false, 0},
{100, 1900, true, false, 0},
{1900, 2200, true, true, 2000},
{2000, 2500, false, false, 0},
}
for _, c := range cases {
t.Run(fmt.Sprintf("mint=%d,maxt=%d", c.mint, c.maxt), func(t *testing.T) {
shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(c.mint, c.maxt)
require.Equal(t, c.expShouldClose, shouldClose)
require.Equal(t, c.expGetNew, getNew)
if getNew {
require.Equal(t, c.expNewMint, newMint)
}
})
}
}
func TestWaitForPendingReadersInTimeRange(t *testing.T) {
db := newTestDB(t)
db.DisableCompactions()
sampleTs := func(i int64) int64 { return i * DefaultBlockDuration / (4 * 120) }
var (
app = db.Appender(context.Background())
ref = storage.SeriesRef(0)
err error
)
for i := int64(0); i <= 3000; i++ {
ts := sampleTs(i)
ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i))
require.NoError(t, err)
}
require.NoError(t, app.Commit())
truncMint, truncMaxt := int64(1000), int64(2000)
cases := []struct {
mint, maxt int64
shouldWait bool
}{
{0, 500, false}, // Before truncation range.
{500, 1500, true}, // Overlaps with truncation at the start.
{1200, 1700, true}, // Within truncation range.
{1800, 2500, true}, // Overlaps with truncation at the end.
{2000, 2500, false}, // After truncation range.
{2100, 2500, false}, // After truncation range.
}
for _, c := range cases {
t.Run(fmt.Sprintf("mint=%d,maxt=%d,shouldWait=%t", c.mint, c.maxt, c.shouldWait), func(t *testing.T) {
checkWaiting := func(cl io.Closer) {
var waitOver atomic.Bool
go func() {
db.head.WaitForPendingReadersInTimeRange(truncMint, truncMaxt)
waitOver.Store(true)
}()
<-time.After(550 * time.Millisecond)
require.Equal(t, !c.shouldWait, waitOver.Load())
require.NoError(t, cl.Close())
<-time.After(550 * time.Millisecond)
require.True(t, waitOver.Load())
}
q, err := db.Querier(context.Background(), c.mint, c.maxt)
require.NoError(t, err)
checkWaiting(q)
cq, err := db.ChunkQuerier(context.Background(), c.mint, c.maxt)
require.NoError(t, err)
checkWaiting(cq)
})
}
}
func TestAppendHistogram(t *testing.T) {
l := labels.FromStrings("a", "b")
for _, numHistograms := range []int{1, 10, 150, 200, 250, 300} {
t.Run(fmt.Sprintf("%d", numHistograms), func(t *testing.T) {
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
ingestTs := int64(0)
app := head.Appender(context.Background())
expHistograms := make([]tsdbutil.Sample, 0, 2*numHistograms)
// Counter integer histograms.
for _, h := range tsdbutil.GenerateTestHistograms(numHistograms) {
_, err := app.AppendHistogram(0, l, ingestTs, h, nil)
require.NoError(t, err)
expHistograms = append(expHistograms, sample{t: ingestTs, h: h})
ingestTs++
if ingestTs%50 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
// Gauge integer histograms.
for _, h := range tsdbutil.GenerateTestGaugeHistograms(numHistograms) {
_, err := app.AppendHistogram(0, l, ingestTs, h, nil)
require.NoError(t, err)
expHistograms = append(expHistograms, sample{t: ingestTs, h: h})
ingestTs++
if ingestTs%50 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
expFloatHistograms := make([]tsdbutil.Sample, 0, 2*numHistograms)
// Counter float histograms.
for _, fh := range tsdbutil.GenerateTestFloatHistograms(numHistograms) {
_, err := app.AppendHistogram(0, l, ingestTs, nil, fh)
require.NoError(t, err)
expFloatHistograms = append(expFloatHistograms, sample{t: ingestTs, fh: fh})
ingestTs++
if ingestTs%50 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
// Gauge float histograms.
for _, fh := range tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms) {
_, err := app.AppendHistogram(0, l, ingestTs, nil, fh)
require.NoError(t, err)
expFloatHistograms = append(expFloatHistograms, sample{t: ingestTs, fh: fh})
ingestTs++
if ingestTs%50 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime())
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, q.Close())
})
ss := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.True(t, ss.Next())
s := ss.At()
require.False(t, ss.Next())
it := s.Iterator(nil)
actHistograms := make([]tsdbutil.Sample, 0, len(expHistograms))
actFloatHistograms := make([]tsdbutil.Sample, 0, len(expFloatHistograms))
for typ := it.Next(); typ != chunkenc.ValNone; typ = it.Next() {
switch typ {
case chunkenc.ValHistogram:
ts, h := it.AtHistogram()
actHistograms = append(actHistograms, sample{t: ts, h: h})
case chunkenc.ValFloatHistogram:
ts, fh := it.AtFloatHistogram()
actFloatHistograms = append(actFloatHistograms, sample{t: ts, fh: fh})
}
}
compareSeries(
t,
map[string][]tsdbutil.Sample{"dummy": expHistograms},
map[string][]tsdbutil.Sample{"dummy": actHistograms},
)
compareSeries(
t,
map[string][]tsdbutil.Sample{"dummy": expFloatHistograms},
map[string][]tsdbutil.Sample{"dummy": actFloatHistograms},
)
})
}
}
func TestHistogramInWALAndMmapChunk(t *testing.T) {
head, _ := newTestHead(t, 3000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
// Series with only histograms.
s1 := labels.FromStrings("a", "b1")
k1 := s1.String()
numHistograms := 300
exp := map[string][]tsdbutil.Sample{}
ts := int64(0)
var app storage.Appender
for _, gauge := range []bool{true, false} {
app = head.Appender(context.Background())
var hists []*histogram.Histogram
if gauge {
hists = tsdbutil.GenerateTestGaugeHistograms(numHistograms)
} else {
hists = tsdbutil.GenerateTestHistograms(numHistograms)
}
for _, h := range hists {
h.Count *= 2
h.NegativeSpans = h.PositiveSpans
h.NegativeBuckets = h.PositiveBuckets
_, err := app.AppendHistogram(0, s1, ts, h, nil)
require.NoError(t, err)
exp[k1] = append(exp[k1], sample{t: ts, h: h.Copy()})
ts++
if ts%5 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
for _, gauge := range []bool{true, false} {
app = head.Appender(context.Background())
var hists []*histogram.FloatHistogram
if gauge {
hists = tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms)
} else {
hists = tsdbutil.GenerateTestFloatHistograms(numHistograms)
}
for _, h := range hists {
h.Count *= 2
h.NegativeSpans = h.PositiveSpans
h.NegativeBuckets = h.PositiveBuckets
_, err := app.AppendHistogram(0, s1, ts, nil, h)
require.NoError(t, err)
exp[k1] = append(exp[k1], sample{t: ts, fh: h.Copy()})
ts++
if ts%5 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
// There should be 11 mmap chunks in s1.
ms := head.series.getByHash(s1.Hash(), s1)
require.Len(t, ms.mmappedChunks, 11)
expMmapChunks := make([]*mmappedChunk, 0, 11)
for _, mmap := range ms.mmappedChunks {
require.Greater(t, mmap.numSamples, uint16(0))
cpy := *mmap
expMmapChunks = append(expMmapChunks, &cpy)
}
expHeadChunkSamples := ms.headChunk.chunk.NumSamples()
require.Greater(t, expHeadChunkSamples, 0)
// Series with mix of histograms and float.
s2 := labels.FromStrings("a", "b2")
k2 := s2.String()
ts = 0
for _, gauge := range []bool{true, false} {
app = head.Appender(context.Background())
var hists []*histogram.Histogram
if gauge {
hists = tsdbutil.GenerateTestGaugeHistograms(100)
} else {
hists = tsdbutil.GenerateTestHistograms(100)
}
for _, h := range hists {
ts++
h.Count *= 2
h.NegativeSpans = h.PositiveSpans
h.NegativeBuckets = h.PositiveBuckets
_, err := app.AppendHistogram(0, s2, ts, h, nil)
require.NoError(t, err)
eh := h.Copy()
if !gauge && ts > 30 && (ts-10)%20 == 1 {
// Need "unknown" hint after float sample.
eh.CounterResetHint = histogram.UnknownCounterReset
}
exp[k2] = append(exp[k2], sample{t: ts, h: eh})
if ts%20 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
// Add some float.
for i := 0; i < 10; i++ {
ts++
_, err := app.Append(0, s2, ts, float64(ts))
require.NoError(t, err)
exp[k2] = append(exp[k2], sample{t: ts, f: float64(ts)})
}
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
for _, gauge := range []bool{true, false} {
app = head.Appender(context.Background())
var hists []*histogram.FloatHistogram
if gauge {
hists = tsdbutil.GenerateTestGaugeFloatHistograms(100)
} else {
hists = tsdbutil.GenerateTestFloatHistograms(100)
}
for _, h := range hists {
ts++
h.Count *= 2
h.NegativeSpans = h.PositiveSpans
h.NegativeBuckets = h.PositiveBuckets
_, err := app.AppendHistogram(0, s2, ts, nil, h)
require.NoError(t, err)
eh := h.Copy()
if !gauge && ts > 30 && (ts-10)%20 == 1 {
// Need "unknown" hint after float sample.
eh.CounterResetHint = histogram.UnknownCounterReset
}
exp[k2] = append(exp[k2], sample{t: ts, fh: eh})
if ts%20 == 0 {
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
// Add some float.
for i := 0; i < 10; i++ {
ts++
_, err := app.Append(0, s2, ts, float64(ts))
require.NoError(t, err)
exp[k2] = append(exp[k2], sample{t: ts, f: float64(ts)})
}
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
// Restart head.
require.NoError(t, head.Close())
startHead := func() {
w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
}
startHead()
// Checking contents of s1.
ms = head.series.getByHash(s1.Hash(), s1)
require.Equal(t, expMmapChunks, ms.mmappedChunks)
require.Equal(t, expHeadChunkSamples, ms.headChunk.chunk.NumSamples())
testQuery := func() {
q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime())
require.NoError(t, err)
act := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "a", "b.*"))
compareSeries(t, exp, act)
}
testQuery()
// Restart with no mmap chunks to test WAL replay.
require.NoError(t, head.Close())
require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot)))
startHead()
testQuery()
}
func TestChunkSnapshot(t *testing.T) {
head, _ := newTestHead(t, 120*4, false, false)
defer func() {
head.opts.EnableMemorySnapshotOnShutdown = false
require.NoError(t, head.Close())
}()
type ex struct {
seriesLabels labels.Labels
e exemplar.Exemplar
}
numSeries := 10
expSeries := make(map[string][]tsdbutil.Sample)
expTombstones := make(map[storage.SeriesRef]tombstones.Intervals)
expExemplars := make([]ex, 0)
addExemplar := func(app storage.Appender, ref storage.SeriesRef, lbls labels.Labels, ts int64) {
e := ex{
seriesLabels: lbls,
e: exemplar.Exemplar{
Labels: labels.FromStrings("traceID", fmt.Sprintf("%d", rand.Int())),
Value: rand.Float64(),
Ts: ts,
},
}
expExemplars = append(expExemplars, e)
_, err := app.AppendExemplar(ref, e.seriesLabels, e.e)
require.NoError(t, err)
}
checkSamples := func() {
q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64)
require.NoError(t, err)
series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".*"))
require.Equal(t, expSeries, series)
}
checkTombstones := func() {
tr, err := head.Tombstones()
require.NoError(t, err)
actTombstones := make(map[storage.SeriesRef]tombstones.Intervals)
require.NoError(t, tr.Iter(func(ref storage.SeriesRef, itvs tombstones.Intervals) error {
for _, itv := range itvs {
actTombstones[ref].Add(itv)
}
return nil
}))
require.Equal(t, expTombstones, actTombstones)
}
checkExemplars := func() {
actExemplars := make([]ex, 0, len(expExemplars))
err := head.exemplars.IterateExemplars(func(seriesLabels labels.Labels, e exemplar.Exemplar) error {
actExemplars = append(actExemplars, ex{
seriesLabels: seriesLabels,
e: e,
})
return nil
})
require.NoError(t, err)
// Verifies both existence of right exemplars and order of exemplars in the buffer.
require.Equal(t, expExemplars, actExemplars)
}
var (
wlast, woffset int
err error
)
closeHeadAndCheckSnapshot := func() {
require.NoError(t, head.Close())
_, sidx, soffset, err := LastChunkSnapshot(head.opts.ChunkDirRoot)
require.NoError(t, err)
require.Equal(t, wlast, sidx)
require.Equal(t, woffset, soffset)
}
openHeadAndCheckReplay := func() {
w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
checkSamples()
checkTombstones()
checkExemplars()
}
{ // Initial data that goes into snapshot.
// Add some initial samples with >=1 m-map chunk.
app := head.Appender(context.Background())
for i := 1; i <= numSeries; i++ {
lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i))
lblStr := lbls.String()
// 240 samples should m-map at least 1 chunk.
for ts := int64(1); ts <= 240; ts++ {
val := rand.Float64()
expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil})
ref, err := app.Append(0, lbls, ts, val)
require.NoError(t, err)
// Add an exemplar and to create multiple WAL records.
if ts%10 == 0 {
addExemplar(app, ref, lbls, ts)
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
}
require.NoError(t, app.Commit())
// Add some tombstones.
var enc record.Encoder
for i := 1; i <= numSeries; i++ {
ref := storage.SeriesRef(i)
itvs := tombstones.Intervals{
{Mint: 1234, Maxt: 2345},
{Mint: 3456, Maxt: 4567},
}
for _, itv := range itvs {
expTombstones[ref].Add(itv)
}
head.tombstones.AddInterval(ref, itvs...)
err := head.wal.Log(enc.Tombstones([]tombstones.Stone{
{Ref: ref, Intervals: itvs},
}, nil))
require.NoError(t, err)
}
}
// These references should be the ones used for the snapshot.
wlast, woffset, err = head.wal.LastSegmentAndOffset()
require.NoError(t, err)
if woffset != 0 && woffset < 32*1024 {
// The page is always filled before taking the snapshot.
woffset = 32 * 1024
}
{
// Creating snapshot and verifying it.
head.opts.EnableMemorySnapshotOnShutdown = true
closeHeadAndCheckSnapshot() // This will create a snapshot.
// Test the replay of snapshot.
openHeadAndCheckReplay()
}
{ // Additional data to only include in WAL and m-mapped chunks and not snapshot. This mimics having an old snapshot on disk.
// Add more samples.
app := head.Appender(context.Background())
for i := 1; i <= numSeries; i++ {
lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i))
lblStr := lbls.String()
// 240 samples should m-map at least 1 chunk.
for ts := int64(241); ts <= 480; ts++ {
val := rand.Float64()
expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil})
ref, err := app.Append(0, lbls, ts, val)
require.NoError(t, err)
// Add an exemplar and to create multiple WAL records.
if ts%10 == 0 {
addExemplar(app, ref, lbls, ts)
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
}
}
}
require.NoError(t, app.Commit())
// Add more tombstones.
var enc record.Encoder
for i := 1; i <= numSeries; i++ {
ref := storage.SeriesRef(i)
itvs := tombstones.Intervals{
{Mint: 12345, Maxt: 23456},
{Mint: 34567, Maxt: 45678},
}
for _, itv := range itvs {
expTombstones[ref].Add(itv)
}
head.tombstones.AddInterval(ref, itvs...)
err := head.wal.Log(enc.Tombstones([]tombstones.Stone{
{Ref: ref, Intervals: itvs},
}, nil))
require.NoError(t, err)
}
}
{
// Close Head and verify that new snapshot was not created.
head.opts.EnableMemorySnapshotOnShutdown = false
closeHeadAndCheckSnapshot() // This should not create a snapshot.
// Test the replay of snapshot, m-map chunks, and WAL.
head.opts.EnableMemorySnapshotOnShutdown = true // Enabled to read from snapshot.
openHeadAndCheckReplay()
}
// Creating another snapshot should delete the older snapshot and replay still works fine.
wlast, woffset, err = head.wal.LastSegmentAndOffset()
require.NoError(t, err)
if woffset != 0 && woffset < 32*1024 {
// The page is always filled before taking the snapshot.
woffset = 32 * 1024
}
{
// Close Head and verify that new snapshot was created.
closeHeadAndCheckSnapshot()
// Verify that there is only 1 snapshot.
files, err := os.ReadDir(head.opts.ChunkDirRoot)
require.NoError(t, err)
snapshots := 0
for i := len(files) - 1; i >= 0; i-- {
fi := files[i]
if strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) {
snapshots++
require.Equal(t, chunkSnapshotDir(wlast, woffset), fi.Name())
}
}
require.Equal(t, 1, snapshots)
// Test the replay of snapshot.
head.opts.EnableMemorySnapshotOnShutdown = true // Enabled to read from snapshot.
// Disabling exemplars to check that it does not hard fail replay
// https://github.com/prometheus/prometheus/issues/9437#issuecomment-933285870.
head.opts.EnableExemplarStorage = false
head.opts.MaxExemplars.Store(0)
expExemplars = expExemplars[:0]
openHeadAndCheckReplay()
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal))
}
}
func TestSnapshotError(t *testing.T) {
head, _ := newTestHead(t, 120*4, false, false)
defer func() {
head.opts.EnableMemorySnapshotOnShutdown = false
require.NoError(t, head.Close())
}()
// Add a sample.
app := head.Appender(context.Background())
lbls := labels.FromStrings("foo", "bar")
_, err := app.Append(0, lbls, 99, 99)
require.NoError(t, err)
require.NoError(t, app.Commit())
// Add some tombstones.
itvs := tombstones.Intervals{
{Mint: 1234, Maxt: 2345},
{Mint: 3456, Maxt: 4567},
}
head.tombstones.AddInterval(1, itvs...)
// Check existence of data.
require.NotNil(t, head.series.getByHash(lbls.Hash(), lbls))
tm, err := head.tombstones.Get(1)
require.NoError(t, err)
require.NotEqual(t, 0, len(tm))
head.opts.EnableMemorySnapshotOnShutdown = true
require.NoError(t, head.Close()) // This will create a snapshot.
// Remove the WAL so that we don't load from it.
require.NoError(t, os.RemoveAll(head.wal.Dir()))
// Corrupt the snapshot.
snapDir, _, _, err := LastChunkSnapshot(head.opts.ChunkDirRoot)
require.NoError(t, err)
files, err := os.ReadDir(snapDir)
require.NoError(t, err)
f, err := os.OpenFile(path.Join(snapDir, files[0].Name()), os.O_RDWR, 0)
require.NoError(t, err)
_, err = f.WriteAt([]byte{0b11111111}, 18)
require.NoError(t, err)
require.NoError(t, f.Close())
// Create new Head which should replay this snapshot.
w, err := wlog.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, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
// There should be no series in the memory after snapshot error since WAL was removed.
require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal))
require.Nil(t, head.series.getByHash(lbls.Hash(), lbls))
tm, err = head.tombstones.Get(1)
require.NoError(t, err)
require.Equal(t, 0, len(tm))
}
func TestHistogramMetrics(t *testing.T) {
numHistograms := 10
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
expHSeries, expHSamples := 0, 0
for x := 0; x < 5; x++ {
expHSeries++
l := labels.FromStrings("a", fmt.Sprintf("b%d", x))
for i, h := range tsdbutil.GenerateTestHistograms(numHistograms) {
app := head.Appender(context.Background())
_, err := app.AppendHistogram(0, l, int64(i), h, nil)
require.NoError(t, err)
require.NoError(t, app.Commit())
expHSamples++
}
for i, fh := range tsdbutil.GenerateTestFloatHistograms(numHistograms) {
app := head.Appender(context.Background())
_, err := app.AppendHistogram(0, l, int64(numHistograms+i), nil, fh)
require.NoError(t, err)
require.NoError(t, app.Commit())
expHSamples++
}
}
require.Equal(t, float64(expHSamples), prom_testutil.ToFloat64(head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram)))
require.NoError(t, head.Close())
w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
require.Equal(t, float64(0), prom_testutil.ToFloat64(head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram))) // Counter reset.
}
func TestHistogramStaleSample(t *testing.T) {
t.Run("integer histogram", func(t *testing.T) {
testHistogramStaleSampleHelper(t, false)
})
t.Run("float histogram", func(t *testing.T) {
testHistogramStaleSampleHelper(t, true)
})
}
func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) {
t.Helper()
l := labels.FromStrings("a", "b")
numHistograms := 20
head, _ := newTestHead(t, 100000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
type timedHistogram struct {
t int64
h *histogram.Histogram
fh *histogram.FloatHistogram
}
expHistograms := make([]timedHistogram, 0, numHistograms)
testQuery := func(numStale int) {
q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime())
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, q.Close())
})
ss := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.True(t, ss.Next())
s := ss.At()
require.False(t, ss.Next())
it := s.Iterator(nil)
actHistograms := make([]timedHistogram, 0, len(expHistograms))
for typ := it.Next(); typ != chunkenc.ValNone; typ = it.Next() {
switch typ {
case chunkenc.ValHistogram:
t, h := it.AtHistogram()
actHistograms = append(actHistograms, timedHistogram{t: t, h: h})
case chunkenc.ValFloatHistogram:
t, h := it.AtFloatHistogram()
actHistograms = append(actHistograms, timedHistogram{t: t, fh: h})
}
}
// We cannot compare StaleNAN with require.Equal, hence checking each histogram manually.
require.Equal(t, len(expHistograms), len(actHistograms))
actNumStale := 0
for i, eh := range expHistograms {
ah := actHistograms[i]
if floatHistogram {
switch {
case value.IsStaleNaN(eh.fh.Sum):
actNumStale++
require.True(t, value.IsStaleNaN(ah.fh.Sum))
// To make require.Equal work.
ah.fh.Sum = 0
eh.fh = eh.fh.Copy()
eh.fh.Sum = 0
case i > 0:
prev := expHistograms[i-1]
if prev.fh == nil || value.IsStaleNaN(prev.fh.Sum) {
eh.fh.CounterResetHint = histogram.UnknownCounterReset
}
}
require.Equal(t, eh, ah)
} else {
switch {
case value.IsStaleNaN(eh.h.Sum):
actNumStale++
require.True(t, value.IsStaleNaN(ah.h.Sum))
// To make require.Equal work.
ah.h.Sum = 0
eh.h = eh.h.Copy()
eh.h.Sum = 0
case i > 0:
prev := expHistograms[i-1]
if prev.h == nil || value.IsStaleNaN(prev.h.Sum) {
eh.h.CounterResetHint = histogram.UnknownCounterReset
}
}
require.Equal(t, eh, ah)
}
}
require.Equal(t, numStale, actNumStale)
}
// Adding stale in the same appender.
app := head.Appender(context.Background())
for _, h := range tsdbutil.GenerateTestHistograms(numHistograms) {
var err error
if floatHistogram {
_, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), nil, h.ToFloat())
expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), fh: h.ToFloat()})
} else {
_, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h, nil)
expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), h: h})
}
require.NoError(t, err)
}
// +1 so that delta-of-delta is not 0.
_, err := app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN))
require.NoError(t, err)
if floatHistogram {
expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}})
} else {
expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, h: &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}})
}
require.NoError(t, app.Commit())
// Only 1 chunk in the memory, no m-mapped chunk.
s := head.series.getByHash(l.Hash(), l)
require.NotNil(t, s)
require.Equal(t, 0, len(s.mmappedChunks))
testQuery(1)
// Adding stale in different appender and continuing series after a stale sample.
app = head.Appender(context.Background())
for _, h := range tsdbutil.GenerateTestHistograms(2 * numHistograms)[numHistograms:] {
var err error
if floatHistogram {
_, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), nil, h.ToFloat())
expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), fh: h.ToFloat()})
} else {
_, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h, nil)
expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), h: h})
}
require.NoError(t, err)
}
require.NoError(t, app.Commit())
app = head.Appender(context.Background())
// +1 so that delta-of-delta is not 0.
_, err = app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN))
require.NoError(t, err)
if floatHistogram {
expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}})
} else {
expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, h: &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}})
}
require.NoError(t, app.Commit())
// Total 2 chunks, 1 m-mapped.
s = head.series.getByHash(l.Hash(), l)
require.NotNil(t, s)
require.Equal(t, 1, len(s.mmappedChunks))
testQuery(2)
}
func TestHistogramCounterResetHeader(t *testing.T) {
for _, floatHisto := range []bool{true, false} {
t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) {
l := labels.FromStrings("a", "b")
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
ts := int64(0)
appendHistogram := func(h *histogram.Histogram) {
ts++
app := head.Appender(context.Background())
var err error
if floatHisto {
_, err = app.AppendHistogram(0, l, ts, nil, h.ToFloat())
} else {
_, err = app.AppendHistogram(0, l, ts, h.Copy(), nil)
}
require.NoError(t, err)
require.NoError(t, app.Commit())
}
var expHeaders []chunkenc.CounterResetHeader
checkExpCounterResetHeader := func(newHeaders ...chunkenc.CounterResetHeader) {
expHeaders = append(expHeaders, newHeaders...)
ms, _, err := head.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.Len(t, ms.mmappedChunks, len(expHeaders)-1) // One is the head chunk.
for i, mmapChunk := range ms.mmappedChunks {
chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref)
require.NoError(t, err)
if floatHisto {
require.Equal(t, expHeaders[i], chk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader())
} else {
require.Equal(t, expHeaders[i], chk.(*chunkenc.HistogramChunk).GetCounterResetHeader())
}
}
if floatHisto {
require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunk.chunk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader())
} else {
require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunk.chunk.(*chunkenc.HistogramChunk).GetCounterResetHeader())
}
}
h := tsdbutil.GenerateTestHistograms(1)[0]
h.PositiveBuckets = []int64{100, 1, 1, 1}
h.NegativeBuckets = []int64{100, 1, 1, 1}
h.Count = 1000
// First histogram is UnknownCounterReset.
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.UnknownCounterReset)
// Another normal histogram.
h.Count++
appendHistogram(h)
checkExpCounterResetHeader()
// Counter reset via Count.
h.Count--
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.CounterReset)
// Add 2 non-counter reset histograms.
for i := 0; i < 250; i++ {
appendHistogram(h)
}
checkExpCounterResetHeader(chunkenc.NotCounterReset, chunkenc.NotCounterReset)
// Changing schema will cut a new chunk with unknown counter reset.
h.Schema++
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.UnknownCounterReset)
// Changing schema will zero threshold a new chunk with unknown counter reset.
h.ZeroThreshold += 0.01
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.UnknownCounterReset)
// Counter reset by removing a positive bucket.
h.PositiveSpans[1].Length--
h.PositiveBuckets = h.PositiveBuckets[1:]
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.CounterReset)
// Counter reset by removing a negative bucket.
h.NegativeSpans[1].Length--
h.NegativeBuckets = h.NegativeBuckets[1:]
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.CounterReset)
// Add 2 non-counter reset histograms. Just to have some non-counter reset chunks in between.
for i := 0; i < 250; i++ {
appendHistogram(h)
}
checkExpCounterResetHeader(chunkenc.NotCounterReset, chunkenc.NotCounterReset)
// Counter reset with counter reset in a positive bucket.
h.PositiveBuckets[len(h.PositiveBuckets)-1]--
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.CounterReset)
// Counter reset with counter reset in a negative bucket.
h.NegativeBuckets[len(h.NegativeBuckets)-1]--
appendHistogram(h)
checkExpCounterResetHeader(chunkenc.CounterReset)
})
}
}
func TestAppendingDifferentEncodingToSameSeries(t *testing.T) {
dir := t.TempDir()
opts := DefaultOptions()
opts.EnableNativeHistograms = true
db, err := Open(dir, nil, nil, opts, nil)
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, db.Close())
})
db.DisableCompactions()
hists := tsdbutil.GenerateTestHistograms(10)
floatHists := tsdbutil.GenerateTestFloatHistograms(10)
lbls := labels.FromStrings("a", "b")
var expResult []tsdbutil.Sample
checkExpChunks := func(count int) {
ms, created, err := db.Head().getOrCreate(lbls.Hash(), lbls)
require.NoError(t, err)
require.False(t, created)
require.NotNil(t, ms)
require.Len(t, ms.mmappedChunks, count-1) // One will be the head chunk.
}
appends := []struct {
samples []tsdbutil.Sample
expChunks int
err error
// If this is empty, samples above will be taken instead of this.
addToExp []tsdbutil.Sample
}{
// Histograms that end up in the expected samples are copied here so that we
// can independently set the CounterResetHint later.
{
samples: []tsdbutil.Sample{sample{t: 100, h: hists[0].Copy()}},
expChunks: 1,
},
{
samples: []tsdbutil.Sample{sample{t: 200, f: 2}},
expChunks: 2,
},
{
samples: []tsdbutil.Sample{sample{t: 210, fh: floatHists[0].Copy()}},
expChunks: 3,
},
{
samples: []tsdbutil.Sample{sample{t: 220, h: hists[1].Copy()}},
expChunks: 4,
},
{
samples: []tsdbutil.Sample{sample{t: 230, fh: floatHists[3].Copy()}},
expChunks: 5,
},
{
samples: []tsdbutil.Sample{sample{t: 100, h: hists[2].Copy()}},
err: storage.ErrOutOfOrderSample,
},
{
samples: []tsdbutil.Sample{sample{t: 300, h: hists[3].Copy()}},
expChunks: 6,
},
{
samples: []tsdbutil.Sample{sample{t: 100, f: 2}},
err: storage.ErrOutOfOrderSample,
},
{
samples: []tsdbutil.Sample{sample{t: 100, fh: floatHists[4].Copy()}},
err: storage.ErrOutOfOrderSample,
},
{
// Combination of histograms and float64 in the same commit. The behaviour is undefined, but we want to also
// verify how TSDB would behave. Here the histogram is appended at the end, hence will be considered as out of order.
samples: []tsdbutil.Sample{
sample{t: 400, f: 4},
sample{t: 500, h: hists[5]}, // This won't be committed.
sample{t: 600, f: 6},
},
addToExp: []tsdbutil.Sample{
sample{t: 400, f: 4},
sample{t: 600, f: 6},
},
expChunks: 7, // Only 1 new chunk for float64.
},
{
// Here the histogram is appended at the end, hence the first histogram is out of order.
samples: []tsdbutil.Sample{
sample{t: 700, h: hists[7]}, // Out of order w.r.t. the next float64 sample that is appended first.
sample{t: 800, f: 8},
sample{t: 900, h: hists[9]},
},
addToExp: []tsdbutil.Sample{
sample{t: 800, f: 8},
sample{t: 900, h: hists[9].Copy()},
},
expChunks: 8, // float64 added to old chunk, only 1 new for histograms.
},
{
// Float histogram is appended at the end.
samples: []tsdbutil.Sample{
sample{t: 1000, fh: floatHists[7]}, // Out of order w.r.t. the next histogram.
sample{t: 1100, h: hists[9]},
},
addToExp: []tsdbutil.Sample{
sample{t: 1100, h: hists[9].Copy()},
},
expChunks: 8,
},
}
for _, a := range appends {
app := db.Appender(context.Background())
for _, s := range a.samples {
var err error
if s.H() != nil || s.FH() != nil {
_, err = app.AppendHistogram(0, lbls, s.T(), s.H(), s.FH())
} else {
_, err = app.Append(0, lbls, s.T(), s.F())
}
require.Equal(t, a.err, err)
}
if a.err == nil {
require.NoError(t, app.Commit())
if len(a.addToExp) > 0 {
expResult = append(expResult, a.addToExp...)
} else {
expResult = append(expResult, a.samples...)
}
checkExpChunks(a.expChunks)
} else {
require.NoError(t, app.Rollback())
}
}
for i, s := range expResult[1:] {
switch {
case s.H() != nil && expResult[i].H() == nil:
s.(sample).h.CounterResetHint = histogram.UnknownCounterReset
case s.FH() != nil && expResult[i].FH() == nil:
s.(sample).fh.CounterResetHint = histogram.UnknownCounterReset
}
}
// Query back and expect same order of samples.
q, err := db.Querier(context.Background(), math.MinInt64, math.MaxInt64)
require.NoError(t, err)
series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
require.Equal(t, map[string][]tsdbutil.Sample{lbls.String(): expResult}, series)
}
// Tests https://github.com/prometheus/prometheus/issues/9725.
func TestChunkSnapshotReplayBug(t *testing.T) {
dir := t.TempDir()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
// Write few series records and samples such that the series references are not in order in the WAL
// for status_code="200".
var buf []byte
for i := 1; i <= 1000; i++ {
var ref chunks.HeadSeriesRef
if i <= 500 {
ref = chunks.HeadSeriesRef(i * 100)
} else {
ref = chunks.HeadSeriesRef((i - 500) * 50)
}
seriesRec := record.RefSeries{
Ref: ref,
Labels: labels.FromStrings(
"__name__", "request_duration",
"status_code", "200",
"foo", fmt.Sprintf("baz%d", rand.Int()),
),
}
// Add a sample so that the series is not garbage collected.
samplesRec := record.RefSample{Ref: ref, T: 1000, V: 1000}
var enc record.Encoder
rec := enc.Series([]record.RefSeries{seriesRec}, buf)
buf = rec[:0]
require.NoError(t, wal.Log(rec))
rec = enc.Samples([]record.RefSample{samplesRec}, buf)
buf = rec[:0]
require.NoError(t, wal.Log(rec))
}
// Write a corrupt snapshot to fail the replay on startup.
snapshotName := chunkSnapshotDir(0, 100)
cpdir := filepath.Join(dir, snapshotName)
require.NoError(t, os.MkdirAll(cpdir, 0o777))
err = os.WriteFile(filepath.Join(cpdir, "00000000"), []byte{1, 5, 3, 5, 6, 7, 4, 2, 2}, 0o777)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.EnableMemorySnapshotOnShutdown = true
head, err := NewHead(nil, nil, wal, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
defer func() {
require.NoError(t, head.Close())
}()
// Snapshot replay should error out.
require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal))
// Querying `request_duration{status_code!="200"}` should return no series since all of
// them have status_code="200".
q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64)
require.NoError(t, err)
series := query(t, q,
labels.MustNewMatcher(labels.MatchEqual, "__name__", "request_duration"),
labels.MustNewMatcher(labels.MatchNotEqual, "status_code", "200"),
)
require.Len(t, series, 0, "there should be no series found")
}
func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) {
dir := t.TempDir()
wlTemp, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
// Write a snapshot with .tmp suffix. This used to fail taking any further snapshots or replay of snapshots.
snapshotName := chunkSnapshotDir(0, 100) + ".tmp"
cpdir := filepath.Join(dir, snapshotName)
require.NoError(t, os.MkdirAll(cpdir, 0o777))
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.EnableMemorySnapshotOnShutdown = true
head, err := NewHead(nil, nil, wlTemp, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal))
// Add some samples for the snapshot.
app := head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 10, 10)
require.NoError(t, err)
require.NoError(t, app.Commit())
// Should not return any error for a successful snapshot.
require.NoError(t, head.Close())
// Verify the snapshot.
name, idx, offset, err := LastChunkSnapshot(dir)
require.NoError(t, err)
require.True(t, name != "")
require.Equal(t, 0, idx)
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()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = dir
opts.OutOfOrderTimeWindow.Store(30 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wal, 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, f: 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())
wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
h, err = NewHead(nil, nil, wal, 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.ooo.oooHeadChunk.chunk.ToXOR()
require.NoError(t, err)
it := xor.Iterator(nil)
actOOOSamples := make([]sample, 0, len(expOOOSamples))
for it.Next() == chunkenc.ValFloat {
ts, v := it.At()
actOOOSamples = append(actOOOSamples, sample{t: ts, f: 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()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = 1000
opts.ChunkDirRoot = dir
opts.OutOfOrderCapMax.Store(30)
opts.OutOfOrderTimeWindow.Store(1000 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wal, 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.ooo.oooMmappedChunks, 3)
// Verify that we can access the chunks without error.
for _, m := range ms.ooo.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.ooo.oooMmappedChunks)
// Restart head.
require.NoError(t, h.Close())
wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
h, err = NewHead(nil, nil, wal, 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.ooo.oooMmappedChunks, len(expMmapChunks))
// Verify that we can access the chunks without error.
for _, m := range ms.ooo.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.ooo.oooMmappedChunks)
require.Equal(t, expMmapChunks, actMmapChunks)
require.NoError(t, h.Close())
}
func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) {
h, _ := newTestHead(t, 1000, false, false)
defer func() {
require.NoError(t, h.Close())
}()
require.NoError(t, h.Init(0))
ctx := context.Background()
app := h.Appender(ctx)
seriesLabels := labels.FromStrings("a", "1")
var seriesRef storage.SeriesRef
var err error
for i := 0; i < 400; i++ {
seriesRef, err = app.Append(0, seriesLabels, int64(i), float64(i))
require.NoError(t, err)
}
require.NoError(t, app.Commit())
require.Greater(t, prom_testutil.ToFloat64(h.metrics.chunksCreated), 1.0)
uc := newUnsupportedChunk()
// Make this chunk not overlap with the previous and the next
h.chunkDiskMapper.WriteChunk(chunks.HeadSeriesRef(seriesRef), 500, 600, uc, false, func(err error) { require.NoError(t, err) })
app = h.Appender(ctx)
for i := 700; i < 1200; i++ {
_, err := app.Append(0, seriesLabels, int64(i), float64(i))
require.NoError(t, err)
}
require.NoError(t, app.Commit())
require.Greater(t, prom_testutil.ToFloat64(h.metrics.chunksCreated), 4.0)
series, created, err := h.getOrCreate(seriesLabels.Hash(), seriesLabels)
require.NoError(t, err)
require.False(t, created, "should already exist")
require.NotNil(t, series, "should return the series we created above")
expChunks := make([]*mmappedChunk, len(series.mmappedChunks))
copy(expChunks, series.mmappedChunks)
require.NoError(t, h.Close())
wal, err := wlog.NewSize(nil, nil, filepath.Join(h.opts.ChunkDirRoot, "wal"), 32768, false)
require.NoError(t, err)
h, err = NewHead(nil, nil, wal, nil, h.opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
series, created, err = h.getOrCreate(seriesLabels.Hash(), seriesLabels)
require.NoError(t, err)
require.False(t, created, "should already exist")
require.NotNil(t, series, "should return the series we created above")
require.Equal(t, expChunks, series.mmappedChunks)
}
const (
UnsupportedMask = 0b10000000
EncUnsupportedXOR = chunkenc.EncXOR | UnsupportedMask
)
// unsupportedChunk holds a XORChunk and overrides the Encoding() method.
type unsupportedChunk struct {
*chunkenc.XORChunk
}
func newUnsupportedChunk() *unsupportedChunk {
return &unsupportedChunk{chunkenc.NewXORChunk()}
}
func (c *unsupportedChunk) Encoding() chunkenc.Encoding {
return EncUnsupportedXOR
}
// Tests https://github.com/prometheus/prometheus/issues/10277.
func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) {
dir := t.TempDir()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, false)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = DefaultBlockDuration
opts.ChunkDirRoot = dir
opts.EnableExemplarStorage = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
h, err := NewHead(nil, nil, wal, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
lastTs := int64(0)
var ref storage.SeriesRef
lbls := labels.FromStrings("__name__", "testing", "foo", "bar")
addChunks := func() {
interval := DefaultBlockDuration / (4 * 120)
app := h.Appender(context.Background())
for i := 0; i < 250; i++ {
ref, err = app.Append(ref, lbls, lastTs, float64(lastTs))
lastTs += interval
if i%10 == 0 {
require.NoError(t, app.Commit())
app = h.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
addChunks()
require.NoError(t, h.Close())
wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, false)
require.NoError(t, err)
mmapFilePath := filepath.Join(dir, "chunks_head", "000001")
f, err := os.OpenFile(mmapFilePath, os.O_WRONLY, 0o666)
require.NoError(t, err)
_, err = f.WriteAt([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, 17)
require.NoError(t, err)
require.NoError(t, f.Close())
h, err = NewHead(nil, nil, wal, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
addChunks()
require.NoError(t, h.Close())
}
// Tests https://github.com/prometheus/prometheus/issues/10277.
func TestReplayAfterMmapReplayError(t *testing.T) {
dir := t.TempDir()
var h *Head
var err error
openHead := func() {
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, false)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkRange = DefaultBlockDuration
opts.ChunkDirRoot = dir
opts.EnableMemorySnapshotOnShutdown = true
opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars)
h, err = NewHead(nil, nil, wal, nil, opts, nil)
require.NoError(t, err)
require.NoError(t, h.Init(0))
}
openHead()
itvl := int64(15 * time.Second / time.Millisecond)
lastTs := int64(0)
lbls := labels.FromStrings("__name__", "testing", "foo", "bar")
var expSamples []tsdbutil.Sample
addSamples := func(numSamples int) {
app := h.Appender(context.Background())
var ref storage.SeriesRef
for i := 0; i < numSamples; i++ {
ref, err = app.Append(ref, lbls, lastTs, float64(lastTs))
expSamples = append(expSamples, sample{t: lastTs, f: float64(lastTs)})
require.NoError(t, err)
lastTs += itvl
if i%10 == 0 {
require.NoError(t, app.Commit())
app = h.Appender(context.Background())
}
}
require.NoError(t, app.Commit())
}
// Creating multiple m-map files.
for i := 0; i < 5; i++ {
addSamples(250)
require.NoError(t, h.Close())
if i != 4 {
// Don't open head for the last iteration.
openHead()
}
}
files, err := os.ReadDir(filepath.Join(dir, "chunks_head"))
require.Equal(t, 5, len(files))
// Corrupt a m-map file.
mmapFilePath := filepath.Join(dir, "chunks_head", "000002")
f, err := os.OpenFile(mmapFilePath, os.O_WRONLY, 0o666)
require.NoError(t, err)
_, err = f.WriteAt([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, 17)
require.NoError(t, err)
require.NoError(t, f.Close())
openHead()
// There should be less m-map files due to corruption.
files, err = os.ReadDir(filepath.Join(dir, "chunks_head"))
require.Equal(t, 2, len(files))
// Querying should not panic.
q, err := NewBlockQuerier(h, 0, lastTs)
require.NoError(t, err)
res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "__name__", "testing"))
require.Equal(t, map[string][]tsdbutil.Sample{lbls.String(): expSamples}, res)
require.NoError(t, h.Close())
}
func TestHistogramValidation(t *testing.T) {
tests := map[string]struct {
h *histogram.Histogram
errMsg string
errMsgFloat string // To be considered for float histogram only if it is non-empty.
}{
"valid histogram": {
h: tsdbutil.GenerateTestHistograms(1)[0],
},
"rejects histogram who has too few negative buckets": {
h: &histogram.Histogram{
NegativeSpans: []histogram.Span{{Offset: 0, Length: 1}},
NegativeBuckets: []int64{},
},
errMsg: `negative side: spans need 1 buckets, have 0 buckets`,
},
"rejects histogram who has too few positive buckets": {
h: &histogram.Histogram{
PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}},
PositiveBuckets: []int64{},
},
errMsg: `positive side: spans need 1 buckets, have 0 buckets`,
},
"rejects histogram who has too many negative buckets": {
h: &histogram.Histogram{
NegativeSpans: []histogram.Span{{Offset: 0, Length: 1}},
NegativeBuckets: []int64{1, 2},
},
errMsg: `negative side: spans need 1 buckets, have 2 buckets`,
},
"rejects histogram who has too many positive buckets": {
h: &histogram.Histogram{
PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}},
PositiveBuckets: []int64{1, 2},
},
errMsg: `positive side: spans need 1 buckets, have 2 buckets`,
},
"rejects a histogram which has a negative span with a negative offset": {
h: &histogram.Histogram{
NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}, {Offset: -1, Length: 1}},
NegativeBuckets: []int64{1, 2},
},
errMsg: `negative side: span number 2 with offset -1`,
},
"rejects a histogram which has a positive span with a negative offset": {
h: &histogram.Histogram{
PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}, {Offset: -1, Length: 1}},
PositiveBuckets: []int64{1, 2},
},
errMsg: `positive side: span number 2 with offset -1`,
},
"rejects a histogram which has a negative bucket with a negative count": {
h: &histogram.Histogram{
NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}},
NegativeBuckets: []int64{-1},
},
errMsg: `negative side: bucket number 1 has observation count of -1`,
},
"rejects a histogram which has a positive bucket with a negative count": {
h: &histogram.Histogram{
PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}},
PositiveBuckets: []int64{-1},
},
errMsg: `positive side: bucket number 1 has observation count of -1`,
},
"rejects a histogram which which has a lower count than count in buckets": {
h: &histogram.Histogram{
Count: 0,
NegativeSpans: []histogram.Span{{Offset: -1, Length: 1}},
PositiveSpans: []histogram.Span{{Offset: -1, Length: 1}},
NegativeBuckets: []int64{1},
PositiveBuckets: []int64{1},
},
errMsg: `2 observations found in buckets, but the Count field is 0`,
errMsgFloat: `2.000000 observations found in buckets, but the Count field is 0.000000`,
},
}
for testName, tc := range tests {
t.Run(testName, func(t *testing.T) {
switch err := ValidateHistogram(tc.h); {
case tc.errMsg != "":
require.ErrorContains(t, err, tc.errMsg)
default:
require.NoError(t, err)
}
switch err := ValidateFloatHistogram(tc.h.ToFloat()); {
case tc.errMsgFloat != "":
require.ErrorContains(t, err, tc.errMsgFloat)
case tc.errMsg != "":
require.ErrorContains(t, err, tc.errMsg)
default:
require.NoError(t, err)
}
})
}
}
func BenchmarkHistogramValidation(b *testing.B) {
histograms := generateBigTestHistograms(b.N)
b.ResetTimer()
for _, h := range histograms {
require.NoError(b, ValidateHistogram(h))
}
}
func generateBigTestHistograms(n int) []*histogram.Histogram {
const numBuckets = 500
numSpans := numBuckets / 10
bucketsPerSide := numBuckets / 2
spanLength := uint32(bucketsPerSide / numSpans)
// Given all bucket deltas are 1, sum n + 1.
observationCount := numBuckets / 2 * (1 + numBuckets)
var histograms []*histogram.Histogram
for i := 0; i < n; i++ {
h := &histogram.Histogram{
Count: uint64(i + observationCount),
ZeroCount: uint64(i),
ZeroThreshold: 1e-128,
Sum: 18.4 * float64(i+1),
Schema: 2,
NegativeSpans: make([]histogram.Span, numSpans),
PositiveSpans: make([]histogram.Span, numSpans),
NegativeBuckets: make([]int64, bucketsPerSide),
PositiveBuckets: make([]int64, bucketsPerSide),
}
for j := 0; j < numSpans; j++ {
s := histogram.Span{Offset: 1 + int32(i), Length: spanLength}
h.NegativeSpans[j] = s
h.PositiveSpans[j] = s
}
for j := 0; j < bucketsPerSide; j++ {
h.NegativeBuckets[j] = 1
h.PositiveBuckets[j] = 1
}
histograms = append(histograms, h)
}
return histograms
}
func TestOOOAppendWithNoSeries(t *testing.T) {
dir := t.TempDir()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.OutOfOrderCapMax.Store(30)
opts.OutOfOrderTimeWindow.Store(120 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wal, 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.ooo.oooHeadChunk)
require.Equal(t, expSamples, ms.ooo.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.ooo)
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)
}
func TestHeadMinOOOTimeUpdate(t *testing.T) {
dir := t.TempDir()
wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, true)
require.NoError(t, err)
oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, true)
require.NoError(t, err)
opts := DefaultHeadOptions()
opts.ChunkDirRoot = dir
opts.OutOfOrderTimeWindow.Store(10 * time.Minute.Milliseconds())
h, err := NewHead(nil, nil, wal, oooWlog, opts, nil)
require.NoError(t, err)
t.Cleanup(func() {
require.NoError(t, h.Close())
})
require.NoError(t, h.Init(0))
appendSample := func(ts int64) {
lbls := labels.FromStrings("foo", "bar")
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())
}
appendSample(300) // In-order sample.
require.Equal(t, int64(math.MaxInt64), h.MinOOOTime())
appendSample(295) // OOO sample.
require.Equal(t, 295*time.Minute.Milliseconds(), h.MinOOOTime())
// Allowed window for OOO is >=290, which is before the earliest ooo sample 295, so it gets set to the lower value.
require.NoError(t, h.truncateOOO(0, 1))
require.Equal(t, 290*time.Minute.Milliseconds(), h.MinOOOTime())
appendSample(310) // In-order sample.
appendSample(305) // OOO sample.
require.Equal(t, 290*time.Minute.Milliseconds(), h.MinOOOTime())
// Now the OOO sample 295 was not gc'ed yet. And allowed window for OOO is now >=300.
// So the lowest among them, 295, is set as minOOOTime.
require.NoError(t, h.truncateOOO(0, 2))
require.Equal(t, 295*time.Minute.Milliseconds(), h.MinOOOTime())
}
func TestGaugeHistogramWALAndChunkHeader(t *testing.T) {
l := labels.FromStrings("a", "b")
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
ts := int64(0)
appendHistogram := func(h *histogram.Histogram) {
ts++
app := head.Appender(context.Background())
_, err := app.AppendHistogram(0, l, ts, h.Copy(), nil)
require.NoError(t, err)
require.NoError(t, app.Commit())
}
hists := tsdbutil.GenerateTestGaugeHistograms(5)
hists[0].CounterResetHint = histogram.UnknownCounterReset
appendHistogram(hists[0])
appendHistogram(hists[1])
appendHistogram(hists[2])
hists[3].CounterResetHint = histogram.UnknownCounterReset
appendHistogram(hists[3])
appendHistogram(hists[3])
appendHistogram(hists[4])
checkHeaders := func() {
ms, _, err := head.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.Len(t, ms.mmappedChunks, 3)
expHeaders := []chunkenc.CounterResetHeader{
chunkenc.UnknownCounterReset,
chunkenc.GaugeType,
chunkenc.UnknownCounterReset,
chunkenc.GaugeType,
}
for i, mmapChunk := range ms.mmappedChunks {
chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref)
require.NoError(t, err)
require.Equal(t, expHeaders[i], chk.(*chunkenc.HistogramChunk).GetCounterResetHeader())
}
require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunk.chunk.(*chunkenc.HistogramChunk).GetCounterResetHeader())
}
checkHeaders()
recs := readTestWAL(t, head.wal.Dir())
require.Equal(t, []interface{}{
[]record.RefSeries{
{
Ref: 1,
Labels: labels.FromStrings("a", "b"),
},
},
[]record.RefHistogramSample{{Ref: 1, T: 1, H: hists[0]}},
[]record.RefHistogramSample{{Ref: 1, T: 2, H: hists[1]}},
[]record.RefHistogramSample{{Ref: 1, T: 3, H: hists[2]}},
[]record.RefHistogramSample{{Ref: 1, T: 4, H: hists[3]}},
[]record.RefHistogramSample{{Ref: 1, T: 5, H: hists[3]}},
[]record.RefHistogramSample{{Ref: 1, T: 6, H: hists[4]}},
}, recs)
// Restart Head without mmap chunks to expect the WAL replay to recognize gauge histograms.
require.NoError(t, head.Close())
require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot)))
w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
checkHeaders()
}
func TestGaugeFloatHistogramWALAndChunkHeader(t *testing.T) {
l := labels.FromStrings("a", "b")
head, _ := newTestHead(t, 1000, false, false)
t.Cleanup(func() {
require.NoError(t, head.Close())
})
require.NoError(t, head.Init(0))
ts := int64(0)
appendHistogram := func(h *histogram.FloatHistogram) {
ts++
app := head.Appender(context.Background())
_, err := app.AppendHistogram(0, l, ts, nil, h.Copy())
require.NoError(t, err)
require.NoError(t, app.Commit())
}
hists := tsdbutil.GenerateTestGaugeFloatHistograms(5)
hists[0].CounterResetHint = histogram.UnknownCounterReset
appendHistogram(hists[0])
appendHistogram(hists[1])
appendHistogram(hists[2])
hists[3].CounterResetHint = histogram.UnknownCounterReset
appendHistogram(hists[3])
appendHistogram(hists[3])
appendHistogram(hists[4])
checkHeaders := func() {
ms, _, err := head.getOrCreate(l.Hash(), l)
require.NoError(t, err)
require.Len(t, ms.mmappedChunks, 3)
expHeaders := []chunkenc.CounterResetHeader{
chunkenc.UnknownCounterReset,
chunkenc.GaugeType,
chunkenc.UnknownCounterReset,
chunkenc.GaugeType,
}
for i, mmapChunk := range ms.mmappedChunks {
chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref)
require.NoError(t, err)
require.Equal(t, expHeaders[i], chk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader())
}
require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunk.chunk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader())
}
checkHeaders()
recs := readTestWAL(t, head.wal.Dir())
require.Equal(t, []interface{}{
[]record.RefSeries{
{
Ref: 1,
Labels: labels.FromStrings("a", "b"),
},
},
[]record.RefFloatHistogramSample{{Ref: 1, T: 1, FH: hists[0]}},
[]record.RefFloatHistogramSample{{Ref: 1, T: 2, FH: hists[1]}},
[]record.RefFloatHistogramSample{{Ref: 1, T: 3, FH: hists[2]}},
[]record.RefFloatHistogramSample{{Ref: 1, T: 4, FH: hists[3]}},
[]record.RefFloatHistogramSample{{Ref: 1, T: 5, FH: hists[3]}},
[]record.RefFloatHistogramSample{{Ref: 1, T: 6, FH: hists[4]}},
}, recs)
// Restart Head without mmap chunks to expect the WAL replay to recognize gauge histograms.
require.NoError(t, head.Close())
require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot)))
w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
require.NoError(t, err)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(0))
checkHeaders()
}
func TestSnapshotAheadOfWALError(t *testing.T) {
head, _ := newTestHead(t, 120*4, false, false)
head.opts.EnableMemorySnapshotOnShutdown = true
// Add a sample to fill WAL.
app := head.Appender(context.Background())
_, err := app.Append(0, labels.FromStrings("foo", "bar"), 10, 10)
require.NoError(t, err)
require.NoError(t, app.Commit())
// Increment snapshot index to create sufficiently large difference.
for i := 0; i < 2; i++ {
_, err = head.wal.NextSegment()
require.NoError(t, err)
}
require.NoError(t, head.Close()) // This will create a snapshot.
_, idx, _, err := LastChunkSnapshot(head.opts.ChunkDirRoot)
require.NoError(t, err)
require.Equal(t, 2, idx)
// Restart the WAL while keeping the old snapshot. The new head is created manually in this case in order
// to keep using the same snapshot directory instead of a random one.
require.NoError(t, os.RemoveAll(head.wal.Dir()))
head.opts.EnableMemorySnapshotOnShutdown = false
w, _ := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
// Add a sample to fill WAL.
app = head.Appender(context.Background())
_, err = app.Append(0, labels.FromStrings("foo", "bar"), 10, 10)
require.NoError(t, err)
require.NoError(t, app.Commit())
lastSegment, _, _ := w.LastSegmentAndOffset()
require.Equal(t, 0, lastSegment)
require.NoError(t, head.Close())
// New WAL is saved, but old snapshot still exists.
_, idx, _, err = LastChunkSnapshot(head.opts.ChunkDirRoot)
require.NoError(t, err)
require.Equal(t, 2, idx)
// Create new Head which should detect the incorrect index and delete the snapshot.
head.opts.EnableMemorySnapshotOnShutdown = true
w, _ = wlog.NewSize(nil, nil, head.wal.Dir(), 32768, false)
head, err = NewHead(nil, nil, w, nil, head.opts, nil)
require.NoError(t, err)
require.NoError(t, head.Init(math.MinInt64))
// Verify that snapshot directory does not exist anymore.
_, _, _, err = LastChunkSnapshot(head.opts.ChunkDirRoot)
require.Equal(t, record.ErrNotFound, err)
require.NoError(t, head.Close())
}