mirror of
				https://github.com/prometheus/prometheus.git
				synced 2025-10-23 13:31:01 +02:00 
			
		
		
		
	Add a string remembering which label and limit the cache corresponds to. Signed-off-by: Xiaochao Dong (@damnever) <the.xcdong@gmail.com>
		
			
				
	
	
		
			5644 lines
		
	
	
		
			179 KiB
		
	
	
	
		
			Go
		
	
	
	
	
	
			
		
		
	
	
			5644 lines
		
	
	
		
			179 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.
 | |
| 
 | |
| package tsdb
 | |
| 
 | |
| import (
 | |
| 	"context"
 | |
| 	"fmt"
 | |
| 	"io"
 | |
| 	"math"
 | |
| 	"math/rand"
 | |
| 	"os"
 | |
| 	"path"
 | |
| 	"path/filepath"
 | |
| 	"reflect"
 | |
| 	"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 wlog.CompressionType, 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, wlog.CompressionNone, 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, wlog.CompressionNone, 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 populateTestWL(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)))
 | |
| 		case []record.RefMmapMarker:
 | |
| 			require.NoError(t, w.Log(enc.MmapMarkers(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)
 | |
| 		case record.Exemplars:
 | |
| 			exemplars, err := dec.Exemplars(rec, nil)
 | |
| 			require.NoError(t, err)
 | |
| 			recs = append(recs, exemplars)
 | |
| 		default:
 | |
| 			t.Fatalf("unknown record type")
 | |
| 		}
 | |
| 	}
 | |
| 	require.NoError(t, r.Err())
 | |
| 	return recs
 | |
| }
 | |
| 
 | |
| func BenchmarkLoadWLs(b *testing.B) {
 | |
| 	cases := []struct {
 | |
| 		// Total series is (batches*seriesPerBatch).
 | |
| 		batches          int
 | |
| 		seriesPerBatch   int
 | |
| 		samplesPerSeries int
 | |
| 		mmappedChunkT    int64
 | |
| 		// The first oooSeriesPct*seriesPerBatch series in a batch are selected as "OOO" series.
 | |
| 		oooSeriesPct float64
 | |
| 		// The first oooSamplesPct*samplesPerSeries samples in an OOO series are written as OOO samples.
 | |
| 		oooSamplesPct float64
 | |
| 		oooCapMax     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,
 | |
| 		},
 | |
| 		{ // A lot of OOO samples (50% series with 50% of samples being OOO).
 | |
| 			batches:          10,
 | |
| 			seriesPerBatch:   1000,
 | |
| 			samplesPerSeries: 480,
 | |
| 			oooSeriesPct:     0.5,
 | |
| 			oooSamplesPct:    0.5,
 | |
| 			oooCapMax:        DefaultOutOfOrderCapMax,
 | |
| 		},
 | |
| 		{ // Fewer OOO samples (10% of series with 10% of samples being OOO).
 | |
| 			batches:          10,
 | |
| 			seriesPerBatch:   1000,
 | |
| 			samplesPerSeries: 480,
 | |
| 			oooSeriesPct:     0.1,
 | |
| 			oooSamplesPct:    0.1,
 | |
| 		},
 | |
| 		{ // 2 hour WAL with 15 second scrape interval, and mmapped chunks up to last 100 samples.
 | |
| 			// Four mmap markers per OOO series: 480 * 0.3 = 144, 144 / 32 (DefaultOutOfOrderCapMax) = 4.
 | |
| 			batches:          100,
 | |
| 			seriesPerBatch:   1000,
 | |
| 			samplesPerSeries: 480,
 | |
| 			mmappedChunkT:    3800,
 | |
| 			oooSeriesPct:     0.2,
 | |
| 			oooSamplesPct:    0.3,
 | |
| 			oooCapMax:        DefaultOutOfOrderCapMax,
 | |
| 		},
 | |
| 	}
 | |
| 
 | |
| 	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,oooSeriesPct=%.3f,oooSamplesPct=%.3f,oooCapMax=%d", c.batches, c.seriesPerBatch, c.samplesPerSeries, exemplarsPerSeries, c.mmappedChunkT, c.oooSeriesPct, c.oooSamplesPct, c.oooCapMax),
 | |
| 				func(b *testing.B) {
 | |
| 					dir := b.TempDir()
 | |
| 
 | |
| 					wal, err := wlog.New(nil, nil, dir, wlog.CompressionNone)
 | |
| 					require.NoError(b, err)
 | |
| 					var wbl *wlog.WL
 | |
| 					if c.oooSeriesPct != 0 {
 | |
| 						wbl, err = wlog.New(nil, nil, dir, wlog.CompressionNone)
 | |
| 						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)})
 | |
| 						}
 | |
| 						populateTestWL(b, wal, []interface{}{refSeries})
 | |
| 					}
 | |
| 
 | |
| 					// Write samples.
 | |
| 					refSamples := make([]record.RefSample, 0, c.seriesPerBatch)
 | |
| 
 | |
| 					oooSeriesPerBatch := int(float64(c.seriesPerBatch) * c.oooSeriesPct)
 | |
| 					oooSamplesPerSeries := int(float64(c.samplesPerSeries) * c.oooSamplesPct)
 | |
| 
 | |
| 					for i := 0; i < c.samplesPerSeries; i++ {
 | |
| 						for j := 0; j < c.batches; j++ {
 | |
| 							refSamples = refSamples[:0]
 | |
| 
 | |
| 							k := j * c.seriesPerBatch
 | |
| 							// Skip appending the first oooSamplesPerSeries samples for the series in the batch that
 | |
| 							// should have OOO samples. OOO samples are appended after all the in-order samples.
 | |
| 							if i < oooSamplesPerSeries {
 | |
| 								k += oooSeriesPerBatch
 | |
| 							}
 | |
| 							for ; k < (j+1)*c.seriesPerBatch; k++ {
 | |
| 								refSamples = append(refSamples, record.RefSample{
 | |
| 									Ref: chunks.HeadSeriesRef(k) * 101,
 | |
| 									T:   int64(i) * 10,
 | |
| 									V:   float64(i) * 100,
 | |
| 								})
 | |
| 							}
 | |
| 							populateTestWL(b, wal, []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)
 | |
| 						cOpts := chunkOpts{
 | |
| 							chunkDiskMapper: chunkDiskMapper,
 | |
| 							chunkRange:      c.mmappedChunkT,
 | |
| 							samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 						}
 | |
| 						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)
 | |
| 							s.append(c.mmappedChunkT, 42, 0, cOpts)
 | |
| 							// There's only one head chunk because only a single sample is appended. mmapChunks()
 | |
| 							// ignores the latest chunk, so we need to cut a new head chunk to guarantee the chunk with
 | |
| 							// the sample at c.mmappedChunkT is mmapped.
 | |
| 							s.cutNewHeadChunk(c.mmappedChunkT, chunkenc.EncXOR, c.mmappedChunkT)
 | |
| 							s.mmapChunks(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)),
 | |
| 								})
 | |
| 							}
 | |
| 							populateTestWL(b, wal, []interface{}{refExemplars})
 | |
| 						}
 | |
| 					}
 | |
| 
 | |
| 					// Write OOO samples and mmap markers.
 | |
| 					refMarkers := make([]record.RefMmapMarker, 0, oooSeriesPerBatch)
 | |
| 					refSamples = make([]record.RefSample, 0, oooSeriesPerBatch)
 | |
| 					for i := 0; i < oooSamplesPerSeries; i++ {
 | |
| 						shouldAddMarkers := c.oooCapMax != 0 && i != 0 && int64(i)%c.oooCapMax == 0
 | |
| 
 | |
| 						for j := 0; j < c.batches; j++ {
 | |
| 							refSamples = refSamples[:0]
 | |
| 							if shouldAddMarkers {
 | |
| 								refMarkers = refMarkers[:0]
 | |
| 							}
 | |
| 							for k := j * c.seriesPerBatch; k < (j*c.seriesPerBatch)+oooSeriesPerBatch; k++ {
 | |
| 								ref := chunks.HeadSeriesRef(k) * 101
 | |
| 								if shouldAddMarkers {
 | |
| 									// loadWBL() checks that the marker's MmapRef is less than or equal to the ref
 | |
| 									// for the last mmap chunk. Setting MmapRef to 0 to always pass that check.
 | |
| 									refMarkers = append(refMarkers, record.RefMmapMarker{Ref: ref, MmapRef: 0})
 | |
| 								}
 | |
| 								refSamples = append(refSamples, record.RefSample{
 | |
| 									Ref: ref,
 | |
| 									T:   int64(i) * 10,
 | |
| 									V:   float64(i) * 100,
 | |
| 								})
 | |
| 							}
 | |
| 							if shouldAddMarkers {
 | |
| 								populateTestWL(b, wbl, []interface{}{refMarkers})
 | |
| 							}
 | |
| 							populateTestWL(b, wal, []interface{}{refSamples})
 | |
| 							populateTestWL(b, wbl, []interface{}{refSamples})
 | |
| 						}
 | |
| 					}
 | |
| 
 | |
| 					b.ResetTimer()
 | |
| 
 | |
| 					// Load the WAL.
 | |
| 					for i := 0; i < b.N; i++ {
 | |
| 						opts := DefaultHeadOptions()
 | |
| 						opts.ChunkRange = 1000
 | |
| 						opts.ChunkDirRoot = dir
 | |
| 						if c.oooCapMax > 0 {
 | |
| 							opts.OutOfOrderCapMax.Store(c.oooCapMax)
 | |
| 						}
 | |
| 						h, err := NewHead(nil, nil, wal, wbl, opts, nil)
 | |
| 						require.NoError(b, err)
 | |
| 						h.Init(0)
 | |
| 					}
 | |
| 					b.StopTimer()
 | |
| 					wal.Close()
 | |
| 					if wbl != nil {
 | |
| 						wbl.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, wlog.CompressionNone, 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][]chunks.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 []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} {
 | |
| 		t.Run(fmt.Sprintf("compress=%s", 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())
 | |
| 			}()
 | |
| 
 | |
| 			populateTestWL(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, wlog.CompressionNone, 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(), wlog.CompressionNone)
 | |
| 	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][]chunks.Sample{`{foo="bar"}`: {
 | |
| 		sample{1700, 3, nil, nil},
 | |
| 		sample{2000, 4, nil, nil},
 | |
| 	}}, series)
 | |
| }
 | |
| 
 | |
| func TestHead_ActiveAppenders(t *testing.T) {
 | |
| 	head, _ := newTestHead(t, 1000, wlog.CompressionNone, 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, wlog.CompressionNone, 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, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	h.initTime(0)
 | |
| 
 | |
| 	ctx := context.Background()
 | |
| 
 | |
| 	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(ctx, 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())
 | |
| 	}()
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: chunkDiskMapper,
 | |
| 		chunkRange:      2000,
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	memChunkPool := sync.Pool{
 | |
| 		New: func() interface{} {
 | |
| 			return &memChunk{}
 | |
| 		},
 | |
| 	}
 | |
| 
 | |
| 	s := newMemSeries(labels.FromStrings("a", "b"), 1, defaultIsolationDisabled)
 | |
| 
 | |
| 	for i := 0; i < 4000; i += 5 {
 | |
| 		ok, _ := s.append(int64(i), float64(i), 0, cOpts)
 | |
| 		require.True(t, ok, "sample append failed")
 | |
| 	}
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 
 | |
| 	// 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 TestMemSeries_truncateChunks_scenarios(t *testing.T) {
 | |
| 	const chunkRange = 100
 | |
| 	const chunkStep = 5
 | |
| 
 | |
| 	tests := []struct {
 | |
| 		name                 string
 | |
| 		headChunks           int                // the number of head chubks to create on memSeries by appending enough samples
 | |
| 		mmappedChunks        int                // the number of mmapped chunks to create on memSeries by appending enough samples
 | |
| 		truncateBefore       int64              // the mint to pass to truncateChunksBefore()
 | |
| 		expectedTruncated    int                // the number of chunks that we're expecting be truncated and returned by truncateChunksBefore()
 | |
| 		expectedHead         int                // the expected number of head chunks after truncation
 | |
| 		expectedMmap         int                // the expected number of mmapped chunks after truncation
 | |
| 		expectedFirstChunkID chunks.HeadChunkID // the expected series.firstChunkID after truncation
 | |
| 	}{
 | |
| 		{
 | |
| 			name:           "empty memSeries",
 | |
| 			truncateBefore: chunkRange * 10,
 | |
| 		},
 | |
| 		{
 | |
| 			name:         "single head chunk, not truncated",
 | |
| 			headChunks:   1,
 | |
| 			expectedHead: 1,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "single head chunk, truncated",
 | |
| 			headChunks:           1,
 | |
| 			truncateBefore:       chunkRange,
 | |
| 			expectedTruncated:    1,
 | |
| 			expectedHead:         0,
 | |
| 			expectedFirstChunkID: 1,
 | |
| 		},
 | |
| 		{
 | |
| 			name:         "2 head chunks, not truncated",
 | |
| 			headChunks:   2,
 | |
| 			expectedHead: 2,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "2 head chunks, first truncated",
 | |
| 			headChunks:           2,
 | |
| 			truncateBefore:       chunkRange,
 | |
| 			expectedTruncated:    1,
 | |
| 			expectedHead:         1,
 | |
| 			expectedFirstChunkID: 1,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "2 head chunks, everything truncated",
 | |
| 			headChunks:           2,
 | |
| 			truncateBefore:       chunkRange * 2,
 | |
| 			expectedTruncated:    2,
 | |
| 			expectedHead:         0,
 | |
| 			expectedFirstChunkID: 2,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "no head chunks, 3 mmap chunks, second mmap truncated",
 | |
| 			headChunks:           0,
 | |
| 			mmappedChunks:        3,
 | |
| 			truncateBefore:       chunkRange * 2,
 | |
| 			expectedTruncated:    2,
 | |
| 			expectedHead:         0,
 | |
| 			expectedMmap:         1,
 | |
| 			expectedFirstChunkID: 2,
 | |
| 		},
 | |
| 		{
 | |
| 			name:          "single head chunk, single mmap chunk, not truncated",
 | |
| 			headChunks:    1,
 | |
| 			mmappedChunks: 1,
 | |
| 			expectedHead:  1,
 | |
| 			expectedMmap:  1,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "single head chunk, single mmap chunk, mmap truncated",
 | |
| 			headChunks:           1,
 | |
| 			mmappedChunks:        1,
 | |
| 			truncateBefore:       chunkRange,
 | |
| 			expectedTruncated:    1,
 | |
| 			expectedHead:         1,
 | |
| 			expectedMmap:         0,
 | |
| 			expectedFirstChunkID: 1,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "5 head chunk, 5 mmap chunk, third head truncated",
 | |
| 			headChunks:           5,
 | |
| 			mmappedChunks:        5,
 | |
| 			truncateBefore:       chunkRange * 7,
 | |
| 			expectedTruncated:    7,
 | |
| 			expectedHead:         3,
 | |
| 			expectedMmap:         0,
 | |
| 			expectedFirstChunkID: 7,
 | |
| 		},
 | |
| 		{
 | |
| 			name:                 "2 head chunks, 3 mmap chunks, second mmap truncated",
 | |
| 			headChunks:           2,
 | |
| 			mmappedChunks:        3,
 | |
| 			truncateBefore:       chunkRange * 2,
 | |
| 			expectedTruncated:    2,
 | |
| 			expectedHead:         2,
 | |
| 			expectedMmap:         1,
 | |
| 			expectedFirstChunkID: 2,
 | |
| 		},
 | |
| 	}
 | |
| 
 | |
| 	for _, tc := range tests {
 | |
| 		t.Run(tc.name, func(t *testing.T) {
 | |
| 			dir := t.TempDir()
 | |
| 			chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize)
 | |
| 			require.NoError(t, err)
 | |
| 			defer func() {
 | |
| 				require.NoError(t, chunkDiskMapper.Close())
 | |
| 			}()
 | |
| 
 | |
| 			series := newMemSeries(labels.EmptyLabels(), 1, true)
 | |
| 
 | |
| 			cOpts := chunkOpts{
 | |
| 				chunkDiskMapper: chunkDiskMapper,
 | |
| 				chunkRange:      chunkRange,
 | |
| 				samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 			}
 | |
| 
 | |
| 			var headStart int
 | |
| 			if tc.mmappedChunks > 0 {
 | |
| 				headStart = (tc.mmappedChunks + 1) * chunkRange
 | |
| 				for i := 0; i < (tc.mmappedChunks+1)*chunkRange; i += chunkStep {
 | |
| 					ok, _ := series.append(int64(i), float64(i), 0, cOpts)
 | |
| 					require.True(t, ok, "sample append failed")
 | |
| 				}
 | |
| 				series.mmapChunks(chunkDiskMapper)
 | |
| 			}
 | |
| 
 | |
| 			if tc.headChunks == 0 {
 | |
| 				series.headChunks = nil
 | |
| 			} else {
 | |
| 				for i := headStart; i < chunkRange*(tc.mmappedChunks+tc.headChunks); i += chunkStep {
 | |
| 					ok, _ := series.append(int64(i), float64(i), 0, cOpts)
 | |
| 					require.True(t, ok, "sample append failed: %d", i)
 | |
| 				}
 | |
| 			}
 | |
| 
 | |
| 			if tc.headChunks > 0 {
 | |
| 				require.NotNil(t, series.headChunks, "head chunk is missing")
 | |
| 				require.Equal(t, tc.headChunks, series.headChunks.len(), "wrong number of head chunks")
 | |
| 			} else {
 | |
| 				require.Nil(t, series.headChunks, "head chunk is present")
 | |
| 			}
 | |
| 			require.Equal(t, tc.mmappedChunks, len(series.mmappedChunks), "wrong number of mmapped chunks")
 | |
| 
 | |
| 			truncated := series.truncateChunksBefore(tc.truncateBefore, 0)
 | |
| 			require.Equal(t, tc.expectedTruncated, truncated, "wrong number of truncated chunks returned")
 | |
| 
 | |
| 			require.Equal(t, tc.expectedMmap, len(series.mmappedChunks), "wrong number of mmappedChunks after truncation")
 | |
| 
 | |
| 			if tc.expectedHead > 0 {
 | |
| 				require.NotNil(t, series.headChunks, "headChunks should is nil after truncation")
 | |
| 				require.Equal(t, tc.expectedHead, series.headChunks.len(), "wrong number of head chunks after truncation")
 | |
| 				require.Nil(t, series.headChunks.oldest().prev, "last head chunk cannot have any next chunk set")
 | |
| 			} else {
 | |
| 				require.Nil(t, series.headChunks, "headChunks should is non-nil after truncation")
 | |
| 			}
 | |
| 
 | |
| 			if series.headChunks != nil || len(series.mmappedChunks) > 0 {
 | |
| 				require.GreaterOrEqual(t, series.maxTime(), tc.truncateBefore, "wrong value of series.maxTime() after truncation")
 | |
| 			} else {
 | |
| 				require.Equal(t, int64(math.MinInt64), series.maxTime(), "wrong value of series.maxTime() after truncation")
 | |
| 			}
 | |
| 
 | |
| 			require.Equal(t, tc.expectedFirstChunkID, series.firstChunkID, "wrong firstChunkID after truncation")
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func TestHeadDeleteSeriesWithoutSamples(t *testing.T) {
 | |
| 	for _, compress := range []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} {
 | |
| 		t.Run(fmt.Sprintf("compress=%s", 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())
 | |
| 			}()
 | |
| 
 | |
| 			populateTestWL(t, w, entries)
 | |
| 
 | |
| 			require.NoError(t, head.Init(math.MinInt64))
 | |
| 
 | |
| 			require.NoError(t, head.Delete(context.Background(), 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 []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} {
 | |
| 		t.Run(fmt.Sprintf("compress=%s", 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(context.Background(), 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(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value))
 | |
| 					require.NoError(t, q.Close())
 | |
| 					expSeriesSet := newMockSeriesSet([]storage.Series{
 | |
| 						storage.NewListSeries(lblsDefault, func() []chunks.Sample {
 | |
| 							ss := make([]chunks.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, wlog.CompressionNone, 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(context.Background(), 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(context.Background(), 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(context.Background(), 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, []chunks.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, wlog.CompressionNone, 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(context.Background(), 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][]chunks.Sample{}
 | |
| 	for _, l := range lbls {
 | |
| 		seriesMap[labels.New(l...).String()] = []chunks.Sample{}
 | |
| 	}
 | |
| 
 | |
| 	hb, _ := newTestHead(t, 100000, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, hb.Close())
 | |
| 	}()
 | |
| 
 | |
| 	app := hb.Appender(context.Background())
 | |
| 	for _, l := range lbls {
 | |
| 		ls := labels.New(l...)
 | |
| 		series := []chunks.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(context.Background(), 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(context.Background(), 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 []chunks.Sample, mint, maxt int64) []chunks.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 []chunks.Sample, dranges tombstones.Intervals) []chunks.Sample {
 | |
| 	ds := make([]chunks.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 := map[string]struct {
 | |
| 		start, cur, max int64
 | |
| 		ratioToFull     float64
 | |
| 		res             int64
 | |
| 	}{
 | |
| 		"exactly 1/4 full, even increment": {
 | |
| 			start:       0,
 | |
| 			cur:         250,
 | |
| 			max:         1000,
 | |
| 			ratioToFull: 4,
 | |
| 			res:         1000,
 | |
| 		},
 | |
| 		"exactly 1/4 full, uneven increment": {
 | |
| 			start:       100,
 | |
| 			cur:         200,
 | |
| 			max:         1000,
 | |
| 			ratioToFull: 4,
 | |
| 			res:         550,
 | |
| 		},
 | |
| 		"decimal ratio to full": {
 | |
| 			start:       5000,
 | |
| 			cur:         5110,
 | |
| 			max:         10000,
 | |
| 			ratioToFull: 4.2,
 | |
| 			res:         5500,
 | |
| 		},
 | |
| 		// Case where we fit floored 0 chunks. Must catch division by 0
 | |
| 		// and default to maximum time.
 | |
| 		"fit floored 0 chunks": {
 | |
| 			start:       0,
 | |
| 			cur:         500,
 | |
| 			max:         1000,
 | |
| 			ratioToFull: 4,
 | |
| 			res:         1000,
 | |
| 		},
 | |
| 		// Catch division by zero for cur == start. Strictly not a possible case.
 | |
| 		"cur == start": {
 | |
| 			start:       100,
 | |
| 			cur:         100,
 | |
| 			max:         1000,
 | |
| 			ratioToFull: 4,
 | |
| 			res:         104,
 | |
| 		},
 | |
| 	}
 | |
| 
 | |
| 	for testName, tc := range cases {
 | |
| 		t.Run(testName, func(t *testing.T) {
 | |
| 			got := computeChunkEndTime(tc.start, tc.cur, tc.max, tc.ratioToFull)
 | |
| 			if got != tc.res {
 | |
| 				t.Errorf("expected %d for (start: %d, cur: %d, max: %d, ratioToFull: %f), got %d", tc.res, tc.start, tc.cur, tc.max, tc.ratioToFull, 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())
 | |
| 	}()
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: chunkDiskMapper,
 | |
| 		chunkRange:      500,
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled)
 | |
| 
 | |
| 	// 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, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.True(t, chunkCreated, "first sample created chunk")
 | |
| 
 | |
| 	ok, chunkCreated = s.append(999, 2, 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.False(t, chunkCreated, "second sample should use same chunk")
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 
 | |
| 	ok, chunkCreated = s.append(1000, 3, 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.True(t, chunkCreated, "expected new chunk on boundary")
 | |
| 
 | |
| 	ok, chunkCreated = s.append(1001, 4, 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.False(t, chunkCreated, "second sample should use same chunk")
 | |
| 
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 	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.headChunks.minTime, "wrong chunk range")
 | |
| 	require.Equal(t, int64(1001), s.headChunks.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, cOpts)
 | |
| 		require.True(t, ok, "append failed")
 | |
| 	}
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 
 | |
| 	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())
 | |
| 	}()
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: chunkDiskMapper,
 | |
| 		chunkRange:      int64(1000),
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled)
 | |
| 
 | |
| 	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, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.True(t, chunkCreated, "first sample created chunk")
 | |
| 
 | |
| 	ok, chunkCreated = s.appendHistogram(999, histograms[1], 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.False(t, chunkCreated, "second sample should use same chunk")
 | |
| 
 | |
| 	ok, chunkCreated = s.appendHistogram(1000, histograms[2], 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.True(t, chunkCreated, "expected new chunk on boundary")
 | |
| 
 | |
| 	ok, chunkCreated = s.appendHistogram(1001, histograms[3], 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.False(t, chunkCreated, "second sample should use same chunk")
 | |
| 
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 	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.headChunks.minTime, "wrong chunk range")
 | |
| 	require.Equal(t, int64(1001), s.headChunks.maxTime, "wrong chunk range")
 | |
| 
 | |
| 	ok, chunkCreated = s.appendHistogram(1002, histogramWithOneMoreBucket, 0, cOpts)
 | |
| 	require.True(t, ok, "append failed")
 | |
| 	require.False(t, chunkCreated, "third sample should trigger a re-encoded chunk")
 | |
| 
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 	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.headChunks.minTime, "wrong chunk range")
 | |
| 	require.Equal(t, int64(1002), s.headChunks.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())
 | |
| 	})
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: chunkDiskMapper,
 | |
| 		chunkRange:      DefaultBlockDuration,
 | |
| 		samplesPerChunk: samplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled)
 | |
| 
 | |
| 	// 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, cOpts)
 | |
| 		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, cOpts)
 | |
| 		require.Truef(t, ok, "quick sample %d was not appended", i)
 | |
| 		nextTs++
 | |
| 		totalAppendedSamples++
 | |
| 	}
 | |
| 	ok, chunkCreated := s.append(DefaultBlockDuration, float64(0), 0, cOpts)
 | |
| 	require.True(t, ok, "new chunk sample was not appended")
 | |
| 	require.True(t, chunkCreated, "sample at block duration timestamp should create a new chunk")
 | |
| 
 | |
| 	s.mmapChunks(chunkDiskMapper)
 | |
| 	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, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: h.chunkDiskMapper,
 | |
| 		chunkRange:      chunkRange,
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	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, cOpts)
 | |
| 	require.True(t, ok, "series append failed")
 | |
| 	require.True(t, chunkCreated, "chunks was not created")
 | |
| 	ok, chunkCreated = s.append(999, 999, 0, cOpts)
 | |
| 	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, cOpts)
 | |
| 	require.True(t, ok, "series append failed")
 | |
| 	require.True(t, chunkCreated, "chunks was not created")
 | |
| 	ok, chunkCreated = s.append(1999, 1999, 0, cOpts)
 | |
| 	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.ChunkOrIterable(chunks[0])
 | |
| 	require.NoError(t, err)
 | |
| 	_, _, err = cr.ChunkOrIterable(chunks[1])
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	require.NoError(t, h.Truncate(1500)) // Remove a chunk.
 | |
| 
 | |
| 	_, _, err = cr.ChunkOrIterable(chunks[0])
 | |
| 	require.Equal(t, storage.ErrNotFound, err)
 | |
| 	_, _, err = cr.ChunkOrIterable(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, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: h.chunkDiskMapper,
 | |
| 		chunkRange:      chunkRange,
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	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, cOpts)
 | |
| 	require.True(t, ok, "series append failed")
 | |
| 	require.True(t, chunkCreated, "chunks was not created")
 | |
| 	ok, chunkCreated = s.append(999, 999, 0, cOpts)
 | |
| 	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, cOpts)
 | |
| 	require.True(t, ok, "series append failed")
 | |
| 	require.True(t, chunkCreated, "chunks was not created")
 | |
| 	ok, chunkCreated = s.append(1999, 1999, 0, cOpts)
 | |
| 	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.ChunkOrIterable(chunks[0])
 | |
| 	require.NoError(t, err)
 | |
| 	_, _, err = cr.ChunkOrIterable(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.ChunkOrIterable(chunks[0])
 | |
| 	require.Equal(t, storage.ErrNotFound, err)
 | |
| 	_, _, err = cr.ChunkOrIterable(chunks[1])
 | |
| 	require.Equal(t, storage.ErrNotFound, err)
 | |
| }
 | |
| 
 | |
| func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) {
 | |
| 	h, _ := newTestHead(t, 1000, wlog.CompressionNone, 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(context.Background(), 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, wlog.CompressionNone, 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(context.Background(), 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 []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} {
 | |
| 		t.Run(fmt.Sprintf("compress=%s", 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 []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} {
 | |
| 			t.Run(fmt.Sprintf("%s,compress=%s", 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")
 | |
| 				}
 | |
| 			})
 | |
| 		}
 | |
| 	}
 | |
| }
 | |
| 
 | |
| // TestWblRepair_DecodingError ensures that a repair is run for an error
 | |
| // when decoding a record.
 | |
| func TestWblRepair_DecodingError(t *testing.T) {
 | |
| 	var enc record.Encoder
 | |
| 	corrFunc := func(rec []byte) []byte {
 | |
| 		return rec[:3]
 | |
| 	}
 | |
| 	rec := enc.Samples([]record.RefSample{{Ref: 0, T: 99, V: 1}}, []byte{})
 | |
| 	totalRecs := 9
 | |
| 	expRecs := 5
 | |
| 	dir := t.TempDir()
 | |
| 
 | |
| 	// Fill the wbl and corrupt it.
 | |
| 	{
 | |
| 		wal, err := wlog.New(nil, nil, filepath.Join(dir, "wal"), wlog.CompressionNone)
 | |
| 		require.NoError(t, err)
 | |
| 		wbl, err := wlog.New(nil, nil, filepath.Join(dir, "wbl"), wlog.CompressionNone)
 | |
| 		require.NoError(t, err)
 | |
| 
 | |
| 		for i := 1; i <= totalRecs; i++ {
 | |
| 			// At this point insert a corrupted record.
 | |
| 			if i-1 == expRecs {
 | |
| 				require.NoError(t, wbl.Log(corrFunc(rec)))
 | |
| 				continue
 | |
| 			}
 | |
| 			require.NoError(t, wbl.Log(rec))
 | |
| 		}
 | |
| 
 | |
| 		opts := DefaultHeadOptions()
 | |
| 		opts.ChunkRange = 1
 | |
| 		opts.ChunkDirRoot = wal.Dir()
 | |
| 		opts.OutOfOrderCapMax.Store(30)
 | |
| 		opts.OutOfOrderTimeWindow.Store(1000 * time.Minute.Milliseconds())
 | |
| 		h, err := NewHead(nil, nil, wal, wbl, opts, nil)
 | |
| 		require.NoError(t, err)
 | |
| 		require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.walCorruptionsTotal))
 | |
| 		initErr := h.Init(math.MinInt64)
 | |
| 
 | |
| 		_, ok := initErr.(*errLoadWbl)
 | |
| 		require.True(t, ok) // Wbl errors are wrapped into errLoadWbl, make sure we can unwrap it.
 | |
| 
 | |
| 		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 wbl content after the repair.
 | |
| 	{
 | |
| 		sr, err := wlog.NewSegmentsReader(filepath.Join(dir, "wbl"))
 | |
| 		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, 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, wlog.CompressionNone)
 | |
| 		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))
 | |
| 
 | |
| 		cOpts := chunkOpts{
 | |
| 			chunkDiskMapper: h.chunkDiskMapper,
 | |
| 			chunkRange:      chunkRange,
 | |
| 			samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 		}
 | |
| 
 | |
| 		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, cOpts)
 | |
| 			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, cOpts)
 | |
| 			require.True(t, ok, "series append failed")
 | |
| 			require.False(t, chunkCreated, "chunk was created")
 | |
| 			h.chunkDiskMapper.CutNewFile()
 | |
| 			s.mmapChunks(h.chunkDiskMapper)
 | |
| 		}
 | |
| 		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, wlog.CompressionNone, 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, wlog.CompressionNone, 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(context.Background(), 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())
 | |
| 			h.mmapHeadChunks()
 | |
| 		}
 | |
| 		return i
 | |
| 	}
 | |
| 
 | |
| 	testIsolation := func(h *Head, i int) {
 | |
| 	}
 | |
| 
 | |
| 	// Test isolation without restart of Head.
 | |
| 	hb, _ := newTestHead(t, 1000, wlog.CompressionNone, 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, wlog.CompressionNone, false)
 | |
| 	i = addSamples(hb)
 | |
| 	require.NoError(t, hb.Close())
 | |
| 
 | |
| 	wal, err := wlog.NewSize(nil, nil, w.Dir(), 32768, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone, 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, wlog.CompressionNone, 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, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	h.initTime(0)
 | |
| 
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: h.chunkDiskMapper,
 | |
| 		chunkRange:      h.chunkRange.Load(),
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"))
 | |
| 
 | |
| 	ok, _ := s.append(0, 0, 0, cOpts)
 | |
| 	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, wlog.CompressionNone, 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(ctx))
 | |
| 	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, wlog.CompressionNone, 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(context.Background(), false, nil, matcher)
 | |
| 	for ss.Next() {
 | |
| 	}
 | |
| 	require.NoError(t, ss.Err())
 | |
| 	wg.Wait()
 | |
| }
 | |
| 
 | |
| func TestHeadLabelNamesValuesWithMinMaxRange(t *testing.T) {
 | |
| 	head, _ := newTestHead(t, 1000, wlog.CompressionNone, 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"}
 | |
| 		ctx                 = context.Background()
 | |
| 	)
 | |
| 
 | |
| 	app := head.Appender(ctx)
 | |
| 	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(ctx)
 | |
| 			require.NoError(t, err)
 | |
| 			require.Equal(t, tt.expectedNames, actualLabelNames)
 | |
| 			if len(tt.expectedValues) > 0 {
 | |
| 				for i, name := range expectedLabelNames {
 | |
| 					actualLabelValue, err := headIdxReader.SortedLabelValues(ctx, name)
 | |
| 					require.NoError(t, err)
 | |
| 					require.Equal(t, []string{tt.expectedValues[i]}, actualLabelValue)
 | |
| 				}
 | |
| 			}
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func TestHeadLabelValuesWithMatchers(t *testing.T) {
 | |
| 	head, _ := newTestHead(t, 1000, wlog.CompressionNone, false)
 | |
| 	t.Cleanup(func() { require.NoError(t, head.Close()) })
 | |
| 
 | |
| 	ctx := context.Background()
 | |
| 
 | |
| 	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(ctx, tt.labelName, tt.matchers...)
 | |
| 			require.NoError(t, err)
 | |
| 			require.Equal(t, tt.expectedValues, actualValues)
 | |
| 
 | |
| 			actualValues, err = headIdxReader.LabelValues(ctx, 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, wlog.CompressionNone, 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(context.Background(), tt.matchers...)
 | |
| 			require.NoError(t, err)
 | |
| 			require.Equal(t, tt.expectedNames, actualNames)
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func TestErrReuseAppender(t *testing.T) {
 | |
| 	head, _ := newTestHead(t, 1000, wlog.CompressionNone, 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, wlog.CompressionNone, 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, wlog.CompressionNone, 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, wlog.CompressionNone, false)
 | |
| 	b.Cleanup(func() { require.NoError(b, head.Close()) })
 | |
| 
 | |
| 	ctx := context.Background()
 | |
| 
 | |
| 	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(ctx, "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())
 | |
| 	}()
 | |
| 	cOpts := chunkOpts{
 | |
| 		chunkDiskMapper: chunkDiskMapper,
 | |
| 		chunkRange:      500,
 | |
| 		samplesPerChunk: DefaultSamplesPerChunk,
 | |
| 	}
 | |
| 
 | |
| 	s := newMemSeries(labels.Labels{}, 1, defaultIsolationDisabled)
 | |
| 
 | |
| 	for i := 0; i < 7; i++ {
 | |
| 		ok, _ := s.append(int64(i), float64(i), 0, cOpts)
 | |
| 		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()
 | |
| 	ctx := context.Background()
 | |
| 
 | |
| 	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(mint, maxt)
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	// Query the compacted range and get the first series before compaction.
 | |
| 	ss := q.Select(context.Background(), 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(ctx))
 | |
| 		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()
 | |
| 	ctx := context.Background()
 | |
| 
 | |
| 	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([]chunks.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(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(ctx))
 | |
| 		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][]chunks.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(c.mint, c.maxt)
 | |
| 			require.NoError(t, err)
 | |
| 			checkWaiting(q)
 | |
| 
 | |
| 			cq, err := db.ChunkQuerier(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, wlog.CompressionNone, 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([]chunks.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([]chunks.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(context.Background(), 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([]chunks.Sample, 0, len(expHistograms))
 | |
| 			actFloatHistograms := make([]chunks.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][]chunks.Sample{"dummy": expHistograms},
 | |
| 				map[string][]chunks.Sample{"dummy": actHistograms},
 | |
| 			)
 | |
| 			compareSeries(
 | |
| 				t,
 | |
| 				map[string][]chunks.Sample{"dummy": expFloatHistograms},
 | |
| 				map[string][]chunks.Sample{"dummy": actFloatHistograms},
 | |
| 			)
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func TestHistogramInWALAndMmapChunk(t *testing.T) {
 | |
| 	head, _ := newTestHead(t, 3000, wlog.CompressionNone, 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][]chunks.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.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.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())
 | |
| 		head.mmapHeadChunks()
 | |
| 	}
 | |
| 
 | |
| 	// There should be 20 mmap chunks in s1.
 | |
| 	ms := head.series.getByHash(s1.Hash(), s1)
 | |
| 	require.Len(t, ms.mmappedChunks, 25)
 | |
| 	expMmapChunks := make([]*mmappedChunk, 0, 20)
 | |
| 	for _, mmap := range ms.mmappedChunks {
 | |
| 		require.Greater(t, mmap.numSamples, uint16(0))
 | |
| 		cpy := *mmap
 | |
| 		expMmapChunks = append(expMmapChunks, &cpy)
 | |
| 	}
 | |
| 	expHeadChunkSamples := ms.headChunks.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.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.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, wlog.CompressionNone)
 | |
| 		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.headChunks.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, wlog.CompressionNone, 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][]chunks.Sample)
 | |
| 	expHist := make(map[string][]chunks.Sample)
 | |
| 	expFloatHist := make(map[string][]chunks.Sample)
 | |
| 	expTombstones := make(map[storage.SeriesRef]tombstones.Intervals)
 | |
| 	expExemplars := make([]ex, 0)
 | |
| 	histograms := tsdbutil.GenerateTestGaugeHistograms(481)
 | |
| 	floatHistogram := tsdbutil.GenerateTestGaugeFloatHistograms(481)
 | |
| 
 | |
| 	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", "bar.*"))
 | |
| 		require.Equal(t, expSeries, series)
 | |
| 	}
 | |
| 	checkHistograms := func() {
 | |
| 		q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64)
 | |
| 		require.NoError(t, err)
 | |
| 		series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "hist", "baz.*"))
 | |
| 		require.Equal(t, expHist, series)
 | |
| 	}
 | |
| 	checkFloatHistograms := func() {
 | |
| 		q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64)
 | |
| 		require.NoError(t, err)
 | |
| 		series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "floathist", "bat.*"))
 | |
| 		require.Equal(t, expFloatHist, 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, wlog.CompressionNone)
 | |
| 		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()
 | |
| 		checkHistograms()
 | |
| 		checkFloatHistograms()
 | |
| 		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()
 | |
| 			lblsHist := labels.FromStrings("hist", fmt.Sprintf("baz%d", i))
 | |
| 			lblsHistStr := lblsHist.String()
 | |
| 			lblsFloatHist := labels.FromStrings("floathist", fmt.Sprintf("bat%d", i))
 | |
| 			lblsFloatHistStr := lblsFloatHist.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)
 | |
| 
 | |
| 				hist := histograms[int(ts)]
 | |
| 				expHist[lblsHistStr] = append(expHist[lblsHistStr], sample{ts, 0, hist, nil})
 | |
| 				_, err = app.AppendHistogram(0, lblsHist, ts, hist, nil)
 | |
| 				require.NoError(t, err)
 | |
| 
 | |
| 				floatHist := floatHistogram[int(ts)]
 | |
| 				expFloatHist[lblsFloatHistStr] = append(expFloatHist[lblsFloatHistStr], sample{ts, 0, nil, floatHist})
 | |
| 				_, err = app.AppendHistogram(0, lblsFloatHist, ts, nil, floatHist)
 | |
| 				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()
 | |
| 			lblsHist := labels.FromStrings("hist", fmt.Sprintf("baz%d", i))
 | |
| 			lblsHistStr := lblsHist.String()
 | |
| 			lblsFloatHist := labels.FromStrings("floathist", fmt.Sprintf("bat%d", i))
 | |
| 			lblsFloatHistStr := lblsFloatHist.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)
 | |
| 
 | |
| 				hist := histograms[int(ts)]
 | |
| 				expHist[lblsHistStr] = append(expHist[lblsHistStr], sample{ts, 0, hist, nil})
 | |
| 				_, err = app.AppendHistogram(0, lblsHist, ts, hist, nil)
 | |
| 				require.NoError(t, err)
 | |
| 
 | |
| 				floatHist := floatHistogram[int(ts)]
 | |
| 				expFloatHist[lblsFloatHistStr] = append(expFloatHist[lblsFloatHistStr], sample{ts, 0, nil, floatHist})
 | |
| 				_, err = app.AppendHistogram(0, lblsFloatHist, ts, nil, floatHist)
 | |
| 				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, wlog.CompressionNone, 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)
 | |
| 
 | |
| 	// Add histograms
 | |
| 	hist := tsdbutil.GenerateTestGaugeHistograms(1)[0]
 | |
| 	floatHist := tsdbutil.GenerateTestGaugeFloatHistograms(1)[0]
 | |
| 	lblsHist := labels.FromStrings("hist", "bar")
 | |
| 	lblsFloatHist := labels.FromStrings("floathist", "bar")
 | |
| 
 | |
| 	_, err = app.AppendHistogram(0, lblsHist, 99, hist, nil)
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	_, err = app.AppendHistogram(0, lblsFloatHist, 99, nil, floatHist)
 | |
| 	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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone, 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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone, 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(context.Background(), 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.NotNil(t, s.headChunks)
 | |
| 	require.Equal(t, s.headChunks.len(), 1)
 | |
| 	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())
 | |
| 	head.mmapHeadChunks()
 | |
| 
 | |
| 	// Total 2 chunks, 1 m-mapped.
 | |
| 	s = head.series.getByHash(l.Hash(), l)
 | |
| 	require.NotNil(t, s)
 | |
| 	require.NotNil(t, s.headChunks)
 | |
| 	require.Equal(t, s.headChunks.len(), 1)
 | |
| 	require.Equal(t, 1, len(s.mmappedChunks))
 | |
| 	testQuery(2)
 | |
| }
 | |
| 
 | |
| func TestHistogramCounterResetHeader(t *testing.T) {
 | |
| 	for _, floatHisto := range []bool{true} { // FIXME
 | |
| 		t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) {
 | |
| 			l := labels.FromStrings("a", "b")
 | |
| 			head, _ := newTestHead(t, 1000, wlog.CompressionNone, 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)
 | |
| 				ms.mmapChunks(head.chunkDiskMapper)
 | |
| 				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.headChunks.chunk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader())
 | |
| 				} else {
 | |
| 					require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunks.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 histogram chunks (each chunk targets 1024 bytes which contains ~500 int histogram
 | |
| 			// samples or ~1000 float histogram samples).
 | |
| 			numAppend := 2000
 | |
| 			if floatHisto {
 | |
| 				numAppend = 1000
 | |
| 			}
 | |
| 			for i := 0; i < numAppend; 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 histogram chunks. Just to have some non-counter reset chunks in between.
 | |
| 			for i := 0; i < 2000; 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 []chunks.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.Equal(t, count, ms.headChunks.len())
 | |
| 	}
 | |
| 
 | |
| 	appends := []struct {
 | |
| 		samples   []chunks.Sample
 | |
| 		expChunks int
 | |
| 		err       error
 | |
| 		// If this is empty, samples above will be taken instead of this.
 | |
| 		addToExp []chunks.Sample
 | |
| 	}{
 | |
| 		// Histograms that end up in the expected samples are copied here so that we
 | |
| 		// can independently set the CounterResetHint later.
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 100, h: hists[0].Copy()}},
 | |
| 			expChunks: 1,
 | |
| 		},
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 200, f: 2}},
 | |
| 			expChunks: 2,
 | |
| 		},
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 210, fh: floatHists[0].Copy()}},
 | |
| 			expChunks: 3,
 | |
| 		},
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 220, h: hists[1].Copy()}},
 | |
| 			expChunks: 4,
 | |
| 		},
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 230, fh: floatHists[3].Copy()}},
 | |
| 			expChunks: 5,
 | |
| 		},
 | |
| 		{
 | |
| 			samples: []chunks.Sample{sample{t: 100, h: hists[2].Copy()}},
 | |
| 			err:     storage.ErrOutOfOrderSample,
 | |
| 		},
 | |
| 		{
 | |
| 			samples:   []chunks.Sample{sample{t: 300, h: hists[3].Copy()}},
 | |
| 			expChunks: 6,
 | |
| 		},
 | |
| 		{
 | |
| 			samples: []chunks.Sample{sample{t: 100, f: 2}},
 | |
| 			err:     storage.ErrOutOfOrderSample,
 | |
| 		},
 | |
| 		{
 | |
| 			samples: []chunks.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: []chunks.Sample{
 | |
| 				sample{t: 400, f: 4},
 | |
| 				sample{t: 500, h: hists[5]}, // This won't be committed.
 | |
| 				sample{t: 600, f: 6},
 | |
| 			},
 | |
| 			addToExp: []chunks.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: []chunks.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: []chunks.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: []chunks.Sample{
 | |
| 				sample{t: 1000, fh: floatHists[7]}, // Out of order w.r.t. the next histogram.
 | |
| 				sample{t: 1100, h: hists[9]},
 | |
| 			},
 | |
| 			addToExp: []chunks.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(math.MinInt64, math.MaxInt64)
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
 | |
| 	require.Equal(t, map[string][]chunks.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, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionSnappy)
 | |
| 	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)
 | |
| }
 | |
| 
 | |
| // TestWBLReplay checks the replay at a low level.
 | |
| func TestWBLReplay(t *testing.T) {
 | |
| 	dir := t.TempDir()
 | |
| 	wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionNone, 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")
 | |
| 
 | |
| 	series.mmapChunks(h.chunkDiskMapper)
 | |
| 	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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone)
 | |
| 		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 []chunks.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()
 | |
| 	h.mmapHeadChunks()
 | |
| 
 | |
| 	// 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][]chunks.Sample{lbls.String(): expSamples}, res)
 | |
| 
 | |
| 	require.NoError(t, h.Close())
 | |
| }
 | |
| 
 | |
| func TestOOOAppendWithNoSeries(t *testing.T) {
 | |
| 	dir := t.TempDir()
 | |
| 	wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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.headChunks)
 | |
| 		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.headChunks)
 | |
| 		require.Equal(t, expSamples, ms.headChunks.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, wlog.CompressionSnappy)
 | |
| 	require.NoError(t, err)
 | |
| 	oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy)
 | |
| 	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, wlog.CompressionNone, 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() {
 | |
| 		head.mmapHeadChunks()
 | |
| 		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.headChunks.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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone, 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)
 | |
| 		head.mmapHeadChunks()
 | |
| 		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.headChunks.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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone, 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, wlog.CompressionNone)
 | |
| 	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, wlog.CompressionNone)
 | |
| 	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())
 | |
| }
 | |
| 
 | |
| func BenchmarkCuttingHeadHistogramChunks(b *testing.B) {
 | |
| 	const (
 | |
| 		numSamples = 50000
 | |
| 		numBuckets = 100
 | |
| 	)
 | |
| 	samples := histogram.GenerateBigTestHistograms(numSamples, numBuckets)
 | |
| 
 | |
| 	h, _ := newTestHead(b, DefaultBlockDuration, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(b, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	a := h.Appender(context.Background())
 | |
| 	ts := time.Now().UnixMilli()
 | |
| 	lbls := labels.FromStrings("foo", "bar")
 | |
| 
 | |
| 	b.ResetTimer()
 | |
| 
 | |
| 	for _, s := range samples {
 | |
| 		_, err := a.AppendHistogram(0, lbls, ts, s, nil)
 | |
| 		require.NoError(b, err)
 | |
| 	}
 | |
| }
 | |
| 
 | |
| func TestCuttingNewHeadChunks(t *testing.T) {
 | |
| 	ctx := context.Background()
 | |
| 	testCases := map[string]struct {
 | |
| 		numTotalSamples int
 | |
| 		timestampJitter bool
 | |
| 		floatValFunc    func(i int) float64
 | |
| 		histValFunc     func(i int) *histogram.Histogram
 | |
| 		expectedChks    []struct {
 | |
| 			numSamples int
 | |
| 			numBytes   int
 | |
| 		}
 | |
| 	}{
 | |
| 		"float samples": {
 | |
| 			numTotalSamples: 180,
 | |
| 			floatValFunc: func(i int) float64 {
 | |
| 				return 1.
 | |
| 			},
 | |
| 			expectedChks: []struct {
 | |
| 				numSamples int
 | |
| 				numBytes   int
 | |
| 			}{
 | |
| 				{numSamples: 120, numBytes: 46},
 | |
| 				{numSamples: 60, numBytes: 32},
 | |
| 			},
 | |
| 		},
 | |
| 		"large float samples": {
 | |
| 			// Normally 120 samples would fit into a single chunk but these chunks violate the 1005 byte soft cap.
 | |
| 			numTotalSamples: 120,
 | |
| 			timestampJitter: true,
 | |
| 			floatValFunc: func(i int) float64 {
 | |
| 				// Flipping between these two make each sample val take at least 64 bits.
 | |
| 				vals := []float64{math.MaxFloat64, 0x00}
 | |
| 				return vals[i%len(vals)]
 | |
| 			},
 | |
| 			expectedChks: []struct {
 | |
| 				numSamples int
 | |
| 				numBytes   int
 | |
| 			}{
 | |
| 				{99, 1008},
 | |
| 				{21, 219},
 | |
| 			},
 | |
| 		},
 | |
| 		"small histograms": {
 | |
| 			numTotalSamples: 240,
 | |
| 			histValFunc: func() func(i int) *histogram.Histogram {
 | |
| 				hists := histogram.GenerateBigTestHistograms(240, 10)
 | |
| 				return func(i int) *histogram.Histogram {
 | |
| 					return hists[i]
 | |
| 				}
 | |
| 			}(),
 | |
| 			expectedChks: []struct {
 | |
| 				numSamples int
 | |
| 				numBytes   int
 | |
| 			}{
 | |
| 				{120, 1087},
 | |
| 				{120, 1039},
 | |
| 			},
 | |
| 		},
 | |
| 		"large histograms": {
 | |
| 			numTotalSamples: 240,
 | |
| 			histValFunc: func() func(i int) *histogram.Histogram {
 | |
| 				hists := histogram.GenerateBigTestHistograms(240, 100)
 | |
| 				return func(i int) *histogram.Histogram {
 | |
| 					return hists[i]
 | |
| 				}
 | |
| 			}(),
 | |
| 			expectedChks: []struct {
 | |
| 				numSamples int
 | |
| 				numBytes   int
 | |
| 			}{
 | |
| 				{40, 896},
 | |
| 				{40, 899},
 | |
| 				{40, 896},
 | |
| 				{30, 690},
 | |
| 				{30, 691},
 | |
| 				{30, 694},
 | |
| 				{30, 693},
 | |
| 			},
 | |
| 		},
 | |
| 		"really large histograms": {
 | |
| 			// Really large histograms; each chunk can only contain a single histogram but we have a 10 sample minimum
 | |
| 			// per chunk.
 | |
| 			numTotalSamples: 11,
 | |
| 			histValFunc: func() func(i int) *histogram.Histogram {
 | |
| 				hists := histogram.GenerateBigTestHistograms(11, 100000)
 | |
| 				return func(i int) *histogram.Histogram {
 | |
| 					return hists[i]
 | |
| 				}
 | |
| 			}(),
 | |
| 			expectedChks: []struct {
 | |
| 				numSamples int
 | |
| 				numBytes   int
 | |
| 			}{
 | |
| 				{10, 200103},
 | |
| 				{1, 87540},
 | |
| 			},
 | |
| 		},
 | |
| 	}
 | |
| 	for testName, tc := range testCases {
 | |
| 		t.Run(testName, func(t *testing.T) {
 | |
| 			h, _ := newTestHead(t, DefaultBlockDuration, wlog.CompressionNone, false)
 | |
| 			defer func() {
 | |
| 				require.NoError(t, h.Close())
 | |
| 			}()
 | |
| 
 | |
| 			a := h.Appender(context.Background())
 | |
| 
 | |
| 			ts := int64(10000)
 | |
| 			lbls := labels.FromStrings("foo", "bar")
 | |
| 			jitter := []int64{0, 1} // A bit of jitter to prevent dod=0.
 | |
| 
 | |
| 			for i := 0; i < tc.numTotalSamples; i++ {
 | |
| 				if tc.floatValFunc != nil {
 | |
| 					_, err := a.Append(0, lbls, ts, tc.floatValFunc(i))
 | |
| 					require.NoError(t, err)
 | |
| 				} else if tc.histValFunc != nil {
 | |
| 					_, err := a.AppendHistogram(0, lbls, ts, tc.histValFunc(i), nil)
 | |
| 					require.NoError(t, err)
 | |
| 				}
 | |
| 
 | |
| 				ts += int64(60 * time.Second / time.Millisecond)
 | |
| 				if tc.timestampJitter {
 | |
| 					ts += jitter[i%len(jitter)]
 | |
| 				}
 | |
| 			}
 | |
| 
 | |
| 			require.NoError(t, a.Commit())
 | |
| 
 | |
| 			idxReader, err := h.Index()
 | |
| 			require.NoError(t, err)
 | |
| 
 | |
| 			chkReader, err := h.Chunks()
 | |
| 			require.NoError(t, err)
 | |
| 
 | |
| 			p, err := idxReader.Postings(ctx, "foo", "bar")
 | |
| 			require.NoError(t, err)
 | |
| 
 | |
| 			var lblBuilder labels.ScratchBuilder
 | |
| 
 | |
| 			for p.Next() {
 | |
| 				sRef := p.At()
 | |
| 
 | |
| 				chkMetas := make([]chunks.Meta, len(tc.expectedChks))
 | |
| 				require.NoError(t, idxReader.Series(sRef, &lblBuilder, &chkMetas))
 | |
| 
 | |
| 				require.Len(t, chkMetas, len(tc.expectedChks))
 | |
| 
 | |
| 				for i, expected := range tc.expectedChks {
 | |
| 					chk, iterable, err := chkReader.ChunkOrIterable(chkMetas[i])
 | |
| 					require.NoError(t, err)
 | |
| 					require.Nil(t, iterable)
 | |
| 
 | |
| 					require.Equal(t, expected.numSamples, chk.NumSamples())
 | |
| 					require.Len(t, chk.Bytes(), expected.numBytes)
 | |
| 				}
 | |
| 			}
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| // TestHeadDetectsDuplcateSampleAtSizeLimit tests a regression where a duplicate sample
 | |
| // is appended to the head, right when the head chunk is at the size limit.
 | |
| // The test adds all samples as duplicate, thus expecting that the result has
 | |
| // exactly half of the samples.
 | |
| func TestHeadDetectsDuplicateSampleAtSizeLimit(t *testing.T) {
 | |
| 	numSamples := 1000
 | |
| 	baseTS := int64(1695209650)
 | |
| 
 | |
| 	h, _ := newTestHead(t, DefaultBlockDuration, wlog.CompressionNone, false)
 | |
| 	defer func() {
 | |
| 		require.NoError(t, h.Close())
 | |
| 	}()
 | |
| 
 | |
| 	a := h.Appender(context.Background())
 | |
| 	var err error
 | |
| 	vals := []float64{math.MaxFloat64, 0x00} // Use the worst case scenario for the XOR encoding. Otherwise we hit the sample limit before the size limit.
 | |
| 	for i := 0; i < numSamples; i++ {
 | |
| 		ts := baseTS + int64(i/2)*10000
 | |
| 		a.Append(0, labels.FromStrings("foo", "bar"), ts, vals[(i/2)%len(vals)])
 | |
| 		err = a.Commit()
 | |
| 		require.NoError(t, err)
 | |
| 		a = h.Appender(context.Background())
 | |
| 	}
 | |
| 
 | |
| 	indexReader, err := h.Index()
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	var (
 | |
| 		chunks  []chunks.Meta
 | |
| 		builder labels.ScratchBuilder
 | |
| 	)
 | |
| 	require.NoError(t, indexReader.Series(1, &builder, &chunks))
 | |
| 
 | |
| 	chunkReader, err := h.Chunks()
 | |
| 	require.NoError(t, err)
 | |
| 
 | |
| 	storedSampleCount := 0
 | |
| 	for _, chunkMeta := range chunks {
 | |
| 		chunk, iterable, err := chunkReader.ChunkOrIterable(chunkMeta)
 | |
| 		require.NoError(t, err)
 | |
| 		require.Nil(t, iterable)
 | |
| 		storedSampleCount += chunk.NumSamples()
 | |
| 	}
 | |
| 
 | |
| 	require.Equal(t, numSamples/2, storedSampleCount)
 | |
| }
 | |
| 
 | |
| func TestWALSampleAndExemplarOrder(t *testing.T) {
 | |
| 	lbls := labels.FromStrings("foo", "bar")
 | |
| 	testcases := map[string]struct {
 | |
| 		appendF      func(app storage.Appender, ts int64) (storage.SeriesRef, error)
 | |
| 		expectedType reflect.Type
 | |
| 	}{
 | |
| 		"float sample": {
 | |
| 			appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) {
 | |
| 				return app.Append(0, lbls, ts, 1.0)
 | |
| 			},
 | |
| 			expectedType: reflect.TypeOf([]record.RefSample{}),
 | |
| 		},
 | |
| 		"histogram sample": {
 | |
| 			appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) {
 | |
| 				return app.AppendHistogram(0, lbls, ts, tsdbutil.GenerateTestHistogram(1), nil)
 | |
| 			},
 | |
| 			expectedType: reflect.TypeOf([]record.RefHistogramSample{}),
 | |
| 		},
 | |
| 		"float histogram sample": {
 | |
| 			appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) {
 | |
| 				return app.AppendHistogram(0, lbls, ts, nil, tsdbutil.GenerateTestFloatHistogram(1))
 | |
| 			},
 | |
| 			expectedType: reflect.TypeOf([]record.RefFloatHistogramSample{}),
 | |
| 		},
 | |
| 	}
 | |
| 
 | |
| 	for testName, tc := range testcases {
 | |
| 		t.Run(testName, func(t *testing.T) {
 | |
| 			h, w := newTestHead(t, 1000, wlog.CompressionNone, false)
 | |
| 			defer func() {
 | |
| 				require.NoError(t, h.Close())
 | |
| 			}()
 | |
| 
 | |
| 			app := h.Appender(context.Background())
 | |
| 			ref, err := tc.appendF(app, 10)
 | |
| 			require.NoError(t, err)
 | |
| 			app.AppendExemplar(ref, lbls, exemplar.Exemplar{Value: 1.0, Ts: 5})
 | |
| 
 | |
| 			app.Commit()
 | |
| 
 | |
| 			recs := readTestWAL(t, w.Dir())
 | |
| 			require.Len(t, recs, 3)
 | |
| 			_, ok := recs[0].([]record.RefSeries)
 | |
| 			require.True(t, ok, "expected first record to be a RefSeries")
 | |
| 			actualType := reflect.TypeOf(recs[1])
 | |
| 			require.Equal(t, tc.expectedType, actualType, "expected second record to be a %s", tc.expectedType)
 | |
| 			_, ok = recs[2].([]record.RefExemplar)
 | |
| 			require.True(t, ok, "expected third record to be a RefExemplar")
 | |
| 		})
 | |
| 	}
 | |
| }
 | |
| 
 | |
| // TestHeadCompactionWhileAppendAndCommitExemplar simulates a use case where
 | |
| // a series is removed from the head while an exemplar is being appended to it.
 | |
| // This can happen in theory by compacting the head at the right time due to
 | |
| // a series being idle.
 | |
| // The test cheats a little bit by not appending a sample with the exemplar.
 | |
| // If you also add a sample and run Truncate in a concurrent goroutine and run
 | |
| // the test around a million(!) times, you can get
 | |
| // `unknown HeadSeriesRef when trying to add exemplar: 1` error on push.
 | |
| // It is likely that running the test for much longer and with more time variations
 | |
| // would trigger the
 | |
| // `signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0xbb03d1`
 | |
| // panic, that we have seen in the wild once.
 | |
| func TestHeadCompactionWhileAppendAndCommitExemplar(t *testing.T) {
 | |
| 	h, _ := newTestHead(t, DefaultBlockDuration, wlog.CompressionNone, false)
 | |
| 	app := h.Appender(context.Background())
 | |
| 	lbls := labels.FromStrings("foo", "bar")
 | |
| 	ref, err := app.Append(0, lbls, 1, 1)
 | |
| 	require.NoError(t, err)
 | |
| 	app.Commit()
 | |
| 	// Not adding a sample here to trigger the fault.
 | |
| 	app = h.Appender(context.Background())
 | |
| 	_, err = app.AppendExemplar(ref, lbls, exemplar.Exemplar{Value: 1, Ts: 20})
 | |
| 	require.NoError(t, err)
 | |
| 	h.Truncate(10)
 | |
| 	app.Commit()
 | |
| 	h.Close()
 | |
| }
 | |
| 
 | |
| func labelsWithHashCollision() (labels.Labels, labels.Labels) {
 | |
| 	// These two series have the same XXHash; thanks to https://github.com/pstibrany/labels_hash_collisions
 | |
| 	ls1 := labels.FromStrings("__name__", "metric", "lbl1", "value", "lbl2", "l6CQ5y")
 | |
| 	ls2 := labels.FromStrings("__name__", "metric", "lbl1", "value", "lbl2", "v7uDlF")
 | |
| 
 | |
| 	if ls1.Hash() != ls2.Hash() {
 | |
| 		// These ones are the same when using -tags stringlabels
 | |
| 		ls1 = labels.FromStrings("__name__", "metric", "lbl", "HFnEaGl")
 | |
| 		ls2 = labels.FromStrings("__name__", "metric", "lbl", "RqcXatm")
 | |
| 	}
 | |
| 
 | |
| 	if ls1.Hash() != ls2.Hash() {
 | |
| 		panic("This code needs to be updated: find new labels with colliding hash values.")
 | |
| 	}
 | |
| 
 | |
| 	return ls1, ls2
 | |
| }
 | |
| 
 | |
| // stripeSeriesWithCollidingSeries returns a stripeSeries with two memSeries having the same, colliding, hash.
 | |
| func stripeSeriesWithCollidingSeries(t *testing.T) (*stripeSeries, *memSeries, *memSeries) {
 | |
| 	t.Helper()
 | |
| 
 | |
| 	lbls1, lbls2 := labelsWithHashCollision()
 | |
| 	ms1 := memSeries{
 | |
| 		lset: lbls1,
 | |
| 	}
 | |
| 	ms2 := memSeries{
 | |
| 		lset: lbls2,
 | |
| 	}
 | |
| 	hash := lbls1.Hash()
 | |
| 	s := newStripeSeries(1, noopSeriesLifecycleCallback{})
 | |
| 
 | |
| 	got, created, err := s.getOrSet(hash, lbls1, func() *memSeries {
 | |
| 		return &ms1
 | |
| 	})
 | |
| 	require.NoError(t, err)
 | |
| 	require.True(t, created)
 | |
| 	require.Same(t, &ms1, got)
 | |
| 
 | |
| 	// Add a conflicting series
 | |
| 	got, created, err = s.getOrSet(hash, lbls2, func() *memSeries {
 | |
| 		return &ms2
 | |
| 	})
 | |
| 	require.NoError(t, err)
 | |
| 	require.True(t, created)
 | |
| 	require.Same(t, &ms2, got)
 | |
| 
 | |
| 	return s, &ms1, &ms2
 | |
| }
 | |
| 
 | |
| func TestStripeSeries_getOrSet(t *testing.T) {
 | |
| 	s, ms1, ms2 := stripeSeriesWithCollidingSeries(t)
 | |
| 	hash := ms1.lset.Hash()
 | |
| 
 | |
| 	// Verify that we can get both of the series despite the hash collision
 | |
| 	got := s.getByHash(hash, ms1.lset)
 | |
| 	require.Same(t, ms1, got)
 | |
| 	got = s.getByHash(hash, ms2.lset)
 | |
| 	require.Same(t, ms2, got)
 | |
| }
 | |
| 
 | |
| func TestStripeSeries_gc(t *testing.T) {
 | |
| 	s, ms1, ms2 := stripeSeriesWithCollidingSeries(t)
 | |
| 	hash := ms1.lset.Hash()
 | |
| 
 | |
| 	s.gc(0, 0)
 | |
| 
 | |
| 	// Verify that we can get neither ms1 nor ms2 after gc-ing corresponding series
 | |
| 	got := s.getByHash(hash, ms1.lset)
 | |
| 	require.Nil(t, got)
 | |
| 	got = s.getByHash(hash, ms2.lset)
 | |
| 	require.Nil(t, got)
 | |
| }
 | |
| 
 | |
| func TestPostingsCardinalityStats(t *testing.T) {
 | |
| 	head := &Head{postings: index.NewMemPostings()}
 | |
| 	head.postings.Add(1, labels.FromStrings(labels.MetricName, "t", "n", "v1"))
 | |
| 	head.postings.Add(2, labels.FromStrings(labels.MetricName, "t", "n", "v2"))
 | |
| 
 | |
| 	statsForMetricName := head.PostingsCardinalityStats(labels.MetricName, 10)
 | |
| 	head.postings.Add(3, labels.FromStrings(labels.MetricName, "t", "n", "v3"))
 | |
| 	// Using cache.
 | |
| 	require.Equal(t, statsForMetricName, head.PostingsCardinalityStats(labels.MetricName, 10))
 | |
| 
 | |
| 	statsForSomeLabel := head.PostingsCardinalityStats("n", 10)
 | |
| 	// Cache should be evicted because of the change of label name.
 | |
| 	require.NotEqual(t, statsForMetricName, statsForSomeLabel)
 | |
| 	head.postings.Add(4, labels.FromStrings(labels.MetricName, "t", "n", "v4"))
 | |
| 	// Using cache.
 | |
| 	require.Equal(t, statsForSomeLabel, head.PostingsCardinalityStats("n", 10))
 | |
| 	// Cache should be evicted because of the change of limit parameter.
 | |
| 	statsForSomeLabel1 := head.PostingsCardinalityStats("n", 1)
 | |
| 	require.NotEqual(t, statsForSomeLabel1, statsForSomeLabel)
 | |
| 	// Using cache.
 | |
| 	require.Equal(t, statsForSomeLabel1, head.PostingsCardinalityStats("n", 1))
 | |
| }
 |