From bc286f8b298d6d2504ecaf03b32ea5a08c93a37c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sun, 4 Jun 2023 11:31:42 -0300 Subject: [PATCH 01/84] hacky cached postings --- .../stores/shipper/indexgateway/gateway.go | 4 +- .../stores/tsdb/cached_postings_index.go | 139 ++++++ .../stores/tsdb/cached_postings_index_test.go | 427 ++++++++++++++++++ pkg/storage/stores/tsdb/single_file_index.go | 13 +- 4 files changed, 579 insertions(+), 4 deletions(-) create mode 100644 pkg/storage/stores/tsdb/cached_postings_index.go create mode 100644 pkg/storage/stores/tsdb/cached_postings_index_test.go diff --git a/pkg/storage/stores/shipper/indexgateway/gateway.go b/pkg/storage/stores/shipper/indexgateway/gateway.go index 7c37f4faeb58b..9a15614af579c 100644 --- a/pkg/storage/stores/shipper/indexgateway/gateway.go +++ b/pkg/storage/stores/shipper/indexgateway/gateway.go @@ -85,7 +85,7 @@ func NewIndexGateway(cfg Config, log log.Logger, registerer prometheus.Registere } func (g *Gateway) QueryIndex(request *logproto.QueryIndexRequest, server logproto.IndexGateway_QueryIndexServer) error { - log, _ := spanlogger.New(context.Background(), "IndexGateway.QueryIndex") + log, ctx := spanlogger.New(server.Context(), "IndexGateway.QueryIndex") defer log.Finish() var outerErr, innerErr error @@ -127,7 +127,7 @@ func (g *Gateway) QueryIndex(request *logproto.QueryIndexRequest, server logprot continue } - outerErr = indexClient.QueryPages(server.Context(), queries[start:end], func(query seriesindex.Query, batch seriesindex.ReadBatchResult) bool { + outerErr = indexClient.QueryPages(ctx, queries[start:end], func(query seriesindex.Query, batch seriesindex.ReadBatchResult) bool { innerErr = buildResponses(query, batch, func(response *logproto.QueryIndexResponse) error { // do not send grpc responses concurrently. See https://github.com/grpc/grpc-go/blob/master/stream.go#L120-L123. sendBatchMtx.Lock() diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go new file mode 100644 index 0000000000000..3728e44d1e82a --- /dev/null +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -0,0 +1,139 @@ +package tsdb + +import ( + "bytes" + "context" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" +) + +func NewCachedPostingsTSDBIndex(reader IndexReader) Index { + return &cachedPostingsTSDBIndex{ + reader: reader, + Index: NewTSDBIndex(reader), + postingsCache: make(map[string]map[string]index.Postings), + } +} + +type cachedPostingsTSDBIndex struct { + reader IndexReader + chunkFilter chunk.RequestChunkFilterer + Index + + // postingsCache maps shards -> matchers -> postings. + postingsCache map[string]map[string]index.Postings +} + +func matchersToString(matchers []*labels.Matcher) string { + var b bytes.Buffer + + b.WriteByte('{') + for i, l := range matchers { + if i > 0 { + b.WriteByte(',') + b.WriteByte(' ') + } + b.WriteString(l.String()) + } + b.WriteByte('}') + return b.String() +} + +func (c *cachedPostingsTSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { + var ls labels.Labels + chks := ChunkMetasPool.Get() + defer ChunkMetasPool.Put(chks) + + var filterer chunk.Filterer + if c.chunkFilter != nil { + filterer = c.chunkFilter.ForRequest(ctx) + } + + return c.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + for p.Next() { + hash, err := c.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) + if err != nil { + return err + } + + // skip series that belong to different shards + if shard != nil && !shard.Match(model.Fingerprint(hash)) { + continue + } + + if filterer != nil && filterer.ShouldFilter(ls) { + continue + } + + fn(ls, model.Fingerprint(hash), chks) + } + return p.Err() + }) +} + +func (c *cachedPostingsTSDBIndex) forPostings( + ctx context.Context, + shard *index.ShardAnnotation, + from, through model.Time, + matchers []*labels.Matcher, + fn func(index.Postings) error, +) error { + matchersStr := matchersToString(matchers) + var shardStr string + if shard != nil { + shardStr = shard.String() + } + + if _, ok := c.postingsCache[shardStr]; !ok { + c.postingsCache[shardStr] = make(map[string]index.Postings) + } + + if postings, ok := c.postingsCache[shardStr][matchersStr]; ok { + return fn(postings) + } + + p, err := PostingsForMatchers(c.reader, shard, matchers...) + if err != nil { + return err + } + c.postingsCache[shardStr][matchersStr] = p + return fn(p) +} + +func (c *cachedPostingsTSDBIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { + return c.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + // TODO(owen-d): use pool + var ls labels.Labels + var filterer chunk.Filterer + if c.chunkFilter != nil { + filterer = c.chunkFilter.ForRequest(ctx) + } + + for p.Next() { + fp, stats, err := c.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) + if err != nil { + return err + } + + // skip series that belong to different shards + if shard != nil && !shard.Match(model.Fingerprint(fp)) { + continue + } + + if filterer != nil && filterer.ShouldFilter(ls) { + continue + } + + if stats.Entries > 0 { + // need to add stream + acc.AddStream(model.Fingerprint(fp)) + acc.AddChunkStats(stats) + } + } + return p.Err() + }) +} diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go new file mode 100644 index 0000000000000..9400ae173cc39 --- /dev/null +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -0,0 +1,427 @@ +package tsdb + +import ( + "context" + "math/rand" + "sort" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" +) + +func TestSingleIdxCached(t *testing.T) { + shouldCachePostings = true + cases := []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 3, + Checksum: 0, + }, + { + MinTime: 1, + MaxTime: 4, + Checksum: 1, + }, + { + MinTime: 2, + MaxTime: 5, + Checksum: 2, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 1, + MaxTime: 10, + Checksum: 3, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bard", bazz="bozz", bonk="borb"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 1, + MaxTime: 7, + Checksum: 4, + }, + }, + }, + } + + for _, variant := range []struct { + desc string + fn func() Index + }{ + { + desc: "file", + fn: func() Index { + return BuildIndex(t, t.TempDir(), cases) + }, + }, + { + desc: "head", + fn: func() Index { + head := NewHead("fake", NewMetrics(nil), log.NewNopLogger()) + for _, x := range cases { + _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) + } + reader := head.Index() + return NewTSDBIndex(reader) + }, + }, + } { + t.Run(variant.desc, func(t *testing.T) { + idx := variant.fn() + t.Run("GetChunkRefs", func(t *testing.T) { + var err error + refs := make([]ChunkRef, 0, 8) + refs, err = idx.GetChunkRefs(context.Background(), "fake", 1, 5, refs, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []ChunkRef{ + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 0, + End: 3, + Checksum: 0, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 1, + End: 4, + Checksum: 1, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 2, + End: 5, + Checksum: 2, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + Start: 1, + End: 10, + Checksum: 3, + }, + } + require.Equal(t, expected, refs) + }) + + t.Run("GetChunkRefsSharded", func(t *testing.T) { + shard := index.ShardAnnotation{ + Shard: 1, + Of: 2, + } + var err error + refs := make([]ChunkRef, 0, 8) + refs, err = idx.GetChunkRefs(context.Background(), "fake", 1, 5, refs, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + + require.Nil(t, err) + + require.Equal(t, []ChunkRef{{ + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + Start: 1, + End: 10, + Checksum: 3, + }}, refs) + + }) + + t.Run("Series", func(t *testing.T) { + xs, err := idx.Series(context.Background(), "fake", 8, 9, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []Series{ + { + Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + }, + } + require.Equal(t, expected, xs) + }) + + t.Run("SeriesSharded", func(t *testing.T) { + shard := index.ShardAnnotation{ + Shard: 0, + Of: 2, + } + + xs, err := idx.Series(context.Background(), "fake", 0, 10, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []Series{ + { + Labels: mustParseLabels(`{foo="bar"}`), + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + }, + } + require.Equal(t, expected, xs) + }) + + t.Run("LabelNames", func(t *testing.T) { + // request data at the end of the tsdb range, but it should return all labels present + ls, err := idx.LabelNames(context.Background(), "fake", 9, 10) + require.Nil(t, err) + sort.Strings(ls) + require.Equal(t, []string{"bazz", "bonk", "foo"}, ls) + }) + + t.Run("LabelNamesWithMatchers", func(t *testing.T) { + // request data at the end of the tsdb range, but it should return all labels present + ls, err := idx.LabelNames(context.Background(), "fake", 9, 10, labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) + require.Nil(t, err) + sort.Strings(ls) + require.Equal(t, []string{"bazz", "foo"}, ls) + }) + + t.Run("LabelValues", func(t *testing.T) { + vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo") + require.Nil(t, err) + sort.Strings(vs) + require.Equal(t, []string{"bar", "bard"}, vs) + }) + + t.Run("LabelValuesWithMatchers", func(t *testing.T) { + vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo", labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) + require.Nil(t, err) + require.Equal(t, []string{"bar"}, vs) + }) + + }) + } + +} + +func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { + shouldCachePostings = false + now := model.Now() + queryFrom, queryThrough := now.Add(3*time.Hour).Add(time.Millisecond), now.Add(5*time.Hour).Add(-time.Millisecond) + queryBounds := newBounds(queryFrom, queryThrough) + numChunksToMatch := 0 + + var chunkMetas []index.ChunkMeta + // build a chunk for every second with randomized chunk length + for from, through := now, now.Add(24*time.Hour); from <= through; from = from.Add(time.Second) { + // randomize chunk length between 1-120 mins + chunkLenMin := rand.Intn(120) + if chunkLenMin == 0 { + chunkLenMin = 1 + } + chunkMeta := index.ChunkMeta{ + MinTime: int64(from), + MaxTime: int64(from.Add(time.Duration(chunkLenMin) * time.Minute)), + Checksum: uint32(from), + Entries: 1, + } + chunkMetas = append(chunkMetas, chunkMeta) + if Overlap(chunkMeta, queryBounds) { + numChunksToMatch++ + } + } + + tempDir := b.TempDir() + tsdbIndex := BuildIndex(b, tempDir, []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: chunkMetas, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: chunkMetas, + }, + { + Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), + Chunks: chunkMetas, + }, + }) + + b.ResetTimer() + b.ReportAllocs() + var err error + for i := 0; i < b.N; i++ { + chkRefs := ChunkRefsPool.Get() + chkRefs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, chkRefs, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.NoError(b, err) + require.Len(b, chkRefs, numChunksToMatch*2) + ChunkRefsPool.Put(chkRefs) + } +} + +func TestCacheableTSDBIndex_Stats(t *testing.T) { + shouldCachePostings = true + series := []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 1, + Entries: 10, + KB: 10, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 2, + Entries: 20, + KB: 20, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 3, + Entries: 30, + KB: 30, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 4, + Entries: 40, + KB: 40, + }, + }, + }, + } + + // Create the TSDB index + tempDir := t.TempDir() + + // Create the test cases + testCases := []struct { + name string + from model.Time + through model.Time + expected stats.Stats + expectedErr error + }{ + { + name: "from at the beginning of one chunk and through at the end of another chunk", + from: 0, + through: 20, + expected: stats.Stats{ + Streams: 2, + Chunks: 4, + Bytes: (10 + 20 + 30 + 40) * 1024, + Entries: 10 + 20 + 30 + 40, + }, + }, + { + name: "from inside one chunk and through inside another chunk", + from: 5, + through: 15, + expected: stats.Stats{ + Streams: 2, + Chunks: 4, + Bytes: (10*0.5 + 20*0.5 + 30*0.5 + 40*0.5) * 1024, + Entries: 10*0.5 + 20*0.5 + 30*0.5 + 40*0.5, + }, + }, + { + name: "from inside one chunk and through at the end of another chunk", + from: 5, + through: 20, + expected: stats.Stats{ + Streams: 2, + Chunks: 4, + Bytes: (10*0.5 + 20 + 30*0.5 + 40) * 1024, + Entries: 10*0.5 + 20 + 30*0.5 + 40, + }, + }, + { + name: "from at the beginning of one chunk and through inside another chunk", + from: 0, + through: 15, + expected: stats.Stats{ + Streams: 2, + Chunks: 4, + Bytes: (10 + 20*0.5 + 30 + 40*0.5) * 1024, + Entries: 10 + 20*0.5 + 30 + 40*0.5, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + tsdbIndex := BuildIndex(t, tempDir, series) + acc := &stats.Stats{} + err := tsdbIndex.Stats(context.Background(), "fake", tc.from, tc.through, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Equal(t, tc.expectedErr, err) + require.Equal(t, tc.expected, *acc) + }) + } +} + +func BenchmarkSeriesRepetitive(b *testing.B) { + shouldCachePostings = true + series := []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 1, + Entries: 10, + KB: 10, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 2, + Entries: 20, + KB: 20, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 3, + Entries: 30, + KB: 30, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 4, + Entries: 40, + KB: 40, + }, + }, + }, + } + tempDir := b.TempDir() + tsdbIndex := BuildIndex(b, tempDir, series) + acc := &stats.Stats{} + + for i := 0; i < b.N; i++ { + tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + } +} diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index abbcb00450f03..f79395599e2e8 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -18,6 +18,8 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" ) +var shouldCachePostings = false + var ErrAlreadyOnDesiredVersion = errors.New("tsdb file already on desired version") // GetRawFileReaderFunc returns an io.ReadSeeker for reading raw tsdb file from disk @@ -120,13 +122,20 @@ type TSDBIndex struct { // Return the index as well as the underlying raw file reader which isn't exposed as an index // method but is helpful for building an io.reader for the index shipper -func NewTSDBIndexFromFile(location string) (*TSDBIndex, GetRawFileReaderFunc, error) { +func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) { reader, err := index.NewFileReader(location) if err != nil { return nil, nil, err } - return NewTSDBIndex(reader), func() (io.ReadSeeker, error) { + var idx Index + if shouldCachePostings { + idx = NewCachedPostingsTSDBIndex(reader) + } else { + idx = NewTSDBIndex(reader) + } + + return idx, func() (io.ReadSeeker, error) { return reader.RawFileReader() }, nil } From da40b4020fb5d536af4bbe791b98b54f4fcf0db4 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 5 Jun 2023 09:11:58 -0300 Subject: [PATCH 02/84] Change signatures --- pkg/storage/factory.go | 7 ++++--- pkg/storage/store.go | 2 +- pkg/storage/store_test.go | 5 +++-- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/index.go | 7 +++++++ pkg/storage/stores/tsdb/store.go | 20 ++++++++++++-------- tools/tsdb/index-analyzer/main.go | 2 +- 7 files changed, 29 insertions(+), 16 deletions(-) diff --git a/pkg/storage/factory.go b/pkg/storage/factory.go index 803169cdbc1f3..8b8261b8ba1b5 100644 --- a/pkg/storage/factory.go +++ b/pkg/storage/factory.go @@ -33,6 +33,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores/indexshipper/gatewayclient" "github.com/grafana/loki/pkg/storage/stores/series/index" "github.com/grafana/loki/pkg/storage/stores/shipper" + "github.com/grafana/loki/pkg/storage/stores/tsdb" util_log "github.com/grafana/loki/pkg/util/log" ) @@ -192,9 +193,9 @@ type Config struct { DisableBroadIndexQueries bool `yaml:"disable_broad_index_queries"` MaxParallelGetChunk int `yaml:"max_parallel_get_chunk"` - MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` - BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper" doc:"description=Configures storing index in an Object Store (GCS/S3/Azure/Swift/COS/Filesystem) in the form of boltdb files. Required fields only required when boltdb-shipper is defined in config."` - TSDBShipperConfig indexshipper.Config `yaml:"tsdb_shipper"` + MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` + BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper" doc:"description=Configures storing index in an Object Store (GCS/S3/Azure/Swift/COS/Filesystem) in the form of boltdb files. Required fields only required when boltdb-shipper is defined in config."` + TSDBShipperConfig tsdb.IndexCfg `yaml:"tsdb_shipper"` // Config for using AsyncStore when using async index stores like `boltdb-shipper`. // It is required for getting chunk ids of recently flushed chunks from the ingesters. diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 69208d16eb2e7..f4cf13f06a795 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -215,7 +215,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, tableRange config.TableRan indexClientLogger := log.With(s.logger, "index-store", fmt.Sprintf("%s-%s", p.IndexType, p.From.String())) if p.IndexType == config.TSDBType { - if shouldUseIndexGatewayClient(s.cfg.TSDBShipperConfig) { + if shouldUseIndexGatewayClient(s.cfg.TSDBShipperConfig.Config) { // inject the index-gateway client into the index store gw, err := gatewayclient.NewGatewayClient(s.cfg.TSDBShipperConfig.IndexGatewayClientConfig, indexClientReg, indexClientLogger) if err != nil { diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 29c7e617eb243..f48114de2e3fc 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -31,6 +31,7 @@ import ( "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/shipper" + "github.com/grafana/loki/pkg/storage/stores/tsdb" util_log "github.com/grafana/loki/pkg/util/log" "github.com/grafana/loki/pkg/util/marshal" "github.com/grafana/loki/pkg/validation" @@ -1020,7 +1021,7 @@ func TestStore_MultiPeriod(t *testing.T) { BoltDBShipperConfig: shipper.Config{ Config: shipperConfig, }, - TSDBShipperConfig: shipperConfig, + TSDBShipperConfig: tsdb.IndexCfg{Config: shipperConfig}, NamedStores: NamedStores{ Filesystem: map[string]local.FSConfig{ "named-store": {Directory: path.Join(tempDir, "named-store")}, @@ -1333,7 +1334,7 @@ func TestStore_BoltdbTsdbSameIndexPrefix(t *testing.T) { cfg := Config{ FSConfig: local.FSConfig{Directory: path.Join(tempDir, "chunks")}, BoltDBShipperConfig: boltdbShipperConfig, - TSDBShipperConfig: tsdbShipperConfig, + TSDBShipperConfig: tsdb.IndexCfg{Config: tsdbShipperConfig}, } schemaConfig := config.SchemaConfig{ diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index d5caa261336e3..8de2bbb99db47 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -488,7 +488,7 @@ func TestBuildLegacyWALs(t *testing.T) { }, } { t.Run(tc.name, func(t *testing.T) { - store, stop, err := NewStore(tc.store, shipperCfg, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger()) + store, stop, err := NewStore(tc.store, IndexCfg{Config: shipperCfg}, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger()) require.Nil(t, err) refs, err := store.GetChunkRefs( diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index fc44ef9824b8d..c29052933d3dd 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) @@ -22,6 +23,12 @@ type ChunkRef struct { Checksum uint32 } +type IndexCfg struct { + indexshipper.Config + + cachePostings bool +} + // Compares by (Start, End) // Assumes User is equivalent func (r ChunkRef) Less(x ChunkRef) bool { diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index 12d2ea6d33f42..c0f75ecc51e8e 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -39,7 +39,7 @@ type store struct { // NewStore creates a new tsdb index ReaderWriter. func NewStore( name string, - indexShipperCfg indexshipper.Config, + indexShipperCfg IndexCfg, schemaCfg config.SchemaConfig, f *fetcher.Fetcher, objectClient client.ObjectClient, @@ -69,12 +69,16 @@ func NewStore( return storeInstance, storeInstance.Stop, nil } -func (s *store) init(name string, indexShipperCfg indexshipper.Config, schemaCfg config.SchemaConfig, objectClient client.ObjectClient, +func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConfig, objectClient client.ObjectClient, limits downloads.Limits, tableRange config.TableRange, reg prometheus.Registerer) error { + if indexCfg.cachePostings { + shouldCachePostings = true + } + var err error s.indexShipper, err = indexshipper.NewIndexShipper( - indexShipperCfg, + indexCfg.Config, objectClient, limits, nil, @@ -90,7 +94,7 @@ func (s *store) init(name string, indexShipperCfg indexshipper.Config, schemaCfg var indices []Index opts := DefaultIndexClientOptions() - if indexShipperCfg.Mode == indexshipper.ModeWriteOnly { + if indexCfg.Mode == indexshipper.ModeWriteOnly { // We disable bloom filters on write nodes // for the Stats() methods as it's of relatively little // benefit when compared to the memory cost. The bloom filters @@ -100,8 +104,8 @@ func (s *store) init(name string, indexShipperCfg indexshipper.Config, schemaCfg opts.UseBloomFilters = false } - if indexShipperCfg.Mode != indexshipper.ModeReadOnly { - nodeName, err := indexShipperCfg.GetUniqueUploaderName() + if indexCfg.Mode != indexshipper.ModeReadOnly { + nodeName, err := indexCfg.GetUniqueUploaderName() if err != nil { return err } @@ -110,7 +114,7 @@ func (s *store) init(name string, indexShipperCfg indexshipper.Config, schemaCfg tsdbManager := NewTSDBManager( name, nodeName, - indexShipperCfg.ActiveIndexDirectory, + indexCfg.ActiveIndexDirectory, s.indexShipper, tableRange, schemaCfg, @@ -121,7 +125,7 @@ func (s *store) init(name string, indexShipperCfg indexshipper.Config, schemaCfg headManager := NewHeadManager( name, s.logger, - indexShipperCfg.ActiveIndexDirectory, + indexCfg.ActiveIndexDirectory, tsdbMetrics, tsdbManager, ) diff --git a/tools/tsdb/index-analyzer/main.go b/tools/tsdb/index-analyzer/main.go index 58f650703d046..dbfa61abca960 100644 --- a/tools/tsdb/index-analyzer/main.go +++ b/tools/tsdb/index-analyzer/main.go @@ -34,7 +34,7 @@ func main() { tableRanges := getIndexStoreTableRanges(config.TSDBType, conf.SchemaConfig.Configs) shipper, err := indexshipper.NewIndexShipper( - conf.StorageConfig.TSDBShipperConfig, + conf.StorageConfig.TSDBShipperConfig.Config, objectClient, overrides, nil, From 2753a89bcb2a2f3a08a306e6ffc4ea4650b32682 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 7 Jun 2023 19:06:05 -0300 Subject: [PATCH 03/84] tmp inherit thanos indexcache code --- pkg/storage/store.go | 2 +- .../stores/tsdb/cached_postings_index.go | 132 +-- .../stores/tsdb/cached_postings_index_test.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 3 +- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/indexcache/cache.go | 200 ++++ .../stores/tsdb/indexcache/cache_test.go | 108 ++ .../stores/tsdb/indexcache/inmemory.go | 383 +++++++ .../stores/tsdb/indexcache/inmemory_test.go | 411 ++++++++ pkg/storage/stores/tsdb/indexcache/remote.go | 307 ++++++ .../stores/tsdb/indexcache/remote_test.go | 937 ++++++++++++++++++ pkg/storage/stores/tsdb/indexcache/tracing.go | 144 +++ pkg/storage/stores/tsdb/single_file_index.go | 67 +- .../stores/tsdb/single_file_index_test.go | 2 +- pkg/storage/stores/tsdb/store.go | 9 +- tools/tsdb/tsdb-map/main_test.go | 4 +- 16 files changed, 2565 insertions(+), 148 deletions(-) create mode 100644 pkg/storage/stores/tsdb/indexcache/cache.go create mode 100644 pkg/storage/stores/tsdb/indexcache/cache_test.go create mode 100644 pkg/storage/stores/tsdb/indexcache/inmemory.go create mode 100644 pkg/storage/stores/tsdb/indexcache/inmemory_test.go create mode 100644 pkg/storage/stores/tsdb/indexcache/remote.go create mode 100644 pkg/storage/stores/tsdb/indexcache/remote_test.go create mode 100644 pkg/storage/stores/tsdb/indexcache/tracing.go diff --git a/pkg/storage/store.go b/pkg/storage/store.go index f4cf13f06a795..d81b10aa6241d 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -256,7 +256,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, tableRange config.TableRan } indexReaderWriter, stopTSDBStoreFunc, err := tsdb.NewStore(fmt.Sprintf("%s_%s", p.ObjectType, p.From.String()), s.cfg.TSDBShipperConfig, s.schemaCfg, f, objectClient, s.limits, - tableRange, backupIndexWriter, indexClientReg, indexClientLogger) + tableRange, backupIndexWriter, indexClientReg, indexClientLogger, s.indexReadCache) if err != nil { return nil, nil, nil, err } diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 3728e44d1e82a..6a496575c6c18 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -1,139 +1,47 @@ package tsdb import ( - "bytes" "context" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/tsdb/indexcache" ) -func NewCachedPostingsTSDBIndex(reader IndexReader) Index { - return &cachedPostingsTSDBIndex{ - reader: reader, - Index: NewTSDBIndex(reader), - postingsCache: make(map[string]map[string]index.Postings), - } -} - -type cachedPostingsTSDBIndex struct { - reader IndexReader - chunkFilter chunk.RequestChunkFilterer - Index - - // postingsCache maps shards -> matchers -> postings. - postingsCache map[string]map[string]index.Postings -} +var cacheClient cache.Cache -func matchersToString(matchers []*labels.Matcher) string { - var b bytes.Buffer - - b.WriteByte('{') - for i, l := range matchers { - if i > 0 { - b.WriteByte(',') - b.WriteByte(' ') - } - b.WriteString(l.String()) +func NewCachedPostingsClient(reader IndexReader) PostingsClient { + return &cachedPostingsClient{ + reader: reader, + cacheClient: cacheClient, } - b.WriteByte('}') - return b.String() } -func (c *cachedPostingsTSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { - var ls labels.Labels - chks := ChunkMetasPool.Get() - defer ChunkMetasPool.Put(chks) - - var filterer chunk.Filterer - if c.chunkFilter != nil { - filterer = c.chunkFilter.ForRequest(ctx) - } - - return c.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { - for p.Next() { - hash, err := c.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) - if err != nil { - return err - } - - // skip series that belong to different shards - if shard != nil && !shard.Match(model.Fingerprint(hash)) { - continue - } - - if filterer != nil && filterer.ShouldFilter(ls) { - continue - } - - fn(ls, model.Fingerprint(hash), chks) - } - return p.Err() - }) +type PostingsClient interface { + ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error } -func (c *cachedPostingsTSDBIndex) forPostings( - ctx context.Context, - shard *index.ShardAnnotation, - from, through model.Time, - matchers []*labels.Matcher, - fn func(index.Postings) error, -) error { - matchersStr := matchersToString(matchers) - var shardStr string - if shard != nil { - shardStr = shard.String() - } +type cachedPostingsClient struct { + reader IndexReader + chunkFilter chunk.RequestChunkFilterer - if _, ok := c.postingsCache[shardStr]; !ok { - c.postingsCache[shardStr] = make(map[string]index.Postings) - } + cacheClient cache.Cache + IndexCacheClient indexcache.Client +} - if postings, ok := c.postingsCache[shardStr][matchersStr]; ok { +func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { + if postings, got := c.IndexCacheClient.FetchPostings(matchers); got { return fn(postings) } - p, err := PostingsForMatchers(c.reader, shard, matchers...) + p, err := PostingsForMatchers(c.reader, nil, matchers...) if err != nil { return err } - c.postingsCache[shardStr][matchersStr] = p - return fn(p) -} - -func (c *cachedPostingsTSDBIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { - return c.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { - // TODO(owen-d): use pool - var ls labels.Labels - var filterer chunk.Filterer - if c.chunkFilter != nil { - filterer = c.chunkFilter.ForRequest(ctx) - } - - for p.Next() { - fp, stats, err := c.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) - if err != nil { - return err - } - // skip series that belong to different shards - if shard != nil && !shard.Match(model.Fingerprint(fp)) { - continue - } - - if filterer != nil && filterer.ShouldFilter(ls) { - continue - } - - if stats.Entries > 0 { - // need to add stream - acc.AddStream(model.Fingerprint(fp)) - acc.AddChunkStats(stats) - } - } - return p.Err() - }) + c.IndexCacheClient.StorePostings(matchers, p) + return fn(p) } diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 9400ae173cc39..4049dedb7ca31 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -78,7 +78,7 @@ func TestSingleIdxCached(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader) + return NewTSDBIndex(reader, DefaultPostingsClient(reader)) }, }, } { diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index a68914fd856f1..b2ea683c9dcc3 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -730,7 +730,8 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx return } - idx = NewTSDBIndex(tenant.indexRange(int64(from), int64(through))) + reader := tenant.indexRange(int64(from), int64(through)) + idx = NewTSDBIndex(reader, &simpleForPostingsClient{reader: reader}) if t.chunkFilter != nil { idx.SetChunkFilterer(t.chunkFilter) } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 8de2bbb99db47..da1d68e4f1996 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -488,7 +488,7 @@ func TestBuildLegacyWALs(t *testing.T) { }, } { t.Run(tc.name, func(t *testing.T) { - store, stop, err := NewStore(tc.store, IndexCfg{Config: shipperCfg}, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger()) + store, stop, err := NewStore(tc.store, IndexCfg{Config: shipperCfg}, schemaCfg, nil, fsObjectClient, &zeroValueLimits{}, tc.tableRange, nil, nil, log.NewNopLogger(), nil) require.Nil(t, err) refs, err := store.GetChunkRefs( diff --git a/pkg/storage/stores/tsdb/indexcache/cache.go b/pkg/storage/stores/tsdb/indexcache/cache.go new file mode 100644 index 0000000000000..ccff100d8872c --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/cache.go @@ -0,0 +1,200 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/cache.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package indexcache + +import ( + "encoding/base64" + "reflect" + "sort" + "strings" + "unsafe" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "golang.org/x/crypto/blake2b" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" +) + +const ( + cacheTypePostings = "Postings" + cacheTypeSeriesForRef = "SeriesForRef" + cacheTypeExpandedPostings = "ExpandedPostings" + cacheTypeSeriesForPostings = "SeriesForPostings" + cacheTypeLabelNames = "LabelNames" + cacheTypeLabelValues = "LabelValues" +) + +var ( + allCacheTypes = []string{ + cacheTypePostings, + cacheTypeSeriesForRef, + cacheTypeExpandedPostings, + cacheTypeSeriesForPostings, + cacheTypeLabelNames, + cacheTypeLabelValues, + } +) + +type BytesResult interface { + // Next should return a byte slice if there was a cache hit for the current key; otherwise Next should return nil. + // Next should return false when there are no more keys in the result. + Next() ([]byte, bool) + + // Remaining should return the number of keys left in the result. + // There may or may not be an item for each key. + Remaining() int + + // Size should return the size in bytes of the result. + Size() int +} + +type MapIterator[T comparable] struct { + M map[T][]byte + Keys []T +} + +func (l *MapIterator[T]) Next() ([]byte, bool) { + if len(l.Keys) == 0 { + return nil, false + } + b := l.M[l.Keys[0]] + l.Keys = l.Keys[1:] + return b, true +} + +func (l *MapIterator[T]) Remaining() int { + return len(l.Keys) +} + +func (l *MapIterator[T]) Size() int { + return sumBytes[T](l.M) +} + +func sumBytes[T comparable](res map[T][]byte) int { + sum := 0 + for _, v := range res { + sum += len(v) + } + return sum +} + +// IndexCacheClient is the interface exported by index cache backends. +type Client interface { + StorePostings(matchers []*labels.Matcher, postings index.Postings) + FetchPostings(matchers []*labels.Matcher) (index.Postings, bool) + + StoreSeriesForPostings(matchers []*labels.Matcher, refs []storage.SeriesRef) + + // FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, postingsKey PostingsKey) ([]byte, bool) + + // // StoreSeriesForRef stores a single series. + // StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) + + // // FetchMultiSeriesForRefs fetches multiple series - each identified by ID - from the cache + // // and returns a map containing cache hits, along with a list of missing IDs. + // // The order of the returned misses should be the same as their relative order in the provided ids. + // FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) + + // // StoreExpandedPostings stores the result of ExpandedPostings, encoded with an unspecified codec. + // StoreExpandedPostings(userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string, v []byte) + + // // FetchExpandedPostings fetches the result of ExpandedPostings, encoded with an unspecified codec. + // FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) + + // StoreSeriesForPostings stores a series set for the provided postings. + // StoreSeriesForPostings(matchers []*labels.Matcher, postings index.Postings) + // FetchSeriesForPostings fetches a series set for the provided postings. + // FetchSeriesForPostings(ctx context.Context, matchers []*labels.Matcher) (index.Postings, bool) + + // // StoreLabelNames stores the result of a LabelNames() call. + // StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) + // // FetchLabelNames fetches the result of a LabelNames() call. + // FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) + + // // StoreLabelValues stores the result of a LabelValues() call. + // StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) + // // FetchLabelValues fetches the result of a LabelValues() call. + // FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) +} + +// PostingsKey represents a canonical key for a []storage.SeriesRef slice +type PostingsKey string + +// CanonicalPostingsKey creates a canonical version of PostingsKey +func CanonicalPostingsKey(postings []storage.SeriesRef) PostingsKey { + hashable := unsafeCastPostingsToBytes(postings) + checksum := blake2b.Sum256(hashable) + return PostingsKey(base64.RawURLEncoding.EncodeToString(checksum[:])) +} + +const bytesPerPosting = int(unsafe.Sizeof(storage.SeriesRef(0))) + +// unsafeCastPostingsToBytes returns the postings as a slice of bytes with minimal allocations. +// It casts the memory region of the underlying array to a slice of bytes. The resulting byte slice is only valid as long as the postings slice exists and is unmodified. +func unsafeCastPostingsToBytes(postings []storage.SeriesRef) []byte { + byteSlice := make([]byte, 0) + slicePtr := (*reflect.SliceHeader)(unsafe.Pointer(&byteSlice)) + slicePtr.Data = (*reflect.SliceHeader)(unsafe.Pointer(&postings)).Data + slicePtr.Len = len(postings) * bytesPerPosting + slicePtr.Cap = slicePtr.Len + return byteSlice +} + +// LabelMatchersKey represents a canonical key for a []*matchers.Matchers slice +type LabelMatchersKey string + +// CanonicalLabelMatchersKey creates a canonical version of LabelMatchersKey +func CanonicalLabelMatchersKey(ms []*labels.Matcher) LabelMatchersKey { + sorted := make([]labels.Matcher, len(ms)) + for i := range ms { + sorted[i] = labels.Matcher{Type: ms[i].Type, Name: ms[i].Name, Value: ms[i].Value} + } + sort.Sort(sortedLabelMatchers(sorted)) + + const ( + typeLen = 2 + sepLen = 1 + ) + var size int + for _, m := range sorted { + size += len(m.Name) + len(m.Value) + typeLen + sepLen + } + sb := strings.Builder{} + sb.Grow(size) + for _, m := range sorted { + sb.WriteString(m.Name) + sb.WriteString(m.Type.String()) + sb.WriteString(m.Value) + sb.WriteByte(0) + } + return LabelMatchersKey(sb.String()) +} + +type sortedLabelMatchers []labels.Matcher + +func (c sortedLabelMatchers) Less(i, j int) bool { + if c[i].Name != c[j].Name { + return c[i].Name < c[j].Name + } + if c[i].Type != c[j].Type { + return c[i].Type < c[j].Type + } + return c[i].Value < c[j].Value +} + +func (c sortedLabelMatchers) Len() int { return len(c) } +func (c sortedLabelMatchers) Swap(i, j int) { c[i], c[j] = c[j], c[i] } + +func initLabelValuesForAllCacheTypes(vec *prometheus.MetricVec) { + for _, typ := range allCacheTypes { + _, err := vec.GetMetricWithLabelValues(typ) + if err != nil { + panic(err) + } + } +} diff --git a/pkg/storage/stores/tsdb/indexcache/cache_test.go b/pkg/storage/stores/tsdb/indexcache/cache_test.go new file mode 100644 index 0000000000000..b4189d7c46b5a --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/cache_test.go @@ -0,0 +1,108 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/cache_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package indexcache + +// func TestMain(m *testing.M) { +// test.VerifyNoLeakTestMain(m) +// } + +// func TestCanonicalLabelMatchersKey(t *testing.T) { +// foo := labels.MustNewMatcher(labels.MatchEqual, "foo", "bar") +// bar := labels.MustNewMatcher(labels.MatchEqual, "bar", "foo") + +// assert.Equal(t, CanonicalLabelMatchersKey([]*labels.Matcher{foo, bar}), CanonicalLabelMatchersKey([]*labels.Matcher{bar, foo})) +// } + +// func BenchmarkCanonicalLabelMatchersKey(b *testing.B) { +// ms := make([]*labels.Matcher, 20) +// for i := range ms { +// ms[i] = labels.MustNewMatcher(labels.MatchType(i%4), fmt.Sprintf("%04x", i%3), fmt.Sprintf("%04x", i%2)) +// } +// for _, l := range []int{1, 5, 10, 20} { +// b.Run(fmt.Sprintf("%d matchers", l), func(b *testing.B) { +// b.ResetTimer() +// for i := 0; i < b.N; i++ { +// _ = CanonicalLabelMatchersKey(ms[:l]) +// } +// }) +// } +// } + +// func BenchmarkCanonicalPostingsKey(b *testing.B) { +// ms := make([]storage.SeriesRef, 1_000_000) +// for i := range ms { +// ms[i] = storage.SeriesRef(i) +// } +// for numPostings := 10; numPostings <= len(ms); numPostings *= 10 { +// b.Run(fmt.Sprintf("%d postings", numPostings), func(b *testing.B) { +// b.ReportAllocs() +// for i := 0; i < b.N; i++ { +// _ = CanonicalPostingsKey(ms[:numPostings]) +// } +// }) +// } +// } + +// func TestUnsafeCastPostingsToBytes(t *testing.T) { +// slowPostingsToBytes := func(postings []storage.SeriesRef) []byte { +// byteSlice := make([]byte, len(postings)*8) +// for i, posting := range postings { +// for octet := 0; octet < 8; octet++ { +// byteSlice[i*8+octet] = byte(posting >> (octet * 8)) +// } +// } +// return byteSlice +// } +// t.Run("base case", func(t *testing.T) { +// postings := []storage.SeriesRef{1, 2} +// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) +// }) +// t.Run("zero-length postings", func(t *testing.T) { +// postings := make([]storage.SeriesRef, 0) +// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) +// }) +// t.Run("nil postings", func(t *testing.T) { +// assert.Equal(t, []byte(nil), unsafeCastPostingsToBytes(nil)) +// }) +// t.Run("more than 256 postings", func(t *testing.T) { +// // Only casting a slice pointer truncates all postings to only their last byte. +// postings := make([]storage.SeriesRef, 300) +// for i := range postings { +// postings[i] = storage.SeriesRef(i + 1) +// } +// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) +// }) +// } + +// func TestCanonicalPostingsKey(t *testing.T) { +// t.Run("same length postings have different hashes", func(t *testing.T) { +// postings1 := []storage.SeriesRef{1, 2, 3, 4} +// postings2 := []storage.SeriesRef{5, 6, 7, 8} + +// assert.NotEqual(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) +// }) + +// t.Run("when postings are a subset of each other, they still have different hashes", func(t *testing.T) { +// postings1 := []storage.SeriesRef{1, 2, 3, 4} +// postings2 := []storage.SeriesRef{1, 2, 3, 4, 5} + +// assert.NotEqual(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) +// }) + +// t.Run("same postings with different slice capacities have same hashes", func(t *testing.T) { +// postings1 := []storage.SeriesRef{1, 2, 3, 4} +// postings2 := make([]storage.SeriesRef, 4, 8) +// copy(postings2, postings1) + +// assert.Equal(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) +// }) + +// t.Run("postings key is a base64-encoded string (i.e. is printable)", func(t *testing.T) { +// key := CanonicalPostingsKey([]storage.SeriesRef{1, 2, 3, 4}) +// _, err := base64.RawURLEncoding.DecodeString(string(key)) +// assert.NoError(t, err) +// }) +// } diff --git a/pkg/storage/stores/tsdb/indexcache/inmemory.go b/pkg/storage/stores/tsdb/indexcache/inmemory.go new file mode 100644 index 0000000000000..12ef623052687 --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/inmemory.go @@ -0,0 +1,383 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. +package indexcache + +import ( + "context" + "encoding/binary" + "fmt" + "sync" + + "github.com/dennwc/varint" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/flagext" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/util/encoding" + lru "github.com/hashicorp/golang-lru/simplelru" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" +) + +type codec string + +const ( + codecHeaderSnappy codec = "dvs" // As in "diff+varint+snappy". + codecHeaderSnappyWithMatchers codec = "dm" // As in "dvs+matchers" +) + +var DefaultInMemoryIndexCacheConfig = InMemoryIndexCacheConfig{ + MaxSize: 250 * 1024 * 1024, + MaxItemSize: 125 * 1024 * 1024, +} + +const maxInt = int(^uint(0) >> 1) + +const ( + stringHeaderSize = 8 + sliceHeaderSize = 16 +) + +var ulidSize = uint64(len(ulid.ULID{})) + +type InMemoryIndexCache struct { + mtx sync.Mutex + + logger log.Logger + lru *lru.LRU + maxSizeBytes uint64 + maxItemSizeBytes uint64 + + curSize uint64 + + evicted *prometheus.CounterVec + requests *prometheus.CounterVec + hits *prometheus.CounterVec + added *prometheus.CounterVec + current *prometheus.GaugeVec + currentSize *prometheus.GaugeVec + totalCurrentSize *prometheus.GaugeVec + overflow *prometheus.CounterVec +} + +// InMemoryIndexCacheConfig holds the in-memory index cache config. +type InMemoryIndexCacheConfig struct { + // MaxSize represents overall maximum number of bytes cache can contain. + MaxSize flagext.Bytes `yaml:"max_size"` + // MaxItemSize represents maximum size of single item. + MaxItemSize flagext.Bytes `yaml:"max_item_size"` +} + +// NewInMemoryIndexCache creates a new thread-safe LRU cache for index entries and ensures the total cache +// size approximately does not exceed maxBytes. +func NewInMemoryIndexCache(logger log.Logger, reg prometheus.Registerer, cfg InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) { + return NewInMemoryIndexCacheWithConfig(logger, reg, cfg) +} + +// NewInMemoryIndexCacheWithConfig creates a new thread-safe LRU cache for index entries and ensures the total cache +// size approximately does not exceed maxBytes. +func NewInMemoryIndexCacheWithConfig(logger log.Logger, reg prometheus.Registerer, config InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) { + if config.MaxItemSize > config.MaxSize { + return nil, errors.Errorf("max item size (%v) cannot be bigger than overall cache size (%v)", config.MaxItemSize, config.MaxSize) + } + + c := &InMemoryIndexCache{ + logger: logger, + maxSizeBytes: uint64(config.MaxSize), + maxItemSizeBytes: uint64(config.MaxItemSize), + } + + c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_evicted_total", + Help: "Total number of items that were evicted from the index cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.evicted.MetricVec) + + c.added = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_added_total", + Help: "Total number of items that were added to the index cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.added.MetricVec) + + c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_requests_total", + Help: "Total number of requests to the cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.requests.MetricVec) + + c.overflow = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_overflowed_total", + Help: "Total number of items that could not be added to the cache due to being too big.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.overflow.MetricVec) + + c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_hits_total", + Help: "Total number of requests to the cache that were a hit.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.hits.MetricVec) + + c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items", + Help: "Current number of items in the index cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.current.MetricVec) + + c.currentSize = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_size_bytes", + Help: "Current byte size of items in the index cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.currentSize.MetricVec) + + c.totalCurrentSize = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_total_size_bytes", + Help: "Current byte size of items (both value and key) in the index cache.", + }, []string{"item_type"}) + initLabelValuesForAllCacheTypes(c.totalCurrentSize.MetricVec) + + _ = promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_max_size_bytes", + Help: "Maximum number of bytes to be held in the index cache.", + }, func() float64 { + return float64(c.maxSizeBytes) + }) + _ = promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_max_item_size_bytes", + Help: "Maximum number of bytes for single entry to be held in the index cache.", + }, func() float64 { + return float64(c.maxItemSizeBytes) + }) + + // Initialize LRU cache with a high size limit since we will manage evictions ourselves + // based on stored size using `RemoveOldest` method. + l, err := lru.NewLRU(maxInt, c.onEvict) + if err != nil { + return nil, err + } + c.lru = l + + level.Info(logger).Log( + "msg", "created in-memory index cache", + "maxItemSizeBytes", c.maxItemSizeBytes, + "maxSizeBytes", c.maxSizeBytes, + "maxItems", "maxInt", + ) + return c, nil +} + +func (c *InMemoryIndexCache) onEvict(key, val interface{}) { + k := key.(cacheKey) + typ := k.typ() + entrySize := sliceSize(val.([]byte)) + + c.evicted.WithLabelValues(typ).Inc() + c.current.WithLabelValues(typ).Dec() + c.currentSize.WithLabelValues(typ).Sub(float64(entrySize)) + c.totalCurrentSize.WithLabelValues(typ).Sub(float64(entrySize + k.size())) + + c.curSize -= entrySize +} + +func (c *InMemoryIndexCache) get(key cacheKey) ([]byte, bool) { + typ := key.typ() + c.requests.WithLabelValues(typ).Inc() + + c.mtx.Lock() + defer c.mtx.Unlock() + + v, ok := c.lru.Get(key) + if !ok { + return nil, false + } + c.hits.WithLabelValues(typ).Inc() + return v.([]byte), true +} + +func (c *InMemoryIndexCache) set(key cacheKey, val []byte) { + typ := key.typ() + size := sliceSize(val) + + c.mtx.Lock() + defer c.mtx.Unlock() + + if _, ok := c.lru.Get(key); ok { + return + } + + if !c.ensureFits(size, typ) { + c.overflow.WithLabelValues(typ).Inc() + return + } + + // The caller may be passing in a sub-slice of a huge array. Copy the data + // to ensure we don't waste huge amounts of space for something small. + v := make([]byte, len(val)) + copy(v, val) + c.lru.Add(key, v) + + c.added.WithLabelValues(typ).Inc() + c.currentSize.WithLabelValues(typ).Add(float64(size)) + c.totalCurrentSize.WithLabelValues(typ).Add(float64(size + key.size())) + c.current.WithLabelValues(typ).Inc() + c.curSize += size +} + +// ensureFits tries to make sure that the passed slice will fit into the LRU cache. +// Returns true if it will fit. +func (c *InMemoryIndexCache) ensureFits(size uint64, typ string) bool { + if size > c.maxItemSizeBytes { + level.Debug(c.logger).Log( + "msg", "item bigger than maxItemSizeBytes. Ignoring..", + "maxItemSizeBytes", c.maxItemSizeBytes, + "maxSizeBytes", c.maxSizeBytes, + "curSize", c.curSize, + "itemSize", size, + "cacheType", typ, + ) + return false + } + + for c.curSize+size > c.maxSizeBytes { + if _, _, ok := c.lru.RemoveOldest(); !ok { + level.Error(c.logger).Log( + "msg", "LRU has nothing more to evict, but we still cannot allocate the item. Resetting cache.", + "maxItemSizeBytes", c.maxItemSizeBytes, + "maxSizeBytes", c.maxSizeBytes, + "curSize", c.curSize, + "itemSize", size, + "cacheType", typ, + ) + c.reset() + } + } + return true +} + +func (c *InMemoryIndexCache) reset() { + c.lru.Purge() + c.current.Reset() + c.currentSize.Reset() + c.totalCurrentSize.Reset() + c.curSize = 0 +} + +func (c *InMemoryIndexCache) StorePostings(postings index.Postings, matchers []*labels.Matcher) { + dataToCache, err := diffVarintEncodeNoHeader(postings, 0) + if err != nil { + level.Warn(c.logger).Log("msg", "couldn't encode postings", "err", err, "matchers", CanonicalLabelMatchersKey(matchers)) + } + + c.set(cacheKeyPostings{matchers: matchers}, dataToCache) +} + +// diffVarintEncodeNoHeader encodes postings into diff+varint representation. +// It doesn't add any header to the output bytes. +// Length argument is expected number of postings, used for preallocating buffer. +func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { + buf := encoding.Encbuf{} + + // This encoding uses around ~1 bytes per posting, but let's use + // conservative 1.25 bytes per posting to avoid extra allocations. + if length > 0 { + buf.B = make([]byte, 0, 5*length/4) + } + + prev := storage.SeriesRef(0) + for p.Next() { + v := p.At() + + // TODO(dylanguedes): can we ignore this? + // if v < prev { + // return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) + // } + + // This is the 'diff' part -- compute difference from previous value. + buf.PutUvarint64(uint64(v - prev)) + prev = v + } + if p.Err() != nil { + return nil, p.Err() + } + + return buf.B, nil +} + +func encodedMatchersLen(matchers []*labels.Matcher) int { + matchersLen := varint.UvarintSize(uint64(len(matchers))) + for _, m := range matchers { + matchersLen += varint.UvarintSize(uint64(len(m.Name))) + matchersLen += len(m.Name) + matchersLen++ // 1 byte for the type + matchersLen += varint.UvarintSize(uint64(len(m.Value))) + matchersLen += len(m.Value) + } + return matchersLen +} + +// encodeMatchers needs to be called with the precomputed length of the encoded matchers from encodedMatchersLen +func encodeMatchers(expectedLen int, matchers []*labels.Matcher, dest []byte) (written int, _ error) { + if len(dest) < expectedLen { + return 0, fmt.Errorf("too small buffer to encode matchers: need at least %d, got %d", expectedLen, dest) + } + written += binary.PutUvarint(dest, uint64(len(matchers))) + for _, m := range matchers { + written += binary.PutUvarint(dest[written:], uint64(len(m.Name))) + written += copy(dest[written:], m.Name) + + dest[written] = byte(m.Type) + written++ + + written += binary.PutUvarint(dest[written:], uint64(len(m.Value))) + written += copy(dest[written:], m.Value) + } + return written, nil +} + +// FetchSeriesForPostings fetches a series set for the provided postings. +func (c *InMemoryIndexCache) FetchSeriesForPostings(_ context.Context, matchers []*labels.Matcher) ([]byte, bool) { + return c.get(cacheKeyPostings{matchers: matchers}) +} + +// cacheKey is used by in-memory representation to store cached data. +// The implementations of cacheKey should be hashable, as they will be used as keys for *lru.LRU cache +type cacheKey interface { + // typ is used as label for metrics. + typ() string + // size is used to keep track of the cache size, it represents the footprint of the cache key in memory. + size() uint64 +} + +// cacheKeyPostings implements cacheKey and is used to reference a postings cache entry in the inmemory cache. +type cacheKeyPostings struct { + matchers []*labels.Matcher +} + +func (c cacheKeyPostings) typ() string { return cacheTypePostings } + +func (c cacheKeyPostings) size() uint64 { + return stringSize(string(CanonicalLabelMatchersKey(c.matchers))) +} + +func stringSize(s string) uint64 { + return stringHeaderSize + uint64(len(s)) +} + +func sliceSize(b []byte) uint64 { + return sliceHeaderSize + uint64(len(b)) +} diff --git a/pkg/storage/stores/tsdb/indexcache/inmemory_test.go b/pkg/storage/stores/tsdb/indexcache/inmemory_test.go new file mode 100644 index 0000000000000..e50df7214243a --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/inmemory_test.go @@ -0,0 +1,411 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +// Tests out the index cache implementation. +package indexcache + +// func TestInMemoryIndexCache_AvoidsDeadlock(t *testing.T) { +// user := "tenant" +// metrics := prometheus.NewRegistry() +// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ +// MaxItemSize: sliceHeaderSize + 5, +// MaxSize: sliceHeaderSize + 5, +// }) +// assert.NoError(t, err) + +// l, err := simplelru.NewLRU(math.MaxInt64, func(key, val interface{}) { +// // Hack LRU to simulate broken accounting: evictions do not reduce current size. +// size := cache.curSize +// cache.onEvict(key, val) +// cache.curSize = size +// }) +// assert.NoError(t, err) +// cache.lru = l + +// cache.StorePostings(user, ulid.MustNew(0, nil), labels.Label{Name: "test2", Value: "1"}, []byte{42, 33, 14, 67, 11}) + +// assert.Equal(t, uint64(sliceHeaderSize+5), cache.curSize) +// assert.Equal(t, float64(cache.curSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) + +// // This triggers deadlock logic. +// cache.StorePostings(user, ulid.MustNew(0, nil), labels.Label{Name: "test1", Value: "1"}, []byte{42}) + +// assert.Equal(t, uint64(sliceHeaderSize+1), cache.curSize) +// assert.Equal(t, float64(cache.curSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// } + +// func TestInMemoryIndexCache_UpdateItem(t *testing.T) { +// const maxSize = 2 * (sliceHeaderSize + 1) + +// var errorLogs []string +// errorLogger := log.LoggerFunc(func(kvs ...interface{}) error { +// var lvl string +// for i := 0; i < len(kvs); i += 2 { +// if kvs[i] == "level" { +// lvl = fmt.Sprint(kvs[i+1]) +// break +// } +// } +// if lvl != "error" { +// return nil +// } +// var buf bytes.Buffer +// defer func() { errorLogs = append(errorLogs, buf.String()) }() +// return log.NewLogfmtLogger(&buf).Log(kvs...) +// }) + +// metrics := prometheus.NewRegistry() +// cache, err := NewInMemoryIndexCacheWithConfig(log.NewSyncLogger(errorLogger), metrics, InMemoryIndexCacheConfig{ +// MaxItemSize: maxSize, +// MaxSize: maxSize, +// }) +// assert.NoError(t, err) + +// user := "tenant" +// uid := func(id uint64) ulid.ULID { return ulid.MustNew(uint64(id), nil) } +// lbl := labels.Label{Name: "foo", Value: "bar"} +// matchers := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), labels.MustNewMatcher(labels.MatchNotRegexp, "baz", ".*")} +// shard := &sharding.ShardSelector{ShardIndex: 1, ShardCount: 16} +// ctx := context.Background() + +// for _, tt := range []struct { +// typ string +// set func(uint64, []byte) +// get func(uint64) ([]byte, bool) +// }{ +// { +// typ: cacheTypePostings, +// set: func(id uint64, b []byte) { cache.StorePostings(user, uid(id), lbl, b) }, +// get: func(id uint64) ([]byte, bool) { +// hits := cache.FetchMultiPostings(ctx, user, uid(id), []labels.Label{lbl}) +// b, _ := hits.Next() +// return b, b != nil +// }, +// }, +// { +// typ: cacheTypeSeriesForRef, +// set: func(id uint64, b []byte) { +// cache.StoreSeriesForRef(user, uid(id), storage.SeriesRef(id), b) +// }, +// get: func(id uint64) ([]byte, bool) { +// seriesRef := storage.SeriesRef(id) +// hits, _ := cache.FetchMultiSeriesForRefs(ctx, user, uid(id), []storage.SeriesRef{seriesRef}) +// b, ok := hits[seriesRef] + +// return b, ok +// }, +// }, +// { +// typ: cacheTypeExpandedPostings, +// set: func(id uint64, b []byte) { +// cache.StoreExpandedPostings(user, uid(id), CanonicalLabelMatchersKey(matchers), "strategy", b) +// }, +// get: func(id uint64) ([]byte, bool) { +// return cache.FetchExpandedPostings(ctx, user, uid(id), CanonicalLabelMatchersKey(matchers), "strategy") +// }, +// }, +// { +// typ: cacheTypeSeriesForPostings, +// set: func(id uint64, b []byte) { +// cache.StoreSeriesForPostings(user, uid(id), shard, CanonicalPostingsKey([]storage.SeriesRef{1}), b) +// }, +// get: func(id uint64) ([]byte, bool) { +// return cache.FetchSeriesForPostings(ctx, user, uid(id), shard, CanonicalPostingsKey([]storage.SeriesRef{1})) +// }, +// }, +// { +// typ: cacheTypeLabelNames, +// set: func(id uint64, b []byte) { +// cache.StoreLabelNames(user, uid(id), CanonicalLabelMatchersKey(matchers), b) +// }, +// get: func(id uint64) ([]byte, bool) { +// return cache.FetchLabelNames(ctx, user, uid(id), CanonicalLabelMatchersKey(matchers)) +// }, +// }, +// { +// typ: cacheTypeLabelValues, +// set: func(id uint64, b []byte) { +// cache.StoreLabelValues(user, uid(id), fmt.Sprintf("lbl_%d", id), CanonicalLabelMatchersKey(matchers), b) +// }, +// get: func(id uint64) ([]byte, bool) { +// return cache.FetchLabelValues(ctx, user, uid(id), fmt.Sprintf("lbl_%d", id), CanonicalLabelMatchersKey(matchers)) +// }, +// }, +// } { +// t.Run(tt.typ, func(t *testing.T) { +// defer func() { errorLogs = nil }() + +// // Set value. +// tt.set(0, []byte{0}) +// buf, ok := tt.get(0) +// assert.Equal(t, true, ok) +// assert.Equal(t, []byte{0}, buf) +// assert.Equal(t, float64(sliceHeaderSize+1), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) +// assert.Equal(t, []string(nil), errorLogs) + +// // Set the same value again. +// // NB: This used to over-count the value. +// tt.set(0, []byte{0}) +// buf, ok = tt.get(0) +// assert.Equal(t, true, ok) +// assert.Equal(t, []byte{0}, buf) +// assert.Equal(t, float64(sliceHeaderSize+1), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) +// assert.Equal(t, []string(nil), errorLogs) + +// // Set a larger value. +// // NB: This used to deadlock when enough values were over-counted and it +// // couldn't clear enough space -- repeatedly removing oldest after empty. +// tt.set(1, []byte{0, 1}) +// buf, ok = tt.get(1) +// assert.Equal(t, true, ok) +// assert.Equal(t, []byte{0, 1}, buf) +// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) +// assert.Equal(t, []string(nil), errorLogs) + +// // Mutations to existing values will be ignored. +// tt.set(1, []byte{1, 2}) +// buf, ok = tt.get(1) +// assert.Equal(t, true, ok) +// assert.Equal(t, []byte{0, 1}, buf) +// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) +// assert.Equal(t, []string(nil), errorLogs) +// }) +// } +// } + +// // This should not happen as we hardcode math.MaxInt, but we still add test to check this out. +// func TestInMemoryIndexCache_MaxNumberOfItemsHit(t *testing.T) { +// user := "tenant" +// metrics := prometheus.NewRegistry() +// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ +// MaxItemSize: 2*sliceHeaderSize + 10, +// MaxSize: 2*sliceHeaderSize + 10, +// }) +// assert.NoError(t, err) + +// l, err := simplelru.NewLRU(2, cache.onEvict) +// assert.NoError(t, err) +// cache.lru = l + +// id := ulid.MustNew(0, nil) + +// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "123"}, []byte{42, 33}) +// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "124"}, []byte{42, 33}) +// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "125"}, []byte{42, 33}) + +// assert.Equal(t, uint64(2*sliceHeaderSize+4), cache.curSize) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(3), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings))) +// for _, typ := range remove(allCacheTypes, cacheTypePostings) { +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(typ))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(typ))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.added.WithLabelValues(typ))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(typ))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(typ))) +// } +// } + +// func TestInMemoryIndexCache_Eviction_WithMetrics(t *testing.T) { +// user := "tenant" +// metrics := prometheus.NewRegistry() +// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ +// MaxItemSize: 2*sliceHeaderSize + 5, +// MaxSize: 2*sliceHeaderSize + 5, +// }) +// assert.NoError(t, err) + +// id := ulid.MustNew(0, nil) +// lbls := labels.Label{Name: "test", Value: "123"} +// ctx := context.Background() +// emptySeriesHits := map[storage.SeriesRef][]byte{} +// emptySeriesMisses := []storage.SeriesRef(nil) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, nil) + +// // Add sliceHeaderSize + 2 bytes. +// cache.StorePostings(user, id, lbls, []byte{42, 33}) +// assert.Equal(t, uint64(sliceHeaderSize+2), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize+2+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, map[labels.Label][]byte{lbls: {42, 33}}) + +// testFetchMultiPostings(ctx, t, cache, user, ulid.MustNew(1, nil), []labels.Label{lbls}, nil) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{{Name: "test", Value: "124"}}, nil) + +// // Add sliceHeaderSize + 3 more bytes. +// cache.StoreSeriesForRef(user, id, 1234, []byte{222, 223, 224}) +// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize+2+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(sliceHeaderSize+3), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(sliceHeaderSize+3+cacheKeySeriesForRef{user, id, 1234}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// sHits, sMisses := cache.FetchMultiSeriesForRefs(ctx, user, id, []storage.SeriesRef{1234}) +// assert.Equal(t, map[storage.SeriesRef][]byte{1234: {222, 223, 224}}, sHits, "key exists") +// assert.Equal(t, emptySeriesMisses, sMisses) + +// lbls2 := labels.Label{Name: "test", Value: "124"} + +// // Add sliceHeaderSize + 5 + 16 bytes, should fully evict 2 last items. +// v := []byte{42, 33, 14, 67, 11} +// for i := 0; i < sliceHeaderSize; i++ { +// v = append(v, 3) +// } +// cache.StorePostings(user, id, lbls2, v) + +// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) // Eviction. +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) // Eviction. + +// // Evicted. +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, nil) + +// sHits, sMisses = cache.FetchMultiSeriesForRefs(ctx, user, id, []storage.SeriesRef{1234}) +// assert.Equal(t, emptySeriesHits, sHits, "no such key") +// assert.Equal(t, []storage.SeriesRef{1234}, sMisses) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls2}, map[labels.Label][]byte{lbls2: v}) + +// // Add same item again. +// cache.StorePostings(user, id, lbls2, v) + +// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls2}, map[labels.Label][]byte{lbls2: v}) + +// // Add too big item. +// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "toobig"}, append(v, 5)) +// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) // Overflow. +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// _, _, ok := cache.lru.RemoveOldest() +// assert.True(t, ok, "something to remove") + +// assert.Equal(t, uint64(0), cache.curSize) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// _, _, ok = cache.lru.RemoveOldest() +// assert.True(t, !ok, "nothing to remove") + +// lbls3 := labels.Label{Name: "test", Value: "124"} + +// cache.StorePostings(user, id, lbls3, []byte{}) + +// assert.Equal(t, uint64(sliceHeaderSize), cache.curSize) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(sliceHeaderSize+cacheKeyPostings{user, id, lbls3}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls3}, map[labels.Label][]byte{lbls3: {}}) + +// // nil works and still allocates empty slice. +// lbls4 := labels.Label{Name: "test", Value: "125"} +// cache.StorePostings(user, id, lbls4, []byte(nil)) + +// assert.Equal(t, 2*uint64(sliceHeaderSize), cache.curSize) +// assert.Equal(t, float64(2), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, 2*float64(sliceHeaderSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, 2*float64(sliceHeaderSize+cacheKeyPostings{user, id, lbls4}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) + +// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls4}, map[labels.Label][]byte{lbls4: {}}) + +// // Other metrics. +// assert.Equal(t, float64(4), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(9), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(2), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(5), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(1), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypeSeriesForRef))) +// } + +// func testFetchMultiPostings(ctx context.Context, t *testing.T, cache IndexCache, user string, id ulid.ULID, keys []labels.Label, expectedHits map[labels.Label][]byte) { +// t.Helper() +// pHits := cache.FetchMultiPostings(ctx, user, id, keys) +// expectedResult := &MapIterator[labels.Label]{M: expectedHits, Keys: keys} + +// assert.Equal(t, expectedResult.Remaining(), pHits.Remaining()) +// for exp, hasNext := expectedResult.Next(); hasNext; exp, hasNext = expectedResult.Next() { +// actual, ok := pHits.Next() +// assert.True(t, ok) +// assert.Equal(t, exp, actual) +// } +// _, ok := pHits.Next() +// assert.False(t, ok) +// } diff --git a/pkg/storage/stores/tsdb/indexcache/remote.go b/pkg/storage/stores/tsdb/indexcache/remote.go new file mode 100644 index 0000000000000..0cb3084c64aef --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/remote.go @@ -0,0 +1,307 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/memcached.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package indexcache + +// const ( +// remoteDefaultTTL = 7 * 24 * time.Hour +// ) + +// var ( +// postingsCacheKeyLabelHashBufferPool = sync.Pool{New: func() any { +// // We assume the label name/value pair is typically not longer than 1KB. +// b := make([]byte, 1024) +// return &b +// }} +// ) + +// // RemoteIndexCache is a memcached or redis based index cache. +// type RemoteIndexCache struct { +// logger log.Logger +// remote cache.RemoteCacheClient + +// // Metrics. +// requests *prometheus.CounterVec +// hits *prometheus.CounterVec +// } + +// // NewRemoteIndexCache makes a new RemoteIndexCache. +// func NewRemoteIndexCache(logger log.Logger, remote cache.RemoteCacheClient, reg prometheus.Registerer) (*RemoteIndexCache, error) { +// c := &RemoteIndexCache{ +// logger: logger, +// remote: remote, +// } + +// c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ +// Name: "thanos_store_index_cache_requests_total", +// Help: "Total number of items requests to the cache.", +// }, []string{"item_type"}) +// initLabelValuesForAllCacheTypes(c.requests.MetricVec) + +// c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ +// Name: "thanos_store_index_cache_hits_total", +// Help: "Total number of items requests to the cache that were a hit.", +// }, []string{"item_type"}) +// initLabelValuesForAllCacheTypes(c.hits.MetricVec) + +// level.Info(logger).Log("msg", "created remote index cache") + +// return c, nil +// } + +// // set stores a value for the given key in the remote cache. +// func (c *RemoteIndexCache) set(typ string, key string, val []byte) { +// if err := c.remote.SetAsync(key, val, remoteDefaultTTL); err != nil { +// level.Error(c.logger).Log("msg", "failed to set item in remote cache", "type", typ, "err", err) +// } +// } + +// // get retrieves a single value from the remote cache, returned bool value indicates whether the value was found or not. +// func (c *RemoteIndexCache) get(ctx context.Context, typ string, key string) ([]byte, bool) { +// c.requests.WithLabelValues(typ).Inc() +// results := c.remote.GetMulti(ctx, []string{key}) +// data, ok := results[key] +// if ok { +// c.hits.WithLabelValues(typ).Inc() +// } +// return data, ok +// } + +// // StorePostings sets the postings identified by the ulid and label to the value v. +// // The function enqueues the request and returns immediately: the entry will be +// // asynchronously stored in the cache. +// func (c *RemoteIndexCache) StorePostings(userID string, blockID ulid.ULID, l labels.Label, v []byte) { +// c.set(cacheTypePostings, postingsCacheKey(userID, blockID.String(), l), v) +// } + +// // FetchMultiPostings fetches multiple postings - each identified by a label. +// // In case of error, it logs and return an empty result. +// func (c *RemoteIndexCache) FetchMultiPostings(ctx context.Context, userID string, blockID ulid.ULID, lbls []labels.Label) BytesResult { +// blockIDStr := blockID.String() + +// keys := make([]string, 0, len(lbls)) +// for _, lbl := range lbls { +// keys = append(keys, postingsCacheKey(userID, blockIDStr, lbl)) +// } + +// // Fetch the keys from the remote cache in a single request. +// c.requests.WithLabelValues(cacheTypePostings).Add(float64(len(keys))) +// results := c.remote.GetMulti(ctx, keys) +// c.hits.WithLabelValues(cacheTypePostings).Add(float64(len(results))) + +// return &MapIterator[string]{ +// Keys: keys, +// M: results, +// } +// } + +// // postingsCacheKey returns the cache key used to store postings matching the input +// // label name/value pair in the given block. +// func postingsCacheKey(userID, blockID string, l labels.Label) string { +// const ( +// prefix = "P2:" +// separator = ":" +// ) + +// // Compute the label hash. +// lblHash, hashLen := postingsCacheKeyLabelID(l) + +// // Preallocate the byte slice used to store the cache key. +// encodedHashLen := base64.RawURLEncoding.EncodedLen(hashLen) +// expectedLen := len(prefix) + len(userID) + 1 + ulid.EncodedSize + 1 + encodedHashLen +// key := make([]byte, expectedLen) +// offset := 0 + +// offset += copy(key[offset:], prefix) +// offset += copy(key[offset:], userID) +// offset += copy(key[offset:], separator) +// offset += copy(key[offset:], blockID) +// offset += copy(key[offset:], separator) +// base64.RawURLEncoding.Encode(key[offset:], lblHash[:hashLen]) +// offset += encodedHashLen + +// sizedKey := key[:offset] +// // Convert []byte to string with no extra allocation. +// return *(*string)(unsafe.Pointer(&sizedKey)) +// } + +// // postingsCacheKeyLabelID returns the hash of the input label or the label itself if it is shorter than the size of the hash. +// // This is used as part of the cache key generated by postingsCacheKey(). +// func postingsCacheKeyLabelID(l labels.Label) (out [blake2b.Size256]byte, outLen int) { +// const separator = ":" + +// // Compute the expected length. +// expectedLen := len(l.Name) + len(separator) + len(l.Value) + +// // If the whole label is smaller than the hash, then shortcut hashing and directly write out the result. +// if expectedLen <= blake2b.Size256 { +// offset := 0 +// offset += copy(out[offset:], l.Name) +// offset += copy(out[offset:], separator) +// offset += copy(out[offset:], l.Value) +// return out, offset +// } + +// // Get a buffer from the pool and fill it with the label name/value pair to hash. +// bp := postingsCacheKeyLabelHashBufferPool.Get().(*[]byte) +// buf := *bp + +// if cap(buf) < expectedLen { +// buf = make([]byte, expectedLen) +// } else { +// buf = buf[:expectedLen] +// } + +// offset := 0 +// offset += copy(buf[offset:], l.Name) +// offset += copy(buf[offset:], separator) +// offset += copy(buf[offset:], l.Value) + +// // This is expected to be always equal. If it's not, then it's a severe bug. +// if offset != expectedLen { +// panic(fmt.Sprintf("postingsCacheKeyLabelID() computed an invalid expected length (expected: %d, actual: %d)", expectedLen, offset)) +// } + +// // Use cryptographically hash functions to avoid hash collisions +// // which would end up in wrong query results. +// hash := blake2b.Sum256(buf) + +// // Reuse the same pointer to put the buffer back into the pool. +// *bp = buf +// postingsCacheKeyLabelHashBufferPool.Put(bp) + +// return hash, len(hash) +// } + +// // StoreSeriesForRef sets the series identified by the ulid and id to the value v. +// // The function enqueues the request and returns immediately: the entry will be +// // asynchronously stored in the cache. +// func (c *RemoteIndexCache) StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) { +// c.set(cacheTypeSeriesForRef, seriesForRefCacheKey(userID, blockID, id), v) +// } + +// // FetchMultiSeriesForRefs fetches multiple series - each identified by ID - from the cache +// // and returns a map containing cache hits, along with a list of missing IDs. +// // In case of error, it logs and return an empty cache hits map. +// func (c *RemoteIndexCache) FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) { +// // Build the cache keys, while keeping a map between input id and the cache key +// // so that we can easily reverse it back after the GetMulti(). +// keys := make([]string, 0, len(ids)) +// keysMapping := make(map[storage.SeriesRef]string, len(ids)) + +// for _, id := range ids { +// key := seriesForRefCacheKey(userID, blockID, id) + +// keys = append(keys, key) +// keysMapping[id] = key +// } + +// // Fetch the keys from the remote cache in a single request. +// c.requests.WithLabelValues(cacheTypeSeriesForRef).Add(float64(len(ids))) +// results := c.remote.GetMulti(ctx, keys) +// if len(results) == 0 { +// return nil, ids +// } + +// // Construct the resulting hits map and list of missing keys. We iterate on the input +// // list of ids to be able to easily create the list of ones in a single iteration. +// hits = make(map[storage.SeriesRef][]byte, len(results)) +// if numMisses := len(ids) - len(results); numMisses > 0 { +// misses = make([]storage.SeriesRef, 0, numMisses) +// } + +// for _, id := range ids { +// key, ok := keysMapping[id] +// if !ok { +// _ = level.Error(c.logger).Log("msg", "keys mapping inconsistency found in remote index cache client", "type", "series", "id", id) +// continue +// } + +// // Check if the key has been found in the remote cache. If not, we add it to the list +// // of missing keys. +// value, ok := results[key] +// if !ok { +// misses = append(misses, id) +// continue +// } + +// hits[id] = value +// } + +// c.hits.WithLabelValues(cacheTypeSeriesForRef).Add(float64(len(hits))) +// return hits, misses +// } + +// func seriesForRefCacheKey(userID string, blockID ulid.ULID, id storage.SeriesRef) string { +// // Max uint64 string representation is no longer than 20 characters. +// b := make([]byte, 0, 20) +// return "S:" + userID + ":" + blockID.String() + ":" + string(strconv.AppendUint(b, uint64(id), 10)) +// } + +// // StoreExpandedPostings stores the encoded result of ExpandedPostings for specified matchers identified by the provided LabelMatchersKey. +// func (c *RemoteIndexCache) StoreExpandedPostings(userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string, v []byte) { +// c.set(cacheTypeExpandedPostings, expandedPostingsCacheKey(userID, blockID, lmKey, postingsSelectionStrategy), v) +// } + +// // FetchExpandedPostings fetches the encoded result of ExpandedPostings for specified matchers identified by the provided LabelMatchersKey. +// func (c *RemoteIndexCache) FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) { +// return c.get(ctx, cacheTypeExpandedPostings, expandedPostingsCacheKey(userID, blockID, lmKey, postingsSelectionStrategy)) +// } + +// func expandedPostingsCacheKey(userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string) string { +// hash := blake2b.Sum256([]byte(lmKey)) +// return "E2:" + userID + ":" + blockID.String() + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) + ":" + postingsSelectionStrategy +// } + +// // StoreSeriesForPostings stores a series set for the provided postings. +// func (c *RemoteIndexCache) StoreSeriesForPostings(userID string, blockID ulid.ULID, postingsKey PostingsKey, v []byte) { +// c.set(cacheTypeSeriesForPostings, seriesForPostingsCacheKey(userID, blockID, postingsKey), v) +// } + +// // FetchSeriesForPostings fetches a series set for the provided postings. +// func (c *RemoteIndexCache) FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) ([]byte, bool) { +// return c.get(ctx, cacheTypeSeriesForPostings, seriesForPostingsCacheKey(userID, blockID, shard, postingsKey)) +// } + +// func seriesForPostingsCacheKey(userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) string { +// // We use SP2: as +// // * S: is already used for SeriesForRef +// // * SS: is already used for Series +// // * SP: was in use when using gob encoding +// // +// // "SP2" (3) + userID (150) + blockID (26) + shard (10 with up to 1000 shards) + ":" (4) = 193 +// // Memcached limits key length to 250, so we're left with 57 bytes for the postings key. +// return "SP2:" + userID + ":" + blockID.String() + ":" + shardKey(shard) + ":" + string(postingsKey) +// } + +// // StoreLabelNames stores the result of a LabelNames() call. +// func (c *RemoteIndexCache) StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) { +// c.set(cacheTypeLabelNames, labelNamesCacheKey(userID, blockID, matchersKey), v) +// } + +// // FetchLabelNames fetches the result of a LabelNames() call. +// func (c *RemoteIndexCache) FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) { +// return c.get(ctx, cacheTypeLabelNames, labelNamesCacheKey(userID, blockID, matchersKey)) +// } + +// func labelNamesCacheKey(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) string { +// hash := blake2b.Sum256([]byte(matchersKey)) +// return "LN:" + userID + ":" + blockID.String() + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) +// } + +// // StoreLabelValues stores the result of a LabelValues() call. +// func (c *RemoteIndexCache) StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) { +// c.set(cacheTypeLabelValues, labelValuesCacheKey(userID, blockID, labelName, matchersKey), v) +// } + +// // FetchLabelValues fetches the result of a LabelValues() call. +// func (c *RemoteIndexCache) FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) { +// return c.get(ctx, cacheTypeLabelValues, labelValuesCacheKey(userID, blockID, labelName, matchersKey)) +// } + +// func labelValuesCacheKey(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) string { +// hash := blake2b.Sum256([]byte(matchersKey)) +// return "LV2:" + userID + ":" + blockID.String() + ":" + labelName + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) +// } diff --git a/pkg/storage/stores/tsdb/indexcache/remote_test.go b/pkg/storage/stores/tsdb/indexcache/remote_test.go new file mode 100644 index 0000000000000..ab7276a1fe972 --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/remote_test.go @@ -0,0 +1,937 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/memcached_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package indexcache + +// func TestRemoteIndexCache_FetchMultiPostings(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// label1 := labels.Label{Name: "instance", Value: "a"} +// label2 := labels.Label{Name: "instance", Value: "b"} +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} + +// tests := map[string]struct { +// setup []mockedPostings +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchLabels []labels.Label +// expectedHits map[labels.Label][]byte +// expectedMisses []labels.Label +// }{ +// "should return no hits on empty cache": { +// setup: []mockedPostings{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabels: []labels.Label{label1, label2}, +// expectedHits: nil, +// expectedMisses: []labels.Label{label1, label2}, +// }, +// "should return no misses on 100% hit ratio": { +// setup: []mockedPostings{ +// {userID: user1, block: block1, label: label1, value: value1}, +// {userID: user2, block: block1, label: label1, value: value2}, +// {userID: user1, block: block1, label: label2, value: value2}, +// {userID: user1, block: block2, label: label1, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabels: []labels.Label{label1, label2}, +// expectedHits: map[labels.Label][]byte{ +// label1: value1, +// label2: value2, +// }, +// expectedMisses: []labels.Label{}, +// }, +// "should return hits and misses on partial hits": { +// setup: []mockedPostings{ +// {userID: user1, block: block1, label: label1, value: value1}, +// {userID: user1, block: block2, label: label1, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabels: []labels.Label{label1, label2}, +// expectedHits: map[labels.Label][]byte{label1: value1}, +// expectedMisses: []labels.Label{label2}, +// }, +// "should return no hits on remote cache error": { +// setup: []mockedPostings{ +// {userID: user1, block: block1, label: label1, value: value1}, +// {userID: user1, block: block1, label: label2, value: value2}, +// {userID: user1, block: block2, label: label1, value: value3}, +// }, +// mockedErr: errors.New("mocked error"), +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabels: []labels.Label{label1, label2}, +// expectedHits: nil, +// expectedMisses: []labels.Label{label1, label2}, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the postings expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StorePostings(p.userID, p.block, p.label, p.value) +// } + +// // Fetch postings from cached and assert on it. +// testFetchMultiPostings(ctx, t, c, testData.fetchUserID, testData.fetchBlockID, testData.fetchLabels, testData.expectedHits) + +// // Assert on metrics. +// assert.Equal(t, float64(len(testData.fetchLabels)), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypePostings))) +// assert.Equal(t, float64(len(testData.expectedHits)), prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypePostings))) +// for _, typ := range remove(allCacheTypes, cacheTypePostings) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func BenchmarkRemoteIndexCache_FetchMultiPostings(b *testing.B) { +// const ( +// numHits = 10000 +// numMisses = 10000 + +// numKeys = numHits + numMisses +// ) + +// var ( +// ctx = context.Background() +// userID = "user-1" +// blockID = ulid.MustNew(1, nil) +// ) + +// benchCases := map[string]struct { +// fetchLabels []labels.Label +// }{ +// "short labels": { +// fetchLabels: func() []labels.Label { +// fetchLabels := make([]labels.Label, 0, numKeys) +// for i := 0; i < numKeys; i++ { +// fetchLabels = append(fetchLabels, labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("series_%d", i)}) +// } +// return fetchLabels +// }(), +// }, +// "long labels": { // this should trigger hashing the labels instead of embedding them in the cache key +// fetchLabels: func() []labels.Label { +// fetchLabels := make([]labels.Label, 0, numKeys) +// for i := 0; i < numKeys; i++ { +// fetchLabels = append(fetchLabels, labels.Label{Name: labels.MetricName, Value: "series_" + strings.Repeat(strconv.Itoa(i), 100)}) +// } +// return fetchLabels +// }(), +// }, +// } + +// for name, benchCase := range benchCases { +// fetchLabels := benchCase.fetchLabels +// b.Run(name, func(b *testing.B) { +// client := newMockedRemoteCacheClient(nil) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(b, err) + +// // Store the postings expected before running the benchmark. +// for i := 0; i < numHits; i++ { +// c.StorePostings(userID, blockID, fetchLabels[i], []byte{1}) +// } + +// b.ResetTimer() + +// for n := 0; n < b.N; n++ { +// results := c.FetchMultiPostings(ctx, userID, blockID, fetchLabels) +// assert.Equal(b, numKeys, results.Remaining()) +// actualHits := 0 +// // iterate over the returned map to account for cost of access +// for i := 0; i < numKeys; i++ { +// bytes, ok := results.Next() +// assert.True(b, ok) +// if bytes != nil { +// actualHits++ +// } +// } +// assert.Equal(b, numHits, actualHits) +// } +// }) +// } +// } + +// func TestRemoteIndexCache_FetchMultiSeriesForRef(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} + +// tests := map[string]struct { +// setup []mockedSeriesForRef +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchIds []storage.SeriesRef +// expectedHits map[storage.SeriesRef][]byte +// expectedMisses []storage.SeriesRef +// }{ +// "should return no hits on empty cache": { +// setup: []mockedSeriesForRef{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchIds: []storage.SeriesRef{1, 2}, +// expectedHits: nil, +// expectedMisses: []storage.SeriesRef{1, 2}, +// }, +// "should return no misses on 100% hit ratio": { +// setup: []mockedSeriesForRef{ +// {userID: user1, block: block1, id: 1, value: value1}, +// {userID: user2, block: block1, id: 1, value: value2}, +// {userID: user1, block: block1, id: 1, value: value1}, +// {userID: user1, block: block1, id: 2, value: value2}, +// {userID: user1, block: block2, id: 1, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchIds: []storage.SeriesRef{1, 2}, +// expectedHits: map[storage.SeriesRef][]byte{ +// 1: value1, +// 2: value2, +// }, +// expectedMisses: nil, +// }, +// "should return hits and misses on partial hits": { +// setup: []mockedSeriesForRef{ +// {userID: user1, block: block1, id: 1, value: value1}, +// {userID: user1, block: block2, id: 1, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchIds: []storage.SeriesRef{1, 2}, +// expectedHits: map[storage.SeriesRef][]byte{1: value1}, +// expectedMisses: []storage.SeriesRef{2}, +// }, +// "should return no hits on remote cache error": { +// setup: []mockedSeriesForRef{ +// {userID: user1, block: block1, id: 1, value: value1}, +// {userID: user1, block: block1, id: 2, value: value2}, +// {userID: user1, block: block2, id: 1, value: value3}, +// }, +// mockedErr: errors.New("mocked error"), +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchIds: []storage.SeriesRef{1, 2}, +// expectedHits: nil, +// expectedMisses: []storage.SeriesRef{1, 2}, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the series expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StoreSeriesForRef(p.userID, p.block, p.id, p.value) +// } + +// // Fetch series from cached and assert on it. +// hits, misses := c.FetchMultiSeriesForRefs(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchIds) +// assert.Equal(t, testData.expectedHits, hits) +// assert.Equal(t, testData.expectedMisses, misses) + +// // Assert on metrics. +// assert.Equal(t, float64(len(testData.fetchIds)), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeSeriesForRef))) +// assert.Equal(t, float64(len(testData.expectedHits)), prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeSeriesForRef))) +// for _, typ := range remove(allCacheTypes, cacheTypeSeriesForRef) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func TestRemoteIndexCache_FetchExpandedPostings(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} +// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} +// postingsStrategy1 := "s1" +// postingsStrategy2 := "s2" + +// tests := map[string]struct { +// setup []mockedExpandedPostings +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchKey LabelMatchersKey +// fetchStrategy string +// expectedData []byte +// expectedOk bool +// }{ +// "should return no hit on empty cache": { +// setup: []mockedExpandedPostings{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// "should return no miss on hit": { +// setup: []mockedExpandedPostings{ +// {userID: user1, block: block1, matchers: matchers1, value: value1, postingsStrategy: postingsStrategy1}, +// {userID: user2, block: block1, matchers: matchers1, value: value2, postingsStrategy: postingsStrategy1}, +// {userID: user1, block: block1, matchers: matchers2, value: value2, postingsStrategy: postingsStrategy1}, +// {userID: user1, block: block2, matchers: matchers1, value: value3, postingsStrategy: postingsStrategy1}, +// {userID: user1, block: block1, matchers: matchers1, value: value1, postingsStrategy: postingsStrategy2}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// fetchStrategy: postingsStrategy1, +// expectedData: value1, +// expectedOk: true, +// }, +// "should return no hit on remote cache error": { +// setup: []mockedExpandedPostings{ +// {userID: user1, block: block1, matchers: matchers1, value: value1}, +// {userID: user1, block: block1, matchers: matchers2, value: value2}, +// {userID: user1, block: block2, matchers: matchers1, value: value3}, +// }, +// mockedErr: context.DeadlineExceeded, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the postings expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StoreExpandedPostings(p.userID, p.block, CanonicalLabelMatchersKey(p.matchers), testData.fetchStrategy, p.value) +// } + +// // Fetch postings from cached and assert on it. +// data, ok := c.FetchExpandedPostings(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchKey, testData.fetchStrategy) +// assert.Equal(t, testData.expectedData, data) +// assert.Equal(t, testData.expectedOk, ok) + +// // Assert on metrics. +// expectedHits := 0.0 +// if testData.expectedOk { +// expectedHits = 1.0 +// } +// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeExpandedPostings))) +// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeExpandedPostings))) +// for _, typ := range remove(allCacheTypes, cacheTypeExpandedPostings) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func TestRemoteIndexCache_FetchSeriesForPostings(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} +// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} +// shard1 := (*sharding.ShardSelector)(nil) +// shard2 := &sharding.ShardSelector{ShardIndex: 1, ShardCount: 16} +// postings1 := []storage.SeriesRef{1, 2} +// postings2 := []storage.SeriesRef{2, 3} + +// tests := map[string]struct { +// setup []mockedSeries +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchKey LabelMatchersKey +// fetchShard *sharding.ShardSelector +// postings []storage.SeriesRef +// expectedData []byte +// expectedOk bool +// }{ +// "should return no hit on empty cache": { +// setup: []mockedSeries{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// fetchShard: shard1, +// postings: postings1, +// expectedData: nil, +// expectedOk: false, +// }, +// "should return no miss on hit": { +// setup: []mockedSeries{ +// {userID: user1, block: block1, shard: shard1, postings: postings1, value: value1}, +// {userID: user2, block: block1, shard: shard1, postings: postings1, value: value2}, // different user +// {userID: user1, block: block1, shard: shard2, postings: postings1, value: value2}, // different shard +// {userID: user1, block: block2, shard: shard1, postings: postings1, value: value3}, // different block +// {userID: user1, block: block2, shard: shard1, postings: postings2, value: value3}, // different postings +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// fetchShard: shard1, +// postings: postings1, +// expectedData: value1, +// expectedOk: true, +// }, +// "should return no hit on remote cache error": { +// setup: []mockedSeries{ +// {userID: user1, block: block1, matchers: matchers1, shard: shard1, postings: postings1, value: value1}, +// {userID: user1, block: block1, matchers: matchers2, shard: shard1, postings: postings1, value: value2}, +// {userID: user1, block: block2, matchers: matchers1, shard: shard1, postings: postings1, value: value3}, +// }, +// mockedErr: context.DeadlineExceeded, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// fetchShard: shard1, +// postings: postings1, +// expectedData: nil, +// expectedOk: false, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the postings expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StoreSeriesForPostings(p.userID, p.block, p.shard, CanonicalPostingsKey(p.postings), p.value) +// } + +// // Fetch postings from cached and assert on it. +// data, ok := c.FetchSeriesForPostings(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchShard, CanonicalPostingsKey(testData.postings)) +// assert.Equal(t, testData.expectedData, data) +// assert.Equal(t, testData.expectedOk, ok) + +// // Assert on metrics. +// expectedHits := 0.0 +// if testData.expectedOk { +// expectedHits = 1.0 +// } +// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeSeriesForPostings))) +// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeSeriesForPostings))) +// for _, typ := range remove(allCacheTypes, cacheTypeSeriesForPostings) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func TestRemoteIndexCache_FetchLabelNames(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} +// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} + +// tests := map[string]struct { +// setup []mockedLabelNames +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchKey LabelMatchersKey +// expectedData []byte +// expectedOk bool +// }{ +// "should return no hit on empty cache": { +// setup: []mockedLabelNames{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// "should return no miss on hit": { +// setup: []mockedLabelNames{ +// {userID: user1, block: block1, matchers: matchers1, value: value1}, +// {userID: user2, block: block1, matchers: matchers1, value: value2}, +// {userID: user1, block: block1, matchers: matchers2, value: value2}, +// {userID: user1, block: block2, matchers: matchers1, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: value1, +// expectedOk: true, +// }, +// "should return no hit on remote cache error": { +// setup: []mockedLabelNames{ +// {userID: user1, block: block1, matchers: matchers1, value: value1}, +// {userID: user1, block: block1, matchers: matchers2, value: value2}, +// {userID: user1, block: block2, matchers: matchers1, value: value3}, +// }, +// mockedErr: context.DeadlineExceeded, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the postings expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StoreLabelNames(p.userID, p.block, CanonicalLabelMatchersKey(p.matchers), p.value) +// } + +// // Fetch postings from cached and assert on it. +// data, ok := c.FetchLabelNames(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchKey) +// assert.Equal(t, testData.expectedData, data) +// assert.Equal(t, testData.expectedOk, ok) + +// // Assert on metrics. +// expectedHits := 0.0 +// if testData.expectedOk { +// expectedHits = 1.0 +// } +// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeLabelNames))) +// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeLabelNames))) +// for _, typ := range remove(allCacheTypes, cacheTypeLabelNames) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func TestRemoteIndexCache_FetchLabelValues(t *testing.T) { +// t.Parallel() + +// // Init some data to conveniently define test cases later one. +// user1 := "tenant1" +// user2 := "tenant2" +// block1 := ulid.MustNew(1, nil) +// block2 := ulid.MustNew(2, nil) +// labelName1 := "one" +// labelName2 := "two" +// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} +// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} +// value1 := []byte{1} +// value2 := []byte{2} +// value3 := []byte{3} + +// tests := map[string]struct { +// setup []mockedLabelValues +// mockedErr error +// fetchUserID string +// fetchBlockID ulid.ULID +// fetchLabelName string +// fetchKey LabelMatchersKey +// expectedData []byte +// expectedOk bool +// }{ +// "should return no hit on empty cache": { +// setup: []mockedLabelValues{}, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabelName: labelName1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// "should return no miss on hit": { +// setup: []mockedLabelValues{ +// {userID: user1, block: block1, labelName: labelName1, matchers: matchers1, value: value1}, +// {userID: user2, block: block1, labelName: labelName1, matchers: matchers1, value: value2}, +// {userID: user1, block: block1, labelName: labelName2, matchers: matchers2, value: value2}, +// {userID: user1, block: block2, labelName: labelName1, matchers: matchers1, value: value3}, +// {userID: user1, block: block2, labelName: labelName1, matchers: matchers2, value: value3}, +// }, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabelName: labelName1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: value1, +// expectedOk: true, +// }, +// "should return no hit on remote cache error": { +// setup: []mockedLabelValues{ +// {userID: user1, block: block1, labelName: labelName1, matchers: matchers1, value: value1}, +// {userID: user1, block: block1, labelName: labelName2, matchers: matchers2, value: value2}, +// {userID: user1, block: block2, labelName: labelName1, matchers: matchers1, value: value3}, +// {userID: user1, block: block2, labelName: labelName1, matchers: matchers2, value: value3}, +// }, +// mockedErr: context.DeadlineExceeded, +// fetchUserID: user1, +// fetchBlockID: block1, +// fetchLabelName: labelName1, +// fetchKey: CanonicalLabelMatchersKey(matchers1), +// expectedData: nil, +// expectedOk: false, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// client := newMockedRemoteCacheClient(testData.mockedErr) +// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) +// assert.NoError(t, err) + +// // Store the postings expected before running the test. +// ctx := context.Background() +// for _, p := range testData.setup { +// c.StoreLabelValues(p.userID, p.block, p.labelName, CanonicalLabelMatchersKey(p.matchers), p.value) +// } + +// // Fetch postings from cached and assert on it. +// data, ok := c.FetchLabelValues(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchLabelName, testData.fetchKey) +// assert.Equal(t, testData.expectedData, data) +// assert.Equal(t, testData.expectedOk, ok) + +// // Assert on metrics. +// expectedHits := 0.0 +// if testData.expectedOk { +// expectedHits = 1.0 +// } +// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeLabelValues))) +// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeLabelValues))) +// for _, typ := range remove(allCacheTypes, cacheTypeLabelValues) { +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) +// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) +// } +// }) +// } +// } + +// func TestStringCacheKeys_Values(t *testing.T) { +// t.Parallel() + +// user := "tenant" +// uid := ulid.MustNew(1, nil) + +// tests := map[string]struct { +// key string +// expected string +// }{ +// "should stringify postings cache key": { +// key: postingsCacheKey(user, uid.String(), labels.Label{Name: "foo", Value: "bar"}), +// expected: func() string { +// encodedLabel := base64.RawURLEncoding.EncodeToString([]byte("foo:bar")) +// return fmt.Sprintf("P2:%s:%s:%s", user, uid.String(), encodedLabel) +// }(), +// }, +// "should hash long postings cache key": { +// key: postingsCacheKey(user, uid.String(), labels.Label{Name: "foo", Value: strings.Repeat("bar", 11)}), +// expected: func() string { +// hash := blake2b.Sum256([]byte("foo:" + strings.Repeat("bar", 11))) +// encodedHash := base64.RawURLEncoding.EncodeToString(hash[0:]) + +// return fmt.Sprintf("P2:%s:%s:%s", user, uid.String(), encodedHash) +// }(), +// }, +// "should stringify series cache key": { +// key: seriesForRefCacheKey(user, uid, 12345), +// expected: fmt.Sprintf("S:%s:%s:12345", user, uid.String()), +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// actual := testData.key +// assert.Equal(t, testData.expected, actual) +// }) +// } +// } + +// func TestStringCacheKeys_ShouldGuaranteeReasonablyShortKeysLength(t *testing.T) { +// t.Parallel() + +// user := "tenant" +// uid := ulid.MustNew(1, nil) + +// tests := map[string]struct { +// keys []string +// expectedLen int +// }{ +// "should guarantee reasonably short key length for postings": { +// expectedLen: 80, +// keys: []string{ +// postingsCacheKey(user, uid.String(), labels.Label{Name: "a", Value: "b"}), +// postingsCacheKey(user, uid.String(), labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)}), +// }, +// }, +// "should guarantee reasonably short key length for series": { +// expectedLen: 56, +// keys: []string{ +// seriesForRefCacheKey(user, uid, math.MaxUint64), +// }, +// }, +// } + +// for testName, testData := range tests { +// t.Run(testName, func(t *testing.T) { +// for _, key := range testData.keys { +// assert.LessOrEqual(t, len(key), testData.expectedLen) +// } +// }) +// } +// } + +// func BenchmarkStringCacheKeys(b *testing.B) { +// userID := "tenant" +// uid := ulid.MustNew(1, nil) +// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} +// lmKey := CanonicalLabelMatchersKey([]*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")}) + +// b.Run("postings", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// postingsCacheKey(userID, uid.String(), lbl) +// } +// }) + +// b.Run("series ref", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// seriesForRefCacheKey(userID, uid, math.MaxUint64) +// } +// }) + +// b.Run("expanded postings", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// expandedPostingsCacheKey(userID, uid, lmKey, "strategy") +// } +// }) +// } + +// func TestPostingsCacheKey_ShouldOnlyAllocateOncePerCall(t *testing.T) { +// const numRuns = 1000 + +// blockID := ulid.MustNew(1, nil) +// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} + +// actualAllocs := testing.AllocsPerRun(numRuns, func() { +// postingsCacheKey("user-1", blockID.String(), lbl) +// }) + +// // Allow for 1 extra allocation here, reported when running the test with -race. +// assert.LessOrEqual(t, actualAllocs, 2.0) +// } + +// func TestPostingsCacheKeyLabelHash_ShouldNotAllocateMemory(t *testing.T) { +// const numRuns = 1000 + +// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} + +// actualAllocs := testing.AllocsPerRun(numRuns, func() { +// postingsCacheKeyLabelID(lbl) +// }) + +// // Allow for 1 extra allocation here, reported when running the test with -race. +// assert.LessOrEqual(t, actualAllocs, 1.0) +// } + +// func TestPostingsCacheKeyLabelHash_ShouldBeConcurrencySafe(t *testing.T) { +// const ( +// numWorkers = 10 +// numRunsPerWorker = 10000 +// ) + +// // Generate a different label per worker, and their expected hash. +// inputPerWorker := make([]labels.Label, 0, numWorkers) +// expectedPerWorker := make([][]byte, 0, numWorkers) + +// for w := 0; w < numWorkers; w++ { +// inputPerWorker = append(inputPerWorker, labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("series_%d", w)}) + +// hash, hashLen := postingsCacheKeyLabelID(inputPerWorker[w]) +// expectedPerWorker = append(expectedPerWorker, hash[0:hashLen]) +// } + +// // Sanity check: ensure expected hashes are different for each worker. +// for w := 0; w < numWorkers; w++ { +// for c := 0; c < numWorkers; c++ { +// if w == c { +// continue +// } + +// require.NotEqual(t, expectedPerWorker[w], expectedPerWorker[c]) +// } +// } + +// // Run workers, each generating the hash for their own label. +// wg := sync.WaitGroup{} +// wg.Add(numWorkers) + +// for w := 0; w < numWorkers; w++ { +// go func(workerID int) { +// defer wg.Done() + +// for r := 0; r < numRunsPerWorker; r++ { +// actual, hashLen := postingsCacheKeyLabelID(inputPerWorker[workerID]) +// assert.Equal(t, expectedPerWorker[workerID], actual[0:hashLen]) +// } +// }(w) +// } + +// wg.Wait() +// } + +// type mockedPostings struct { +// userID string +// block ulid.ULID +// label labels.Label +// value []byte +// } + +// type mockedSeriesForRef struct { +// userID string +// block ulid.ULID +// id storage.SeriesRef +// value []byte +// } + +// type mockedExpandedPostings struct { +// userID string +// block ulid.ULID +// matchers []*labels.Matcher +// postingsStrategy string +// value []byte +// } + +// type mockedLabelNames struct { +// userID string +// block ulid.ULID +// matchers []*labels.Matcher +// value []byte +// } + +// type mockedSeries struct { +// userID string +// block ulid.ULID +// matchers []*labels.Matcher +// shard *sharding.ShardSelector +// postings []storage.SeriesRef +// value []byte +// } + +// type mockedLabelValues struct { +// userID string +// block ulid.ULID +// labelName string +// matchers []*labels.Matcher +// value []byte +// } + +// type mockedRemoteCacheClient struct { +// cache map[string][]byte +// mockedGetMultiErr error +// } + +// func newMockedRemoteCacheClient(mockedGetMultiErr error) *mockedRemoteCacheClient { +// return &mockedRemoteCacheClient{ +// cache: map[string][]byte{}, +// mockedGetMultiErr: mockedGetMultiErr, +// } +// } + +// func (c *mockedRemoteCacheClient) GetMulti(_ context.Context, keys []string, _ ...cache.Option) map[string][]byte { +// if c.mockedGetMultiErr != nil { +// return nil +// } + +// hits := map[string][]byte{} + +// for _, key := range keys { +// if value, ok := c.cache[key]; ok { +// hits[key] = value +// } +// } + +// return hits +// } + +// func (c *mockedRemoteCacheClient) SetAsync(key string, value []byte, _ time.Duration) error { +// c.cache[key] = value + +// return nil +// } + +// func (c *mockedRemoteCacheClient) Delete(_ context.Context, key string) error { +// delete(c.cache, key) + +// return nil +// } + +// func (c *mockedRemoteCacheClient) Stop() { +// // Nothing to do. +// } + +// // remove a string from a slice of strings +// func remove(slice []string, needle string) []string { +// res := make([]string, 0, len(slice)) +// for _, s := range slice { +// if s != needle { +// res = append(res, s) +// } +// } +// return res +// } diff --git a/pkg/storage/stores/tsdb/indexcache/tracing.go b/pkg/storage/stores/tsdb/indexcache/tracing.go new file mode 100644 index 0000000000000..fb97795f6575a --- /dev/null +++ b/pkg/storage/stores/tsdb/indexcache/tracing.go @@ -0,0 +1,144 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package indexcache + +// type TracingIndexCache struct { +// c IndexCache +// logger log.Logger +// } + +// func NewTracingIndexCache(cache IndexCache, logger log.Logger) IndexCache { +// return &TracingIndexCache{ +// c: cache, +// logger: logger, +// } +// } + +// func (t *TracingIndexCache) StorePostings(userID string, blockID ulid.ULID, l labels.Label, v []byte) { +// t.c.StorePostings(userID, blockID, l, v) +// } + +// func (t *TracingIndexCache) FetchMultiPostings(ctx context.Context, userID string, blockID ulid.ULID, keys []labels.Label) (hits BytesResult) { +// t0 := time.Now() +// hits = t.c.FetchMultiPostings(ctx, userID, blockID, keys) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log( +// "msg", "IndexCache.FetchMultiPostings", +// "requested keys", len(keys), +// "cache hits", hits.Remaining(), +// "cache misses", len(keys)-hits.Remaining(), +// "time elapsed", time.Since(t0), +// "returned bytes", hits.Size(), +// "user_id", userID, +// ) +// return hits +// } + +// func (t *TracingIndexCache) StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) { +// t.c.StoreSeriesForRef(userID, blockID, id, v) +// } + +// func (t *TracingIndexCache) FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) { +// t0 := time.Now() +// hits, misses = t.c.FetchMultiSeriesForRefs(ctx, userID, blockID, ids) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log("msg", "IndexCache.FetchMultiSeriesForRefs", +// "requested series", len(ids), +// "cache hits", len(hits), +// "cache misses", len(misses), +// "time elapsed", time.Since(t0), +// "returned bytes", sumBytes(hits), +// "user_id", userID, +// ) + +// return hits, misses +// } + +// func (t *TracingIndexCache) StoreExpandedPostings(userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string, v []byte) { +// t.c.StoreExpandedPostings(userID, blockID, key, postingsSelectionStrategy, v) +// } + +// func (t *TracingIndexCache) FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) { +// t0 := time.Now() +// data, found := t.c.FetchExpandedPostings(ctx, userID, blockID, key, postingsSelectionStrategy) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log( +// "msg", "IndexCache.FetchExpandedPostings", +// "requested key", key, +// "postings selection strategy", postingsSelectionStrategy, +// "found", found, +// "time elapsed", time.Since(t0), +// "returned bytes", len(data), +// "user_id", userID, +// ) + +// return data, found +// } + +// func (t *TracingIndexCache) StoreSeriesForPostings(userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey, v []byte) { +// t.c.StoreSeriesForPostings(userID, blockID, shard, postingsKey, v) +// } + +// func (t *TracingIndexCache) FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) ([]byte, bool) { +// t0 := time.Now() +// data, found := t.c.FetchSeriesForPostings(ctx, userID, blockID, shard, postingsKey) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log( +// "msg", "IndexCache.FetchSeriesForPostings", +// "shard", shardKey(shard), +// "found", found, +// "time_elapsed", time.Since(t0), +// "returned_bytes", len(data), +// "user_id", userID, +// "postings_key", postingsKey, +// ) + +// return data, found +// } + +// func (t *TracingIndexCache) StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) { +// t.c.StoreLabelNames(userID, blockID, matchersKey, v) +// } + +// func (t *TracingIndexCache) FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) { +// t0 := time.Now() +// data, found := t.c.FetchLabelNames(ctx, userID, blockID, matchersKey) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log( +// "msg", "IndexCache.FetchLabelNames", +// "requested key", matchersKey, +// "found", found, +// "time elapsed", time.Since(t0), +// "returned bytes", len(data), +// "user_id", userID, +// ) + +// return data, found +// } + +// func (t *TracingIndexCache) StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) { +// t.c.StoreLabelValues(userID, blockID, labelName, matchersKey, v) +// } + +// func (t *TracingIndexCache) FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) { +// t0 := time.Now() +// data, found := t.c.FetchLabelValues(ctx, userID, blockID, labelName, matchersKey) + +// spanLogger := spanlogger.FromContext(ctx, t.logger) +// level.Debug(spanLogger).Log( +// "msg", "IndexCache.FetchLabelValues", +// "label name", labelName, +// "requested key", matchersKey, +// "found", found, +// "time elapsed", time.Since(t0), +// "returned bytes", len(data), +// "user_id", userID, +// ) + +// return data, found +// } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index f79395599e2e8..ebebbbe8ea967 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -116,8 +116,9 @@ func (f *TSDBFile) Reader() (io.ReadSeeker, error) { // and translates the IndexReader to an Index implementation // It loads the file into memory and doesn't keep a file descriptor open type TSDBIndex struct { - reader IndexReader - chunkFilter chunk.RequestChunkFilterer + reader IndexReader + chunkFilter chunk.RequestChunkFilterer + postingsClient PostingsClient } // Return the index as well as the underlying raw file reader which isn't exposed as an index @@ -128,21 +129,47 @@ func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) return nil, nil, err } - var idx Index - if shouldCachePostings { - idx = NewCachedPostingsTSDBIndex(reader) - } else { - idx = NewTSDBIndex(reader) - } + tsdbIdx := NewTSDBIndex(reader, getPostingsClient(reader)) - return idx, func() (io.ReadSeeker, error) { + return tsdbIdx, func() (io.ReadSeeker, error) { return reader.RawFileReader() }, nil } -func NewTSDBIndex(reader IndexReader) *TSDBIndex { +func getPostingsClient(reader IndexReader) PostingsClient { + var postingsClient PostingsClient + + if shouldCachePostings && cacheClient != nil { + postingsClient = NewCachedPostingsClient(reader) + } + + if postingsClient == nil { + postingsClient = DefaultPostingsClient(reader) + } + + return postingsClient +} + +func DefaultPostingsClient(reader IndexReader) PostingsClient { + return &simpleForPostingsClient{reader: reader} +} + +type simpleForPostingsClient struct { + reader IndexReader +} + +func (s *simpleForPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { + p, err := PostingsForMatchers(s.reader, nil, matchers...) + if err != nil { + return err + } + return fn(p) +} + +func NewTSDBIndex(reader IndexReader, postingsClient PostingsClient) *TSDBIndex { return &TSDBIndex{ - reader: reader, + reader: reader, + postingsClient: postingsClient, } } @@ -173,7 +200,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, filterer = i.chunkFilter.ForRequest(ctx) } - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + return i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { for p.Next() { hash, err := i.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) if err != nil { @@ -196,20 +223,6 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, } -func (i *TSDBIndex) forPostings( - ctx context.Context, - shard *index.ShardAnnotation, - from, through model.Time, - matchers []*labels.Matcher, - fn func(index.Postings) error, -) error { - p, err := PostingsForMatchers(i.reader, shard, matchers...) - if err != nil { - return err - } - return fn(p) -} - func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { if err := i.ForSeries(ctx, shard, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { for _, chk := range chks { @@ -279,7 +292,7 @@ func (i *TSDBIndex) Identifier(string) SingleTenantTSDBIdentifier { } func (i *TSDBIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + return i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { // TODO(owen-d): use pool var ls labels.Labels var filterer chunk.Filterer diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 8707e66583b6f..b1abfb890539d 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -78,7 +78,7 @@ func TestSingleIdx(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader) + return NewTSDBIndex(reader, DefaultPostingsClient(reader)) }, }, } { diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index c0f75ecc51e8e..9bd000aea1bf0 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -14,6 +14,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/chunk/fetcher" "github.com/grafana/loki/pkg/storage/config" @@ -48,6 +49,7 @@ func NewStore( backupIndexWriter index.Writer, reg prometheus.Registerer, logger log.Logger, + idxCache cache.Cache, ) ( index.ReaderWriter, func(), @@ -62,7 +64,7 @@ func NewStore( logger: logger, } - if err := storeInstance.init(name, indexShipperCfg, schemaCfg, objectClient, limits, tableRange, reg); err != nil { + if err := storeInstance.init(name, indexShipperCfg, schemaCfg, objectClient, limits, tableRange, reg, idxCache); err != nil { return nil, nil, err } @@ -70,7 +72,9 @@ func NewStore( } func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConfig, objectClient client.ObjectClient, - limits downloads.Limits, tableRange config.TableRange, reg prometheus.Registerer) error { + limits downloads.Limits, tableRange config.TableRange, reg prometheus.Registerer, idxCache cache.Cache) error { + + cacheClient = idxCache if indexCfg.cachePostings { shouldCachePostings = true @@ -87,6 +91,7 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf prometheus.WrapRegistererWithPrefix("loki_tsdb_shipper_", reg), s.logger, ) + if err != nil { return err } diff --git a/tools/tsdb/tsdb-map/main_test.go b/tools/tsdb/tsdb-map/main_test.go index 5d5b9bdaa118d..ee12cf7201abb 100644 --- a/tools/tsdb/tsdb-map/main_test.go +++ b/tools/tsdb/tsdb-map/main_test.go @@ -93,7 +93,7 @@ func BenchmarkQuery_GetChunkRefs(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader) + idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsClient(reader)) b.Run(bm.name, func(b *testing.B) { refs := tsdb.ChunkRefsPool.Get() for i := 0; i < b.N; i++ { @@ -118,7 +118,7 @@ func BenchmarkQuery_GetChunkRefsSharded(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader) + idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsClient(reader)) shardFactor := 16 b.Run(bm.name, func(b *testing.B) { From 64a0af7bd9297e96f66568307beacd1866e90c15 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 8 Jun 2023 17:42:46 -0300 Subject: [PATCH 04/84] make caching configurable --- pkg/storage/stores/tsdb/index.go | 2 +- pkg/storage/stores/tsdb/store.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index c29052933d3dd..4e9af8c4d8e75 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -26,7 +26,7 @@ type ChunkRef struct { type IndexCfg struct { indexshipper.Config - cachePostings bool + CachePostings bool `yaml:"cache_postings"` } // Compares by (Start, End) diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index 9bd000aea1bf0..c7aac55ec3a2e 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -76,7 +76,7 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf cacheClient = idxCache - if indexCfg.cachePostings { + if indexCfg.CachePostings { shouldCachePostings = true } From 369869f1b0adcf10bac5d6150846605959de3e24 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 15:19:07 -0300 Subject: [PATCH 05/84] Implement LRU as a possible cache option --- pkg/storage/chunk/cache/cache.go | 13 + pkg/storage/chunk/cache/lru_cache.go | 271 ++++++++++++++++++ .../stores/tsdb/cached_postings_index.go | 167 ++++++++++- pkg/storage/stores/tsdb/head_manager.go | 2 +- pkg/storage/stores/tsdb/indexcache/cache.go | 41 --- .../stores/tsdb/indexcache/inmemory.go | 10 + pkg/storage/stores/tsdb/single_file_index.go | 11 +- pkg/storage/stores/tsdb/store.go | 2 +- 8 files changed, 455 insertions(+), 62 deletions(-) create mode 100644 pkg/storage/chunk/cache/lru_cache.go diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index d12240c87b091..c6c05e06130eb 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -35,6 +35,7 @@ type Config struct { MemcacheClient MemcachedClientConfig `yaml:"memcached_client"` Redis RedisConfig `yaml:"redis"` EmbeddedCache EmbeddedCacheConfig `yaml:"embedded_cache"` + LRUCache LRUCacheConfig `yaml:"lru_cache"` Fifocache FifoCacheConfig `yaml:"fifocache"` // deprecated // This is to name the cache metrics properly. @@ -145,6 +146,18 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger, cacheType sta } } + if cfg.LRUCache.Enabled { + cache, err := NewLRUCache(cfg.Prefix+"embedded-cache", cfg.LRUCache, reg, logger, cacheType) + if err != nil { + level.Error(logger).Log("msg", "failed to initialize LRU cache", "err", err) + return nil, err + } + + if cache != nil { + caches = append(caches, CollectStats(Instrument(cfg.Prefix+"embedded-cache", cache, reg))) + } + } + if IsMemcacheSet(cfg) && IsRedisSet(cfg) { return nil, errors.New("use of multiple cache storage systems is not supported") } diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go new file mode 100644 index 0000000000000..e69f8dc2f671f --- /dev/null +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -0,0 +1,271 @@ +package cache + +import ( + "context" + "fmt" + "sync" + "time" + "unsafe" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/loki/pkg/logqlmodel/stats" + util_log "github.com/grafana/loki/pkg/util/log" + lru "github.com/hashicorp/golang-lru/simplelru" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type codec string + +const ( + codecHeaderSnappy codec = "dvs" // As in "diff+varint+snappy". + codecHeaderSnappyWithMatchers codec = "dm" // As in "dvs+matchers" +) + +var DefaultLRUCacheConfig = LRUCacheConfig{ + MaxSizeBytes: "250MB", +} + +const maxInt = int(^uint(0) >> 1) + +const ( + stringHeaderSize = 8 + sliceHeaderSize = 16 +) + +var ulidSize = uint64(len(ulid.ULID{})) + +// This FIFO cache implementation supports two eviction methods - based on number of items in the cache, and based on memory usage. +// For the memory-based eviction, set FifoCacheConfig.MaxSizeBytes to a positive integer, indicating upper limit of memory allocated by items in the cache. +// Alternatively, set FifoCacheConfig.MaxSizeItems to a positive integer, indicating maximum number of items in the cache. +// If both parameters are set, both methods are enforced, whichever hits first. + +// FifoCacheConfig holds config for the FifoCache. +type LRUCacheConfig struct { + MaxSizeBytes string `yaml:"max_size_bytes"` + + Enabled bool + + PurgeInterval time.Duration +} + +// RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet +// func (cfg *FifoCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { +// f.StringVar(&cfg.MaxSizeBytes, prefix+"fifocache.max-size-bytes", "1GB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") +// f.IntVar(&cfg.MaxSizeItems, prefix+"fifocache.max-size-items", 0, description+"deprecated: Maximum number of entries in the cache.") +// f.DurationVar(&cfg.TTL, prefix+"fifocache.ttl", time.Hour, description+"The time to live for items in the cache before they get purged.") + +// f.DurationVar(&cfg.DeprecatedValidity, prefix+"fifocache.duration", 0, "Deprecated (use ttl instead): "+description+"The expiry duration for the cache.") +// f.IntVar(&cfg.DeprecatedSize, prefix+"fifocache.size", 0, "Deprecated (use max-size-items or max-size-bytes instead): "+description+"The number of entries to cache.") +// } + +// func (cfg *FifoCacheConfig) Validate() error { +// _, err := parsebytes(cfg.MaxSizeBytes) +// return err +// } + +// FifoCache is a simple string -> interface{} cache which uses a fifo slide to +// manage evictions. O(1) inserts and updates, O(1) gets. +type LRUCache struct { + cacheType stats.CacheType + + done chan struct{} + + // important ones below + mtx sync.Mutex + + logger log.Logger + lru *lru.LRU + maxSizeBytes uint64 + maxItemSizeBytes uint64 + + evicted *prometheus.CounterVec + requests *prometheus.CounterVec + hits *prometheus.CounterVec + totalMisses prometheus.Counter + added *prometheus.CounterVec + current *prometheus.GaugeVec + bytesInUse prometheus.Gauge + overflow *prometheus.CounterVec +} + +// TODO: better description: NewLRUCache returns a new initialised LRU cache of size. +func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (*LRUCache, error) { + util_log.WarnExperimentalUse(fmt.Sprintf("In-memory (LRU) cache - %s", name), logger) + + maxSizeBytes, _ := parsebytes(cfg.MaxSizeBytes) + + // This can be overwritten to a smaller value in tests + if cfg.PurgeInterval == 0 { + cfg.PurgeInterval = 1 * time.Minute + } + + c := &LRUCache{ + cacheType: cacheType, + + maxSizeBytes: maxSizeBytes, + logger: logger, + + done: make(chan struct{}), + } + + c.totalMisses = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Namespace: "querier", + Subsystem: "cache", + Name: "misses_total", + Help: "The total number of Get calls that had no valid entry", + ConstLabels: prometheus.Labels{"cache": name}, + }) + + c.bytesInUse = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Namespace: "querier", + Subsystem: "cache", + Name: "memory_bytes", + Help: "The current cache size in bytes", + ConstLabels: prometheus.Labels{"cache": name}, + }) + + c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_evicted_total", + Help: "Total number of items that were evicted from the index cache.", + }, []string{}) + + c.added = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_added_total", + Help: "Total number of items that were added to the index cache.", + }, []string{}) + + c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_requests_total", + Help: "Total number of requests to the cache.", + }, []string{}) + + c.overflow = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items_overflowed_total", + Help: "Total number of items that could not be added to the cache due to being too big.", + }, []string{}) + + c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_hits_total", + Help: "Total number of requests to the cache that were a hit.", + }, []string{}) + + c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "loki", + Name: "index_gateway_index_cache_items", + Help: "Current number of items in the index cache.", + }, []string{}) + + // Initialize LRU cache with a high size limit since we will manage evictions ourselves + // based on stored size using `RemoveOldest` method. + l, err := lru.NewLRU(maxInt, c.onEvict) + if err != nil { + return nil, err + } + c.lru = l + + level.Info(logger).Log( + "msg", "created in-memory index cache", + "maxItemSizeBytes", c.maxItemSizeBytes, + "maxSizeBytes", c.maxSizeBytes, + "maxItems", "maxInt", + ) + + return c, nil +} + +// Fetch implements Cache. +func (c *LRUCache) Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { + found, missing, bufs = make([]string, 0, len(keys)), make([]string, 0, len(keys)), make([][]byte, 0, len(keys)) + for _, key := range keys { + val, ok := c.get(key) + if !ok { + missing = append(missing, key) + continue + } + + found = append(found, key) + bufs = append(bufs, val) + } + return +} + +// Store implements Cache. +func (c *LRUCache) Store(ctx context.Context, keys []string, values [][]byte) error { + for i := range keys { + c.set(keys[i], values[i]) + } + + return nil +} + +// Stop implements Cache. +func (c *LRUCache) Stop() { + c.mtx.Lock() + defer c.mtx.Unlock() + + close(c.done) + + c.reset() +} + +func (c *LRUCache) GetCacheType() stats.CacheType { + return c.cacheType +} + +func (c *LRUCache) onEvict(key, val interface{}) { + c.evicted.WithLabelValues().Inc() + c.current.WithLabelValues().Dec() + c.bytesInUse.Sub(float64(c.entryMemoryUsage(key.(string), val.([]byte)))) +} + +func (c *LRUCache) get(key string) ([]byte, bool) { + c.requests.WithLabelValues().Inc() + + c.mtx.Lock() + defer c.mtx.Unlock() + + v, ok := c.lru.Get(key) + if !ok { + c.totalMisses.Inc() + return nil, false + } + c.hits.WithLabelValues().Inc() + return v.([]byte), true +} + +func (c *LRUCache) set(key string, val []byte) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if _, ok := c.lru.Get(key); ok { + return + } + + // The caller may be passing in a sub-slice of a huge array. Copy the data + // to ensure we don't waste huge amounts of space for something small. + v := make([]byte, len(val)) + copy(v, val) + c.lru.Add(key, v) + + c.bytesInUse.Add(float64(c.entryMemoryUsage(key, val))) + c.added.WithLabelValues().Inc() + c.current.WithLabelValues().Inc() +} + +func (c *LRUCache) entryMemoryUsage(key string, val []byte) int { + return int(unsafe.Sizeof(val)) + len(key) +} + +func (c *LRUCache) reset() { + c.lru.Purge() + c.current.Reset() + c.bytesInUse.Set(0) +} diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 6a496575c6c18..388f80393ac9a 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -2,38 +2,46 @@ package tsdb import ( "context" + "sort" + "strings" + "github.com/dennwc/varint" + "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + promEncoding "github.com/prometheus/prometheus/tsdb/encoding" - "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/grafana/loki/pkg/storage/stores/tsdb/indexcache" + "github.com/grafana/loki/pkg/util/encoding" ) -var cacheClient cache.Cache +type PostingsClient interface { + ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error +} + +var sharedCacheClient cache.Cache -func NewCachedPostingsClient(reader IndexReader) PostingsClient { +func NewCachedPostingsClient(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsClient { return &cachedPostingsClient{ reader: reader, cacheClient: cacheClient, + log: logger, } } -type PostingsClient interface { - ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error -} - type cachedPostingsClient struct { - reader IndexReader - chunkFilter chunk.RequestChunkFilterer + reader IndexReader + + cacheClient cache.Cache - cacheClient cache.Cache - IndexCacheClient indexcache.Client + log log.Logger } func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { - if postings, got := c.IndexCacheClient.FetchPostings(matchers); got { + key := CanonicalLabelMatchersKey(matchers) + if postings, got := c.fetchPostings(key); got { return fn(postings) } @@ -42,6 +50,137 @@ func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labe return err } - c.IndexCacheClient.StorePostings(matchers, p) + if err := c.storePostings(p, matchers); err != nil { + level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) + } return fn(p) } + +// diffVarintEncodeNoHeader encodes postings into diff+varint representation. +// It doesn't add any header to the output bytes. +// Length argument is expected number of postings, used for preallocating buffer. +func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { + buf := encoding.Encbuf{} + + // This encoding uses around ~1 bytes per posting, but let's use + // conservative 1.25 bytes per posting to avoid extra allocations. + if length > 0 { + buf.B = make([]byte, 0, 5*length/4) + } + + prev := storage.SeriesRef(0) + for p.Next() { + v := p.At() + + // TODO(dylanguedes): can we ignore this? + // if v < prev { + // return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) + // } + + // This is the 'diff' part -- compute difference from previous value. + buf.PutUvarint64(uint64(v - prev)) + prev = v + } + if p.Err() != nil { + return nil, p.Err() + } + + return buf.B, nil +} + +func decodeToPostings(b []byte) index.Postings { + decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) + + refs := []storage.SeriesRef{} + prev := storage.SeriesRef(0) + for i := 0; i < decoder.Len(); i++ { + v := storage.SeriesRef(decoder.Uvarint64()) + refs = append(refs, v+prev) + prev = v + } + + return index.NewListPostings(refs) +} + +func encodedMatchersLen(matchers []*labels.Matcher) int { + matchersLen := varint.UvarintSize(uint64(len(matchers))) + for _, m := range matchers { + matchersLen += varint.UvarintSize(uint64(len(m.Name))) + matchersLen += len(m.Name) + matchersLen++ // 1 byte for the type + matchersLen += varint.UvarintSize(uint64(len(m.Value))) + matchersLen += len(m.Value) + } + return matchersLen +} + +func (c *cachedPostingsClient) storePostings(postings index.Postings, matchers []*labels.Matcher) error { + dataToCache, err := diffVarintEncodeNoHeader(postings, 0) + if err != nil { + level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", CanonicalLabelMatchersKey(matchers)) + } + + return c.cacheClient.Store(context.TODO(), []string{CanonicalLabelMatchersKey(matchers)}, [][]byte{dataToCache}) +} + +func (c *cachedPostingsClient) fetchPostings(key string) (index.Postings, bool) { + found, bufs, _, err := c.cacheClient.Fetch(context.TODO(), []string{key}) + + if err != nil { + level.Error(c.log).Log("msg", "error on fetching postings", "err", err, "matchers", key) + return nil, false + } + + if len(found) > 0 { + var postings []index.Postings + for _, b := range bufs { + postings = append(postings, decodeToPostings(b)) + } + + return index.Merge(postings...), true + } + + return nil, false +} + +// CanonicalLabelMatchersKey creates a canonical version of LabelMatchersKey +func CanonicalLabelMatchersKey(ms []*labels.Matcher) string { + sorted := make([]labels.Matcher, len(ms)) + for i := range ms { + sorted[i] = labels.Matcher{Type: ms[i].Type, Name: ms[i].Name, Value: ms[i].Value} + } + sort.Sort(sortedLabelMatchers(sorted)) + + const ( + typeLen = 2 + sepLen = 1 + ) + var size int + for _, m := range sorted { + size += len(m.Name) + len(m.Value) + typeLen + sepLen + } + sb := strings.Builder{} + sb.Grow(size) + for _, m := range sorted { + sb.WriteString(m.Name) + sb.WriteString(m.Type.String()) + sb.WriteString(m.Value) + sb.WriteByte(0) + } + return sb.String() +} + +type sortedLabelMatchers []labels.Matcher + +func (c sortedLabelMatchers) Less(i, j int) bool { + if c[i].Name != c[j].Name { + return c[i].Name < c[j].Name + } + if c[i].Type != c[j].Type { + return c[i].Type < c[j].Type + } + return c[i].Value < c[j].Value +} + +func (c sortedLabelMatchers) Len() int { return len(c) } +func (c sortedLabelMatchers) Swap(i, j int) { c[i], c[j] = c[j], c[i] } diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index b2ea683c9dcc3..2fcde6f455a91 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -731,7 +731,7 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } reader := tenant.indexRange(int64(from), int64(through)) - idx = NewTSDBIndex(reader, &simpleForPostingsClient{reader: reader}) + idx = NewTSDBIndex(reader, &simplePostingsClient{reader: reader}) if t.chunkFilter != nil { idx.SetChunkFilterer(t.chunkFilter) } diff --git a/pkg/storage/stores/tsdb/indexcache/cache.go b/pkg/storage/stores/tsdb/indexcache/cache.go index ccff100d8872c..33eed15c09ee8 100644 --- a/pkg/storage/stores/tsdb/indexcache/cache.go +++ b/pkg/storage/stores/tsdb/indexcache/cache.go @@ -16,8 +16,6 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "golang.org/x/crypto/blake2b" - - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) const ( @@ -83,45 +81,6 @@ func sumBytes[T comparable](res map[T][]byte) int { return sum } -// IndexCacheClient is the interface exported by index cache backends. -type Client interface { - StorePostings(matchers []*labels.Matcher, postings index.Postings) - FetchPostings(matchers []*labels.Matcher) (index.Postings, bool) - - StoreSeriesForPostings(matchers []*labels.Matcher, refs []storage.SeriesRef) - - // FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, postingsKey PostingsKey) ([]byte, bool) - - // // StoreSeriesForRef stores a single series. - // StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) - - // // FetchMultiSeriesForRefs fetches multiple series - each identified by ID - from the cache - // // and returns a map containing cache hits, along with a list of missing IDs. - // // The order of the returned misses should be the same as their relative order in the provided ids. - // FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) - - // // StoreExpandedPostings stores the result of ExpandedPostings, encoded with an unspecified codec. - // StoreExpandedPostings(userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string, v []byte) - - // // FetchExpandedPostings fetches the result of ExpandedPostings, encoded with an unspecified codec. - // FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) - - // StoreSeriesForPostings stores a series set for the provided postings. - // StoreSeriesForPostings(matchers []*labels.Matcher, postings index.Postings) - // FetchSeriesForPostings fetches a series set for the provided postings. - // FetchSeriesForPostings(ctx context.Context, matchers []*labels.Matcher) (index.Postings, bool) - - // // StoreLabelNames stores the result of a LabelNames() call. - // StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) - // // FetchLabelNames fetches the result of a LabelNames() call. - // FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) - - // // StoreLabelValues stores the result of a LabelValues() call. - // StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) - // // FetchLabelValues fetches the result of a LabelValues() call. - // FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) -} - // PostingsKey represents a canonical key for a []storage.SeriesRef slice type PostingsKey string diff --git a/pkg/storage/stores/tsdb/indexcache/inmemory.go b/pkg/storage/stores/tsdb/indexcache/inmemory.go index 12ef623052687..798cccb2d7f6c 100644 --- a/pkg/storage/stores/tsdb/indexcache/inmemory.go +++ b/pkg/storage/stores/tsdb/indexcache/inmemory.go @@ -209,6 +209,16 @@ func (c *InMemoryIndexCache) get(key cacheKey) ([]byte, bool) { return v.([]byte), true } +// Store(ctx context.Context, key []string, buf [][]byte) error +// Stop() +// // GetCacheType returns a string indicating the cache "type" for the purpose of grouping cache usage statistics +// GetCacheType() stats.CacheType + +// func (c *InMemoryIndexCache) Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { +// got, ok := c.get(cacheKeyPostings{}) + +// } + func (c *InMemoryIndexCache) set(key cacheKey, val []byte) { typ := key.typ() size := sliceSize(val) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index ebebbbe8ea967..275eefd438459 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -91,6 +91,7 @@ type TSDBFile struct { } func NewShippableTSDBFile(id Identifier) (*TSDBFile, error) { + level.Warn(util_log.Logger).Log("msg", "NEW SHIPPABLE TSDB FILE") idx, getRawFileReader, err := NewTSDBIndexFromFile(id.Path()) if err != nil { return nil, err @@ -139,8 +140,8 @@ func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) func getPostingsClient(reader IndexReader) PostingsClient { var postingsClient PostingsClient - if shouldCachePostings && cacheClient != nil { - postingsClient = NewCachedPostingsClient(reader) + if shouldCachePostings && sharedCacheClient != nil { + postingsClient = NewCachedPostingsClient(reader, util_log.Logger, sharedCacheClient) } if postingsClient == nil { @@ -151,14 +152,14 @@ func getPostingsClient(reader IndexReader) PostingsClient { } func DefaultPostingsClient(reader IndexReader) PostingsClient { - return &simpleForPostingsClient{reader: reader} + return &simplePostingsClient{reader: reader} } -type simpleForPostingsClient struct { +type simplePostingsClient struct { reader IndexReader } -func (s *simpleForPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { +func (s *simplePostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { p, err := PostingsForMatchers(s.reader, nil, matchers...) if err != nil { return err diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index c7aac55ec3a2e..94ae6b94a0759 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -74,7 +74,7 @@ func NewStore( func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConfig, objectClient client.ObjectClient, limits downloads.Limits, tableRange config.TableRange, reg prometheus.Registerer, idxCache cache.Cache) error { - cacheClient = idxCache + sharedCacheClient = idxCache if indexCfg.CachePostings { shouldCachePostings = true From 6878e4bf8d439c9f91dcce6800d97ed1d7b2a2ee Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 18:01:43 -0300 Subject: [PATCH 06/84] Add tests --- pkg/storage/chunk/cache/lru_cache_test.go | 148 +++ pkg/storage/stores/tsdb/indexcache/remote.go | 307 ------ .../stores/tsdb/indexcache/remote_test.go | 937 ------------------ pkg/storage/stores/tsdb/single_file_index.go | 1 - 4 files changed, 148 insertions(+), 1245 deletions(-) create mode 100644 pkg/storage/chunk/cache/lru_cache_test.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/remote.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/remote_test.go diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go new file mode 100644 index 0000000000000..9a0aa2f95e558 --- /dev/null +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -0,0 +1,148 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +// Tests out the LRU index cache implementation. +package cache + +import ( + "context" + "fmt" + "strconv" + "testing" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestLRUCacheEviction(t *testing.T) { + const ( + cnt = 10 + evicted = 5 + ) + itemTemplate := &cacheEntry{ + key: "00", + value: []byte("00"), + } + + tests := []struct { + name string + cfg LRUCacheConfig + }{ + { + name: "test-memory-eviction", + cfg: LRUCacheConfig{MaxSizeBytes: strconv.FormatInt(int64(cnt*sizeOf(itemTemplate)), 10)}, + }, + } + + for _, test := range tests { + c, err := NewLRUCache(test.name, test.cfg, nil, log.NewNopLogger(), "test") + require.NoError(t, err) + ctx := context.Background() + + // Check put / get works + keys := []string{} + values := [][]byte{} + for i := 0; i < cnt; i++ { + key := fmt.Sprintf("%02d", i) + value := make([]byte, len(key)) + copy(value, key) + keys = append(keys, key) + values = append(values, value) + } + require.NoError(t, c.Store(ctx, keys, values)) + require.Len(t, c.lru.Len(), cnt) + + assert.Equal(t, testutil.ToFloat64(c.added), float64(1)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) + + for i := 0; i < cnt; i++ { + key := fmt.Sprintf("%02d", i) + value, ok := c.get(key) + require.True(t, ok) + require.Equal(t, []byte(key), value) + } + + assert.Equal(t, testutil.ToFloat64(c.added), float64(1)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) + + // Check evictions + keys = []string{} + values = [][]byte{} + for i := cnt - evicted; i < cnt+evicted; i++ { + key := fmt.Sprintf("%02d", i) + value := make([]byte, len(key)) + copy(value, key) + keys = append(keys, key) + values = append(values, value) + } + err = c.Store(ctx, keys, values) + require.NoError(t, err) + require.Len(t, c.lru.Len(), cnt) + + assert.Equal(t, testutil.ToFloat64(c.added), float64(2)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) + + for i := 0; i < cnt-evicted; i++ { + _, ok := c.get(fmt.Sprintf("%02d", i)) + require.False(t, ok) + } + for i := cnt - evicted; i < cnt+evicted; i++ { + key := fmt.Sprintf("%02d", i) + value, ok := c.get(key) + require.True(t, ok) + require.Equal(t, []byte(key), value) + } + + assert.Equal(t, testutil.ToFloat64(c.added), float64(2)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) + + // Check updates work + keys = []string{} + values = [][]byte{} + for i := cnt; i < cnt+evicted; i++ { + keys = append(keys, fmt.Sprintf("%02d", i)) + vstr := fmt.Sprintf("%02d", i*2) + value := make([]byte, len(vstr)) + copy(value, vstr) + values = append(values, value) + } + err = c.Store(ctx, keys, values) + require.NoError(t, err) + require.Len(t, c.lru.Len(), cnt) + + for i := cnt; i < cnt+evicted; i++ { + value, ok := c.get(fmt.Sprintf("%02d", i)) + require.True(t, ok) + require.Equal(t, []byte(fmt.Sprintf("%02d", i*2)), value) + } + + assert.Equal(t, testutil.ToFloat64(c.added), float64(3)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted*2)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) + + c.Stop() + } +} diff --git a/pkg/storage/stores/tsdb/indexcache/remote.go b/pkg/storage/stores/tsdb/indexcache/remote.go deleted file mode 100644 index 0cb3084c64aef..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/remote.go +++ /dev/null @@ -1,307 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/memcached.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -package indexcache - -// const ( -// remoteDefaultTTL = 7 * 24 * time.Hour -// ) - -// var ( -// postingsCacheKeyLabelHashBufferPool = sync.Pool{New: func() any { -// // We assume the label name/value pair is typically not longer than 1KB. -// b := make([]byte, 1024) -// return &b -// }} -// ) - -// // RemoteIndexCache is a memcached or redis based index cache. -// type RemoteIndexCache struct { -// logger log.Logger -// remote cache.RemoteCacheClient - -// // Metrics. -// requests *prometheus.CounterVec -// hits *prometheus.CounterVec -// } - -// // NewRemoteIndexCache makes a new RemoteIndexCache. -// func NewRemoteIndexCache(logger log.Logger, remote cache.RemoteCacheClient, reg prometheus.Registerer) (*RemoteIndexCache, error) { -// c := &RemoteIndexCache{ -// logger: logger, -// remote: remote, -// } - -// c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ -// Name: "thanos_store_index_cache_requests_total", -// Help: "Total number of items requests to the cache.", -// }, []string{"item_type"}) -// initLabelValuesForAllCacheTypes(c.requests.MetricVec) - -// c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ -// Name: "thanos_store_index_cache_hits_total", -// Help: "Total number of items requests to the cache that were a hit.", -// }, []string{"item_type"}) -// initLabelValuesForAllCacheTypes(c.hits.MetricVec) - -// level.Info(logger).Log("msg", "created remote index cache") - -// return c, nil -// } - -// // set stores a value for the given key in the remote cache. -// func (c *RemoteIndexCache) set(typ string, key string, val []byte) { -// if err := c.remote.SetAsync(key, val, remoteDefaultTTL); err != nil { -// level.Error(c.logger).Log("msg", "failed to set item in remote cache", "type", typ, "err", err) -// } -// } - -// // get retrieves a single value from the remote cache, returned bool value indicates whether the value was found or not. -// func (c *RemoteIndexCache) get(ctx context.Context, typ string, key string) ([]byte, bool) { -// c.requests.WithLabelValues(typ).Inc() -// results := c.remote.GetMulti(ctx, []string{key}) -// data, ok := results[key] -// if ok { -// c.hits.WithLabelValues(typ).Inc() -// } -// return data, ok -// } - -// // StorePostings sets the postings identified by the ulid and label to the value v. -// // The function enqueues the request and returns immediately: the entry will be -// // asynchronously stored in the cache. -// func (c *RemoteIndexCache) StorePostings(userID string, blockID ulid.ULID, l labels.Label, v []byte) { -// c.set(cacheTypePostings, postingsCacheKey(userID, blockID.String(), l), v) -// } - -// // FetchMultiPostings fetches multiple postings - each identified by a label. -// // In case of error, it logs and return an empty result. -// func (c *RemoteIndexCache) FetchMultiPostings(ctx context.Context, userID string, blockID ulid.ULID, lbls []labels.Label) BytesResult { -// blockIDStr := blockID.String() - -// keys := make([]string, 0, len(lbls)) -// for _, lbl := range lbls { -// keys = append(keys, postingsCacheKey(userID, blockIDStr, lbl)) -// } - -// // Fetch the keys from the remote cache in a single request. -// c.requests.WithLabelValues(cacheTypePostings).Add(float64(len(keys))) -// results := c.remote.GetMulti(ctx, keys) -// c.hits.WithLabelValues(cacheTypePostings).Add(float64(len(results))) - -// return &MapIterator[string]{ -// Keys: keys, -// M: results, -// } -// } - -// // postingsCacheKey returns the cache key used to store postings matching the input -// // label name/value pair in the given block. -// func postingsCacheKey(userID, blockID string, l labels.Label) string { -// const ( -// prefix = "P2:" -// separator = ":" -// ) - -// // Compute the label hash. -// lblHash, hashLen := postingsCacheKeyLabelID(l) - -// // Preallocate the byte slice used to store the cache key. -// encodedHashLen := base64.RawURLEncoding.EncodedLen(hashLen) -// expectedLen := len(prefix) + len(userID) + 1 + ulid.EncodedSize + 1 + encodedHashLen -// key := make([]byte, expectedLen) -// offset := 0 - -// offset += copy(key[offset:], prefix) -// offset += copy(key[offset:], userID) -// offset += copy(key[offset:], separator) -// offset += copy(key[offset:], blockID) -// offset += copy(key[offset:], separator) -// base64.RawURLEncoding.Encode(key[offset:], lblHash[:hashLen]) -// offset += encodedHashLen - -// sizedKey := key[:offset] -// // Convert []byte to string with no extra allocation. -// return *(*string)(unsafe.Pointer(&sizedKey)) -// } - -// // postingsCacheKeyLabelID returns the hash of the input label or the label itself if it is shorter than the size of the hash. -// // This is used as part of the cache key generated by postingsCacheKey(). -// func postingsCacheKeyLabelID(l labels.Label) (out [blake2b.Size256]byte, outLen int) { -// const separator = ":" - -// // Compute the expected length. -// expectedLen := len(l.Name) + len(separator) + len(l.Value) - -// // If the whole label is smaller than the hash, then shortcut hashing and directly write out the result. -// if expectedLen <= blake2b.Size256 { -// offset := 0 -// offset += copy(out[offset:], l.Name) -// offset += copy(out[offset:], separator) -// offset += copy(out[offset:], l.Value) -// return out, offset -// } - -// // Get a buffer from the pool and fill it with the label name/value pair to hash. -// bp := postingsCacheKeyLabelHashBufferPool.Get().(*[]byte) -// buf := *bp - -// if cap(buf) < expectedLen { -// buf = make([]byte, expectedLen) -// } else { -// buf = buf[:expectedLen] -// } - -// offset := 0 -// offset += copy(buf[offset:], l.Name) -// offset += copy(buf[offset:], separator) -// offset += copy(buf[offset:], l.Value) - -// // This is expected to be always equal. If it's not, then it's a severe bug. -// if offset != expectedLen { -// panic(fmt.Sprintf("postingsCacheKeyLabelID() computed an invalid expected length (expected: %d, actual: %d)", expectedLen, offset)) -// } - -// // Use cryptographically hash functions to avoid hash collisions -// // which would end up in wrong query results. -// hash := blake2b.Sum256(buf) - -// // Reuse the same pointer to put the buffer back into the pool. -// *bp = buf -// postingsCacheKeyLabelHashBufferPool.Put(bp) - -// return hash, len(hash) -// } - -// // StoreSeriesForRef sets the series identified by the ulid and id to the value v. -// // The function enqueues the request and returns immediately: the entry will be -// // asynchronously stored in the cache. -// func (c *RemoteIndexCache) StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) { -// c.set(cacheTypeSeriesForRef, seriesForRefCacheKey(userID, blockID, id), v) -// } - -// // FetchMultiSeriesForRefs fetches multiple series - each identified by ID - from the cache -// // and returns a map containing cache hits, along with a list of missing IDs. -// // In case of error, it logs and return an empty cache hits map. -// func (c *RemoteIndexCache) FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) { -// // Build the cache keys, while keeping a map between input id and the cache key -// // so that we can easily reverse it back after the GetMulti(). -// keys := make([]string, 0, len(ids)) -// keysMapping := make(map[storage.SeriesRef]string, len(ids)) - -// for _, id := range ids { -// key := seriesForRefCacheKey(userID, blockID, id) - -// keys = append(keys, key) -// keysMapping[id] = key -// } - -// // Fetch the keys from the remote cache in a single request. -// c.requests.WithLabelValues(cacheTypeSeriesForRef).Add(float64(len(ids))) -// results := c.remote.GetMulti(ctx, keys) -// if len(results) == 0 { -// return nil, ids -// } - -// // Construct the resulting hits map and list of missing keys. We iterate on the input -// // list of ids to be able to easily create the list of ones in a single iteration. -// hits = make(map[storage.SeriesRef][]byte, len(results)) -// if numMisses := len(ids) - len(results); numMisses > 0 { -// misses = make([]storage.SeriesRef, 0, numMisses) -// } - -// for _, id := range ids { -// key, ok := keysMapping[id] -// if !ok { -// _ = level.Error(c.logger).Log("msg", "keys mapping inconsistency found in remote index cache client", "type", "series", "id", id) -// continue -// } - -// // Check if the key has been found in the remote cache. If not, we add it to the list -// // of missing keys. -// value, ok := results[key] -// if !ok { -// misses = append(misses, id) -// continue -// } - -// hits[id] = value -// } - -// c.hits.WithLabelValues(cacheTypeSeriesForRef).Add(float64(len(hits))) -// return hits, misses -// } - -// func seriesForRefCacheKey(userID string, blockID ulid.ULID, id storage.SeriesRef) string { -// // Max uint64 string representation is no longer than 20 characters. -// b := make([]byte, 0, 20) -// return "S:" + userID + ":" + blockID.String() + ":" + string(strconv.AppendUint(b, uint64(id), 10)) -// } - -// // StoreExpandedPostings stores the encoded result of ExpandedPostings for specified matchers identified by the provided LabelMatchersKey. -// func (c *RemoteIndexCache) StoreExpandedPostings(userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string, v []byte) { -// c.set(cacheTypeExpandedPostings, expandedPostingsCacheKey(userID, blockID, lmKey, postingsSelectionStrategy), v) -// } - -// // FetchExpandedPostings fetches the encoded result of ExpandedPostings for specified matchers identified by the provided LabelMatchersKey. -// func (c *RemoteIndexCache) FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) { -// return c.get(ctx, cacheTypeExpandedPostings, expandedPostingsCacheKey(userID, blockID, lmKey, postingsSelectionStrategy)) -// } - -// func expandedPostingsCacheKey(userID string, blockID ulid.ULID, lmKey LabelMatchersKey, postingsSelectionStrategy string) string { -// hash := blake2b.Sum256([]byte(lmKey)) -// return "E2:" + userID + ":" + blockID.String() + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) + ":" + postingsSelectionStrategy -// } - -// // StoreSeriesForPostings stores a series set for the provided postings. -// func (c *RemoteIndexCache) StoreSeriesForPostings(userID string, blockID ulid.ULID, postingsKey PostingsKey, v []byte) { -// c.set(cacheTypeSeriesForPostings, seriesForPostingsCacheKey(userID, blockID, postingsKey), v) -// } - -// // FetchSeriesForPostings fetches a series set for the provided postings. -// func (c *RemoteIndexCache) FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) ([]byte, bool) { -// return c.get(ctx, cacheTypeSeriesForPostings, seriesForPostingsCacheKey(userID, blockID, shard, postingsKey)) -// } - -// func seriesForPostingsCacheKey(userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) string { -// // We use SP2: as -// // * S: is already used for SeriesForRef -// // * SS: is already used for Series -// // * SP: was in use when using gob encoding -// // -// // "SP2" (3) + userID (150) + blockID (26) + shard (10 with up to 1000 shards) + ":" (4) = 193 -// // Memcached limits key length to 250, so we're left with 57 bytes for the postings key. -// return "SP2:" + userID + ":" + blockID.String() + ":" + shardKey(shard) + ":" + string(postingsKey) -// } - -// // StoreLabelNames stores the result of a LabelNames() call. -// func (c *RemoteIndexCache) StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) { -// c.set(cacheTypeLabelNames, labelNamesCacheKey(userID, blockID, matchersKey), v) -// } - -// // FetchLabelNames fetches the result of a LabelNames() call. -// func (c *RemoteIndexCache) FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) { -// return c.get(ctx, cacheTypeLabelNames, labelNamesCacheKey(userID, blockID, matchersKey)) -// } - -// func labelNamesCacheKey(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) string { -// hash := blake2b.Sum256([]byte(matchersKey)) -// return "LN:" + userID + ":" + blockID.String() + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) -// } - -// // StoreLabelValues stores the result of a LabelValues() call. -// func (c *RemoteIndexCache) StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) { -// c.set(cacheTypeLabelValues, labelValuesCacheKey(userID, blockID, labelName, matchersKey), v) -// } - -// // FetchLabelValues fetches the result of a LabelValues() call. -// func (c *RemoteIndexCache) FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) { -// return c.get(ctx, cacheTypeLabelValues, labelValuesCacheKey(userID, blockID, labelName, matchersKey)) -// } - -// func labelValuesCacheKey(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) string { -// hash := blake2b.Sum256([]byte(matchersKey)) -// return "LV2:" + userID + ":" + blockID.String() + ":" + labelName + ":" + base64.RawURLEncoding.EncodeToString(hash[0:]) -// } diff --git a/pkg/storage/stores/tsdb/indexcache/remote_test.go b/pkg/storage/stores/tsdb/indexcache/remote_test.go deleted file mode 100644 index ab7276a1fe972..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/remote_test.go +++ /dev/null @@ -1,937 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/memcached_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -package indexcache - -// func TestRemoteIndexCache_FetchMultiPostings(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// label1 := labels.Label{Name: "instance", Value: "a"} -// label2 := labels.Label{Name: "instance", Value: "b"} -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} - -// tests := map[string]struct { -// setup []mockedPostings -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchLabels []labels.Label -// expectedHits map[labels.Label][]byte -// expectedMisses []labels.Label -// }{ -// "should return no hits on empty cache": { -// setup: []mockedPostings{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabels: []labels.Label{label1, label2}, -// expectedHits: nil, -// expectedMisses: []labels.Label{label1, label2}, -// }, -// "should return no misses on 100% hit ratio": { -// setup: []mockedPostings{ -// {userID: user1, block: block1, label: label1, value: value1}, -// {userID: user2, block: block1, label: label1, value: value2}, -// {userID: user1, block: block1, label: label2, value: value2}, -// {userID: user1, block: block2, label: label1, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabels: []labels.Label{label1, label2}, -// expectedHits: map[labels.Label][]byte{ -// label1: value1, -// label2: value2, -// }, -// expectedMisses: []labels.Label{}, -// }, -// "should return hits and misses on partial hits": { -// setup: []mockedPostings{ -// {userID: user1, block: block1, label: label1, value: value1}, -// {userID: user1, block: block2, label: label1, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabels: []labels.Label{label1, label2}, -// expectedHits: map[labels.Label][]byte{label1: value1}, -// expectedMisses: []labels.Label{label2}, -// }, -// "should return no hits on remote cache error": { -// setup: []mockedPostings{ -// {userID: user1, block: block1, label: label1, value: value1}, -// {userID: user1, block: block1, label: label2, value: value2}, -// {userID: user1, block: block2, label: label1, value: value3}, -// }, -// mockedErr: errors.New("mocked error"), -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabels: []labels.Label{label1, label2}, -// expectedHits: nil, -// expectedMisses: []labels.Label{label1, label2}, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the postings expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StorePostings(p.userID, p.block, p.label, p.value) -// } - -// // Fetch postings from cached and assert on it. -// testFetchMultiPostings(ctx, t, c, testData.fetchUserID, testData.fetchBlockID, testData.fetchLabels, testData.expectedHits) - -// // Assert on metrics. -// assert.Equal(t, float64(len(testData.fetchLabels)), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(len(testData.expectedHits)), prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypePostings))) -// for _, typ := range remove(allCacheTypes, cacheTypePostings) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func BenchmarkRemoteIndexCache_FetchMultiPostings(b *testing.B) { -// const ( -// numHits = 10000 -// numMisses = 10000 - -// numKeys = numHits + numMisses -// ) - -// var ( -// ctx = context.Background() -// userID = "user-1" -// blockID = ulid.MustNew(1, nil) -// ) - -// benchCases := map[string]struct { -// fetchLabels []labels.Label -// }{ -// "short labels": { -// fetchLabels: func() []labels.Label { -// fetchLabels := make([]labels.Label, 0, numKeys) -// for i := 0; i < numKeys; i++ { -// fetchLabels = append(fetchLabels, labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("series_%d", i)}) -// } -// return fetchLabels -// }(), -// }, -// "long labels": { // this should trigger hashing the labels instead of embedding them in the cache key -// fetchLabels: func() []labels.Label { -// fetchLabels := make([]labels.Label, 0, numKeys) -// for i := 0; i < numKeys; i++ { -// fetchLabels = append(fetchLabels, labels.Label{Name: labels.MetricName, Value: "series_" + strings.Repeat(strconv.Itoa(i), 100)}) -// } -// return fetchLabels -// }(), -// }, -// } - -// for name, benchCase := range benchCases { -// fetchLabels := benchCase.fetchLabels -// b.Run(name, func(b *testing.B) { -// client := newMockedRemoteCacheClient(nil) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(b, err) - -// // Store the postings expected before running the benchmark. -// for i := 0; i < numHits; i++ { -// c.StorePostings(userID, blockID, fetchLabels[i], []byte{1}) -// } - -// b.ResetTimer() - -// for n := 0; n < b.N; n++ { -// results := c.FetchMultiPostings(ctx, userID, blockID, fetchLabels) -// assert.Equal(b, numKeys, results.Remaining()) -// actualHits := 0 -// // iterate over the returned map to account for cost of access -// for i := 0; i < numKeys; i++ { -// bytes, ok := results.Next() -// assert.True(b, ok) -// if bytes != nil { -// actualHits++ -// } -// } -// assert.Equal(b, numHits, actualHits) -// } -// }) -// } -// } - -// func TestRemoteIndexCache_FetchMultiSeriesForRef(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} - -// tests := map[string]struct { -// setup []mockedSeriesForRef -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchIds []storage.SeriesRef -// expectedHits map[storage.SeriesRef][]byte -// expectedMisses []storage.SeriesRef -// }{ -// "should return no hits on empty cache": { -// setup: []mockedSeriesForRef{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchIds: []storage.SeriesRef{1, 2}, -// expectedHits: nil, -// expectedMisses: []storage.SeriesRef{1, 2}, -// }, -// "should return no misses on 100% hit ratio": { -// setup: []mockedSeriesForRef{ -// {userID: user1, block: block1, id: 1, value: value1}, -// {userID: user2, block: block1, id: 1, value: value2}, -// {userID: user1, block: block1, id: 1, value: value1}, -// {userID: user1, block: block1, id: 2, value: value2}, -// {userID: user1, block: block2, id: 1, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchIds: []storage.SeriesRef{1, 2}, -// expectedHits: map[storage.SeriesRef][]byte{ -// 1: value1, -// 2: value2, -// }, -// expectedMisses: nil, -// }, -// "should return hits and misses on partial hits": { -// setup: []mockedSeriesForRef{ -// {userID: user1, block: block1, id: 1, value: value1}, -// {userID: user1, block: block2, id: 1, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchIds: []storage.SeriesRef{1, 2}, -// expectedHits: map[storage.SeriesRef][]byte{1: value1}, -// expectedMisses: []storage.SeriesRef{2}, -// }, -// "should return no hits on remote cache error": { -// setup: []mockedSeriesForRef{ -// {userID: user1, block: block1, id: 1, value: value1}, -// {userID: user1, block: block1, id: 2, value: value2}, -// {userID: user1, block: block2, id: 1, value: value3}, -// }, -// mockedErr: errors.New("mocked error"), -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchIds: []storage.SeriesRef{1, 2}, -// expectedHits: nil, -// expectedMisses: []storage.SeriesRef{1, 2}, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the series expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StoreSeriesForRef(p.userID, p.block, p.id, p.value) -// } - -// // Fetch series from cached and assert on it. -// hits, misses := c.FetchMultiSeriesForRefs(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchIds) -// assert.Equal(t, testData.expectedHits, hits) -// assert.Equal(t, testData.expectedMisses, misses) - -// // Assert on metrics. -// assert.Equal(t, float64(len(testData.fetchIds)), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(len(testData.expectedHits)), prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeSeriesForRef))) -// for _, typ := range remove(allCacheTypes, cacheTypeSeriesForRef) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func TestRemoteIndexCache_FetchExpandedPostings(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} -// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} -// postingsStrategy1 := "s1" -// postingsStrategy2 := "s2" - -// tests := map[string]struct { -// setup []mockedExpandedPostings -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchKey LabelMatchersKey -// fetchStrategy string -// expectedData []byte -// expectedOk bool -// }{ -// "should return no hit on empty cache": { -// setup: []mockedExpandedPostings{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// "should return no miss on hit": { -// setup: []mockedExpandedPostings{ -// {userID: user1, block: block1, matchers: matchers1, value: value1, postingsStrategy: postingsStrategy1}, -// {userID: user2, block: block1, matchers: matchers1, value: value2, postingsStrategy: postingsStrategy1}, -// {userID: user1, block: block1, matchers: matchers2, value: value2, postingsStrategy: postingsStrategy1}, -// {userID: user1, block: block2, matchers: matchers1, value: value3, postingsStrategy: postingsStrategy1}, -// {userID: user1, block: block1, matchers: matchers1, value: value1, postingsStrategy: postingsStrategy2}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// fetchStrategy: postingsStrategy1, -// expectedData: value1, -// expectedOk: true, -// }, -// "should return no hit on remote cache error": { -// setup: []mockedExpandedPostings{ -// {userID: user1, block: block1, matchers: matchers1, value: value1}, -// {userID: user1, block: block1, matchers: matchers2, value: value2}, -// {userID: user1, block: block2, matchers: matchers1, value: value3}, -// }, -// mockedErr: context.DeadlineExceeded, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the postings expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StoreExpandedPostings(p.userID, p.block, CanonicalLabelMatchersKey(p.matchers), testData.fetchStrategy, p.value) -// } - -// // Fetch postings from cached and assert on it. -// data, ok := c.FetchExpandedPostings(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchKey, testData.fetchStrategy) -// assert.Equal(t, testData.expectedData, data) -// assert.Equal(t, testData.expectedOk, ok) - -// // Assert on metrics. -// expectedHits := 0.0 -// if testData.expectedOk { -// expectedHits = 1.0 -// } -// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeExpandedPostings))) -// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeExpandedPostings))) -// for _, typ := range remove(allCacheTypes, cacheTypeExpandedPostings) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func TestRemoteIndexCache_FetchSeriesForPostings(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} -// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} -// shard1 := (*sharding.ShardSelector)(nil) -// shard2 := &sharding.ShardSelector{ShardIndex: 1, ShardCount: 16} -// postings1 := []storage.SeriesRef{1, 2} -// postings2 := []storage.SeriesRef{2, 3} - -// tests := map[string]struct { -// setup []mockedSeries -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchKey LabelMatchersKey -// fetchShard *sharding.ShardSelector -// postings []storage.SeriesRef -// expectedData []byte -// expectedOk bool -// }{ -// "should return no hit on empty cache": { -// setup: []mockedSeries{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// fetchShard: shard1, -// postings: postings1, -// expectedData: nil, -// expectedOk: false, -// }, -// "should return no miss on hit": { -// setup: []mockedSeries{ -// {userID: user1, block: block1, shard: shard1, postings: postings1, value: value1}, -// {userID: user2, block: block1, shard: shard1, postings: postings1, value: value2}, // different user -// {userID: user1, block: block1, shard: shard2, postings: postings1, value: value2}, // different shard -// {userID: user1, block: block2, shard: shard1, postings: postings1, value: value3}, // different block -// {userID: user1, block: block2, shard: shard1, postings: postings2, value: value3}, // different postings -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// fetchShard: shard1, -// postings: postings1, -// expectedData: value1, -// expectedOk: true, -// }, -// "should return no hit on remote cache error": { -// setup: []mockedSeries{ -// {userID: user1, block: block1, matchers: matchers1, shard: shard1, postings: postings1, value: value1}, -// {userID: user1, block: block1, matchers: matchers2, shard: shard1, postings: postings1, value: value2}, -// {userID: user1, block: block2, matchers: matchers1, shard: shard1, postings: postings1, value: value3}, -// }, -// mockedErr: context.DeadlineExceeded, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// fetchShard: shard1, -// postings: postings1, -// expectedData: nil, -// expectedOk: false, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the postings expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StoreSeriesForPostings(p.userID, p.block, p.shard, CanonicalPostingsKey(p.postings), p.value) -// } - -// // Fetch postings from cached and assert on it. -// data, ok := c.FetchSeriesForPostings(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchShard, CanonicalPostingsKey(testData.postings)) -// assert.Equal(t, testData.expectedData, data) -// assert.Equal(t, testData.expectedOk, ok) - -// // Assert on metrics. -// expectedHits := 0.0 -// if testData.expectedOk { -// expectedHits = 1.0 -// } -// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeSeriesForPostings))) -// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeSeriesForPostings))) -// for _, typ := range remove(allCacheTypes, cacheTypeSeriesForPostings) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func TestRemoteIndexCache_FetchLabelNames(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} -// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} - -// tests := map[string]struct { -// setup []mockedLabelNames -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchKey LabelMatchersKey -// expectedData []byte -// expectedOk bool -// }{ -// "should return no hit on empty cache": { -// setup: []mockedLabelNames{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// "should return no miss on hit": { -// setup: []mockedLabelNames{ -// {userID: user1, block: block1, matchers: matchers1, value: value1}, -// {userID: user2, block: block1, matchers: matchers1, value: value2}, -// {userID: user1, block: block1, matchers: matchers2, value: value2}, -// {userID: user1, block: block2, matchers: matchers1, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: value1, -// expectedOk: true, -// }, -// "should return no hit on remote cache error": { -// setup: []mockedLabelNames{ -// {userID: user1, block: block1, matchers: matchers1, value: value1}, -// {userID: user1, block: block1, matchers: matchers2, value: value2}, -// {userID: user1, block: block2, matchers: matchers1, value: value3}, -// }, -// mockedErr: context.DeadlineExceeded, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the postings expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StoreLabelNames(p.userID, p.block, CanonicalLabelMatchersKey(p.matchers), p.value) -// } - -// // Fetch postings from cached and assert on it. -// data, ok := c.FetchLabelNames(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchKey) -// assert.Equal(t, testData.expectedData, data) -// assert.Equal(t, testData.expectedOk, ok) - -// // Assert on metrics. -// expectedHits := 0.0 -// if testData.expectedOk { -// expectedHits = 1.0 -// } -// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeLabelNames))) -// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeLabelNames))) -// for _, typ := range remove(allCacheTypes, cacheTypeLabelNames) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func TestRemoteIndexCache_FetchLabelValues(t *testing.T) { -// t.Parallel() - -// // Init some data to conveniently define test cases later one. -// user1 := "tenant1" -// user2 := "tenant2" -// block1 := ulid.MustNew(1, nil) -// block2 := ulid.MustNew(2, nil) -// labelName1 := "one" -// labelName2 := "two" -// matchers1 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")} -// matchers2 := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "baz", "boo")} -// value1 := []byte{1} -// value2 := []byte{2} -// value3 := []byte{3} - -// tests := map[string]struct { -// setup []mockedLabelValues -// mockedErr error -// fetchUserID string -// fetchBlockID ulid.ULID -// fetchLabelName string -// fetchKey LabelMatchersKey -// expectedData []byte -// expectedOk bool -// }{ -// "should return no hit on empty cache": { -// setup: []mockedLabelValues{}, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabelName: labelName1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// "should return no miss on hit": { -// setup: []mockedLabelValues{ -// {userID: user1, block: block1, labelName: labelName1, matchers: matchers1, value: value1}, -// {userID: user2, block: block1, labelName: labelName1, matchers: matchers1, value: value2}, -// {userID: user1, block: block1, labelName: labelName2, matchers: matchers2, value: value2}, -// {userID: user1, block: block2, labelName: labelName1, matchers: matchers1, value: value3}, -// {userID: user1, block: block2, labelName: labelName1, matchers: matchers2, value: value3}, -// }, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabelName: labelName1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: value1, -// expectedOk: true, -// }, -// "should return no hit on remote cache error": { -// setup: []mockedLabelValues{ -// {userID: user1, block: block1, labelName: labelName1, matchers: matchers1, value: value1}, -// {userID: user1, block: block1, labelName: labelName2, matchers: matchers2, value: value2}, -// {userID: user1, block: block2, labelName: labelName1, matchers: matchers1, value: value3}, -// {userID: user1, block: block2, labelName: labelName1, matchers: matchers2, value: value3}, -// }, -// mockedErr: context.DeadlineExceeded, -// fetchUserID: user1, -// fetchBlockID: block1, -// fetchLabelName: labelName1, -// fetchKey: CanonicalLabelMatchersKey(matchers1), -// expectedData: nil, -// expectedOk: false, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// client := newMockedRemoteCacheClient(testData.mockedErr) -// c, err := NewRemoteIndexCache(log.NewNopLogger(), client, nil) -// assert.NoError(t, err) - -// // Store the postings expected before running the test. -// ctx := context.Background() -// for _, p := range testData.setup { -// c.StoreLabelValues(p.userID, p.block, p.labelName, CanonicalLabelMatchersKey(p.matchers), p.value) -// } - -// // Fetch postings from cached and assert on it. -// data, ok := c.FetchLabelValues(ctx, testData.fetchUserID, testData.fetchBlockID, testData.fetchLabelName, testData.fetchKey) -// assert.Equal(t, testData.expectedData, data) -// assert.Equal(t, testData.expectedOk, ok) - -// // Assert on metrics. -// expectedHits := 0.0 -// if testData.expectedOk { -// expectedHits = 1.0 -// } -// assert.Equal(t, float64(1), prom_testutil.ToFloat64(c.requests.WithLabelValues(cacheTypeLabelValues))) -// assert.Equal(t, expectedHits, prom_testutil.ToFloat64(c.hits.WithLabelValues(cacheTypeLabelValues))) -// for _, typ := range remove(allCacheTypes, cacheTypeLabelValues) { -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.requests.WithLabelValues(typ))) -// assert.Equal(t, 0.0, prom_testutil.ToFloat64(c.hits.WithLabelValues(typ))) -// } -// }) -// } -// } - -// func TestStringCacheKeys_Values(t *testing.T) { -// t.Parallel() - -// user := "tenant" -// uid := ulid.MustNew(1, nil) - -// tests := map[string]struct { -// key string -// expected string -// }{ -// "should stringify postings cache key": { -// key: postingsCacheKey(user, uid.String(), labels.Label{Name: "foo", Value: "bar"}), -// expected: func() string { -// encodedLabel := base64.RawURLEncoding.EncodeToString([]byte("foo:bar")) -// return fmt.Sprintf("P2:%s:%s:%s", user, uid.String(), encodedLabel) -// }(), -// }, -// "should hash long postings cache key": { -// key: postingsCacheKey(user, uid.String(), labels.Label{Name: "foo", Value: strings.Repeat("bar", 11)}), -// expected: func() string { -// hash := blake2b.Sum256([]byte("foo:" + strings.Repeat("bar", 11))) -// encodedHash := base64.RawURLEncoding.EncodeToString(hash[0:]) - -// return fmt.Sprintf("P2:%s:%s:%s", user, uid.String(), encodedHash) -// }(), -// }, -// "should stringify series cache key": { -// key: seriesForRefCacheKey(user, uid, 12345), -// expected: fmt.Sprintf("S:%s:%s:12345", user, uid.String()), -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// actual := testData.key -// assert.Equal(t, testData.expected, actual) -// }) -// } -// } - -// func TestStringCacheKeys_ShouldGuaranteeReasonablyShortKeysLength(t *testing.T) { -// t.Parallel() - -// user := "tenant" -// uid := ulid.MustNew(1, nil) - -// tests := map[string]struct { -// keys []string -// expectedLen int -// }{ -// "should guarantee reasonably short key length for postings": { -// expectedLen: 80, -// keys: []string{ -// postingsCacheKey(user, uid.String(), labels.Label{Name: "a", Value: "b"}), -// postingsCacheKey(user, uid.String(), labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)}), -// }, -// }, -// "should guarantee reasonably short key length for series": { -// expectedLen: 56, -// keys: []string{ -// seriesForRefCacheKey(user, uid, math.MaxUint64), -// }, -// }, -// } - -// for testName, testData := range tests { -// t.Run(testName, func(t *testing.T) { -// for _, key := range testData.keys { -// assert.LessOrEqual(t, len(key), testData.expectedLen) -// } -// }) -// } -// } - -// func BenchmarkStringCacheKeys(b *testing.B) { -// userID := "tenant" -// uid := ulid.MustNew(1, nil) -// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} -// lmKey := CanonicalLabelMatchersKey([]*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")}) - -// b.Run("postings", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// postingsCacheKey(userID, uid.String(), lbl) -// } -// }) - -// b.Run("series ref", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// seriesForRefCacheKey(userID, uid, math.MaxUint64) -// } -// }) - -// b.Run("expanded postings", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// expandedPostingsCacheKey(userID, uid, lmKey, "strategy") -// } -// }) -// } - -// func TestPostingsCacheKey_ShouldOnlyAllocateOncePerCall(t *testing.T) { -// const numRuns = 1000 - -// blockID := ulid.MustNew(1, nil) -// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} - -// actualAllocs := testing.AllocsPerRun(numRuns, func() { -// postingsCacheKey("user-1", blockID.String(), lbl) -// }) - -// // Allow for 1 extra allocation here, reported when running the test with -race. -// assert.LessOrEqual(t, actualAllocs, 2.0) -// } - -// func TestPostingsCacheKeyLabelHash_ShouldNotAllocateMemory(t *testing.T) { -// const numRuns = 1000 - -// lbl := labels.Label{Name: strings.Repeat("a", 100), Value: strings.Repeat("a", 1000)} - -// actualAllocs := testing.AllocsPerRun(numRuns, func() { -// postingsCacheKeyLabelID(lbl) -// }) - -// // Allow for 1 extra allocation here, reported when running the test with -race. -// assert.LessOrEqual(t, actualAllocs, 1.0) -// } - -// func TestPostingsCacheKeyLabelHash_ShouldBeConcurrencySafe(t *testing.T) { -// const ( -// numWorkers = 10 -// numRunsPerWorker = 10000 -// ) - -// // Generate a different label per worker, and their expected hash. -// inputPerWorker := make([]labels.Label, 0, numWorkers) -// expectedPerWorker := make([][]byte, 0, numWorkers) - -// for w := 0; w < numWorkers; w++ { -// inputPerWorker = append(inputPerWorker, labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("series_%d", w)}) - -// hash, hashLen := postingsCacheKeyLabelID(inputPerWorker[w]) -// expectedPerWorker = append(expectedPerWorker, hash[0:hashLen]) -// } - -// // Sanity check: ensure expected hashes are different for each worker. -// for w := 0; w < numWorkers; w++ { -// for c := 0; c < numWorkers; c++ { -// if w == c { -// continue -// } - -// require.NotEqual(t, expectedPerWorker[w], expectedPerWorker[c]) -// } -// } - -// // Run workers, each generating the hash for their own label. -// wg := sync.WaitGroup{} -// wg.Add(numWorkers) - -// for w := 0; w < numWorkers; w++ { -// go func(workerID int) { -// defer wg.Done() - -// for r := 0; r < numRunsPerWorker; r++ { -// actual, hashLen := postingsCacheKeyLabelID(inputPerWorker[workerID]) -// assert.Equal(t, expectedPerWorker[workerID], actual[0:hashLen]) -// } -// }(w) -// } - -// wg.Wait() -// } - -// type mockedPostings struct { -// userID string -// block ulid.ULID -// label labels.Label -// value []byte -// } - -// type mockedSeriesForRef struct { -// userID string -// block ulid.ULID -// id storage.SeriesRef -// value []byte -// } - -// type mockedExpandedPostings struct { -// userID string -// block ulid.ULID -// matchers []*labels.Matcher -// postingsStrategy string -// value []byte -// } - -// type mockedLabelNames struct { -// userID string -// block ulid.ULID -// matchers []*labels.Matcher -// value []byte -// } - -// type mockedSeries struct { -// userID string -// block ulid.ULID -// matchers []*labels.Matcher -// shard *sharding.ShardSelector -// postings []storage.SeriesRef -// value []byte -// } - -// type mockedLabelValues struct { -// userID string -// block ulid.ULID -// labelName string -// matchers []*labels.Matcher -// value []byte -// } - -// type mockedRemoteCacheClient struct { -// cache map[string][]byte -// mockedGetMultiErr error -// } - -// func newMockedRemoteCacheClient(mockedGetMultiErr error) *mockedRemoteCacheClient { -// return &mockedRemoteCacheClient{ -// cache: map[string][]byte{}, -// mockedGetMultiErr: mockedGetMultiErr, -// } -// } - -// func (c *mockedRemoteCacheClient) GetMulti(_ context.Context, keys []string, _ ...cache.Option) map[string][]byte { -// if c.mockedGetMultiErr != nil { -// return nil -// } - -// hits := map[string][]byte{} - -// for _, key := range keys { -// if value, ok := c.cache[key]; ok { -// hits[key] = value -// } -// } - -// return hits -// } - -// func (c *mockedRemoteCacheClient) SetAsync(key string, value []byte, _ time.Duration) error { -// c.cache[key] = value - -// return nil -// } - -// func (c *mockedRemoteCacheClient) Delete(_ context.Context, key string) error { -// delete(c.cache, key) - -// return nil -// } - -// func (c *mockedRemoteCacheClient) Stop() { -// // Nothing to do. -// } - -// // remove a string from a slice of strings -// func remove(slice []string, needle string) []string { -// res := make([]string, 0, len(slice)) -// for _, s := range slice { -// if s != needle { -// res = append(res, s) -// } -// } -// return res -// } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 275eefd438459..6f4ab7d7c9685 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -91,7 +91,6 @@ type TSDBFile struct { } func NewShippableTSDBFile(id Identifier) (*TSDBFile, error) { - level.Warn(util_log.Logger).Log("msg", "NEW SHIPPABLE TSDB FILE") idx, getRawFileReader, err := NewTSDBIndexFromFile(id.Path()) if err != nil { return nil, err From b5be8fe91ccef396857c3ac88e2d899def972746 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 18:03:40 -0300 Subject: [PATCH 07/84] delete indexcache folder --- pkg/storage/stores/tsdb/indexcache/cache.go | 159 ------- .../stores/tsdb/indexcache/cache_test.go | 108 ----- .../stores/tsdb/indexcache/inmemory.go | 393 ----------------- .../stores/tsdb/indexcache/inmemory_test.go | 411 ------------------ pkg/storage/stores/tsdb/indexcache/tracing.go | 144 ------ 5 files changed, 1215 deletions(-) delete mode 100644 pkg/storage/stores/tsdb/indexcache/cache.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/cache_test.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/inmemory.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/inmemory_test.go delete mode 100644 pkg/storage/stores/tsdb/indexcache/tracing.go diff --git a/pkg/storage/stores/tsdb/indexcache/cache.go b/pkg/storage/stores/tsdb/indexcache/cache.go deleted file mode 100644 index 33eed15c09ee8..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/cache.go +++ /dev/null @@ -1,159 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/cache.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -package indexcache - -import ( - "encoding/base64" - "reflect" - "sort" - "strings" - "unsafe" - - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" - "golang.org/x/crypto/blake2b" -) - -const ( - cacheTypePostings = "Postings" - cacheTypeSeriesForRef = "SeriesForRef" - cacheTypeExpandedPostings = "ExpandedPostings" - cacheTypeSeriesForPostings = "SeriesForPostings" - cacheTypeLabelNames = "LabelNames" - cacheTypeLabelValues = "LabelValues" -) - -var ( - allCacheTypes = []string{ - cacheTypePostings, - cacheTypeSeriesForRef, - cacheTypeExpandedPostings, - cacheTypeSeriesForPostings, - cacheTypeLabelNames, - cacheTypeLabelValues, - } -) - -type BytesResult interface { - // Next should return a byte slice if there was a cache hit for the current key; otherwise Next should return nil. - // Next should return false when there are no more keys in the result. - Next() ([]byte, bool) - - // Remaining should return the number of keys left in the result. - // There may or may not be an item for each key. - Remaining() int - - // Size should return the size in bytes of the result. - Size() int -} - -type MapIterator[T comparable] struct { - M map[T][]byte - Keys []T -} - -func (l *MapIterator[T]) Next() ([]byte, bool) { - if len(l.Keys) == 0 { - return nil, false - } - b := l.M[l.Keys[0]] - l.Keys = l.Keys[1:] - return b, true -} - -func (l *MapIterator[T]) Remaining() int { - return len(l.Keys) -} - -func (l *MapIterator[T]) Size() int { - return sumBytes[T](l.M) -} - -func sumBytes[T comparable](res map[T][]byte) int { - sum := 0 - for _, v := range res { - sum += len(v) - } - return sum -} - -// PostingsKey represents a canonical key for a []storage.SeriesRef slice -type PostingsKey string - -// CanonicalPostingsKey creates a canonical version of PostingsKey -func CanonicalPostingsKey(postings []storage.SeriesRef) PostingsKey { - hashable := unsafeCastPostingsToBytes(postings) - checksum := blake2b.Sum256(hashable) - return PostingsKey(base64.RawURLEncoding.EncodeToString(checksum[:])) -} - -const bytesPerPosting = int(unsafe.Sizeof(storage.SeriesRef(0))) - -// unsafeCastPostingsToBytes returns the postings as a slice of bytes with minimal allocations. -// It casts the memory region of the underlying array to a slice of bytes. The resulting byte slice is only valid as long as the postings slice exists and is unmodified. -func unsafeCastPostingsToBytes(postings []storage.SeriesRef) []byte { - byteSlice := make([]byte, 0) - slicePtr := (*reflect.SliceHeader)(unsafe.Pointer(&byteSlice)) - slicePtr.Data = (*reflect.SliceHeader)(unsafe.Pointer(&postings)).Data - slicePtr.Len = len(postings) * bytesPerPosting - slicePtr.Cap = slicePtr.Len - return byteSlice -} - -// LabelMatchersKey represents a canonical key for a []*matchers.Matchers slice -type LabelMatchersKey string - -// CanonicalLabelMatchersKey creates a canonical version of LabelMatchersKey -func CanonicalLabelMatchersKey(ms []*labels.Matcher) LabelMatchersKey { - sorted := make([]labels.Matcher, len(ms)) - for i := range ms { - sorted[i] = labels.Matcher{Type: ms[i].Type, Name: ms[i].Name, Value: ms[i].Value} - } - sort.Sort(sortedLabelMatchers(sorted)) - - const ( - typeLen = 2 - sepLen = 1 - ) - var size int - for _, m := range sorted { - size += len(m.Name) + len(m.Value) + typeLen + sepLen - } - sb := strings.Builder{} - sb.Grow(size) - for _, m := range sorted { - sb.WriteString(m.Name) - sb.WriteString(m.Type.String()) - sb.WriteString(m.Value) - sb.WriteByte(0) - } - return LabelMatchersKey(sb.String()) -} - -type sortedLabelMatchers []labels.Matcher - -func (c sortedLabelMatchers) Less(i, j int) bool { - if c[i].Name != c[j].Name { - return c[i].Name < c[j].Name - } - if c[i].Type != c[j].Type { - return c[i].Type < c[j].Type - } - return c[i].Value < c[j].Value -} - -func (c sortedLabelMatchers) Len() int { return len(c) } -func (c sortedLabelMatchers) Swap(i, j int) { c[i], c[j] = c[j], c[i] } - -func initLabelValuesForAllCacheTypes(vec *prometheus.MetricVec) { - for _, typ := range allCacheTypes { - _, err := vec.GetMetricWithLabelValues(typ) - if err != nil { - panic(err) - } - } -} diff --git a/pkg/storage/stores/tsdb/indexcache/cache_test.go b/pkg/storage/stores/tsdb/indexcache/cache_test.go deleted file mode 100644 index b4189d7c46b5a..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/cache_test.go +++ /dev/null @@ -1,108 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/cache_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -package indexcache - -// func TestMain(m *testing.M) { -// test.VerifyNoLeakTestMain(m) -// } - -// func TestCanonicalLabelMatchersKey(t *testing.T) { -// foo := labels.MustNewMatcher(labels.MatchEqual, "foo", "bar") -// bar := labels.MustNewMatcher(labels.MatchEqual, "bar", "foo") - -// assert.Equal(t, CanonicalLabelMatchersKey([]*labels.Matcher{foo, bar}), CanonicalLabelMatchersKey([]*labels.Matcher{bar, foo})) -// } - -// func BenchmarkCanonicalLabelMatchersKey(b *testing.B) { -// ms := make([]*labels.Matcher, 20) -// for i := range ms { -// ms[i] = labels.MustNewMatcher(labels.MatchType(i%4), fmt.Sprintf("%04x", i%3), fmt.Sprintf("%04x", i%2)) -// } -// for _, l := range []int{1, 5, 10, 20} { -// b.Run(fmt.Sprintf("%d matchers", l), func(b *testing.B) { -// b.ResetTimer() -// for i := 0; i < b.N; i++ { -// _ = CanonicalLabelMatchersKey(ms[:l]) -// } -// }) -// } -// } - -// func BenchmarkCanonicalPostingsKey(b *testing.B) { -// ms := make([]storage.SeriesRef, 1_000_000) -// for i := range ms { -// ms[i] = storage.SeriesRef(i) -// } -// for numPostings := 10; numPostings <= len(ms); numPostings *= 10 { -// b.Run(fmt.Sprintf("%d postings", numPostings), func(b *testing.B) { -// b.ReportAllocs() -// for i := 0; i < b.N; i++ { -// _ = CanonicalPostingsKey(ms[:numPostings]) -// } -// }) -// } -// } - -// func TestUnsafeCastPostingsToBytes(t *testing.T) { -// slowPostingsToBytes := func(postings []storage.SeriesRef) []byte { -// byteSlice := make([]byte, len(postings)*8) -// for i, posting := range postings { -// for octet := 0; octet < 8; octet++ { -// byteSlice[i*8+octet] = byte(posting >> (octet * 8)) -// } -// } -// return byteSlice -// } -// t.Run("base case", func(t *testing.T) { -// postings := []storage.SeriesRef{1, 2} -// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) -// }) -// t.Run("zero-length postings", func(t *testing.T) { -// postings := make([]storage.SeriesRef, 0) -// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) -// }) -// t.Run("nil postings", func(t *testing.T) { -// assert.Equal(t, []byte(nil), unsafeCastPostingsToBytes(nil)) -// }) -// t.Run("more than 256 postings", func(t *testing.T) { -// // Only casting a slice pointer truncates all postings to only their last byte. -// postings := make([]storage.SeriesRef, 300) -// for i := range postings { -// postings[i] = storage.SeriesRef(i + 1) -// } -// assert.Equal(t, slowPostingsToBytes(postings), unsafeCastPostingsToBytes(postings)) -// }) -// } - -// func TestCanonicalPostingsKey(t *testing.T) { -// t.Run("same length postings have different hashes", func(t *testing.T) { -// postings1 := []storage.SeriesRef{1, 2, 3, 4} -// postings2 := []storage.SeriesRef{5, 6, 7, 8} - -// assert.NotEqual(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) -// }) - -// t.Run("when postings are a subset of each other, they still have different hashes", func(t *testing.T) { -// postings1 := []storage.SeriesRef{1, 2, 3, 4} -// postings2 := []storage.SeriesRef{1, 2, 3, 4, 5} - -// assert.NotEqual(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) -// }) - -// t.Run("same postings with different slice capacities have same hashes", func(t *testing.T) { -// postings1 := []storage.SeriesRef{1, 2, 3, 4} -// postings2 := make([]storage.SeriesRef, 4, 8) -// copy(postings2, postings1) - -// assert.Equal(t, CanonicalPostingsKey(postings1), CanonicalPostingsKey(postings2)) -// }) - -// t.Run("postings key is a base64-encoded string (i.e. is printable)", func(t *testing.T) { -// key := CanonicalPostingsKey([]storage.SeriesRef{1, 2, 3, 4}) -// _, err := base64.RawURLEncoding.DecodeString(string(key)) -// assert.NoError(t, err) -// }) -// } diff --git a/pkg/storage/stores/tsdb/indexcache/inmemory.go b/pkg/storage/stores/tsdb/indexcache/inmemory.go deleted file mode 100644 index 798cccb2d7f6c..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/inmemory.go +++ /dev/null @@ -1,393 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. -package indexcache - -import ( - "context" - "encoding/binary" - "fmt" - "sync" - - "github.com/dennwc/varint" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/flagext" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/grafana/loki/pkg/util/encoding" - lru "github.com/hashicorp/golang-lru/simplelru" - "github.com/oklog/ulid" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" -) - -type codec string - -const ( - codecHeaderSnappy codec = "dvs" // As in "diff+varint+snappy". - codecHeaderSnappyWithMatchers codec = "dm" // As in "dvs+matchers" -) - -var DefaultInMemoryIndexCacheConfig = InMemoryIndexCacheConfig{ - MaxSize: 250 * 1024 * 1024, - MaxItemSize: 125 * 1024 * 1024, -} - -const maxInt = int(^uint(0) >> 1) - -const ( - stringHeaderSize = 8 - sliceHeaderSize = 16 -) - -var ulidSize = uint64(len(ulid.ULID{})) - -type InMemoryIndexCache struct { - mtx sync.Mutex - - logger log.Logger - lru *lru.LRU - maxSizeBytes uint64 - maxItemSizeBytes uint64 - - curSize uint64 - - evicted *prometheus.CounterVec - requests *prometheus.CounterVec - hits *prometheus.CounterVec - added *prometheus.CounterVec - current *prometheus.GaugeVec - currentSize *prometheus.GaugeVec - totalCurrentSize *prometheus.GaugeVec - overflow *prometheus.CounterVec -} - -// InMemoryIndexCacheConfig holds the in-memory index cache config. -type InMemoryIndexCacheConfig struct { - // MaxSize represents overall maximum number of bytes cache can contain. - MaxSize flagext.Bytes `yaml:"max_size"` - // MaxItemSize represents maximum size of single item. - MaxItemSize flagext.Bytes `yaml:"max_item_size"` -} - -// NewInMemoryIndexCache creates a new thread-safe LRU cache for index entries and ensures the total cache -// size approximately does not exceed maxBytes. -func NewInMemoryIndexCache(logger log.Logger, reg prometheus.Registerer, cfg InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) { - return NewInMemoryIndexCacheWithConfig(logger, reg, cfg) -} - -// NewInMemoryIndexCacheWithConfig creates a new thread-safe LRU cache for index entries and ensures the total cache -// size approximately does not exceed maxBytes. -func NewInMemoryIndexCacheWithConfig(logger log.Logger, reg prometheus.Registerer, config InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) { - if config.MaxItemSize > config.MaxSize { - return nil, errors.Errorf("max item size (%v) cannot be bigger than overall cache size (%v)", config.MaxItemSize, config.MaxSize) - } - - c := &InMemoryIndexCache{ - logger: logger, - maxSizeBytes: uint64(config.MaxSize), - maxItemSizeBytes: uint64(config.MaxItemSize), - } - - c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_evicted_total", - Help: "Total number of items that were evicted from the index cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.evicted.MetricVec) - - c.added = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_added_total", - Help: "Total number of items that were added to the index cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.added.MetricVec) - - c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_requests_total", - Help: "Total number of requests to the cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.requests.MetricVec) - - c.overflow = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_overflowed_total", - Help: "Total number of items that could not be added to the cache due to being too big.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.overflow.MetricVec) - - c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_hits_total", - Help: "Total number of requests to the cache that were a hit.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.hits.MetricVec) - - c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items", - Help: "Current number of items in the index cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.current.MetricVec) - - c.currentSize = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_size_bytes", - Help: "Current byte size of items in the index cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.currentSize.MetricVec) - - c.totalCurrentSize = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_total_size_bytes", - Help: "Current byte size of items (both value and key) in the index cache.", - }, []string{"item_type"}) - initLabelValuesForAllCacheTypes(c.totalCurrentSize.MetricVec) - - _ = promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_max_size_bytes", - Help: "Maximum number of bytes to be held in the index cache.", - }, func() float64 { - return float64(c.maxSizeBytes) - }) - _ = promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_max_item_size_bytes", - Help: "Maximum number of bytes for single entry to be held in the index cache.", - }, func() float64 { - return float64(c.maxItemSizeBytes) - }) - - // Initialize LRU cache with a high size limit since we will manage evictions ourselves - // based on stored size using `RemoveOldest` method. - l, err := lru.NewLRU(maxInt, c.onEvict) - if err != nil { - return nil, err - } - c.lru = l - - level.Info(logger).Log( - "msg", "created in-memory index cache", - "maxItemSizeBytes", c.maxItemSizeBytes, - "maxSizeBytes", c.maxSizeBytes, - "maxItems", "maxInt", - ) - return c, nil -} - -func (c *InMemoryIndexCache) onEvict(key, val interface{}) { - k := key.(cacheKey) - typ := k.typ() - entrySize := sliceSize(val.([]byte)) - - c.evicted.WithLabelValues(typ).Inc() - c.current.WithLabelValues(typ).Dec() - c.currentSize.WithLabelValues(typ).Sub(float64(entrySize)) - c.totalCurrentSize.WithLabelValues(typ).Sub(float64(entrySize + k.size())) - - c.curSize -= entrySize -} - -func (c *InMemoryIndexCache) get(key cacheKey) ([]byte, bool) { - typ := key.typ() - c.requests.WithLabelValues(typ).Inc() - - c.mtx.Lock() - defer c.mtx.Unlock() - - v, ok := c.lru.Get(key) - if !ok { - return nil, false - } - c.hits.WithLabelValues(typ).Inc() - return v.([]byte), true -} - -// Store(ctx context.Context, key []string, buf [][]byte) error -// Stop() -// // GetCacheType returns a string indicating the cache "type" for the purpose of grouping cache usage statistics -// GetCacheType() stats.CacheType - -// func (c *InMemoryIndexCache) Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { -// got, ok := c.get(cacheKeyPostings{}) - -// } - -func (c *InMemoryIndexCache) set(key cacheKey, val []byte) { - typ := key.typ() - size := sliceSize(val) - - c.mtx.Lock() - defer c.mtx.Unlock() - - if _, ok := c.lru.Get(key); ok { - return - } - - if !c.ensureFits(size, typ) { - c.overflow.WithLabelValues(typ).Inc() - return - } - - // The caller may be passing in a sub-slice of a huge array. Copy the data - // to ensure we don't waste huge amounts of space for something small. - v := make([]byte, len(val)) - copy(v, val) - c.lru.Add(key, v) - - c.added.WithLabelValues(typ).Inc() - c.currentSize.WithLabelValues(typ).Add(float64(size)) - c.totalCurrentSize.WithLabelValues(typ).Add(float64(size + key.size())) - c.current.WithLabelValues(typ).Inc() - c.curSize += size -} - -// ensureFits tries to make sure that the passed slice will fit into the LRU cache. -// Returns true if it will fit. -func (c *InMemoryIndexCache) ensureFits(size uint64, typ string) bool { - if size > c.maxItemSizeBytes { - level.Debug(c.logger).Log( - "msg", "item bigger than maxItemSizeBytes. Ignoring..", - "maxItemSizeBytes", c.maxItemSizeBytes, - "maxSizeBytes", c.maxSizeBytes, - "curSize", c.curSize, - "itemSize", size, - "cacheType", typ, - ) - return false - } - - for c.curSize+size > c.maxSizeBytes { - if _, _, ok := c.lru.RemoveOldest(); !ok { - level.Error(c.logger).Log( - "msg", "LRU has nothing more to evict, but we still cannot allocate the item. Resetting cache.", - "maxItemSizeBytes", c.maxItemSizeBytes, - "maxSizeBytes", c.maxSizeBytes, - "curSize", c.curSize, - "itemSize", size, - "cacheType", typ, - ) - c.reset() - } - } - return true -} - -func (c *InMemoryIndexCache) reset() { - c.lru.Purge() - c.current.Reset() - c.currentSize.Reset() - c.totalCurrentSize.Reset() - c.curSize = 0 -} - -func (c *InMemoryIndexCache) StorePostings(postings index.Postings, matchers []*labels.Matcher) { - dataToCache, err := diffVarintEncodeNoHeader(postings, 0) - if err != nil { - level.Warn(c.logger).Log("msg", "couldn't encode postings", "err", err, "matchers", CanonicalLabelMatchersKey(matchers)) - } - - c.set(cacheKeyPostings{matchers: matchers}, dataToCache) -} - -// diffVarintEncodeNoHeader encodes postings into diff+varint representation. -// It doesn't add any header to the output bytes. -// Length argument is expected number of postings, used for preallocating buffer. -func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { - buf := encoding.Encbuf{} - - // This encoding uses around ~1 bytes per posting, but let's use - // conservative 1.25 bytes per posting to avoid extra allocations. - if length > 0 { - buf.B = make([]byte, 0, 5*length/4) - } - - prev := storage.SeriesRef(0) - for p.Next() { - v := p.At() - - // TODO(dylanguedes): can we ignore this? - // if v < prev { - // return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) - // } - - // This is the 'diff' part -- compute difference from previous value. - buf.PutUvarint64(uint64(v - prev)) - prev = v - } - if p.Err() != nil { - return nil, p.Err() - } - - return buf.B, nil -} - -func encodedMatchersLen(matchers []*labels.Matcher) int { - matchersLen := varint.UvarintSize(uint64(len(matchers))) - for _, m := range matchers { - matchersLen += varint.UvarintSize(uint64(len(m.Name))) - matchersLen += len(m.Name) - matchersLen++ // 1 byte for the type - matchersLen += varint.UvarintSize(uint64(len(m.Value))) - matchersLen += len(m.Value) - } - return matchersLen -} - -// encodeMatchers needs to be called with the precomputed length of the encoded matchers from encodedMatchersLen -func encodeMatchers(expectedLen int, matchers []*labels.Matcher, dest []byte) (written int, _ error) { - if len(dest) < expectedLen { - return 0, fmt.Errorf("too small buffer to encode matchers: need at least %d, got %d", expectedLen, dest) - } - written += binary.PutUvarint(dest, uint64(len(matchers))) - for _, m := range matchers { - written += binary.PutUvarint(dest[written:], uint64(len(m.Name))) - written += copy(dest[written:], m.Name) - - dest[written] = byte(m.Type) - written++ - - written += binary.PutUvarint(dest[written:], uint64(len(m.Value))) - written += copy(dest[written:], m.Value) - } - return written, nil -} - -// FetchSeriesForPostings fetches a series set for the provided postings. -func (c *InMemoryIndexCache) FetchSeriesForPostings(_ context.Context, matchers []*labels.Matcher) ([]byte, bool) { - return c.get(cacheKeyPostings{matchers: matchers}) -} - -// cacheKey is used by in-memory representation to store cached data. -// The implementations of cacheKey should be hashable, as they will be used as keys for *lru.LRU cache -type cacheKey interface { - // typ is used as label for metrics. - typ() string - // size is used to keep track of the cache size, it represents the footprint of the cache key in memory. - size() uint64 -} - -// cacheKeyPostings implements cacheKey and is used to reference a postings cache entry in the inmemory cache. -type cacheKeyPostings struct { - matchers []*labels.Matcher -} - -func (c cacheKeyPostings) typ() string { return cacheTypePostings } - -func (c cacheKeyPostings) size() uint64 { - return stringSize(string(CanonicalLabelMatchersKey(c.matchers))) -} - -func stringSize(s string) uint64 { - return stringHeaderSize + uint64(len(s)) -} - -func sliceSize(b []byte) uint64 { - return sliceHeaderSize + uint64(len(b)) -} diff --git a/pkg/storage/stores/tsdb/indexcache/inmemory_test.go b/pkg/storage/stores/tsdb/indexcache/inmemory_test.go deleted file mode 100644 index e50df7214243a..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/inmemory_test.go +++ /dev/null @@ -1,411 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -// Tests out the index cache implementation. -package indexcache - -// func TestInMemoryIndexCache_AvoidsDeadlock(t *testing.T) { -// user := "tenant" -// metrics := prometheus.NewRegistry() -// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ -// MaxItemSize: sliceHeaderSize + 5, -// MaxSize: sliceHeaderSize + 5, -// }) -// assert.NoError(t, err) - -// l, err := simplelru.NewLRU(math.MaxInt64, func(key, val interface{}) { -// // Hack LRU to simulate broken accounting: evictions do not reduce current size. -// size := cache.curSize -// cache.onEvict(key, val) -// cache.curSize = size -// }) -// assert.NoError(t, err) -// cache.lru = l - -// cache.StorePostings(user, ulid.MustNew(0, nil), labels.Label{Name: "test2", Value: "1"}, []byte{42, 33, 14, 67, 11}) - -// assert.Equal(t, uint64(sliceHeaderSize+5), cache.curSize) -// assert.Equal(t, float64(cache.curSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) - -// // This triggers deadlock logic. -// cache.StorePostings(user, ulid.MustNew(0, nil), labels.Label{Name: "test1", Value: "1"}, []byte{42}) - -// assert.Equal(t, uint64(sliceHeaderSize+1), cache.curSize) -// assert.Equal(t, float64(cache.curSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// } - -// func TestInMemoryIndexCache_UpdateItem(t *testing.T) { -// const maxSize = 2 * (sliceHeaderSize + 1) - -// var errorLogs []string -// errorLogger := log.LoggerFunc(func(kvs ...interface{}) error { -// var lvl string -// for i := 0; i < len(kvs); i += 2 { -// if kvs[i] == "level" { -// lvl = fmt.Sprint(kvs[i+1]) -// break -// } -// } -// if lvl != "error" { -// return nil -// } -// var buf bytes.Buffer -// defer func() { errorLogs = append(errorLogs, buf.String()) }() -// return log.NewLogfmtLogger(&buf).Log(kvs...) -// }) - -// metrics := prometheus.NewRegistry() -// cache, err := NewInMemoryIndexCacheWithConfig(log.NewSyncLogger(errorLogger), metrics, InMemoryIndexCacheConfig{ -// MaxItemSize: maxSize, -// MaxSize: maxSize, -// }) -// assert.NoError(t, err) - -// user := "tenant" -// uid := func(id uint64) ulid.ULID { return ulid.MustNew(uint64(id), nil) } -// lbl := labels.Label{Name: "foo", Value: "bar"} -// matchers := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), labels.MustNewMatcher(labels.MatchNotRegexp, "baz", ".*")} -// shard := &sharding.ShardSelector{ShardIndex: 1, ShardCount: 16} -// ctx := context.Background() - -// for _, tt := range []struct { -// typ string -// set func(uint64, []byte) -// get func(uint64) ([]byte, bool) -// }{ -// { -// typ: cacheTypePostings, -// set: func(id uint64, b []byte) { cache.StorePostings(user, uid(id), lbl, b) }, -// get: func(id uint64) ([]byte, bool) { -// hits := cache.FetchMultiPostings(ctx, user, uid(id), []labels.Label{lbl}) -// b, _ := hits.Next() -// return b, b != nil -// }, -// }, -// { -// typ: cacheTypeSeriesForRef, -// set: func(id uint64, b []byte) { -// cache.StoreSeriesForRef(user, uid(id), storage.SeriesRef(id), b) -// }, -// get: func(id uint64) ([]byte, bool) { -// seriesRef := storage.SeriesRef(id) -// hits, _ := cache.FetchMultiSeriesForRefs(ctx, user, uid(id), []storage.SeriesRef{seriesRef}) -// b, ok := hits[seriesRef] - -// return b, ok -// }, -// }, -// { -// typ: cacheTypeExpandedPostings, -// set: func(id uint64, b []byte) { -// cache.StoreExpandedPostings(user, uid(id), CanonicalLabelMatchersKey(matchers), "strategy", b) -// }, -// get: func(id uint64) ([]byte, bool) { -// return cache.FetchExpandedPostings(ctx, user, uid(id), CanonicalLabelMatchersKey(matchers), "strategy") -// }, -// }, -// { -// typ: cacheTypeSeriesForPostings, -// set: func(id uint64, b []byte) { -// cache.StoreSeriesForPostings(user, uid(id), shard, CanonicalPostingsKey([]storage.SeriesRef{1}), b) -// }, -// get: func(id uint64) ([]byte, bool) { -// return cache.FetchSeriesForPostings(ctx, user, uid(id), shard, CanonicalPostingsKey([]storage.SeriesRef{1})) -// }, -// }, -// { -// typ: cacheTypeLabelNames, -// set: func(id uint64, b []byte) { -// cache.StoreLabelNames(user, uid(id), CanonicalLabelMatchersKey(matchers), b) -// }, -// get: func(id uint64) ([]byte, bool) { -// return cache.FetchLabelNames(ctx, user, uid(id), CanonicalLabelMatchersKey(matchers)) -// }, -// }, -// { -// typ: cacheTypeLabelValues, -// set: func(id uint64, b []byte) { -// cache.StoreLabelValues(user, uid(id), fmt.Sprintf("lbl_%d", id), CanonicalLabelMatchersKey(matchers), b) -// }, -// get: func(id uint64) ([]byte, bool) { -// return cache.FetchLabelValues(ctx, user, uid(id), fmt.Sprintf("lbl_%d", id), CanonicalLabelMatchersKey(matchers)) -// }, -// }, -// } { -// t.Run(tt.typ, func(t *testing.T) { -// defer func() { errorLogs = nil }() - -// // Set value. -// tt.set(0, []byte{0}) -// buf, ok := tt.get(0) -// assert.Equal(t, true, ok) -// assert.Equal(t, []byte{0}, buf) -// assert.Equal(t, float64(sliceHeaderSize+1), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) -// assert.Equal(t, []string(nil), errorLogs) - -// // Set the same value again. -// // NB: This used to over-count the value. -// tt.set(0, []byte{0}) -// buf, ok = tt.get(0) -// assert.Equal(t, true, ok) -// assert.Equal(t, []byte{0}, buf) -// assert.Equal(t, float64(sliceHeaderSize+1), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) -// assert.Equal(t, []string(nil), errorLogs) - -// // Set a larger value. -// // NB: This used to deadlock when enough values were over-counted and it -// // couldn't clear enough space -- repeatedly removing oldest after empty. -// tt.set(1, []byte{0, 1}) -// buf, ok = tt.get(1) -// assert.Equal(t, true, ok) -// assert.Equal(t, []byte{0, 1}, buf) -// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) -// assert.Equal(t, []string(nil), errorLogs) - -// // Mutations to existing values will be ignored. -// tt.set(1, []byte{1, 2}) -// buf, ok = tt.get(1) -// assert.Equal(t, true, ok) -// assert.Equal(t, []byte{0, 1}, buf) -// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(tt.typ))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(tt.typ))) -// assert.Equal(t, []string(nil), errorLogs) -// }) -// } -// } - -// // This should not happen as we hardcode math.MaxInt, but we still add test to check this out. -// func TestInMemoryIndexCache_MaxNumberOfItemsHit(t *testing.T) { -// user := "tenant" -// metrics := prometheus.NewRegistry() -// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ -// MaxItemSize: 2*sliceHeaderSize + 10, -// MaxSize: 2*sliceHeaderSize + 10, -// }) -// assert.NoError(t, err) - -// l, err := simplelru.NewLRU(2, cache.onEvict) -// assert.NoError(t, err) -// cache.lru = l - -// id := ulid.MustNew(0, nil) - -// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "123"}, []byte{42, 33}) -// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "124"}, []byte{42, 33}) -// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "125"}, []byte{42, 33}) - -// assert.Equal(t, uint64(2*sliceHeaderSize+4), cache.curSize) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(3), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings))) -// for _, typ := range remove(allCacheTypes, cacheTypePostings) { -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(typ))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(typ))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.added.WithLabelValues(typ))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(typ))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(typ))) -// } -// } - -// func TestInMemoryIndexCache_Eviction_WithMetrics(t *testing.T) { -// user := "tenant" -// metrics := prometheus.NewRegistry() -// cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{ -// MaxItemSize: 2*sliceHeaderSize + 5, -// MaxSize: 2*sliceHeaderSize + 5, -// }) -// assert.NoError(t, err) - -// id := ulid.MustNew(0, nil) -// lbls := labels.Label{Name: "test", Value: "123"} -// ctx := context.Background() -// emptySeriesHits := map[storage.SeriesRef][]byte{} -// emptySeriesMisses := []storage.SeriesRef(nil) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, nil) - -// // Add sliceHeaderSize + 2 bytes. -// cache.StorePostings(user, id, lbls, []byte{42, 33}) -// assert.Equal(t, uint64(sliceHeaderSize+2), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize+2+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, map[labels.Label][]byte{lbls: {42, 33}}) - -// testFetchMultiPostings(ctx, t, cache, user, ulid.MustNew(1, nil), []labels.Label{lbls}, nil) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{{Name: "test", Value: "124"}}, nil) - -// // Add sliceHeaderSize + 3 more bytes. -// cache.StoreSeriesForRef(user, id, 1234, []byte{222, 223, 224}) -// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize+2), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize+2+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(sliceHeaderSize+3), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(sliceHeaderSize+3+cacheKeySeriesForRef{user, id, 1234}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// sHits, sMisses := cache.FetchMultiSeriesForRefs(ctx, user, id, []storage.SeriesRef{1234}) -// assert.Equal(t, map[storage.SeriesRef][]byte{1234: {222, 223, 224}}, sHits, "key exists") -// assert.Equal(t, emptySeriesMisses, sMisses) - -// lbls2 := labels.Label{Name: "test", Value: "124"} - -// // Add sliceHeaderSize + 5 + 16 bytes, should fully evict 2 last items. -// v := []byte{42, 33, 14, 67, 11} -// for i := 0; i < sliceHeaderSize; i++ { -// v = append(v, 3) -// } -// cache.StorePostings(user, id, lbls2, v) - -// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) // Eviction. -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) // Eviction. - -// // Evicted. -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls}, nil) - -// sHits, sMisses = cache.FetchMultiSeriesForRefs(ctx, user, id, []storage.SeriesRef{1234}) -// assert.Equal(t, emptySeriesHits, sHits, "no such key") -// assert.Equal(t, []storage.SeriesRef{1234}, sMisses) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls2}, map[labels.Label][]byte{lbls2: v}) - -// // Add same item again. -// cache.StorePostings(user, id, lbls2, v) - -// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls2}, map[labels.Label][]byte{lbls2: v}) - -// // Add too big item. -// cache.StorePostings(user, id, labels.Label{Name: "test", Value: "toobig"}, append(v, 5)) -// assert.Equal(t, uint64(2*sliceHeaderSize+5), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2*sliceHeaderSize+5+cacheKeyPostings{user, id, lbls}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) // Overflow. -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// _, _, ok := cache.lru.RemoveOldest() -// assert.True(t, ok, "something to remove") - -// assert.Equal(t, uint64(0), cache.curSize) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// _, _, ok = cache.lru.RemoveOldest() -// assert.True(t, !ok, "nothing to remove") - -// lbls3 := labels.Label{Name: "test", Value: "124"} - -// cache.StorePostings(user, id, lbls3, []byte{}) - -// assert.Equal(t, uint64(sliceHeaderSize), cache.curSize) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(sliceHeaderSize+cacheKeyPostings{user, id, lbls3}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls3}, map[labels.Label][]byte{lbls3: {}}) - -// // nil works and still allocates empty slice. -// lbls4 := labels.Label{Name: "test", Value: "125"} -// cache.StorePostings(user, id, lbls4, []byte(nil)) - -// assert.Equal(t, 2*uint64(sliceHeaderSize), cache.curSize) -// assert.Equal(t, float64(2), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, 2*float64(sliceHeaderSize), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, 2*float64(sliceHeaderSize+cacheKeyPostings{user, id, lbls4}.size()), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.current.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.currentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.totalCurrentSize.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(0), promtest.ToFloat64(cache.overflow.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(2), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeriesForRef))) - -// testFetchMultiPostings(ctx, t, cache, user, id, []labels.Label{lbls4}, map[labels.Label][]byte{lbls4: {}}) - -// // Other metrics. -// assert.Equal(t, float64(4), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(9), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(2), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypeSeriesForRef))) -// assert.Equal(t, float64(5), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings))) -// assert.Equal(t, float64(1), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypeSeriesForRef))) -// } - -// func testFetchMultiPostings(ctx context.Context, t *testing.T, cache IndexCache, user string, id ulid.ULID, keys []labels.Label, expectedHits map[labels.Label][]byte) { -// t.Helper() -// pHits := cache.FetchMultiPostings(ctx, user, id, keys) -// expectedResult := &MapIterator[labels.Label]{M: expectedHits, Keys: keys} - -// assert.Equal(t, expectedResult.Remaining(), pHits.Remaining()) -// for exp, hasNext := expectedResult.Next(); hasNext; exp, hasNext = expectedResult.Next() { -// actual, ok := pHits.Next() -// assert.True(t, ok) -// assert.Equal(t, exp, actual) -// } -// _, ok := pHits.Next() -// assert.False(t, ok) -// } diff --git a/pkg/storage/stores/tsdb/indexcache/tracing.go b/pkg/storage/stores/tsdb/indexcache/tracing.go deleted file mode 100644 index fb97795f6575a..0000000000000 --- a/pkg/storage/stores/tsdb/indexcache/tracing.go +++ /dev/null @@ -1,144 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only - -package indexcache - -// type TracingIndexCache struct { -// c IndexCache -// logger log.Logger -// } - -// func NewTracingIndexCache(cache IndexCache, logger log.Logger) IndexCache { -// return &TracingIndexCache{ -// c: cache, -// logger: logger, -// } -// } - -// func (t *TracingIndexCache) StorePostings(userID string, blockID ulid.ULID, l labels.Label, v []byte) { -// t.c.StorePostings(userID, blockID, l, v) -// } - -// func (t *TracingIndexCache) FetchMultiPostings(ctx context.Context, userID string, blockID ulid.ULID, keys []labels.Label) (hits BytesResult) { -// t0 := time.Now() -// hits = t.c.FetchMultiPostings(ctx, userID, blockID, keys) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log( -// "msg", "IndexCache.FetchMultiPostings", -// "requested keys", len(keys), -// "cache hits", hits.Remaining(), -// "cache misses", len(keys)-hits.Remaining(), -// "time elapsed", time.Since(t0), -// "returned bytes", hits.Size(), -// "user_id", userID, -// ) -// return hits -// } - -// func (t *TracingIndexCache) StoreSeriesForRef(userID string, blockID ulid.ULID, id storage.SeriesRef, v []byte) { -// t.c.StoreSeriesForRef(userID, blockID, id, v) -// } - -// func (t *TracingIndexCache) FetchMultiSeriesForRefs(ctx context.Context, userID string, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef) { -// t0 := time.Now() -// hits, misses = t.c.FetchMultiSeriesForRefs(ctx, userID, blockID, ids) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log("msg", "IndexCache.FetchMultiSeriesForRefs", -// "requested series", len(ids), -// "cache hits", len(hits), -// "cache misses", len(misses), -// "time elapsed", time.Since(t0), -// "returned bytes", sumBytes(hits), -// "user_id", userID, -// ) - -// return hits, misses -// } - -// func (t *TracingIndexCache) StoreExpandedPostings(userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string, v []byte) { -// t.c.StoreExpandedPostings(userID, blockID, key, postingsSelectionStrategy, v) -// } - -// func (t *TracingIndexCache) FetchExpandedPostings(ctx context.Context, userID string, blockID ulid.ULID, key LabelMatchersKey, postingsSelectionStrategy string) ([]byte, bool) { -// t0 := time.Now() -// data, found := t.c.FetchExpandedPostings(ctx, userID, blockID, key, postingsSelectionStrategy) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log( -// "msg", "IndexCache.FetchExpandedPostings", -// "requested key", key, -// "postings selection strategy", postingsSelectionStrategy, -// "found", found, -// "time elapsed", time.Since(t0), -// "returned bytes", len(data), -// "user_id", userID, -// ) - -// return data, found -// } - -// func (t *TracingIndexCache) StoreSeriesForPostings(userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey, v []byte) { -// t.c.StoreSeriesForPostings(userID, blockID, shard, postingsKey, v) -// } - -// func (t *TracingIndexCache) FetchSeriesForPostings(ctx context.Context, userID string, blockID ulid.ULID, shard *sharding.ShardSelector, postingsKey PostingsKey) ([]byte, bool) { -// t0 := time.Now() -// data, found := t.c.FetchSeriesForPostings(ctx, userID, blockID, shard, postingsKey) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log( -// "msg", "IndexCache.FetchSeriesForPostings", -// "shard", shardKey(shard), -// "found", found, -// "time_elapsed", time.Since(t0), -// "returned_bytes", len(data), -// "user_id", userID, -// "postings_key", postingsKey, -// ) - -// return data, found -// } - -// func (t *TracingIndexCache) StoreLabelNames(userID string, blockID ulid.ULID, matchersKey LabelMatchersKey, v []byte) { -// t.c.StoreLabelNames(userID, blockID, matchersKey, v) -// } - -// func (t *TracingIndexCache) FetchLabelNames(ctx context.Context, userID string, blockID ulid.ULID, matchersKey LabelMatchersKey) ([]byte, bool) { -// t0 := time.Now() -// data, found := t.c.FetchLabelNames(ctx, userID, blockID, matchersKey) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log( -// "msg", "IndexCache.FetchLabelNames", -// "requested key", matchersKey, -// "found", found, -// "time elapsed", time.Since(t0), -// "returned bytes", len(data), -// "user_id", userID, -// ) - -// return data, found -// } - -// func (t *TracingIndexCache) StoreLabelValues(userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey, v []byte) { -// t.c.StoreLabelValues(userID, blockID, labelName, matchersKey, v) -// } - -// func (t *TracingIndexCache) FetchLabelValues(ctx context.Context, userID string, blockID ulid.ULID, labelName string, matchersKey LabelMatchersKey) ([]byte, bool) { -// t0 := time.Now() -// data, found := t.c.FetchLabelValues(ctx, userID, blockID, labelName, matchersKey) - -// spanLogger := spanlogger.FromContext(ctx, t.logger) -// level.Debug(spanLogger).Log( -// "msg", "IndexCache.FetchLabelValues", -// "label name", labelName, -// "requested key", matchersKey, -// "found", found, -// "time elapsed", time.Since(t0), -// "returned bytes", len(data), -// "user_id", userID, -// ) - -// return data, found -// } From f795168042315404c5633132f3d44241dbe75fbc Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 18:04:49 -0300 Subject: [PATCH 08/84] undo change --- pkg/storage/stores/shipper/indexgateway/gateway.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/shipper/indexgateway/gateway.go b/pkg/storage/stores/shipper/indexgateway/gateway.go index 9a15614af579c..7c37f4faeb58b 100644 --- a/pkg/storage/stores/shipper/indexgateway/gateway.go +++ b/pkg/storage/stores/shipper/indexgateway/gateway.go @@ -85,7 +85,7 @@ func NewIndexGateway(cfg Config, log log.Logger, registerer prometheus.Registere } func (g *Gateway) QueryIndex(request *logproto.QueryIndexRequest, server logproto.IndexGateway_QueryIndexServer) error { - log, ctx := spanlogger.New(server.Context(), "IndexGateway.QueryIndex") + log, _ := spanlogger.New(context.Background(), "IndexGateway.QueryIndex") defer log.Finish() var outerErr, innerErr error @@ -127,7 +127,7 @@ func (g *Gateway) QueryIndex(request *logproto.QueryIndexRequest, server logprot continue } - outerErr = indexClient.QueryPages(ctx, queries[start:end], func(query seriesindex.Query, batch seriesindex.ReadBatchResult) bool { + outerErr = indexClient.QueryPages(server.Context(), queries[start:end], func(query seriesindex.Query, batch seriesindex.ReadBatchResult) bool { innerErr = buildResponses(query, batch, func(response *logproto.QueryIndexResponse) error { // do not send grpc responses concurrently. See https://github.com/grpc/grpc-go/blob/master/stream.go#L120-L123. sendBatchMtx.Lock() From e9d265000e02d9955f62fc87888c760fc4bfa03c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 18:17:53 -0300 Subject: [PATCH 09/84] trim down code --- pkg/storage/chunk/cache/lru_cache.go | 44 +++++++--------------------- 1 file changed, 11 insertions(+), 33 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index e69f8dc2f671f..041bcb0d96122 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -2,9 +2,9 @@ package cache import ( "context" + "flag" "fmt" "sync" - "time" "unsafe" "github.com/go-kit/log" @@ -37,43 +37,27 @@ const ( var ulidSize = uint64(len(ulid.ULID{})) -// This FIFO cache implementation supports two eviction methods - based on number of items in the cache, and based on memory usage. -// For the memory-based eviction, set FifoCacheConfig.MaxSizeBytes to a positive integer, indicating upper limit of memory allocated by items in the cache. -// Alternatively, set FifoCacheConfig.MaxSizeItems to a positive integer, indicating maximum number of items in the cache. -// If both parameters are set, both methods are enforced, whichever hits first. - -// FifoCacheConfig holds config for the FifoCache. type LRUCacheConfig struct { MaxSizeBytes string `yaml:"max_size_bytes"` - Enabled bool - - PurgeInterval time.Duration + Enabled bool `yaml:"enabled"` } // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet -// func (cfg *FifoCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { -// f.StringVar(&cfg.MaxSizeBytes, prefix+"fifocache.max-size-bytes", "1GB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") -// f.IntVar(&cfg.MaxSizeItems, prefix+"fifocache.max-size-items", 0, description+"deprecated: Maximum number of entries in the cache.") -// f.DurationVar(&cfg.TTL, prefix+"fifocache.ttl", time.Hour, description+"The time to live for items in the cache before they get purged.") - -// f.DurationVar(&cfg.DeprecatedValidity, prefix+"fifocache.duration", 0, "Deprecated (use ttl instead): "+description+"The expiry duration for the cache.") -// f.IntVar(&cfg.DeprecatedSize, prefix+"fifocache.size", 0, "Deprecated (use max-size-items or max-size-bytes instead): "+description+"The number of entries to cache.") -// } - -// func (cfg *FifoCacheConfig) Validate() error { -// _, err := parsebytes(cfg.MaxSizeBytes) -// return err -// } - -// FifoCache is a simple string -> interface{} cache which uses a fifo slide to -// manage evictions. O(1) inserts and updates, O(1) gets. +func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { + f.StringVar(&cfg.MaxSizeBytes, prefix+"fifocache.max-size-bytes", "500MB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") +} + +func (cfg *LRUCacheConfig) Validate() error { + _, err := parsebytes(cfg.MaxSizeBytes) + return err +} + type LRUCache struct { cacheType stats.CacheType done chan struct{} - // important ones below mtx sync.Mutex logger log.Logger @@ -91,17 +75,11 @@ type LRUCache struct { overflow *prometheus.CounterVec } -// TODO: better description: NewLRUCache returns a new initialised LRU cache of size. func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (*LRUCache, error) { util_log.WarnExperimentalUse(fmt.Sprintf("In-memory (LRU) cache - %s", name), logger) maxSizeBytes, _ := parsebytes(cfg.MaxSizeBytes) - // This can be overwritten to a smaller value in tests - if cfg.PurgeInterval == 0 { - cfg.PurgeInterval = 1 * time.Minute - } - c := &LRUCache{ cacheType: cacheType, From 3b5ab8f5f4f85ff7a0b253f7e87050c198f7f806 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 12 Jun 2023 18:26:16 -0300 Subject: [PATCH 10/84] Use postingsclient. --- pkg/storage/stores/tsdb/single_file_index.go | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 4ad1629121d85..ee6b956360d09 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -223,20 +223,6 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, } -func (i *TSDBIndex) forPostings( - _ context.Context, - shard *index.ShardAnnotation, - _, _ model.Time, - matchers []*labels.Matcher, - fn func(index.Postings) error, -) error { - p, err := PostingsForMatchers(i.reader, shard, matchers...) - if err != nil { - return err - } - return fn(p) -} - func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() @@ -312,7 +298,7 @@ func (i *TSDBIndex) Identifier(string) SingleTenantTSDBIdentifier { } func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error { - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + return i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { // TODO(owen-d): use pool var ls labels.Labels var filterer chunk.Filterer @@ -348,7 +334,7 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim func (i *TSDBIndex) LabelVolume(ctx context.Context, _ string, from, through model.Time, acc LabelVolumeAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error { volumes := make(map[string]map[string]uint64) - err := i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + err := i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { var ls labels.Labels var filterer chunk.Filterer if i.chunkFilter != nil { From 317a8799c178890d8ce6847624df113d160fe9c6 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 14:57:31 -0300 Subject: [PATCH 11/84] Use right cache name. --- pkg/storage/chunk/cache/cache.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index c6c05e06130eb..49146e2a4f271 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -147,14 +147,14 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger, cacheType sta } if cfg.LRUCache.Enabled { - cache, err := NewLRUCache(cfg.Prefix+"embedded-cache", cfg.LRUCache, reg, logger, cacheType) + cache, err := NewLRUCache(cfg.Prefix+"inmemory-lru-cache", cfg.LRUCache, reg, logger, cacheType) if err != nil { level.Error(logger).Log("msg", "failed to initialize LRU cache", "err", err) return nil, err } if cache != nil { - caches = append(caches, CollectStats(Instrument(cfg.Prefix+"embedded-cache", cache, reg))) + caches = append(caches, CollectStats(Instrument(cfg.Prefix+"inmemory-lru-cache", cache, reg))) } } From 66c7f0d6c0a2ca7b1ad82a7fb051ee2400fdcc8f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 14:58:38 -0300 Subject: [PATCH 12/84] Rename flag and remove unused var. --- pkg/storage/chunk/cache/lru_cache.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 041bcb0d96122..2c66868a33eda 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -24,10 +24,6 @@ const ( codecHeaderSnappyWithMatchers codec = "dm" // As in "dvs+matchers" ) -var DefaultLRUCacheConfig = LRUCacheConfig{ - MaxSizeBytes: "250MB", -} - const maxInt = int(^uint(0) >> 1) const ( @@ -45,7 +41,7 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { - f.StringVar(&cfg.MaxSizeBytes, prefix+"fifocache.max-size-bytes", "500MB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") + f.StringVar(&cfg.MaxSizeBytes, prefix+"lrucache.max-size-bytes", "500MB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") } func (cfg *LRUCacheConfig) Validate() error { From 71cac4ec07306d27df254644954987258463c927 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 15:11:55 -0300 Subject: [PATCH 13/84] Make the metrics more consistent. --- pkg/storage/chunk/cache/lru_cache.go | 56 +++++++++++------------ pkg/storage/chunk/cache/lru_cache_test.go | 2 +- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 2c66868a33eda..d54e17d009cd9 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -68,7 +68,6 @@ type LRUCache struct { added *prometheus.CounterVec current *prometheus.GaugeVec bytesInUse prometheus.Gauge - overflow *prometheus.CounterVec } func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (*LRUCache, error) { @@ -86,7 +85,7 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log } c.totalMisses = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "querier", + Namespace: "loki", Subsystem: "cache", Name: "misses_total", Help: "The total number of Get calls that had no valid entry", @@ -94,7 +93,7 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log }) c.bytesInUse = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Namespace: "querier", + Namespace: "loki", Subsystem: "cache", Name: "memory_bytes", Help: "The current cache size in bytes", @@ -102,39 +101,43 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log }) c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_evicted_total", - Help: "Total number of items that were evicted from the index cache.", + Namespace: "loki", + Subsystem: "cache", + Name: "evicted_total", + Help: "Total number of items that were evicted.", + ConstLabels: prometheus.Labels{"cache": name}, }, []string{}) c.added = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_added_total", - Help: "Total number of items that were added to the index cache.", + Namespace: "loki", + Subsystem: "cache", + Name: "added_total", + Help: "Total number of items that were added to the cache.", + ConstLabels: prometheus.Labels{"cache": name}, }, []string{}) c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_requests_total", - Help: "Total number of requests to the cache.", - }, []string{}) - - c.overflow = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items_overflowed_total", - Help: "Total number of items that could not be added to the cache due to being too big.", + Namespace: "loki", + Subsystem: "cache", + Name: "gets_total", + Help: "Total number of requests to the cache.", + ConstLabels: prometheus.Labels{"cache": name}, }, []string{}) c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_hits_total", - Help: "Total number of requests to the cache that were a hit.", + Namespace: "loki", + Subsystem: "cache", + Name: "hits_total", + Help: "Total number of requests to the cache that were a hit.", + ConstLabels: prometheus.Labels{"cache": name}, }, []string{}) c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Namespace: "loki", - Name: "index_gateway_index_cache_items", - Help: "Current number of items in the index cache.", + Namespace: "loki", + Subsystem: "cache", + Name: "entries", + Help: "Current number of items in the cache.", + ConstLabels: prometheus.Labels{"cache": name}, }, []string{}) // Initialize LRU cache with a high size limit since we will manage evictions ourselves @@ -146,10 +149,7 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log c.lru = l level.Info(logger).Log( - "msg", "created in-memory index cache", - "maxItemSizeBytes", c.maxItemSizeBytes, - "maxSizeBytes", c.maxSizeBytes, - "maxItems", "maxInt", + "msg", "created in-memory LRU cache", ) return c, nil diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go index 9a0aa2f95e558..bf8030bd7da29 100644 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -3,7 +3,7 @@ // Provenance-includes-license: Apache-2.0 // Provenance-includes-copyright: The Thanos Authors. -// Tests out the LRU index cache implementation. +// Tests out the LRU cache implementation. package cache import ( From bf330fcf709e9780ff72a90d72b21fbc12c0ea8c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 15:14:38 -0300 Subject: [PATCH 14/84] Pass canonical keys directly. --- pkg/storage/stores/tsdb/cached_postings_index.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 388f80393ac9a..fde84150b0e1c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -50,7 +50,7 @@ func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labe return err } - if err := c.storePostings(p, matchers); err != nil { + if err := c.storePostings(p, key); err != nil { level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } return fn(p) @@ -114,13 +114,13 @@ func encodedMatchersLen(matchers []*labels.Matcher) int { return matchersLen } -func (c *cachedPostingsClient) storePostings(postings index.Postings, matchers []*labels.Matcher) error { +func (c *cachedPostingsClient) storePostings(postings index.Postings, canonicalMatchers string) error { dataToCache, err := diffVarintEncodeNoHeader(postings, 0) if err != nil { - level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", CanonicalLabelMatchersKey(matchers)) + level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) } - return c.cacheClient.Store(context.TODO(), []string{CanonicalLabelMatchersKey(matchers)}, [][]byte{dataToCache}) + return c.cacheClient.Store(context.Background(), []string{canonicalMatchers}, [][]byte{dataToCache}) } func (c *cachedPostingsClient) fetchPostings(key string) (index.Postings, bool) { From 4a4d1fdcefa07e7cee156dab9cee702509c4af8c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 15:15:39 -0300 Subject: [PATCH 15/84] Pass ctx directly. --- pkg/storage/stores/tsdb/cached_postings_index.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index fde84150b0e1c..3dee690fc8b80 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -50,7 +50,7 @@ func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labe return err } - if err := c.storePostings(p, key); err != nil { + if err := c.storePostings(ctx, p, key); err != nil { level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } return fn(p) @@ -114,13 +114,13 @@ func encodedMatchersLen(matchers []*labels.Matcher) int { return matchersLen } -func (c *cachedPostingsClient) storePostings(postings index.Postings, canonicalMatchers string) error { +func (c *cachedPostingsClient) storePostings(ctx context.Context, postings index.Postings, canonicalMatchers string) error { dataToCache, err := diffVarintEncodeNoHeader(postings, 0) if err != nil { level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) } - return c.cacheClient.Store(context.Background(), []string{canonicalMatchers}, [][]byte{dataToCache}) + return c.cacheClient.Store(ctx, []string{canonicalMatchers}, [][]byte{dataToCache}) } func (c *cachedPostingsClient) fetchPostings(key string) (index.Postings, bool) { From 88fe691cd2bba65c7f236a6f55994fba718ebc36 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 16:02:07 -0300 Subject: [PATCH 16/84] Reuse ctx --- pkg/storage/stores/tsdb/cached_postings_index.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 3dee690fc8b80..d474d2dc4e720 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -41,7 +41,7 @@ type cachedPostingsClient struct { func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) - if postings, got := c.fetchPostings(key); got { + if postings, got := c.fetchPostings(ctx, key); got { return fn(postings) } @@ -123,8 +123,8 @@ func (c *cachedPostingsClient) storePostings(ctx context.Context, postings index return c.cacheClient.Store(ctx, []string{canonicalMatchers}, [][]byte{dataToCache}) } -func (c *cachedPostingsClient) fetchPostings(key string) (index.Postings, bool) { - found, bufs, _, err := c.cacheClient.Fetch(context.TODO(), []string{key}) +func (c *cachedPostingsClient) fetchPostings(ctx context.Context, key string) (index.Postings, bool) { + found, bufs, _, err := c.cacheClient.Fetch(ctx, []string{key}) if err != nil { level.Error(c.log).Log("msg", "error on fetching postings", "err", err, "matchers", key) From 604de2f062b1a7f232d626912d23d6c0fb8b0330 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 15 Jun 2023 18:13:01 -0300 Subject: [PATCH 17/84] Append series numbers to encoded binary --- .../stores/tsdb/cached_postings_index.go | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index d474d2dc4e720..eea363ee1f0f3 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "encoding/binary" "sort" "strings" @@ -61,6 +62,7 @@ func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labe // Length argument is expected number of postings, used for preallocating buffer. func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { buf := encoding.Encbuf{} + buf.PutUvarint64(uint64(length)) // This encoding uses around ~1 bytes per posting, but let's use // conservative 1.25 bytes per posting to avoid extra allocations. @@ -69,6 +71,7 @@ func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { } prev := storage.SeriesRef(0) + var total uint64 = 0 for p.Next() { v := p.At() @@ -80,20 +83,26 @@ func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { // This is the 'diff' part -- compute difference from previous value. buf.PutUvarint64(uint64(v - prev)) prev = v + total += 1 } if p.Err() != nil { return nil, p.Err() } - return buf.B, nil + // add number of postings at the beginning of the buffer, used when decoding. + lenBuf := make([]byte, binary.MaxVarintLen64+len(buf.B)) + binary.PutUvarint(lenBuf, total) + + return append(lenBuf, buf.B...), nil } func decodeToPostings(b []byte) index.Postings { decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) - - refs := []storage.SeriesRef{} + postingsLen := storage.SeriesRef(decoder.Uvarint64()) + refs := make([]storage.SeriesRef, postingsLen) prev := storage.SeriesRef(0) - for i := 0; i < decoder.Len(); i++ { + + for i := 0; i < int(postingsLen); i++ { v := storage.SeriesRef(decoder.Uvarint64()) refs = append(refs, v+prev) prev = v From 4e784f358e635b06ebe9e9f66691626294cb75ab Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 19 Jun 2023 07:11:06 -0300 Subject: [PATCH 18/84] Rename to postingsReader --- pkg/storage/stores/tsdb/cached_postings_index.go | 4 ++-- pkg/storage/stores/tsdb/single_file_index.go | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index eea363ee1f0f3..592cb607e4890 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -18,13 +18,13 @@ import ( "github.com/grafana/loki/pkg/util/encoding" ) -type PostingsClient interface { +type PostingsReader interface { ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error } var sharedCacheClient cache.Cache -func NewCachedPostingsClient(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsClient { +func NewCachedPostingsClient(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsReader { return &cachedPostingsClient{ reader: reader, cacheClient: cacheClient, diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 62d0e17f2abfc..d9864eaf89a5b 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -120,7 +120,7 @@ func (f *TSDBFile) Reader() (io.ReadSeeker, error) { type TSDBIndex struct { reader IndexReader chunkFilter chunk.RequestChunkFilterer - postingsClient PostingsClient + postingsClient PostingsReader } // Return the index as well as the underlying raw file reader which isn't exposed as an index @@ -138,8 +138,8 @@ func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) }, nil } -func getPostingsClient(reader IndexReader) PostingsClient { - var postingsClient PostingsClient +func getPostingsClient(reader IndexReader) PostingsReader { + var postingsClient PostingsReader if shouldCachePostings && sharedCacheClient != nil { postingsClient = NewCachedPostingsClient(reader, util_log.Logger, sharedCacheClient) @@ -152,7 +152,7 @@ func getPostingsClient(reader IndexReader) PostingsClient { return postingsClient } -func DefaultPostingsClient(reader IndexReader) PostingsClient { +func DefaultPostingsClient(reader IndexReader) PostingsReader { return &simplePostingsClient{reader: reader} } @@ -168,7 +168,7 @@ func (s *simplePostingsClient) ForPostings(ctx context.Context, matchers []*labe return fn(p) } -func NewTSDBIndex(reader IndexReader, postingsClient PostingsClient) *TSDBIndex { +func NewTSDBIndex(reader IndexReader, postingsClient PostingsReader) *TSDBIndex { return &TSDBIndex{ reader: reader, postingsClient: postingsClient, From a7d48a8a108621db91c137e9b6651ec25be20ea1 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 19 Jun 2023 07:29:37 -0300 Subject: [PATCH 19/84] lint fix --- pkg/storage/chunk/cache/lru_cache.go | 4 ++-- pkg/storage/stores/tsdb/cached_postings_index.go | 4 ++-- pkg/storage/stores/tsdb/cached_postings_index_test.go | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index d54e17d009cd9..4cbb2c784cacd 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -156,7 +156,7 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log } // Fetch implements Cache. -func (c *LRUCache) Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { +func (c *LRUCache) Fetch(_ context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { found, missing, bufs = make([]string, 0, len(keys)), make([]string, 0, len(keys)), make([][]byte, 0, len(keys)) for _, key := range keys { val, ok := c.get(key) @@ -172,7 +172,7 @@ func (c *LRUCache) Fetch(ctx context.Context, keys []string) (found []string, bu } // Store implements Cache. -func (c *LRUCache) Store(ctx context.Context, keys []string, values [][]byte) error { +func (c *LRUCache) Store(_ context.Context, keys []string, values [][]byte) error { for i := range keys { c.set(keys[i], values[i]) } diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 592cb607e4890..769f0a1b0aba8 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -71,7 +71,7 @@ func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { } prev := storage.SeriesRef(0) - var total uint64 = 0 + var total uint64 for p.Next() { v := p.At() @@ -83,7 +83,7 @@ func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { // This is the 'diff' part -- compute difference from previous value. buf.PutUvarint64(uint64(v - prev)) prev = v - total += 1 + total++ } if p.Err() != nil { return nil, p.Err() diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 4049dedb7ca31..85b8a6885c989 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -422,6 +422,6 @@ func BenchmarkSeriesRepetitive(b *testing.B) { acc := &stats.Stats{} for i := 0; i < b.N; i++ { - tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck } } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index d9864eaf89a5b..3d1e5b6bf3486 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -160,7 +160,7 @@ type simplePostingsClient struct { reader IndexReader } -func (s *simplePostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { +func (s *simplePostingsClient) ForPostings(_ context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { p, err := PostingsForMatchers(s.reader, nil, matchers...) if err != nil { return err From bba1de5dbd784dd2d43862693f23dd5582fac95b Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 19 Jun 2023 16:14:35 -0300 Subject: [PATCH 20/84] Implement overflow logic --- pkg/storage/chunk/cache/lru_cache.go | 140 ++++++++++---- pkg/storage/chunk/cache/lru_cache_test.go | 221 +++++++++++----------- pkg/storage/stores/tsdb/index.go | 2 +- 3 files changed, 208 insertions(+), 155 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 4cbb2c784cacd..8e47e310fddb0 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -9,12 +9,14 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/grafana/loki/pkg/logqlmodel/stats" - util_log "github.com/grafana/loki/pkg/util/log" lru "github.com/hashicorp/golang-lru/simplelru" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/util/flagext" + util_log "github.com/grafana/loki/pkg/util/log" ) type codec string @@ -34,19 +36,27 @@ const ( var ulidSize = uint64(len(ulid.ULID{})) type LRUCacheConfig struct { - MaxSizeBytes string `yaml:"max_size_bytes"` + MaxSizeBytes flagext.ByteSize `yaml:"max_size_bytes"` + MaxItemSizeBytes flagext.ByteSize `yaml:"max_item_size_bytes"` + + MaxItems int `yaml:"max_items"` Enabled bool `yaml:"enabled"` } // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { - f.StringVar(&cfg.MaxSizeBytes, prefix+"lrucache.max-size-bytes", "500MB", description+"Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be applied.") + f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") + cfg.MaxItemSizeBytes.Set("5MB") + + f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") + cfg.MaxSizeBytes.Set("500MB") + + f.IntVar(&cfg.MaxItems, prefix+".max-items", 5000, description+"Maximum items in the cache.") } func (cfg *LRUCacheConfig) Validate() error { - _, err := parsebytes(cfg.MaxSizeBytes) - return err + return nil } type LRUCache struct { @@ -56,30 +66,35 @@ type LRUCache struct { mtx sync.Mutex - logger log.Logger - lru *lru.LRU - maxSizeBytes uint64 + logger log.Logger + lru *lru.LRU + + maxCacheBytes uint64 maxItemSizeBytes uint64 + maxItems int + curSize uint64 - evicted *prometheus.CounterVec - requests *prometheus.CounterVec - hits *prometheus.CounterVec + evicted prometheus.Counter + requests prometheus.Counter + hits prometheus.Counter totalMisses prometheus.Counter - added *prometheus.CounterVec - current *prometheus.GaugeVec + added prometheus.Counter + current prometheus.Gauge bytesInUse prometheus.Gauge + overflow prometheus.Counter } func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (*LRUCache, error) { util_log.WarnExperimentalUse(fmt.Sprintf("In-memory (LRU) cache - %s", name), logger) - maxSizeBytes, _ := parsebytes(cfg.MaxSizeBytes) - c := &LRUCache{ cacheType: cacheType, - maxSizeBytes: maxSizeBytes, - logger: logger, + maxItemSizeBytes: uint64(cfg.MaxItemSizeBytes), + maxCacheBytes: uint64(cfg.MaxSizeBytes), + maxItems: cfg.MaxItems, + + logger: logger, done: make(chan struct{}), } @@ -100,49 +115,57 @@ func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, log ConstLabels: prometheus.Labels{"cache": name}, }) - c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + c.evicted = promauto.With(reg).NewCounter(prometheus.CounterOpts{ Namespace: "loki", Subsystem: "cache", Name: "evicted_total", Help: "Total number of items that were evicted.", ConstLabels: prometheus.Labels{"cache": name}, - }, []string{}) + }) - c.added = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + c.added = promauto.With(reg).NewCounter(prometheus.CounterOpts{ Namespace: "loki", Subsystem: "cache", Name: "added_total", Help: "Total number of items that were added to the cache.", ConstLabels: prometheus.Labels{"cache": name}, - }, []string{}) + }) - c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + c.requests = promauto.With(reg).NewCounter(prometheus.CounterOpts{ Namespace: "loki", Subsystem: "cache", Name: "gets_total", Help: "Total number of requests to the cache.", ConstLabels: prometheus.Labels{"cache": name}, - }, []string{}) + }) - c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + c.hits = promauto.With(reg).NewCounter(prometheus.CounterOpts{ Namespace: "loki", Subsystem: "cache", Name: "hits_total", Help: "Total number of requests to the cache that were a hit.", ConstLabels: prometheus.Labels{"cache": name}, - }, []string{}) + }) - c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + c.current = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ Namespace: "loki", Subsystem: "cache", Name: "entries", Help: "Current number of items in the cache.", ConstLabels: prometheus.Labels{"cache": name}, - }, []string{}) + }) + + c.overflow = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Namespace: "loki", + Subsystem: "cache", + Name: "overflow", + Help: "Total number of items that could not be added to the cache due to being too big.", + ConstLabels: prometheus.Labels{"cache": name}, + }) // Initialize LRU cache with a high size limit since we will manage evictions ourselves // based on stored size using `RemoveOldest` method. - l, err := lru.NewLRU(maxInt, c.onEvict) + l, err := lru.NewLRU(c.maxItems, c.onEvict) if err != nil { return nil, err } @@ -195,13 +218,16 @@ func (c *LRUCache) GetCacheType() stats.CacheType { } func (c *LRUCache) onEvict(key, val interface{}) { - c.evicted.WithLabelValues().Inc() - c.current.WithLabelValues().Dec() - c.bytesInUse.Sub(float64(c.entryMemoryUsage(key.(string), val.([]byte)))) + c.evicted.Inc() + c.current.Dec() + + size := entryMemoryUsage(key.(string), val.([]byte)) + c.bytesInUse.Sub(float64(size)) + c.curSize -= uint64(size) } func (c *LRUCache) get(key string) ([]byte, bool) { - c.requests.WithLabelValues().Inc() + c.requests.Inc() c.mtx.Lock() defer c.mtx.Unlock() @@ -211,7 +237,7 @@ func (c *LRUCache) get(key string) ([]byte, bool) { c.totalMisses.Inc() return nil, false } - c.hits.WithLabelValues().Inc() + c.hits.Inc() return v.([]byte), true } @@ -223,23 +249,59 @@ func (c *LRUCache) set(key string, val []byte) { return } + if !c.ensureFits(key, val) { + c.overflow.Inc() + return + } + // The caller may be passing in a sub-slice of a huge array. Copy the data // to ensure we don't waste huge amounts of space for something small. v := make([]byte, len(val)) copy(v, val) c.lru.Add(key, v) - c.bytesInUse.Add(float64(c.entryMemoryUsage(key, val))) - c.added.WithLabelValues().Inc() - c.current.WithLabelValues().Inc() + size := entryMemoryUsage(key, val) + + c.bytesInUse.Add(float64(size)) + c.added.Inc() + c.current.Inc() + c.curSize += uint64(size) +} + +func (c *LRUCache) ensureFits(key string, val []byte) bool { + size := entryMemoryUsage(key, val) + if size > int(c.maxItemSizeBytes) { + level.Debug(c.logger).Log( + "msg", "item bigger than maxItemSizeBytes. Ignoring..", + "max_item_size_bytes", c.maxItemSizeBytes, + "max_cache_bytes", c.maxCacheBytes, + "cur_size", c.curSize, + "item_size", size, + ) + return false + } + + for c.curSize+uint64(size) > c.maxCacheBytes { + if _, _, ok := c.lru.RemoveOldest(); !ok { + level.Error(c.logger).Log( + "msg", "LRU has nothing more to evict, but we still cannot allocate the item. Resetting cache.", + "max_item_size_bytes", c.maxItemSizeBytes, + "max_size_bytes", c.maxCacheBytes, + "cur_size", c.curSize, + "item_size", size, + ) + c.reset() + } + } + return true } -func (c *LRUCache) entryMemoryUsage(key string, val []byte) int { +func entryMemoryUsage(key string, val []byte) int { return int(unsafe.Sizeof(val)) + len(key) } func (c *LRUCache) reset() { c.lru.Purge() - c.current.Reset() + c.current.Set(0) c.bytesInUse.Set(0) } diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go index bf8030bd7da29..a8a4d52b4d1a3 100644 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -9,10 +9,10 @@ package cache import ( "context" "fmt" - "strconv" "testing" "github.com/go-kit/log" + "github.com/grafana/loki/pkg/util/flagext" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -27,122 +27,113 @@ func TestLRUCacheEviction(t *testing.T) { key: "00", value: []byte("00"), } + entrySize := entryMemoryUsage(itemTemplate.key, itemTemplate.value) + + cfg := LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(entrySize * cnt), MaxItems: cnt, MaxItemSizeBytes: flagext.ByteSize(entrySize + 1), Enabled: true} + + c, err := NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(t, err) + defer c.Stop() + ctx := context.Background() + + // Check put / get works. Put/get 10 different items. + keys := []string{} + values := [][]byte{} + for i := 0; i < cnt; i++ { + key := fmt.Sprintf("%02d", i) + value := make([]byte, len(key)) + copy(value, key) + keys = append(keys, key) + values = append(values, value) + } + require.NoError(t, c.Store(ctx, keys, values)) + require.Equal(t, cnt, c.lru.Len()) + + assert.Equal(t, float64(10), testutil.ToFloat64(c.added), float64(10)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.evicted), float64(0)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.requests), float64(0)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses), float64(0)) + assert.Equal(t, float64(cnt*entryMemoryUsage(itemTemplate.key, itemTemplate.value)), testutil.ToFloat64(c.bytesInUse)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.overflow)) + + for i := 0; i < cnt; i++ { + key := fmt.Sprintf("%02d", i) + value, ok := c.get(key) + require.True(t, ok) + require.Equal(t, []byte(key), value) + } - tests := []struct { - name string - cfg LRUCacheConfig - }{ - { - name: "test-memory-eviction", - cfg: LRUCacheConfig{MaxSizeBytes: strconv.FormatInt(int64(cnt*sizeOf(itemTemplate)), 10)}, - }, + assert.Equal(t, float64(10), testutil.ToFloat64(c.added)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.evicted)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.requests)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses)) + assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) + + // Check evictions + keys = []string{} + values = [][]byte{} + for i := cnt - evicted; i < cnt+evicted; i++ { + key := fmt.Sprintf("%02d", i) + value := make([]byte, len(key)) + copy(value, key) + keys = append(keys, key) + values = append(values, value) + } + require.NoError(t, c.Store(ctx, keys, values)) + require.NoError(t, err) + require.Equal(t, cnt, c.lru.Len()) + + assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.requests)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses)) + assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) + + for i := 0; i < cnt-evicted; i++ { + _, ok := c.get(fmt.Sprintf("%02d", i)) + require.False(t, ok) + } + for i := cnt - evicted; i < cnt+evicted; i++ { + key := fmt.Sprintf("%02d", i) + value, ok := c.get(key) + require.True(t, ok) + require.Equal(t, []byte(key), value) } - for _, test := range tests { - c, err := NewLRUCache(test.name, test.cfg, nil, log.NewNopLogger(), "test") - require.NoError(t, err) - ctx := context.Background() - - // Check put / get works - keys := []string{} - values := [][]byte{} - for i := 0; i < cnt; i++ { - key := fmt.Sprintf("%02d", i) - value := make([]byte, len(key)) - copy(value, key) - keys = append(keys, key) - values = append(values, value) - } - require.NoError(t, c.Store(ctx, keys, values)) - require.Len(t, c.lru.Len(), cnt) - - assert.Equal(t, testutil.ToFloat64(c.added), float64(1)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) - - for i := 0; i < cnt; i++ { - key := fmt.Sprintf("%02d", i) - value, ok := c.get(key) - require.True(t, ok) - require.Equal(t, []byte(key), value) - } - - assert.Equal(t, testutil.ToFloat64(c.added), float64(1)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) - - // Check evictions - keys = []string{} - values = [][]byte{} - for i := cnt - evicted; i < cnt+evicted; i++ { - key := fmt.Sprintf("%02d", i) - value := make([]byte, len(key)) - copy(value, key) - keys = append(keys, key) - values = append(values, value) - } - err = c.Store(ctx, keys, values) - require.NoError(t, err) - require.Len(t, c.lru.Len(), cnt) - - assert.Equal(t, testutil.ToFloat64(c.added), float64(2)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) - - for i := 0; i < cnt-evicted; i++ { - _, ok := c.get(fmt.Sprintf("%02d", i)) - require.False(t, ok) - } - for i := cnt - evicted; i < cnt+evicted; i++ { - key := fmt.Sprintf("%02d", i) - value, ok := c.get(key) - require.True(t, ok) - require.Equal(t, []byte(key), value) - } - - assert.Equal(t, testutil.ToFloat64(c.added), float64(2)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) - - // Check updates work - keys = []string{} - values = [][]byte{} - for i := cnt; i < cnt+evicted; i++ { - keys = append(keys, fmt.Sprintf("%02d", i)) - vstr := fmt.Sprintf("%02d", i*2) - value := make([]byte, len(vstr)) - copy(value, vstr) - values = append(values, value) - } - err = c.Store(ctx, keys, values) - require.NoError(t, err) - require.Len(t, c.lru.Len(), cnt) - - for i := cnt; i < cnt+evicted; i++ { - value, ok := c.get(fmt.Sprintf("%02d", i)) - require.True(t, ok) - require.Equal(t, []byte(fmt.Sprintf("%02d", i*2)), value) - } - - assert.Equal(t, testutil.ToFloat64(c.added), float64(3)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted*2)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*sizeOf(itemTemplate))) - - c.Stop() + assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) + assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) + assert.Equal(t, float64(cnt*2+evicted), testutil.ToFloat64(c.requests)) + assert.Equal(t, float64(cnt-evicted), testutil.ToFloat64(c.totalMisses)) + assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) + + // Check updates work + keys = []string{} + values = [][]byte{} + for i := cnt; i < cnt+evicted; i++ { + keys = append(keys, fmt.Sprintf("%02d", i)) + vstr := fmt.Sprintf("%02d", i*2) + value := make([]byte, len(vstr)) + copy(value, vstr) + values = append(values, value) + } + require.NoError(t, c.Store(ctx, keys, values)) + require.Equal(t, cnt, c.lru.Len()) + + assert.Equal(t, testutil.ToFloat64(c.added), float64(15)) + assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted*2)) + assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) + assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*entrySize)) + + for i := cnt; i < cnt+evicted; i++ { + value, ok := c.get(fmt.Sprintf("%02d", i)) + require.True(t, ok) + require.Equal(t, []byte(fmt.Sprintf("%02d", i*2)), value) } + } diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 9881fc614effd..8fda8a987832e 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -24,7 +24,7 @@ type ChunkRef struct { } type IndexCfg struct { - indexshipper.Config + indexshipper.Config `yaml:",inline"` CachePostings bool `yaml:"cache_postings"` } From 03d0cc85e296687b37c492a74d19b3c0e59be7c5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 19 Jun 2023 17:25:42 -0300 Subject: [PATCH 21/84] Fix tests --- pkg/storage/chunk/cache/lru_cache.go | 24 ++++++++++++++++++----- pkg/storage/chunk/cache/lru_cache_test.go | 10 +++++----- 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 8e47e310fddb0..3e5f5fe6a2a41 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -245,8 +245,11 @@ func (c *LRUCache) set(key string, val []byte) { c.mtx.Lock() defer c.mtx.Unlock() - if _, ok := c.lru.Get(key); ok { - return + wasUpdate := false + var oldValue []byte + if v, ok := c.lru.Get(key); ok { + wasUpdate = true + oldValue = v.([]byte) } if !c.ensureFits(key, val) { @@ -261,11 +264,19 @@ func (c *LRUCache) set(key string, val []byte) { c.lru.Add(key, v) size := entryMemoryUsage(key, val) - c.bytesInUse.Add(float64(size)) - c.added.Inc() - c.current.Inc() c.curSize += uint64(size) + + if wasUpdate { + // it was an update - discount previous value. + previousSize := entryMemoryUsage(key, oldValue) + c.bytesInUse.Add(float64(-previousSize)) + c.curSize -= uint64(previousSize) + return + } + + c.current.Inc() + c.added.Inc() } func (c *LRUCache) ensureFits(key string, val []byte) bool { @@ -297,6 +308,9 @@ func (c *LRUCache) ensureFits(key string, val []byte) bool { } func entryMemoryUsage(key string, val []byte) int { + if len(val) == 0 { + return 0 + } return int(unsafe.Sizeof(val)) + len(key) } diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go index a8a4d52b4d1a3..43df787b5de3f 100644 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -82,11 +82,10 @@ func TestLRUCacheEviction(t *testing.T) { values = append(values, value) } require.NoError(t, c.Store(ctx, keys, values)) - require.NoError(t, err) require.Equal(t, cnt, c.lru.Len()) assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) + assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) assert.Equal(t, float64(cnt), testutil.ToFloat64(c.requests)) assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses)) @@ -121,14 +120,15 @@ func TestLRUCacheEviction(t *testing.T) { values = append(values, value) } require.NoError(t, c.Store(ctx, keys, values)) - require.Equal(t, cnt, c.lru.Len()) + assert.Equal(t, cnt, c.lru.Len()) + assert.Equal(t, testutil.ToFloat64(c.overflow), float64(0)) assert.Equal(t, testutil.ToFloat64(c.added), float64(15)) assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted*2)) + assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted)) assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) - assert.Equal(t, testutil.ToFloat64(c.bytesInUse), float64(cnt*entrySize)) + assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) for i := cnt; i < cnt+evicted; i++ { value, ok := c.get(fmt.Sprintf("%02d", i)) From a98cfea2af8099cfd98e3769ae7dcd4b824858c5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 19 Jun 2023 18:08:56 -0300 Subject: [PATCH 22/84] Finish fixing tests. --- pkg/storage/chunk/cache/lru_cache.go | 13 ++++++------- pkg/storage/chunk/cache/lru_cache_test.go | 13 +++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 3e5f5fe6a2a41..c2724e099e089 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -246,10 +246,11 @@ func (c *LRUCache) set(key string, val []byte) { defer c.mtx.Unlock() wasUpdate := false - var oldValue []byte + previousSize := 0 if v, ok := c.lru.Get(key); ok { wasUpdate = true - oldValue = v.([]byte) + previousSize = entryMemoryUsage(key, v.([]byte)) + c.curSize -= uint64(previousSize) } if !c.ensureFits(key, val) { @@ -264,14 +265,12 @@ func (c *LRUCache) set(key string, val []byte) { c.lru.Add(key, v) size := entryMemoryUsage(key, val) - c.bytesInUse.Add(float64(size)) + + c.bytesInUse.Add(float64(size - previousSize)) c.curSize += uint64(size) if wasUpdate { - // it was an update - discount previous value. - previousSize := entryMemoryUsage(key, oldValue) - c.bytesInUse.Add(float64(-previousSize)) - c.curSize -= uint64(previousSize) + // it was an update - don't update other metrics. return } diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go index 43df787b5de3f..7e76d2e287fc8 100644 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -119,15 +119,16 @@ func TestLRUCacheEviction(t *testing.T) { copy(value, vstr) values = append(values, value) } + assert.Equal(t, cnt, c.lru.Len()) require.NoError(t, c.Store(ctx, keys, values)) assert.Equal(t, cnt, c.lru.Len()) - assert.Equal(t, testutil.ToFloat64(c.overflow), float64(0)) - assert.Equal(t, testutil.ToFloat64(c.added), float64(15)) - assert.Equal(t, testutil.ToFloat64(c.evicted), float64(evicted)) - assert.Equal(t, testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, testutil.ToFloat64(c.requests), float64(cnt*2+evicted)) - assert.Equal(t, testutil.ToFloat64(c.totalMisses), float64(cnt-evicted)) + assert.Equal(t, float64(0), testutil.ToFloat64(c.overflow)) + assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) + assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) + assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) + assert.Equal(t, float64(cnt*2+evicted), testutil.ToFloat64(c.requests)) + assert.Equal(t, float64(cnt-evicted), testutil.ToFloat64(c.totalMisses)) assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) for i := cnt; i < cnt+evicted; i++ { From 7f4e582cb94ddc3ad5f738cd27f06d89dd8594b9 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 20 Jun 2023 15:09:41 -0300 Subject: [PATCH 23/84] Fix lint --- pkg/storage/chunk/cache/lru_cache.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index c2724e099e089..874a18eefa672 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -47,10 +47,10 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") - cfg.MaxItemSizeBytes.Set("5MB") + cfg.MaxItemSizeBytes.Set("5MB") //nolint:errcheck f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") - cfg.MaxSizeBytes.Set("500MB") + cfg.MaxSizeBytes.Set("500MB") //nolint:errcheck f.IntVar(&cfg.MaxItems, prefix+".max-items", 5000, description+"Maximum items in the cache.") } From 8c71ebf612459bd7ec3c3c66c96dac94868bda0b Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 20 Jun 2023 15:14:19 -0300 Subject: [PATCH 24/84] Fix lint --- docs/sources/configuration/_index.md | 21 +++++++++++++++++++++ pkg/storage/chunk/cache/cache.go | 1 + pkg/storage/chunk/cache/lru_cache.go | 4 ++-- 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/docs/sources/configuration/_index.md b/docs/sources/configuration/_index.md index 05a708e927a40..dc13f149ba3c1 100644 --- a/docs/sources/configuration/_index.md +++ b/docs/sources/configuration/_index.md @@ -2025,6 +2025,8 @@ tsdb_shipper: [mode: | default = ""] [ingesterdbretainperiod: ] + + [cache_postings: ] ``` ### chunk_store_config @@ -3918,6 +3920,25 @@ embedded_cache: # CLI flag: -.embedded-cache.ttl [ttl: | default = 1h] +lru_cache: + # Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be + # applied. + # CLI flag: -.lrucache.max-size-bytes + [max_size_bytes: | default = 0B] + + # Maximum memory size of a single item in the cache. A unit suffix (KB, MB, + # GB) may be applied. + # CLI flag: -.lrucache.max-item-size-bytes + [max_item_size_bytes: | default = 0B] + + # Maximum items in the cache. + # CLI flag: -.lrucache.max-items + [max_items: | default = 5000] + + # Whether LRUCache should be enabled or not. + # CLI flag: -.lrucache.enabled + [enabled: | default = false] + fifocache: # Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be # applied. diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index 49146e2a4f271..4ee4db73d78e2 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -57,6 +57,7 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, description string, f cfg.MemcacheClient.RegisterFlagsWithPrefix(prefix, description, f) cfg.Redis.RegisterFlagsWithPrefix(prefix, description, f) cfg.Fifocache.RegisterFlagsWithPrefix(prefix, description, f) + cfg.LRUCache.RegisterFlagsWithPrefix(prefix+"lrucache", description, f) cfg.EmbeddedCache.RegisterFlagsWithPrefix(prefix, description, f) f.IntVar(&cfg.AsyncCacheWriteBackConcurrency, prefix+"max-async-cache-write-back-concurrency", 16, "The maximum number of concurrent asynchronous writeback cache can occur.") f.IntVar(&cfg.AsyncCacheWriteBackBufferSize, prefix+"max-async-cache-write-back-buffer-size", 500, "The maximum number of enqueued asynchronous writeback cache allowed.") diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 874a18eefa672..6bb31f476f1a7 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -47,12 +47,12 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") - cfg.MaxItemSizeBytes.Set("5MB") //nolint:errcheck f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") - cfg.MaxSizeBytes.Set("500MB") //nolint:errcheck f.IntVar(&cfg.MaxItems, prefix+".max-items", 5000, description+"Maximum items in the cache.") + + f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether LRUCache should be enabled or not.") } func (cfg *LRUCacheConfig) Validate() error { From 64ad6ca8a741557dfe82a838a6a8d4bd1dedc57f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 20 Jun 2023 15:19:27 -0300 Subject: [PATCH 25/84] Rename client->reader --- .../stores/tsdb/cached_postings_index.go | 12 +++---- .../stores/tsdb/cached_postings_index_test.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 34 +++++++++---------- .../stores/tsdb/single_file_index_test.go | 2 +- tools/tsdb/tsdb-map/main_test.go | 4 +-- 6 files changed, 28 insertions(+), 28 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 769f0a1b0aba8..dc5c3baf6dc67 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -24,15 +24,15 @@ type PostingsReader interface { var sharedCacheClient cache.Cache -func NewCachedPostingsClient(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsReader { - return &cachedPostingsClient{ +func NewCachedPostingsReader(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsReader { + return &cachedPostingsReader{ reader: reader, cacheClient: cacheClient, log: logger, } } -type cachedPostingsClient struct { +type cachedPostingsReader struct { reader IndexReader cacheClient cache.Cache @@ -40,7 +40,7 @@ type cachedPostingsClient struct { log log.Logger } -func (c *cachedPostingsClient) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { +func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) if postings, got := c.fetchPostings(ctx, key); got { return fn(postings) @@ -123,7 +123,7 @@ func encodedMatchersLen(matchers []*labels.Matcher) int { return matchersLen } -func (c *cachedPostingsClient) storePostings(ctx context.Context, postings index.Postings, canonicalMatchers string) error { +func (c *cachedPostingsReader) storePostings(ctx context.Context, postings index.Postings, canonicalMatchers string) error { dataToCache, err := diffVarintEncodeNoHeader(postings, 0) if err != nil { level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) @@ -132,7 +132,7 @@ func (c *cachedPostingsClient) storePostings(ctx context.Context, postings index return c.cacheClient.Store(ctx, []string{canonicalMatchers}, [][]byte{dataToCache}) } -func (c *cachedPostingsClient) fetchPostings(ctx context.Context, key string) (index.Postings, bool) { +func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (index.Postings, bool) { found, bufs, _, err := c.cacheClient.Fetch(ctx, []string{key}) if err != nil { diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 85b8a6885c989..bc22ce223378c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -78,7 +78,7 @@ func TestSingleIdxCached(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader, DefaultPostingsClient(reader)) + return NewTSDBIndex(reader, DefaultPostingsReader(reader)) }, }, } { diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index c725cee4e5d48..135fb573f0f9a 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -731,7 +731,7 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } reader := tenant.indexRange(int64(from), int64(through)) - idx = NewTSDBIndex(reader, &simplePostingsClient{reader: reader}) + idx = NewTSDBIndex(reader, &simplePostingsReader{reader: reader}) if t.chunkFilter != nil { idx.SetChunkFilterer(t.chunkFilter) } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 3d1e5b6bf3486..0f2da1a7698f8 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -120,7 +120,7 @@ func (f *TSDBFile) Reader() (io.ReadSeeker, error) { type TSDBIndex struct { reader IndexReader chunkFilter chunk.RequestChunkFilterer - postingsClient PostingsReader + postingsReader PostingsReader } // Return the index as well as the underlying raw file reader which isn't exposed as an index @@ -131,36 +131,36 @@ func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) return nil, nil, err } - tsdbIdx := NewTSDBIndex(reader, getPostingsClient(reader)) + tsdbIdx := NewTSDBIndex(reader, getPostingsReader(reader)) return tsdbIdx, func() (io.ReadSeeker, error) { return reader.RawFileReader() }, nil } -func getPostingsClient(reader IndexReader) PostingsReader { - var postingsClient PostingsReader +func getPostingsReader(reader IndexReader) PostingsReader { + var postingsReader PostingsReader if shouldCachePostings && sharedCacheClient != nil { - postingsClient = NewCachedPostingsClient(reader, util_log.Logger, sharedCacheClient) + postingsReader = NewCachedPostingsReader(reader, util_log.Logger, sharedCacheClient) } - if postingsClient == nil { - postingsClient = DefaultPostingsClient(reader) + if postingsReader == nil { + postingsReader = DefaultPostingsReader(reader) } - return postingsClient + return postingsReader } -func DefaultPostingsClient(reader IndexReader) PostingsReader { - return &simplePostingsClient{reader: reader} +func DefaultPostingsReader(reader IndexReader) PostingsReader { + return &simplePostingsReader{reader: reader} } -type simplePostingsClient struct { +type simplePostingsReader struct { reader IndexReader } -func (s *simplePostingsClient) ForPostings(_ context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { +func (s *simplePostingsReader) ForPostings(_ context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { p, err := PostingsForMatchers(s.reader, nil, matchers...) if err != nil { return err @@ -168,10 +168,10 @@ func (s *simplePostingsClient) ForPostings(_ context.Context, matchers []*labels return fn(p) } -func NewTSDBIndex(reader IndexReader, postingsClient PostingsReader) *TSDBIndex { +func NewTSDBIndex(reader IndexReader, postingsReader PostingsReader) *TSDBIndex { return &TSDBIndex{ reader: reader, - postingsClient: postingsClient, + postingsReader: postingsReader, } } @@ -202,7 +202,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, filterer = i.chunkFilter.ForRequest(ctx) } - return i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { + return i.postingsReader.ForPostings(ctx, matchers, func(p index.Postings) error { for p.Next() { hash, err := i.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) if err != nil { @@ -300,7 +300,7 @@ func (i *TSDBIndex) Identifier(string) SingleTenantTSDBIdentifier { } func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error { - return i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { + return i.postingsReader.ForPostings(ctx, matchers, func(p index.Postings) error { // TODO(owen-d): use pool var ls labels.Labels var filterer chunk.Filterer @@ -369,7 +369,7 @@ func (i *TSDBIndex) SeriesVolume(ctx context.Context, _ string, from, through mo seriesLabels := labels.Labels(make([]labels.Label, 0, len(labelsToMatch))) volumes := make(map[string]uint64) - err := i.postingsClient.ForPostings(ctx, matchers, func(p index.Postings) error { + err := i.postingsReader.ForPostings(ctx, matchers, func(p index.Postings) error { var ls labels.Labels var filterer chunk.Filterer if i.chunkFilter != nil { diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index f9c5988f378df..d0628f71ec6db 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -82,7 +82,7 @@ func TestSingleIdx(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader, DefaultPostingsClient(reader)) + return NewTSDBIndex(reader, DefaultPostingsReader(reader)) }, }, } { diff --git a/tools/tsdb/tsdb-map/main_test.go b/tools/tsdb/tsdb-map/main_test.go index ee12cf7201abb..b87864d35c1e9 100644 --- a/tools/tsdb/tsdb-map/main_test.go +++ b/tools/tsdb/tsdb-map/main_test.go @@ -93,7 +93,7 @@ func BenchmarkQuery_GetChunkRefs(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsClient(reader)) + idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsReader(reader)) b.Run(bm.name, func(b *testing.B) { refs := tsdb.ChunkRefsPool.Get() for i := 0; i < b.N; i++ { @@ -118,7 +118,7 @@ func BenchmarkQuery_GetChunkRefsSharded(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsClient(reader)) + idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsReader(reader)) shardFactor := 16 b.Run(bm.name, func(b *testing.B) { From e9ef21a9c76aa70fbac66bfba4a20582acbd6b7f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 20 Jun 2023 17:33:30 -0300 Subject: [PATCH 26/84] Update calls (haudi suggestions) --- .../stores/tsdb/cached_postings_index_test.go | 2 +- pkg/storage/stores/tsdb/compactor.go | 9 +++-- pkg/storage/stores/tsdb/manager.go | 4 +- pkg/storage/stores/tsdb/single_file_index.go | 40 ++++++++++--------- .../stores/tsdb/single_file_index_test.go | 2 +- pkg/storage/stores/tsdb/store.go | 8 ++-- pkg/storage/stores/tsdb/util_test.go | 2 +- tools/tsdb/index-analyzer/main.go | 7 +++- tools/tsdb/migrate-versions/main_test.go | 2 +- tools/tsdb/tsdb-map/main_test.go | 4 +- 10 files changed, 46 insertions(+), 34 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index bc22ce223378c..3bc2d97c8fed9 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -78,7 +78,7 @@ func TestSingleIdxCached(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader, DefaultPostingsReader(reader)) + return NewTSDBIndex(reader, NewPostingsReader(reader)) }, }, } { diff --git a/pkg/storage/stores/tsdb/compactor.go b/pkg/storage/stores/tsdb/compactor.go index 928252e0255ca..0d8e0a930a1d2 100644 --- a/pkg/storage/stores/tsdb/compactor.go +++ b/pkg/storage/stores/tsdb/compactor.go @@ -36,7 +36,8 @@ func (i indexProcessor) NewTableCompactor(ctx context.Context, commonIndexSet co } func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableName, userID, workingDir string, periodConfig config.PeriodConfig, logger log.Logger) (compactor.CompactedIndex, error) { - indexFile, err := OpenShippableTSDB(path) + opts := TSDBIndexOpts{UsePostingsCache: false} + indexFile, err := OpenShippableTSDB(path, opts) if err != nil { return nil, err } @@ -100,7 +101,7 @@ func (t *tableCompactor) CompactTable() error { } downloadPaths[job] = downloadedAt - idx, err := OpenShippableTSDB(downloadedAt) + idx, err := OpenShippableTSDB(downloadedAt, TSDBIndexOpts{UsePostingsCache: false}) if err != nil { return err } @@ -218,7 +219,7 @@ func setupBuilder(ctx context.Context, userID string, sourceIndexSet compactor.I } }() - indexFile, err := OpenShippableTSDB(path) + indexFile, err := OpenShippableTSDB(path, TSDBIndexOpts{UsePostingsCache: false}) if err != nil { return nil, err } @@ -387,7 +388,7 @@ func (c *compactedIndex) ToIndexFile() (index_shipper.Index, error) { return nil, err } - return NewShippableTSDBFile(id) + return NewShippableTSDBFile(id, TSDBIndexOpts{UsePostingsCache: false}) } func getUnsafeBytes(s string) []byte { diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index c5b970e46d05b..7c685077eda1b 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -129,7 +129,7 @@ func (m *tsdbManager) Start() (err error) { indices++ prefixed := NewPrefixedIdentifier(id, filepath.Join(mulitenantDir, bucket), "") - loaded, err := NewShippableTSDBFile(prefixed) + loaded, err := NewShippableTSDBFile(prefixed, TSDBIndexOpts{UsePostingsCache: false}) if err != nil { level.Warn(m.log).Log( @@ -221,7 +221,7 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, shipper indexshipper.Ind level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dst.Path(), "duration", time.Since(start)) - loaded, err := NewShippableTSDBFile(dst) + loaded, err := NewShippableTSDBFile(dst, TSDBIndexOpts{UsePostingsCache: false}) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 0f2da1a7698f8..814685e6a6620 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -20,24 +20,27 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" ) -var shouldCachePostings = false - var ErrAlreadyOnDesiredVersion = errors.New("tsdb file already on desired version") // GetRawFileReaderFunc returns an io.ReadSeeker for reading raw tsdb file from disk type GetRawFileReaderFunc func() (io.ReadSeeker, error) -func OpenShippableTSDB(p string) (index_shipper.Index, error) { +type TSDBIndexOpts struct { + UsePostingsCache bool +} + +func OpenShippableTSDB(p string, opts TSDBIndexOpts) (index_shipper.Index, error) { id, err := identifierFromPath(p) if err != nil { return nil, err } - return NewShippableTSDBFile(id) + return NewShippableTSDBFile(id, opts) } func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index_shipper.Index, error) { - indexFile, err := OpenShippableTSDB(path) + opts := TSDBIndexOpts{UsePostingsCache: false} + indexFile, err := OpenShippableTSDB(path, opts) if err != nil { return nil, err } @@ -76,7 +79,7 @@ func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index if err != nil { return nil, err } - return NewShippableTSDBFile(id) + return NewShippableTSDBFile(id, TSDBIndexOpts{UsePostingsCache: false}) } // nolint @@ -92,8 +95,8 @@ type TSDBFile struct { getRawFileReader GetRawFileReaderFunc } -func NewShippableTSDBFile(id Identifier) (*TSDBFile, error) { - idx, getRawFileReader, err := NewTSDBIndexFromFile(id.Path()) +func NewShippableTSDBFile(id Identifier, opts TSDBIndexOpts) (*TSDBFile, error) { + idx, getRawFileReader, err := NewTSDBIndexFromFile(id.Path(), opts) if err != nil { return nil, err } @@ -125,34 +128,35 @@ type TSDBIndex struct { // Return the index as well as the underlying raw file reader which isn't exposed as an index // method but is helpful for building an io.reader for the index shipper -func NewTSDBIndexFromFile(location string) (Index, GetRawFileReaderFunc, error) { +func NewTSDBIndexFromFile(location string, opts TSDBIndexOpts) (Index, GetRawFileReaderFunc, error) { reader, err := index.NewFileReader(location) if err != nil { return nil, nil, err } - tsdbIdx := NewTSDBIndex(reader, getPostingsReader(reader)) + postingsReader := getPostingsReader(reader, opts.UsePostingsCache) + tsdbIdx := NewTSDBIndex(reader, postingsReader) return tsdbIdx, func() (io.ReadSeeker, error) { return reader.RawFileReader() }, nil } -func getPostingsReader(reader IndexReader) PostingsReader { - var postingsReader PostingsReader +func getPostingsReader(reader IndexReader, usePostingsCache bool) PostingsReader { + var pr PostingsReader - if shouldCachePostings && sharedCacheClient != nil { - postingsReader = NewCachedPostingsReader(reader, util_log.Logger, sharedCacheClient) + if usePostingsCache && sharedCacheClient != nil { + pr = NewCachedPostingsReader(reader, util_log.Logger, sharedCacheClient) } - if postingsReader == nil { - postingsReader = DefaultPostingsReader(reader) + if pr == nil { + pr = NewPostingsReader(reader) } - return postingsReader + return pr } -func DefaultPostingsReader(reader IndexReader) PostingsReader { +func NewPostingsReader(reader IndexReader) PostingsReader { return &simplePostingsReader{reader: reader} } diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index d0628f71ec6db..ed8b4ae26c065 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -82,7 +82,7 @@ func TestSingleIdx(t *testing.T) { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) } reader := head.Index() - return NewTSDBIndex(reader, DefaultPostingsReader(reader)) + return NewTSDBIndex(reader, NewPostingsReader(reader)) }, }, } { diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index d848216c91ece..03c84db128903 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores/index" "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/indexshipper/downloads" + indexshipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" tsdb_index "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) @@ -76,8 +77,9 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf sharedCacheClient = idxCache - if indexCfg.CachePostings { - shouldCachePostings = true + usePostingsCache := indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil + openFn := func(p string) (indexshipper_index.Index, error) { + return OpenShippableTSDB(p, TSDBIndexOpts{UsePostingsCache: usePostingsCache}) } var err error @@ -86,7 +88,7 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf objectClient, limits, nil, - OpenShippableTSDB, + openFn, tableRange, prometheus.WrapRegistererWithPrefix("loki_tsdb_shipper_", reg), s.logger, diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index f3223f161b15d..198462477ac51 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -35,7 +35,7 @@ func BuildIndex(t testing.TB, dir string, cases []LoadableSeries) *TSDBFile { }) require.Nil(t, err) - idx, err := NewShippableTSDBFile(dst) + idx, err := NewShippableTSDBFile(dst, TSDBIndexOpts{UsePostingsCache: false}) require.Nil(t, err) return idx } diff --git a/tools/tsdb/index-analyzer/main.go b/tools/tsdb/index-analyzer/main.go index dbfa61abca960..f8369db6ce93c 100644 --- a/tools/tsdb/index-analyzer/main.go +++ b/tools/tsdb/index-analyzer/main.go @@ -13,6 +13,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/indexshipper" + indexshipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/tsdb" "github.com/grafana/loki/pkg/util/cfg" util_log "github.com/grafana/loki/pkg/util/log" @@ -33,12 +34,16 @@ func main() { tableRanges := getIndexStoreTableRanges(config.TSDBType, conf.SchemaConfig.Configs) + openFn := func(p string) (indexshipper_index.Index, error) { + return tsdb.OpenShippableTSDB(p, tsdb.TSDBIndexOpts{UsePostingsCache: false}) + } + shipper, err := indexshipper.NewIndexShipper( conf.StorageConfig.TSDBShipperConfig.Config, objectClient, overrides, nil, - tsdb.OpenShippableTSDB, + openFn, tableRanges[len(tableRanges)-1], prometheus.WrapRegistererWithPrefix("loki_tsdb_shipper_", prometheus.DefaultRegisterer), util_log.Logger, diff --git a/tools/tsdb/migrate-versions/main_test.go b/tools/tsdb/migrate-versions/main_test.go index 3b2d1fc33a738..872d6ee477a6e 100644 --- a/tools/tsdb/migrate-versions/main_test.go +++ b/tools/tsdb/migrate-versions/main_test.go @@ -88,7 +88,7 @@ func TestMigrateTables(t *testing.T) { require.NoError(t, err) tableName := fmt.Sprintf("%s%d", indexPrefix, i) - idx, err := tsdb.NewShippableTSDBFile(id) + idx, err := tsdb.NewShippableTSDBFile(id, tsdb.TSDBIndexOpts{UsePostingsCache: false}) require.NoError(t, err) require.NoError(t, uploadFile(idx, indexStorageClient, tableName, userID)) diff --git a/tools/tsdb/tsdb-map/main_test.go b/tools/tsdb/tsdb-map/main_test.go index b87864d35c1e9..4d49085290d32 100644 --- a/tools/tsdb/tsdb-map/main_test.go +++ b/tools/tsdb/tsdb-map/main_test.go @@ -93,7 +93,7 @@ func BenchmarkQuery_GetChunkRefs(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsReader(reader)) + idx := tsdb.NewTSDBIndex(reader, tsdb.NewPostingsReader(reader)) b.Run(bm.name, func(b *testing.B) { refs := tsdb.ChunkRefsPool.Get() for i := 0; i < b.N; i++ { @@ -118,7 +118,7 @@ func BenchmarkQuery_GetChunkRefsSharded(b *testing.B) { if err != nil { panic(err) } - idx := tsdb.NewTSDBIndex(reader, tsdb.DefaultPostingsReader(reader)) + idx := tsdb.NewTSDBIndex(reader, tsdb.NewPostingsReader(reader)) shardFactor := 16 b.Run(bm.name, func(b *testing.B) { From 84f68df397592161c451ed098ca52e326bb3ae53 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 20 Jun 2023 17:53:59 -0300 Subject: [PATCH 27/84] Fix tests --- .../stores/tsdb/cached_postings_index_test.go | 12 ++++-------- pkg/storage/stores/tsdb/index_client_test.go | 12 ++++++------ pkg/storage/stores/tsdb/multi_file_index_test.go | 2 +- pkg/storage/stores/tsdb/single_file_index_test.go | 8 ++++---- pkg/storage/stores/tsdb/util_test.go | 4 ++-- 5 files changed, 17 insertions(+), 21 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 3bc2d97c8fed9..d881d7ae2bf1b 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -16,7 +16,6 @@ import ( ) func TestSingleIdxCached(t *testing.T) { - shouldCachePostings = true cases := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar"}`), @@ -67,7 +66,7 @@ func TestSingleIdxCached(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases) + return BuildIndex(t, t.TempDir(), cases, TSDBIndexOpts{UsePostingsCache: true}) }, }, { @@ -210,7 +209,6 @@ func TestSingleIdxCached(t *testing.T) { } func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { - shouldCachePostings = false now := model.Now() queryFrom, queryThrough := now.Add(3*time.Hour).Add(time.Millisecond), now.Add(5*time.Hour).Add(-time.Millisecond) queryBounds := newBounds(queryFrom, queryThrough) @@ -250,7 +248,7 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }) + }, TSDBIndexOpts{UsePostingsCache: true}) b.ResetTimer() b.ReportAllocs() @@ -265,7 +263,6 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { } func TestCacheableTSDBIndex_Stats(t *testing.T) { - shouldCachePostings = true series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -366,7 +363,7 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - tsdbIndex := BuildIndex(t, tempDir, series) + tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: true}) acc := &stats.Stats{} err := tsdbIndex.Stats(context.Background(), "fake", tc.from, tc.through, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Equal(t, tc.expectedErr, err) @@ -376,7 +373,6 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { } func BenchmarkSeriesRepetitive(b *testing.B) { - shouldCachePostings = true series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -418,7 +414,7 @@ func BenchmarkSeriesRepetitive(b *testing.B) { }, } tempDir := b.TempDir() - tsdbIndex := BuildIndex(b, tempDir, series) + tsdbIndex := BuildIndex(b, tempDir, series, TSDBIndexOpts{UsePostingsCache: true}) acc := &stats.Stats{} for i := 0; i < b.N; i++ { diff --git a/pkg/storage/stores/tsdb/index_client_test.go b/pkg/storage/stores/tsdb/index_client_test.go index 74c4853d72da1..55a3a9f38913f 100644 --- a/pkg/storage/stores/tsdb/index_client_test.go +++ b/pkg/storage/stores/tsdb/index_client_test.go @@ -57,7 +57,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99)), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -66,7 +66,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99)), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, } @@ -115,7 +115,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -132,7 +132,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, } @@ -243,7 +243,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -260,7 +260,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }), + }, TSDBIndexOpts{UsePostingsCache: false}), }, } diff --git a/pkg/storage/stores/tsdb/multi_file_index_test.go b/pkg/storage/stores/tsdb/multi_file_index_test.go index 0f6d5f306ec19..9f2d1f9781e56 100644 --- a/pkg/storage/stores/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/tsdb/multi_file_index_test.go @@ -61,7 +61,7 @@ func TestMultiIndex(t *testing.T) { var indices []Index dir := t.TempDir() for i := 0; i < n; i++ { - indices = append(indices, BuildIndex(t, dir, cases)) + indices = append(indices, BuildIndex(t, dir, cases, TSDBIndexOpts{UsePostingsCache: false})) } idx := NewMultiIndex(IndexSlice(indices)) diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index ed8b4ae26c065..8131b059282ec 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -71,7 +71,7 @@ func TestSingleIdx(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases) + return BuildIndex(t, t.TempDir(), cases, TSDBIndexOpts{UsePostingsCache: false}) }, }, { @@ -249,7 +249,7 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }) + }, TSDBIndexOpts{UsePostingsCache: false}) b.ResetTimer() b.ReportAllocs() @@ -304,7 +304,7 @@ func TestTSDBIndex_Stats(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series) + tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: false}) // Create the test cases testCases := []struct { @@ -415,7 +415,7 @@ func TestTSDBIndex_SeriesVolume(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series) + tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: false}) t.Run("it matches all the series when the match all matcher is passed", func(t *testing.T) { matcher := labels.MustNewMatcher(labels.MatchEqual, "", "") diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 198462477ac51..3e81e57dd5107 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -17,7 +17,7 @@ type LoadableSeries struct { Chunks index.ChunkMetas } -func BuildIndex(t testing.TB, dir string, cases []LoadableSeries) *TSDBFile { +func BuildIndex(t testing.TB, dir string, cases []LoadableSeries, opts TSDBIndexOpts) *TSDBFile { b := NewBuilder() for _, s := range cases { @@ -35,7 +35,7 @@ func BuildIndex(t testing.TB, dir string, cases []LoadableSeries) *TSDBFile { }) require.Nil(t, err) - idx, err := NewShippableTSDBFile(dst, TSDBIndexOpts{UsePostingsCache: false}) + idx, err := NewShippableTSDBFile(dst, opts) require.Nil(t, err) return idx } From 210ac079d156ee8f3cd88712390548d39d81fdd0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 16:28:30 -0300 Subject: [PATCH 28/84] Make sure cache is used on tests --- .../stores/tsdb/cached_postings_index.go | 52 +++++++++---------- .../stores/tsdb/cached_postings_index_test.go | 23 ++++++-- .../stores/tsdb/single_file_index_test.go | 15 ++++-- 3 files changed, 55 insertions(+), 35 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index dc5c3baf6dc67..817f1a138b221 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -9,6 +9,7 @@ import ( "github.com/dennwc/varint" "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" promEncoding "github.com/prometheus/prometheus/tsdb/encoding" @@ -51,55 +52,52 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return err } - if err := c.storePostings(ctx, p, key); err != nil { + expandedPosts, err := index.ExpandPostings(p) + if err != nil { + return err + } + + if err := c.storePostings(ctx, expandedPosts, key); err != nil { level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } - return fn(p) + + // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. + return fn(index.NewListPostings(expandedPosts)) } // diffVarintEncodeNoHeader encodes postings into diff+varint representation. // It doesn't add any header to the output bytes. // Length argument is expected number of postings, used for preallocating buffer. -func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { +func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) { buf := encoding.Encbuf{} buf.PutUvarint64(uint64(length)) // This encoding uses around ~1 bytes per posting, but let's use // conservative 1.25 bytes per posting to avoid extra allocations. if length > 0 { - buf.B = make([]byte, 0, 5*length/4) + buf.B = make([]byte, 0, binary.MaxVarintLen64+5*length/4) } - prev := storage.SeriesRef(0) - var total uint64 - for p.Next() { - v := p.At() + buf.PutUvarint64(uint64(length)) // first we put the postings length so we can use it when decoding. - // TODO(dylanguedes): can we ignore this? - // if v < prev { - // return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) - // } + prev := storage.SeriesRef(0) + for _, ref := range p { + if ref < prev { + return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", ref, prev) + } // This is the 'diff' part -- compute difference from previous value. - buf.PutUvarint64(uint64(v - prev)) - prev = v - total++ + buf.PutUvarint64(uint64(ref - prev)) + prev = ref } - if p.Err() != nil { - return nil, p.Err() - } - - // add number of postings at the beginning of the buffer, used when decoding. - lenBuf := make([]byte, binary.MaxVarintLen64+len(buf.B)) - binary.PutUvarint(lenBuf, total) - return append(lenBuf, buf.B...), nil + return buf.B, nil } func decodeToPostings(b []byte) index.Postings { decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) - postingsLen := storage.SeriesRef(decoder.Uvarint64()) - refs := make([]storage.SeriesRef, postingsLen) + postingsLen := decoder.Uvarint64() + refs := make([]storage.SeriesRef, 0, postingsLen) prev := storage.SeriesRef(0) for i := 0; i < int(postingsLen); i++ { @@ -123,8 +121,8 @@ func encodedMatchersLen(matchers []*labels.Matcher) int { return matchersLen } -func (c *cachedPostingsReader) storePostings(ctx context.Context, postings index.Postings, canonicalMatchers string) error { - dataToCache, err := diffVarintEncodeNoHeader(postings, 0) +func (c *cachedPostingsReader) storePostings(ctx context.Context, expandedPostings []storage.SeriesRef, canonicalMatchers string) error { + dataToCache, err := diffVarintEncodeNoHeader(expandedPostings, len(expandedPostings)) if err != nil { level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) } diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index d881d7ae2bf1b..e6dc133776ebc 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -8,14 +8,25 @@ import ( "time" "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/index/stats" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/util/flagext" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" ) +func runTSDBIndexCache(t testing.TB) { + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(t, e) + sharedCacheClient = c +} + func TestSingleIdxCached(t *testing.T) { + runTSDBIndexCache(t) + defer sharedCacheClient.Stop() cases := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar"}`), @@ -209,6 +220,9 @@ func TestSingleIdxCached(t *testing.T) { } func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { + runTSDBIndexCache(b) + defer sharedCacheClient.Stop() + now := model.Now() queryFrom, queryThrough := now.Add(3*time.Hour).Add(time.Millisecond), now.Add(5*time.Hour).Add(-time.Millisecond) queryBounds := newBounds(queryFrom, queryThrough) @@ -252,17 +266,16 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { b.ResetTimer() b.ReportAllocs() - var err error for i := 0; i < b.N; i++ { chkRefs := ChunkRefsPool.Get() - chkRefs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, chkRefs, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - require.NoError(b, err) - require.Len(b, chkRefs, numChunksToMatch*2) + chkRefs, _ = tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, chkRefs, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) ChunkRefsPool.Put(chkRefs) } } func TestCacheableTSDBIndex_Stats(t *testing.T) { + runTSDBIndexCache(t) + defer sharedCacheClient.Stop() series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -373,6 +386,8 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { } func BenchmarkSeriesRepetitive(b *testing.B) { + runTSDBIndexCache(b) + defer sharedCacheClient.Stop() series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 8131b059282ec..217aeeda029db 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -9,8 +9,10 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/util/flagext" "github.com/go-kit/log" "github.com/prometheus/common/model" @@ -215,6 +217,11 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { queryBounds := newBounds(queryFrom, queryThrough) numChunksToMatch := 0 + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, err := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(b, err) + sharedCacheClient = c + var chunkMetas []index.ChunkMeta // build a chunk for every second with randomized chunk length for from, through := now, now.Add(24*time.Hour); from <= through; from = from.Add(time.Second) { @@ -249,14 +256,14 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, TSDBIndexOpts{UsePostingsCache: false}) + }, TSDBIndexOpts{UsePostingsCache: true}) b.ResetTimer() b.ReportAllocs() for i := 0; i < b.N; i++ { - chkRefs, err := tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - require.NoError(b, err) - require.Len(b, chkRefs, numChunksToMatch*2) + chkRefs := ChunkRefsPool.Get() + chkRefs, _ = tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, chkRefs, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + ChunkRefsPool.Put(chkRefs) } } From eef25e51bb702007b9ca5113dba20b33d5ebb7e1 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 16:29:52 -0300 Subject: [PATCH 29/84] remvoe consts only used by mimir --- pkg/storage/chunk/cache/lru_cache.go | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 6bb31f476f1a7..2fc75c0ee2ce2 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -10,7 +10,6 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" lru "github.com/hashicorp/golang-lru/simplelru" - "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -19,22 +18,6 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" ) -type codec string - -const ( - codecHeaderSnappy codec = "dvs" // As in "diff+varint+snappy". - codecHeaderSnappyWithMatchers codec = "dm" // As in "dvs+matchers" -) - -const maxInt = int(^uint(0) >> 1) - -const ( - stringHeaderSize = 8 - sliceHeaderSize = 16 -) - -var ulidSize = uint64(len(ulid.ULID{})) - type LRUCacheConfig struct { MaxSizeBytes flagext.ByteSize `yaml:"max_size_bytes"` MaxItemSizeBytes flagext.ByteSize `yaml:"max_item_size_bytes"` From d27286cfa515eca01d92c160e3754f22761801fd Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 16:49:22 -0300 Subject: [PATCH 30/84] appease lint (remove TSDB from struct name) --- .../stores/tsdb/cached_postings_index_test.go | 8 ++++---- pkg/storage/stores/tsdb/compactor.go | 8 ++++---- pkg/storage/stores/tsdb/index_client_test.go | 12 ++++++------ pkg/storage/stores/tsdb/manager.go | 4 ++-- pkg/storage/stores/tsdb/multi_file_index_test.go | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 12 ++++++------ pkg/storage/stores/tsdb/single_file_index_test.go | 8 ++++---- pkg/storage/stores/tsdb/store.go | 2 +- pkg/storage/stores/tsdb/util_test.go | 2 +- 9 files changed, 29 insertions(+), 29 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index e6dc133776ebc..e5e028fa91fd9 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -77,7 +77,7 @@ func TestSingleIdxCached(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases, TSDBIndexOpts{UsePostingsCache: true}) + return BuildIndex(t, t.TempDir(), cases, IndexOpts{UsePostingsCache: true}) }, }, { @@ -262,7 +262,7 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, TSDBIndexOpts{UsePostingsCache: true}) + }, IndexOpts{UsePostingsCache: true}) b.ResetTimer() b.ReportAllocs() @@ -376,7 +376,7 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: true}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) acc := &stats.Stats{} err := tsdbIndex.Stats(context.Background(), "fake", tc.from, tc.through, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Equal(t, tc.expectedErr, err) @@ -429,7 +429,7 @@ func BenchmarkSeriesRepetitive(b *testing.B) { }, } tempDir := b.TempDir() - tsdbIndex := BuildIndex(b, tempDir, series, TSDBIndexOpts{UsePostingsCache: true}) + tsdbIndex := BuildIndex(b, tempDir, series, IndexOpts{UsePostingsCache: true}) acc := &stats.Stats{} for i := 0; i < b.N; i++ { diff --git a/pkg/storage/stores/tsdb/compactor.go b/pkg/storage/stores/tsdb/compactor.go index 1a531f52dd6a3..746fa8bca66ba 100644 --- a/pkg/storage/stores/tsdb/compactor.go +++ b/pkg/storage/stores/tsdb/compactor.go @@ -36,7 +36,7 @@ func (i indexProcessor) NewTableCompactor(ctx context.Context, commonIndexSet co } func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableName, userID, workingDir string, periodConfig config.PeriodConfig, logger log.Logger) (compactor.CompactedIndex, error) { - opts := TSDBIndexOpts{UsePostingsCache: false} + opts := IndexOpts{UsePostingsCache: false} indexFile, err := OpenShippableTSDB(path, opts) if err != nil { return nil, err @@ -101,7 +101,7 @@ func (t *tableCompactor) CompactTable() error { } downloadPaths[job] = downloadedAt - idx, err := OpenShippableTSDB(downloadedAt, TSDBIndexOpts{UsePostingsCache: false}) + idx, err := OpenShippableTSDB(downloadedAt, IndexOpts{UsePostingsCache: false}) if err != nil { return err } @@ -219,7 +219,7 @@ func setupBuilder(ctx context.Context, userID string, sourceIndexSet compactor.I } }() - indexFile, err := OpenShippableTSDB(path, TSDBIndexOpts{UsePostingsCache: false}) + indexFile, err := OpenShippableTSDB(path, IndexOpts{UsePostingsCache: false}) if err != nil { return nil, err } @@ -388,7 +388,7 @@ func (c *compactedIndex) ToIndexFile() (index_shipper.Index, error) { return nil, err } - return NewShippableTSDBFile(id, TSDBIndexOpts{UsePostingsCache: false}) + return NewShippableTSDBFile(id, IndexOpts{UsePostingsCache: false}) } func getUnsafeBytes(s string) []byte { diff --git a/pkg/storage/stores/tsdb/index_client_test.go b/pkg/storage/stores/tsdb/index_client_test.go index 19331110e87a2..0bba97c467c15 100644 --- a/pkg/storage/stores/tsdb/index_client_test.go +++ b/pkg/storage/stores/tsdb/index_client_test.go @@ -57,7 +57,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99)), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -66,7 +66,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99)), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, } @@ -115,7 +115,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -132,7 +132,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, } @@ -243,7 +243,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -260,7 +260,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }, TSDBIndexOpts{UsePostingsCache: false}), + }, IndexOpts{UsePostingsCache: false}), }, } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 23b1f7e26eb09..9d70a8d846bd0 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -129,7 +129,7 @@ func (m *tsdbManager) Start() (err error) { indices++ prefixed := NewPrefixedIdentifier(id, filepath.Join(mulitenantDir, bucket), "") - loaded, err := NewShippableTSDBFile(prefixed, TSDBIndexOpts{UsePostingsCache: false}) + loaded, err := NewShippableTSDBFile(prefixed, IndexOpts{UsePostingsCache: false}) if err != nil { level.Warn(m.log).Log( @@ -221,7 +221,7 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, shipper indexshipper.Ind level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dst.Path(), "duration", time.Since(start)) - loaded, err := NewShippableTSDBFile(dst, TSDBIndexOpts{UsePostingsCache: false}) + loaded, err := NewShippableTSDBFile(dst, IndexOpts{UsePostingsCache: false}) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/multi_file_index_test.go b/pkg/storage/stores/tsdb/multi_file_index_test.go index 9f2d1f9781e56..49ae1efff3d38 100644 --- a/pkg/storage/stores/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/tsdb/multi_file_index_test.go @@ -61,7 +61,7 @@ func TestMultiIndex(t *testing.T) { var indices []Index dir := t.TempDir() for i := 0; i < n; i++ { - indices = append(indices, BuildIndex(t, dir, cases, TSDBIndexOpts{UsePostingsCache: false})) + indices = append(indices, BuildIndex(t, dir, cases, IndexOpts{UsePostingsCache: false})) } idx := NewMultiIndex(IndexSlice(indices)) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 087ccfda4040a..2b83ba8e7544f 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -25,11 +25,11 @@ var ErrAlreadyOnDesiredVersion = errors.New("tsdb file already on desired versio // GetRawFileReaderFunc returns an io.ReadSeeker for reading raw tsdb file from disk type GetRawFileReaderFunc func() (io.ReadSeeker, error) -type TSDBIndexOpts struct { +type IndexOpts struct { UsePostingsCache bool } -func OpenShippableTSDB(p string, opts TSDBIndexOpts) (index_shipper.Index, error) { +func OpenShippableTSDB(p string, opts IndexOpts) (index_shipper.Index, error) { id, err := identifierFromPath(p) if err != nil { return nil, err @@ -39,7 +39,7 @@ func OpenShippableTSDB(p string, opts TSDBIndexOpts) (index_shipper.Index, error } func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index_shipper.Index, error) { - opts := TSDBIndexOpts{UsePostingsCache: false} + opts := IndexOpts{UsePostingsCache: false} indexFile, err := OpenShippableTSDB(path, opts) if err != nil { return nil, err @@ -79,7 +79,7 @@ func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index if err != nil { return nil, err } - return NewShippableTSDBFile(id, TSDBIndexOpts{UsePostingsCache: false}) + return NewShippableTSDBFile(id, IndexOpts{UsePostingsCache: false}) } // nolint @@ -95,7 +95,7 @@ type TSDBFile struct { getRawFileReader GetRawFileReaderFunc } -func NewShippableTSDBFile(id Identifier, opts TSDBIndexOpts) (*TSDBFile, error) { +func NewShippableTSDBFile(id Identifier, opts IndexOpts) (*TSDBFile, error) { idx, getRawFileReader, err := NewTSDBIndexFromFile(id.Path(), opts) if err != nil { return nil, err @@ -128,7 +128,7 @@ type TSDBIndex struct { // Return the index as well as the underlying raw file reader which isn't exposed as an index // method but is helpful for building an io.reader for the index shipper -func NewTSDBIndexFromFile(location string, opts TSDBIndexOpts) (Index, GetRawFileReaderFunc, error) { +func NewTSDBIndexFromFile(location string, opts IndexOpts) (Index, GetRawFileReaderFunc, error) { reader, err := index.NewFileReader(location) if err != nil { return nil, nil, err diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 83ede0407604b..e65509cacfc43 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -73,7 +73,7 @@ func TestSingleIdx(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases, TSDBIndexOpts{UsePostingsCache: false}) + return BuildIndex(t, t.TempDir(), cases, IndexOpts{UsePostingsCache: false}) }, }, { @@ -256,7 +256,7 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, TSDBIndexOpts{UsePostingsCache: true}) + }, IndexOpts{UsePostingsCache: true}) b.ResetTimer() b.ReportAllocs() @@ -311,7 +311,7 @@ func TestTSDBIndex_Stats(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: false}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: false}) // Create the test cases testCases := []struct { @@ -426,7 +426,7 @@ func TestTSDBIndex_SeriesVolume(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, TSDBIndexOpts{UsePostingsCache: false}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: false}) from := model.TimeFromUnixNano(t1.UnixNano()) through := model.TimeFromUnixNano(t2.UnixNano()) diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index 03c84db128903..5e68edb187e0e 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -79,7 +79,7 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf usePostingsCache := indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil openFn := func(p string) (indexshipper_index.Index, error) { - return OpenShippableTSDB(p, TSDBIndexOpts{UsePostingsCache: usePostingsCache}) + return OpenShippableTSDB(p, IndexOpts{UsePostingsCache: usePostingsCache}) } var err error diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 2fdad9ab79596..8fa8ee3588942 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -17,7 +17,7 @@ type LoadableSeries struct { Chunks index.ChunkMetas } -func BuildIndex(t testing.TB, dir string, cases []LoadableSeries, opts TSDBIndexOpts) *TSDBFile { +func BuildIndex(t testing.TB, dir string, cases []LoadableSeries, opts IndexOpts) *TSDBFile { b := NewBuilder(index.LiveFormat) for _, s := range cases { From 1e6fd338a9b01a1e1ce00ec5f9385d0483880064 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 17:19:45 -0300 Subject: [PATCH 31/84] fix lint --- tools/tsdb/index-analyzer/main.go | 2 +- tools/tsdb/migrate-versions/main_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/tsdb/index-analyzer/main.go b/tools/tsdb/index-analyzer/main.go index f8369db6ce93c..4171d29b4a946 100644 --- a/tools/tsdb/index-analyzer/main.go +++ b/tools/tsdb/index-analyzer/main.go @@ -35,7 +35,7 @@ func main() { tableRanges := getIndexStoreTableRanges(config.TSDBType, conf.SchemaConfig.Configs) openFn := func(p string) (indexshipper_index.Index, error) { - return tsdb.OpenShippableTSDB(p, tsdb.TSDBIndexOpts{UsePostingsCache: false}) + return tsdb.OpenShippableTSDB(p, tsdb.IndexOpts{UsePostingsCache: false}) } shipper, err := indexshipper.NewIndexShipper( diff --git a/tools/tsdb/migrate-versions/main_test.go b/tools/tsdb/migrate-versions/main_test.go index e695ebbf587b7..6da2c92bc7977 100644 --- a/tools/tsdb/migrate-versions/main_test.go +++ b/tools/tsdb/migrate-versions/main_test.go @@ -88,7 +88,7 @@ func TestMigrateTables(t *testing.T) { require.NoError(t, err) tableName := fmt.Sprintf("%s%d", indexPrefix, i) - idx, err := tsdb.NewShippableTSDBFile(id, tsdb.TSDBIndexOpts{UsePostingsCache: false}) + idx, err := tsdb.NewShippableTSDBFile(id, tsdb.IndexOpts{UsePostingsCache: false}) require.NoError(t, err) require.NoError(t, uploadFile(idx, indexStorageClient, tableName, userID)) From 203022419d34a52511cf5cc3f5775f6a3adfdfce Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 17:51:25 -0300 Subject: [PATCH 32/84] fix import order --- pkg/storage/stores/tsdb/cached_postings_index_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index e5e028fa91fd9..82dc1dc84d85d 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -8,13 +8,14 @@ import ( "time" "github.com/go-kit/log" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/index/stats" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/flagext" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/stretchr/testify/require" ) func runTSDBIndexCache(t testing.TB) { From 33521a119c2d12b5a0c5904480f6157a1b43c7d5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 17:51:48 -0300 Subject: [PATCH 33/84] fix import order again --- pkg/storage/chunk/cache/lru_cache_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go index 7e76d2e287fc8..e051b2fe3f02f 100644 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ b/pkg/storage/chunk/cache/lru_cache_test.go @@ -12,10 +12,11 @@ import ( "testing" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/util/flagext" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/util/flagext" ) func TestLRUCacheEviction(t *testing.T) { From 52ca301fdaa02f384996a65395224537f1f3fd33 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 22 Jun 2023 18:49:45 -0300 Subject: [PATCH 34/84] encode with 32b instead of 64b (tsdb uses 32b internally) --- .../stores/tsdb/cached_postings_index.go | 25 +++++-------------- 1 file changed, 6 insertions(+), 19 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 817f1a138b221..487b3d9b75a85 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -6,7 +6,6 @@ import ( "sort" "strings" - "github.com/dennwc/varint" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/pkg/errors" @@ -70,15 +69,15 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe // Length argument is expected number of postings, used for preallocating buffer. func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) { buf := encoding.Encbuf{} - buf.PutUvarint64(uint64(length)) + buf.PutUvarint32(uint32(length)) // This encoding uses around ~1 bytes per posting, but let's use // conservative 1.25 bytes per posting to avoid extra allocations. if length > 0 { - buf.B = make([]byte, 0, binary.MaxVarintLen64+5*length/4) + buf.B = make([]byte, 0, binary.MaxVarintLen32+5*length/4) } - buf.PutUvarint64(uint64(length)) // first we put the postings length so we can use it when decoding. + buf.PutUvarint32(uint32(length)) // first we put the postings length so we can use it when decoding. prev := storage.SeriesRef(0) for _, ref := range p { @@ -87,7 +86,7 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) } // This is the 'diff' part -- compute difference from previous value. - buf.PutUvarint64(uint64(ref - prev)) + buf.PutUvarint32(uint32(ref - prev)) prev = ref } @@ -96,12 +95,12 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) func decodeToPostings(b []byte) index.Postings { decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) - postingsLen := decoder.Uvarint64() + postingsLen := decoder.Uvarint32() refs := make([]storage.SeriesRef, 0, postingsLen) prev := storage.SeriesRef(0) for i := 0; i < int(postingsLen); i++ { - v := storage.SeriesRef(decoder.Uvarint64()) + v := storage.SeriesRef(decoder.Uvarint32()) refs = append(refs, v+prev) prev = v } @@ -109,18 +108,6 @@ func decodeToPostings(b []byte) index.Postings { return index.NewListPostings(refs) } -func encodedMatchersLen(matchers []*labels.Matcher) int { - matchersLen := varint.UvarintSize(uint64(len(matchers))) - for _, m := range matchers { - matchersLen += varint.UvarintSize(uint64(len(m.Name))) - matchersLen += len(m.Name) - matchersLen++ // 1 byte for the type - matchersLen += varint.UvarintSize(uint64(len(m.Value))) - matchersLen += len(m.Value) - } - return matchersLen -} - func (c *cachedPostingsReader) storePostings(ctx context.Context, expandedPostings []storage.SeriesRef, canonicalMatchers string) error { dataToCache, err := diffVarintEncodeNoHeader(expandedPostings, len(expandedPostings)) if err != nil { From 2adc870c86b6521f08dfc6c9a690da21184ec4d1 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 07:39:47 -0300 Subject: [PATCH 35/84] Use "," to separate matchers. --- pkg/storage/stores/tsdb/cached_postings_index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 487b3d9b75a85..6cdd77e1eb729 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -159,7 +159,7 @@ func CanonicalLabelMatchersKey(ms []*labels.Matcher) string { sb.WriteString(m.Name) sb.WriteString(m.Type.String()) sb.WriteString(m.Value) - sb.WriteByte(0) + sb.WriteByte(',') } return sb.String() } From 423692b3c5484acb320058d46f16cc23732b7d5f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 13:45:21 -0300 Subject: [PATCH 36/84] better defaults --- pkg/storage/chunk/cache/lru_cache.go | 7 ++++++- pkg/storage/stores/shipper/indexgateway/shufflesharding.go | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 2fc75c0ee2ce2..596bf5d055b6a 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -29,11 +29,16 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { + megabyte := 1000000 + gigabyte := megabyte * 1000 + f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") + cfg.MaxItemSizeBytes = flagext.ByteSize(100 * megabyte) f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") + cfg.MaxSizeBytes = flagext.ByteSize(1 * gigabyte) - f.IntVar(&cfg.MaxItems, prefix+".max-items", 5000, description+"Maximum items in the cache.") + f.IntVar(&cfg.MaxItems, prefix+".max-items", 50000, description+"Maximum items in the cache.") f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether LRUCache should be enabled or not.") } diff --git a/pkg/storage/stores/shipper/indexgateway/shufflesharding.go b/pkg/storage/stores/shipper/indexgateway/shufflesharding.go index 03dfb61b200fb..b563869d3719c 100644 --- a/pkg/storage/stores/shipper/indexgateway/shufflesharding.go +++ b/pkg/storage/stores/shipper/indexgateway/shufflesharding.go @@ -101,7 +101,7 @@ func (s *NoopStrategy) FilterTenants(tenantIDs []string) ([]string, error) { return tenantIDs, nil } -// GetShardingStrategy returns the correct ShardingStrategy implementaion based +// GetShardingStrategy returns the correct ShardingStrategy implementation based // on provided configuration. func GetShardingStrategy(cfg Config, indexGatewayRingManager *RingManager, o Limits) ShardingStrategy { if cfg.Mode != RingMode || indexGatewayRingManager.Mode == ClientMode { From 474261da3657f3848a9872cb5626afdde75e3fc6 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 14:02:21 -0300 Subject: [PATCH 37/84] bugged linter --- pkg/storage/store_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 9a4ef12c73883..ad7e0ddecc2bd 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -1021,7 +1021,7 @@ func TestStore_MultiPeriod(t *testing.T) { BoltDBShipperConfig: shipper.Config{ Config: shipperConfig, }, - TSDBShipperConfig: tsdb.IndexCfg{Config: shipperConfig}, + TSDBShipperConfig: tsdb.IndexCfg{Config: shipperConfig, CachePostings: false}, NamedStores: NamedStores{ Filesystem: map[string]NamedFSConfig{ "named-store": {Directory: path.Join(tempDir, "named-store")}, From 2baf30d75ed596a172c52222122efc19e1e9cfa8 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 14:10:22 -0300 Subject: [PATCH 38/84] fix docs --- docs/sources/configuration/_index.md | 6 +++--- pkg/storage/chunk/cache/lru_cache.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sources/configuration/_index.md b/docs/sources/configuration/_index.md index 7e240e5060d6d..d90c9adb8482b 100644 --- a/docs/sources/configuration/_index.md +++ b/docs/sources/configuration/_index.md @@ -3933,18 +3933,18 @@ embedded_cache: lru_cache: # Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be - # applied. + # applied. Default: 1GB. # CLI flag: -.lrucache.max-size-bytes [max_size_bytes: | default = 0B] # Maximum memory size of a single item in the cache. A unit suffix (KB, MB, - # GB) may be applied. + # GB) may be applied. Default: 100MB. # CLI flag: -.lrucache.max-item-size-bytes [max_item_size_bytes: | default = 0B] # Maximum items in the cache. # CLI flag: -.lrucache.max-items - [max_items: | default = 5000] + [max_items: | default = 50000] # Whether LRUCache should be enabled or not. # CLI flag: -.lrucache.enabled diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 596bf5d055b6a..886f68735fa61 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -32,10 +32,10 @@ func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f megabyte := 1000000 gigabyte := megabyte * 1000 - f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") + f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied. Default: 100MB.") cfg.MaxItemSizeBytes = flagext.ByteSize(100 * megabyte) - f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") + f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied. Default: 1GB.") cfg.MaxSizeBytes = flagext.ByteSize(1 * gigabyte) f.IntVar(&cfg.MaxItems, prefix+".max-items", 50000, description+"Maximum items in the cache.") From 571c6a8ca665fe58abba0b9554f41b8c7d7e6337 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 14:17:14 -0300 Subject: [PATCH 39/84] fix config type --- pkg/storage/store_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 6aadc296cc2c6..3412727f58416 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -1006,7 +1006,7 @@ func TestStore_indexPrefixChange(t *testing.T) { cfg := Config{ FSConfig: local.FSConfig{Directory: path.Join(tempDir, "chunks")}, - TSDBShipperConfig: shipperConfig, + TSDBShipperConfig: tsdb.IndexCfg{Config: shipperConfig}, NamedStores: NamedStores{ Filesystem: map[string]NamedFSConfig{ "named-store": {Directory: path.Join(tempDir, "named-store")}, From 4139abd7b7bfd4d68a660f05fc038be1834c0015 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 14:44:03 -0300 Subject: [PATCH 40/84] Register flags --- pkg/storage/stores/tsdb/index.go | 7 +++++++ pkg/storage/stores/tsdb/store.go | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 8fda8a987832e..049eb5cfd9f34 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "flag" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -29,6 +30,12 @@ type IndexCfg struct { CachePostings bool `yaml:"cache_postings"` } +func (cfg *IndexCfg) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.BoolVar(&cfg.CachePostings, prefix+"cache-postings", false, "Whether TSDB should cache postings or not.") + + cfg.Config.RegisterFlagsWithPrefix(prefix, f) +} + // Compares by (Start, End) // Assumes User is equivalent func (r ChunkRef) Less(x ChunkRef) bool { diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index 5e68edb187e0e..50556f88a8d8c 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -77,7 +77,7 @@ func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConf sharedCacheClient = idxCache - usePostingsCache := indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil + usePostingsCache := indexCfg.CachePostings && indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil openFn := func(p string) (indexshipper_index.Index, error) { return OpenShippableTSDB(p, IndexOpts{UsePostingsCache: usePostingsCache}) } From 3d30b1e9afdf16326d8c286d19224383de803192 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 16:00:18 -0300 Subject: [PATCH 41/84] wrap error on vendor --- .../github.com/prometheus/prometheus/tsdb/encoding/encoding.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go index ab97876a362ef..5fa5b59c65417 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go @@ -15,6 +15,7 @@ package encoding import ( "encoding/binary" + "fmt" "hash" "hash/crc32" "math" @@ -241,7 +242,7 @@ func (d *Decbuf) Uvarint64() uint64 { } x, n := varint.Uvarint(d.B) if n < 1 { - d.E = ErrInvalidSize + d.E = fmt.Errorf("Uvarint64: %w", ErrInvalidSize) return 0 } d.B = d.B[n:] From 7d9a24bd36ce0b6b8113d51ae2d6eec2164d6db0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 16:02:59 -0300 Subject: [PATCH 42/84] test other thing --- pkg/storage/stores/tsdb/cached_postings_index.go | 3 +++ .../github.com/prometheus/prometheus/tsdb/encoding/encoding.go | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 6cdd77e1eb729..1d0fcfdaec318 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -94,6 +94,9 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) } func decodeToPostings(b []byte) index.Postings { + if len(b) <= 0 { + return index.EmptyPostings() + } decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) postingsLen := decoder.Uvarint32() refs := make([]storage.SeriesRef, 0, postingsLen) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go index 5fa5b59c65417..8dbbb13ad6af2 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go @@ -304,7 +304,7 @@ func (d *Decbuf) ConsumePadding() { } } -func (d *Decbuf) Err() error { return d.E } +func (d *Decbuf) Err() error { return fmt.Errorf("decbuf err: %w", d.E) } func (d *Decbuf) Len() int { return len(d.B) } func (d *Decbuf) Get() []byte { return d.B } From 7edc7bd72a92a75019ddfc5c38976906adfab1f0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 16:22:30 -0300 Subject: [PATCH 43/84] wrap errors --- pkg/storage/stores/tsdb/index/index.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index b6e2c749cf41d..deba5f9fea59f 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -1453,7 +1453,7 @@ func (r *Reader) PostingsRanges() (map[labels.Label]Range, error) { } d := encoding.DecWrap(tsdb_enc.NewDecbufAt(r.b, int(off), castagnoliTable)) if d.Err() != nil { - return d.Err() + return errors.Wrap(d.Err(), "postings ranges") } m[labels.Label{Name: key[0], Value: key[1]}] = Range{ Start: int64(off) + 4, @@ -1499,7 +1499,7 @@ func NewSymbols(bs ByteSlice, version, off int) (*Symbols, error) { s.seen++ } if d.Err() != nil { - return nil, d.Err() + return nil, errors.Wrap(d.Err(), "new symbols") } return s, nil } @@ -1599,7 +1599,7 @@ func (s *symbolsIter) Next() bool { s.cur = yoloString(s.d.UvarintBytes()) s.cnt-- if s.d.Err() != nil { - s.err = s.d.Err() + s.err = errors.Wrap(s.d.Err(), "symbols iter err") return false } return true @@ -1635,7 +1635,7 @@ func ReadOffsetTable(bs ByteSlice, off uint64, f func([]string, uint64, int) err } cnt-- } - return d.Err() + return errors.Wrap(d.Err(), "read offset table") } func readFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets, error) { @@ -1648,7 +1648,7 @@ func readFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets, cnt-- } - return res, d.Err() + return res, errors.Wrap(d.Err(), "read fingerprint offsets table") } @@ -1827,7 +1827,7 @@ func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *l } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) if d.Err() != nil { - return 0, d.Err() + return 0, errors.Wrap(d.Err(), "series") } fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks) @@ -1846,7 +1846,7 @@ func (r *Reader) ChunkStats(id storage.SeriesRef, from, through int64, lbls *lab } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) if d.Err() != nil { - return 0, ChunkStats{}, d.Err() + return 0, ChunkStats{}, errors.Wrap(d.Err(), "chunk stats") } return r.dec.ChunkStats(r.version, d.Get(), id, from, through, lbls) From 8c9fcefbe770c79c5fc1f93c967578ffc06d6942 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 19:11:14 -0300 Subject: [PATCH 44/84] careful wrapping --- .../prometheus/prometheus/tsdb/encoding/encoding.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go index 8dbbb13ad6af2..68e3164329b1c 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go @@ -304,7 +304,13 @@ func (d *Decbuf) ConsumePadding() { } } -func (d *Decbuf) Err() error { return fmt.Errorf("decbuf err: %w", d.E) } +func (d *Decbuf) Err() error { + if d.E != nil { + return fmt.Errorf("decbuf err: %w", d.E) + } + return nil +} + func (d *Decbuf) Len() int { return len(d.B) } func (d *Decbuf) Get() []byte { return d.B } From 5521d822693185b209c82eef517e34540ed80be5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 19:46:22 -0300 Subject: [PATCH 45/84] wrap at different place --- pkg/storage/stores/tsdb/cached_postings_index.go | 7 +------ pkg/storage/stores/tsdb/single_file_index.go | 5 +++-- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 1d0fcfdaec318..24b601911689c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -129,12 +129,7 @@ func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (i } if len(found) > 0 { - var postings []index.Postings - for _, b := range bufs { - postings = append(postings, decodeToPostings(b)) - } - - return index.Merge(postings...), true + return decodeToPostings(bufs[0]), true } return nil, false diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 2b83ba8e7544f..388ac0669610c 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -3,6 +3,7 @@ package tsdb import ( "context" "errors" + "fmt" "io" "math" "path/filepath" @@ -315,7 +316,7 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim for p.Next() { fp, stats, err := i.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) if err != nil { - return err + return fmt.Errorf("chunk stats: %w", err) } // skip series that belong to different shards @@ -383,7 +384,7 @@ func (i *TSDBIndex) SeriesVolume(ctx context.Context, _ string, from, through mo for p.Next() { fp, stats, err := i.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) if err != nil { - return err + return fmt.Errorf("series volume: %w", err) } // skip series that belong to different shards From 0ef38b7d694157f06df4ad376728d804bacdef70 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 19:47:54 -0300 Subject: [PATCH 46/84] wrap different --- pkg/storage/stores/tsdb/single_file_index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 388ac0669610c..70c4ec0f65c33 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -316,7 +316,7 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim for p.Next() { fp, stats, err := i.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) if err != nil { - return fmt.Errorf("chunk stats: %w", err) + return fmt.Errorf("stats: chunk stats: %w", err) } // skip series that belong to different shards From dcdf6d233dbba219763b3c7548454c52c9d8d942 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 20:08:05 -0300 Subject: [PATCH 47/84] use sharded postings? --- pkg/storage/stores/tsdb/cached_postings_index.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 24b601911689c..aad88acfde6fc 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -3,6 +3,7 @@ package tsdb import ( "context" "encoding/binary" + "fmt" "sort" "strings" @@ -48,20 +49,22 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe p, err := PostingsForMatchers(c.reader, nil, matchers...) if err != nil { - return err + return fmt.Errorf("cached postings reader for postings: %w", err) } expandedPosts, err := index.ExpandPostings(p) if err != nil { - return err + return fmt.Errorf("expanded postings: %w", err) } if err := c.storePostings(ctx, expandedPosts, key); err != nil { level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } + idx := index.NewShardedPostings(index.NewListPostings(expandedPosts), index.ShardAnnotation{}, nil) + // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. - return fn(index.NewListPostings(expandedPosts)) + return fn(idx) } // diffVarintEncodeNoHeader encodes postings into diff+varint representation. From 6c3676794f594254aac323490c4a8f6664dddb6d Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 23 Jun 2023 20:21:35 -0300 Subject: [PATCH 48/84] sanity check --- .../stores/tsdb/cached_postings_index.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index aad88acfde6fc..f58f98157e317 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -52,19 +52,19 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return fmt.Errorf("cached postings reader for postings: %w", err) } - expandedPosts, err := index.ExpandPostings(p) - if err != nil { - return fmt.Errorf("expanded postings: %w", err) - } + // expandedPosts, err := index.ExpandPostings(p) + // if err != nil { + // return fmt.Errorf("expanded postings: %w", err) + // } - if err := c.storePostings(ctx, expandedPosts, key); err != nil { - level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) - } + // if err := c.storePostings(ctx, expandedPosts, key); err != nil { + // level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) + // } - idx := index.NewShardedPostings(index.NewListPostings(expandedPosts), index.ShardAnnotation{}, nil) + // idx := index.NewShardedPostings(index.NewListPostings(expandedPosts), index.ShardAnnotation{}, nil) // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. - return fn(idx) + return fn(p) } // diffVarintEncodeNoHeader encodes postings into diff+varint representation. From b84e257c9a2af204e4fab29fa9674c0e39f97c5c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 08:43:39 -0300 Subject: [PATCH 49/84] reset postings by calling PostingsForMatcher again. --- .../stores/tsdb/cached_postings_index.go | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index f58f98157e317..aacfce57b8388 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -52,16 +52,19 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return fmt.Errorf("cached postings reader for postings: %w", err) } - // expandedPosts, err := index.ExpandPostings(p) - // if err != nil { - // return fmt.Errorf("expanded postings: %w", err) - // } + expandedPosts, err := index.ExpandPostings(p) + if err != nil { + return fmt.Errorf("expanded postings: %w", err) + } - // if err := c.storePostings(ctx, expandedPosts, key); err != nil { - // level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) - // } + if err := c.storePostings(ctx, expandedPosts, key); err != nil { + level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) + } - // idx := index.NewShardedPostings(index.NewListPostings(expandedPosts), index.ShardAnnotation{}, nil) + p, err = PostingsForMatchers(c.reader, nil, matchers...) + if err != nil { + return fmt.Errorf("cached postings reader for postings: %w", err) + } // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. return fn(p) From 5ff3a36fb477abf1a88ae746495fb11b06bdf9ab Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 15:10:11 -0300 Subject: [PATCH 50/84] sanity check --- .../stores/tsdb/cached_postings_index.go | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index aacfce57b8388..a946931ad815d 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -52,19 +52,19 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return fmt.Errorf("cached postings reader for postings: %w", err) } - expandedPosts, err := index.ExpandPostings(p) - if err != nil { - return fmt.Errorf("expanded postings: %w", err) - } - - if err := c.storePostings(ctx, expandedPosts, key); err != nil { - level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) - } - - p, err = PostingsForMatchers(c.reader, nil, matchers...) - if err != nil { - return fmt.Errorf("cached postings reader for postings: %w", err) - } + // expandedPosts, err := index.ExpandPostings(p) + // if err != nil { + // return fmt.Errorf("expanded postings: %w", err) + // } + + // if err := c.storePostings(ctx, expandedPosts, key); err != nil { + // level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) + // } + + // p, err = PostingsForMatchers(c.reader, nil, matchers...) + // if err != nil { + // return fmt.Errorf("cached postings reader for postings: %w", err) + // } // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. return fn(p) From d9354f5de6fa8d6ffaff310713c66b714f988f2b Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 16:01:49 -0300 Subject: [PATCH 51/84] Try calling PostingsForMatchers after cache hit too. --- .../stores/tsdb/cached_postings_index.go | 25 ++++----- .../stores/tsdb/cached_postings_index_test.go | 55 +++++++++++++++++++ pkg/storage/stores/tsdb/index/index.go | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 5 +- .../prometheus/tsdb/encoding/encoding.go | 6 +- 5 files changed, 73 insertions(+), 20 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index a946931ad815d..af1158402d39e 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -44,6 +44,8 @@ type cachedPostingsReader struct { func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) if postings, got := c.fetchPostings(ctx, key); got { + // call PostingsForMatchers just to populate things. + PostingsForMatchers(c.reader, nil, matchers...) return fn(postings) } @@ -52,22 +54,17 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return fmt.Errorf("cached postings reader for postings: %w", err) } - // expandedPosts, err := index.ExpandPostings(p) - // if err != nil { - // return fmt.Errorf("expanded postings: %w", err) - // } - - // if err := c.storePostings(ctx, expandedPosts, key); err != nil { - // level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) - // } + expandedPosts, err := index.ExpandPostings(p) + if err != nil { + return fmt.Errorf("expanded postings: %w", err) + } - // p, err = PostingsForMatchers(c.reader, nil, matchers...) - // if err != nil { - // return fmt.Errorf("cached postings reader for postings: %w", err) - // } + if err := c.storePostings(ctx, expandedPosts, key); err != nil { + level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) + } - // `index.ExpandedPostings` makes the iterator to walk, so we have to reset it by instantiating a new NewListPostings. - return fn(p) + // because `index.ExpandedPostings` walks with the iterator, so we have to reset it by instantiating a new ListPostings. + return fn(index.NewListPostings(expandedPosts)) } // diffVarintEncodeNoHeader encodes postings into diff+varint representation. diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 82dc1dc84d85d..ee46b4530097d 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -386,6 +386,61 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { } } +func TestBla(t *testing.T) { + runTSDBIndexCache(t) + defer sharedCacheClient.Stop() + series := []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 1, + Entries: 10, + KB: 10, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 2, + Entries: 20, + KB: 20, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 3, + Entries: 30, + KB: 30, + }, + { + MinTime: 10, + MaxTime: 20, + Checksum: 4, + Entries: 40, + KB: 40, + }, + }, + }, + } + tempDir := t.TempDir() + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + acc := &stats.Stats{} + acc2 := []ChunkRef{} + + tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 15, acc2, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 15, acc2, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) +} + func BenchmarkSeriesRepetitive(b *testing.B) { runTSDBIndexCache(b) defer sharedCacheClient.Stop() diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index deba5f9fea59f..b7c97f3702b7e 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -1846,7 +1846,7 @@ func (r *Reader) ChunkStats(id storage.SeriesRef, from, through int64, lbls *lab } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) if d.Err() != nil { - return 0, ChunkStats{}, errors.Wrap(d.Err(), "chunk stats") + return 0, ChunkStats{}, errors.Wrap(d.Err(), "reader chunk stats") } return r.dec.ChunkStats(r.version, d.Get(), id, from, through, lbls) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 70c4ec0f65c33..cec1f77a7544f 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -314,9 +314,10 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim } for p.Next() { - fp, stats, err := i.reader.ChunkStats(p.At(), int64(from), int64(through), &ls) + seriesRef := p.At() + fp, stats, err := i.reader.ChunkStats(seriesRef, int64(from), int64(through), &ls) if err != nil { - return fmt.Errorf("stats: chunk stats: %w", err) + return fmt.Errorf("stats: chunk stats: %w, seriesRef: %d", err, seriesRef) } // skip series that belong to different shards diff --git a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go index 68e3164329b1c..647d2b1fa94ac 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go @@ -148,7 +148,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec // We never have to access this method at the far end of the byte slice. Thus just checking // against the MaxVarintLen32 is sufficient. if bs.Len() < off+binary.MaxVarintLen32 { - return Decbuf{E: ErrInvalidSize} + return Decbuf{E: fmt.Errorf("bs.Len() < off+binary.MaxVarintLen32: %w", ErrInvalidSize)} } b := bs.Range(off, off+binary.MaxVarintLen32) @@ -158,7 +158,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec } if bs.Len() < off+n+int(l)+4 { - return Decbuf{E: ErrInvalidSize} + return Decbuf{E: fmt.Errorf("bs.Len() < off+n+int(l)+4: %w", ErrInvalidSize)} } // Load bytes holding the contents plus a CRC32 checksum. @@ -166,7 +166,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec dec := Decbuf{B: b[:len(b)-4]} if dec.Crc32(castagnoliTable) != binary.BigEndian.Uint32(b[len(b)-4:]) { - return Decbuf{E: ErrInvalidChecksum} + return Decbuf{E: fmt.Errorf("dec.Crc32() != binary.BigEndian: %w", ErrInvalidChecksum)} } return dec } From fb9b5413a74f2a4e7b0606f9d4d4bd4e94fe89f0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 16:33:28 -0300 Subject: [PATCH 52/84] another sanity check --- pkg/storage/stores/tsdb/cached_postings_index.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index af1158402d39e..9a8695eba1577 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -43,10 +43,10 @@ type cachedPostingsReader struct { func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) - if postings, got := c.fetchPostings(ctx, key); got { + if _, got := c.fetchPostings(ctx, key); got { // call PostingsForMatchers just to populate things. - PostingsForMatchers(c.reader, nil, matchers...) - return fn(postings) + p, _ := PostingsForMatchers(c.reader, nil, matchers...) + return fn(p) } p, err := PostingsForMatchers(c.reader, nil, matchers...) From d3a2a5bcb4fc9679c225860e3a97c04e8b3743cd Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 19:23:24 -0300 Subject: [PATCH 53/84] debug decoded/encoded series --- .../stores/tsdb/cached_postings_index.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 9a8695eba1577..f885b6f1fdb03 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -5,6 +5,7 @@ import ( "encoding/binary" "fmt" "sort" + "strconv" "strings" "github.com/go-kit/log" @@ -17,6 +18,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/encoding" + util_log "github.com/grafana/loki/pkg/util/log" ) type PostingsReader interface { @@ -46,6 +48,7 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe if _, got := c.fetchPostings(ctx, key); got { // call PostingsForMatchers just to populate things. p, _ := PostingsForMatchers(c.reader, nil, matchers...) + return fn(p) } @@ -71,6 +74,7 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe // It doesn't add any header to the output bytes. // Length argument is expected number of postings, used for preallocating buffer. func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) { + buf := encoding.Encbuf{} buf.PutUvarint32(uint32(length)) @@ -82,6 +86,7 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) buf.PutUvarint32(uint32(length)) // first we put the postings length so we can use it when decoding. + refsStr := strings.Builder{} prev := storage.SeriesRef(0) for _, ref := range p { if ref < prev { @@ -91,8 +96,14 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) // This is the 'diff' part -- compute difference from previous value. buf.PutUvarint32(uint32(ref - prev)) prev = ref + + nitString := strconv.Itoa(int(ref)) + refsStr.WriteString(nitString) + refsStr.WriteString(",") } + level.Debug(util_log.Logger).Log("msg", "series to be encoded", "refs", refsStr.String()) + return buf.B, nil } @@ -104,13 +115,20 @@ func decodeToPostings(b []byte) index.Postings { postingsLen := decoder.Uvarint32() refs := make([]storage.SeriesRef, 0, postingsLen) prev := storage.SeriesRef(0) + refsStr := strings.Builder{} for i := 0; i < int(postingsLen); i++ { v := storage.SeriesRef(decoder.Uvarint32()) refs = append(refs, v+prev) prev = v + + nitString := strconv.Itoa(int(v + prev)) + refsStr.WriteString(nitString) + refsStr.WriteString(",") } + level.Debug(util_log.Logger).Log("msg", "refs from postings", "refs", refsStr.String()) + return index.NewListPostings(refs) } From cd0d3a905de7c583c0011db6d9dff7ff6752c68d Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 20:05:18 -0300 Subject: [PATCH 54/84] was my decoding wrong? --- .../stores/tsdb/cached_postings_index.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index f885b6f1fdb03..058c9783f7a16 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -73,7 +73,7 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe // diffVarintEncodeNoHeader encodes postings into diff+varint representation. // It doesn't add any header to the output bytes. // Length argument is expected number of postings, used for preallocating buffer. -func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) { +func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int, k string) ([]byte, error) { buf := encoding.Encbuf{} buf.PutUvarint32(uint32(length)) @@ -102,12 +102,12 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int) ([]byte, error) refsStr.WriteString(",") } - level.Debug(util_log.Logger).Log("msg", "series to be encoded", "refs", refsStr.String()) + level.Warn(util_log.Logger).Log("msg", "series to be encoded", "key", k, "refs", refsStr.String()) return buf.B, nil } -func decodeToPostings(b []byte) index.Postings { +func decodeToPostings(b []byte, k string) index.Postings { if len(b) <= 0 { return index.EmptyPostings() } @@ -118,22 +118,22 @@ func decodeToPostings(b []byte) index.Postings { refsStr := strings.Builder{} for i := 0; i < int(postingsLen); i++ { - v := storage.SeriesRef(decoder.Uvarint32()) - refs = append(refs, v+prev) + v := storage.SeriesRef(decoder.Uvarint32()) + prev + refs = append(refs, v) prev = v - nitString := strconv.Itoa(int(v + prev)) + nitString := strconv.Itoa(int(v)) refsStr.WriteString(nitString) refsStr.WriteString(",") } - level.Debug(util_log.Logger).Log("msg", "refs from postings", "refs", refsStr.String()) + level.Warn(util_log.Logger).Log("msg", "refs from postings", "key", k, "refs", refsStr.String()) return index.NewListPostings(refs) } func (c *cachedPostingsReader) storePostings(ctx context.Context, expandedPostings []storage.SeriesRef, canonicalMatchers string) error { - dataToCache, err := diffVarintEncodeNoHeader(expandedPostings, len(expandedPostings)) + dataToCache, err := diffVarintEncodeNoHeader(expandedPostings, len(expandedPostings), canonicalMatchers) if err != nil { level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) } @@ -150,7 +150,7 @@ func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (i } if len(found) > 0 { - return decodeToPostings(bufs[0]), true + return decodeToPostings(bufs[0], key), true } return nil, false From f7d8013240b509a63fa6baa46b9efdc5e00146f3 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Mon, 26 Jun 2023 20:16:47 -0300 Subject: [PATCH 55/84] cleanup --- pkg/storage/stores/tsdb/cached_postings_index.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 058c9783f7a16..94496550cea4f 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -18,7 +18,6 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/encoding" - util_log "github.com/grafana/loki/pkg/util/log" ) type PostingsReader interface { @@ -45,11 +44,11 @@ type cachedPostingsReader struct { func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) - if _, got := c.fetchPostings(ctx, key); got { + if postings, got := c.fetchPostings(ctx, key); got { // call PostingsForMatchers just to populate things. - p, _ := PostingsForMatchers(c.reader, nil, matchers...) + // p, _ := PostingsForMatchers(c.reader, nil, matchers...) - return fn(p) + return fn(postings) } p, err := PostingsForMatchers(c.reader, nil, matchers...) @@ -102,7 +101,7 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int, k string) ([]by refsStr.WriteString(",") } - level.Warn(util_log.Logger).Log("msg", "series to be encoded", "key", k, "refs", refsStr.String()) + // level.Warn(util_log.Logger).Log("msg", "series to be encoded", "key", k, "refs", refsStr.String()) return buf.B, nil } @@ -127,7 +126,7 @@ func decodeToPostings(b []byte, k string) index.Postings { refsStr.WriteString(",") } - level.Warn(util_log.Logger).Log("msg", "refs from postings", "key", k, "refs", refsStr.String()) + // level.Warn(util_log.Logger).Log("msg", "refs from postings", "key", k, "refs", refsStr.String()) return index.NewListPostings(refs) } From fe659c8b08865681cd09652837d6dbd43de5328d Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 27 Jun 2023 10:49:57 -0300 Subject: [PATCH 56/84] cleanup cached postings file. --- .../stores/tsdb/cached_postings_index.go | 39 ++++++------------- 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 94496550cea4f..6a0fecc3dec0d 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -5,7 +5,6 @@ import ( "encoding/binary" "fmt" "sort" - "strconv" "strings" "github.com/go-kit/log" @@ -45,34 +44,32 @@ type cachedPostingsReader struct { func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { key := CanonicalLabelMatchersKey(matchers) if postings, got := c.fetchPostings(ctx, key); got { - // call PostingsForMatchers just to populate things. - // p, _ := PostingsForMatchers(c.reader, nil, matchers...) - return fn(postings) } p, err := PostingsForMatchers(c.reader, nil, matchers...) if err != nil { - return fmt.Errorf("cached postings reader for postings: %w", err) + return fmt.Errorf("failed to evaluate postings for matchers: %w", err) } expandedPosts, err := index.ExpandPostings(p) if err != nil { - return fmt.Errorf("expanded postings: %w", err) + return fmt.Errorf("failed to expand postings: %w", err) } if err := c.storePostings(ctx, expandedPosts, key); err != nil { level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } - // because `index.ExpandedPostings` walks with the iterator, so we have to reset it by instantiating a new ListPostings. + // because `index.ExpandedPostings` walks with the iterator, we have to reset it current index by instantiating a new ListPostings. return fn(index.NewListPostings(expandedPosts)) } // diffVarintEncodeNoHeader encodes postings into diff+varint representation. // It doesn't add any header to the output bytes. // Length argument is expected number of postings, used for preallocating buffer. -func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int, k string) ([]byte, error) { +func diffVarintEncodeNoHeader(p []storage.SeriesRef) ([]byte, error) { + length := len(p) buf := encoding.Encbuf{} buf.PutUvarint32(uint32(length)) @@ -83,9 +80,8 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int, k string) ([]by buf.B = make([]byte, 0, binary.MaxVarintLen32+5*length/4) } - buf.PutUvarint32(uint32(length)) // first we put the postings length so we can use it when decoding. + buf.PutUvarint32(uint32(length)) // first we put the postings length used when decoding. - refsStr := strings.Builder{} prev := storage.SeriesRef(0) for _, ref := range p { if ref < prev { @@ -95,44 +91,32 @@ func diffVarintEncodeNoHeader(p []storage.SeriesRef, length int, k string) ([]by // This is the 'diff' part -- compute difference from previous value. buf.PutUvarint32(uint32(ref - prev)) prev = ref - - nitString := strconv.Itoa(int(ref)) - refsStr.WriteString(nitString) - refsStr.WriteString(",") } - // level.Warn(util_log.Logger).Log("msg", "series to be encoded", "key", k, "refs", refsStr.String()) - - return buf.B, nil + return buf.Get(), nil } -func decodeToPostings(b []byte, k string) index.Postings { +func decodeToPostings(b []byte) index.Postings { if len(b) <= 0 { return index.EmptyPostings() } + decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) postingsLen := decoder.Uvarint32() refs := make([]storage.SeriesRef, 0, postingsLen) prev := storage.SeriesRef(0) - refsStr := strings.Builder{} for i := 0; i < int(postingsLen); i++ { v := storage.SeriesRef(decoder.Uvarint32()) + prev refs = append(refs, v) prev = v - - nitString := strconv.Itoa(int(v)) - refsStr.WriteString(nitString) - refsStr.WriteString(",") } - // level.Warn(util_log.Logger).Log("msg", "refs from postings", "key", k, "refs", refsStr.String()) - return index.NewListPostings(refs) } func (c *cachedPostingsReader) storePostings(ctx context.Context, expandedPostings []storage.SeriesRef, canonicalMatchers string) error { - dataToCache, err := diffVarintEncodeNoHeader(expandedPostings, len(expandedPostings), canonicalMatchers) + dataToCache, err := diffVarintEncodeNoHeader(expandedPostings) if err != nil { level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) } @@ -149,7 +133,8 @@ func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (i } if len(found) > 0 { - return decodeToPostings(bufs[0], key), true + // we only use a single key so we only care about index=0. + return decodeToPostings(bufs[0]), true } return nil, false From f07228dc502be2a5a1b025b39fb4478cd50475a5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 27 Jun 2023 10:51:50 -0300 Subject: [PATCH 57/84] revert vendor changes --- .../prometheus/tsdb/encoding/encoding.go | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go index 647d2b1fa94ac..ab97876a362ef 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/encoding/encoding.go @@ -15,7 +15,6 @@ package encoding import ( "encoding/binary" - "fmt" "hash" "hash/crc32" "math" @@ -148,7 +147,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec // We never have to access this method at the far end of the byte slice. Thus just checking // against the MaxVarintLen32 is sufficient. if bs.Len() < off+binary.MaxVarintLen32 { - return Decbuf{E: fmt.Errorf("bs.Len() < off+binary.MaxVarintLen32: %w", ErrInvalidSize)} + return Decbuf{E: ErrInvalidSize} } b := bs.Range(off, off+binary.MaxVarintLen32) @@ -158,7 +157,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec } if bs.Len() < off+n+int(l)+4 { - return Decbuf{E: fmt.Errorf("bs.Len() < off+n+int(l)+4: %w", ErrInvalidSize)} + return Decbuf{E: ErrInvalidSize} } // Load bytes holding the contents plus a CRC32 checksum. @@ -166,7 +165,7 @@ func NewDecbufUvarintAt(bs ByteSlice, off int, castagnoliTable *crc32.Table) Dec dec := Decbuf{B: b[:len(b)-4]} if dec.Crc32(castagnoliTable) != binary.BigEndian.Uint32(b[len(b)-4:]) { - return Decbuf{E: fmt.Errorf("dec.Crc32() != binary.BigEndian: %w", ErrInvalidChecksum)} + return Decbuf{E: ErrInvalidChecksum} } return dec } @@ -242,7 +241,7 @@ func (d *Decbuf) Uvarint64() uint64 { } x, n := varint.Uvarint(d.B) if n < 1 { - d.E = fmt.Errorf("Uvarint64: %w", ErrInvalidSize) + d.E = ErrInvalidSize return 0 } d.B = d.B[n:] @@ -304,13 +303,7 @@ func (d *Decbuf) ConsumePadding() { } } -func (d *Decbuf) Err() error { - if d.E != nil { - return fmt.Errorf("decbuf err: %w", d.E) - } - return nil -} - +func (d *Decbuf) Err() error { return d.E } func (d *Decbuf) Len() int { return len(d.B) } func (d *Decbuf) Get() []byte { return d.B } From 2f00fb7e77984dd165fbd6880c01ab84b8d49ce8 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 27 Jun 2023 11:08:59 -0300 Subject: [PATCH 58/84] Undo change to error messages --- pkg/storage/stores/tsdb/head_manager.go | 2 +- pkg/storage/stores/tsdb/index/index.go | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 135fb573f0f9a..fe1d8baf95a5d 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -731,7 +731,7 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } reader := tenant.indexRange(int64(from), int64(through)) - idx = NewTSDBIndex(reader, &simplePostingsReader{reader: reader}) + idx = NewTSDBIndex(reader, NewPostingsReader(reader)) if t.chunkFilter != nil { idx.SetChunkFilterer(t.chunkFilter) } diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index b7c97f3702b7e..b6e2c749cf41d 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -1453,7 +1453,7 @@ func (r *Reader) PostingsRanges() (map[labels.Label]Range, error) { } d := encoding.DecWrap(tsdb_enc.NewDecbufAt(r.b, int(off), castagnoliTable)) if d.Err() != nil { - return errors.Wrap(d.Err(), "postings ranges") + return d.Err() } m[labels.Label{Name: key[0], Value: key[1]}] = Range{ Start: int64(off) + 4, @@ -1499,7 +1499,7 @@ func NewSymbols(bs ByteSlice, version, off int) (*Symbols, error) { s.seen++ } if d.Err() != nil { - return nil, errors.Wrap(d.Err(), "new symbols") + return nil, d.Err() } return s, nil } @@ -1599,7 +1599,7 @@ func (s *symbolsIter) Next() bool { s.cur = yoloString(s.d.UvarintBytes()) s.cnt-- if s.d.Err() != nil { - s.err = errors.Wrap(s.d.Err(), "symbols iter err") + s.err = s.d.Err() return false } return true @@ -1635,7 +1635,7 @@ func ReadOffsetTable(bs ByteSlice, off uint64, f func([]string, uint64, int) err } cnt-- } - return errors.Wrap(d.Err(), "read offset table") + return d.Err() } func readFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets, error) { @@ -1648,7 +1648,7 @@ func readFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets, cnt-- } - return res, errors.Wrap(d.Err(), "read fingerprint offsets table") + return res, d.Err() } @@ -1827,7 +1827,7 @@ func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *l } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) if d.Err() != nil { - return 0, errors.Wrap(d.Err(), "series") + return 0, d.Err() } fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks) @@ -1846,7 +1846,7 @@ func (r *Reader) ChunkStats(id storage.SeriesRef, from, through int64, lbls *lab } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) if d.Err() != nil { - return 0, ChunkStats{}, errors.Wrap(d.Err(), "reader chunk stats") + return 0, ChunkStats{}, d.Err() } return r.dec.ChunkStats(r.version, d.Get(), id, from, through, lbls) From 53162c83a04c08c28d82ebe8dcf67c337304c369 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 27 Jun 2023 11:22:44 -0300 Subject: [PATCH 59/84] Add flag docs --- docs/sources/configuration/_index.md | 4 +++- pkg/storage/stores/tsdb/index.go | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/sources/configuration/_index.md b/docs/sources/configuration/_index.md index d90c9adb8482b..c3031b7f9b565 100644 --- a/docs/sources/configuration/_index.md +++ b/docs/sources/configuration/_index.md @@ -2028,7 +2028,9 @@ tsdb_shipper: [ingesterdbretainperiod: ] - [cache_postings: ] + # Experimental. Whether TSDB should cache postings or not. + # CLI flag: -tsdb.cache-postings + [cache_postings: | default = false] ``` ### chunk_store_config diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 049eb5cfd9f34..0601fe0ece7fe 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -27,11 +27,11 @@ type ChunkRef struct { type IndexCfg struct { indexshipper.Config `yaml:",inline"` - CachePostings bool `yaml:"cache_postings"` + CachePostings bool `yaml:"cache_postings" category:"experimental"` } func (cfg *IndexCfg) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.BoolVar(&cfg.CachePostings, prefix+"cache-postings", false, "Whether TSDB should cache postings or not.") + f.BoolVar(&cfg.CachePostings, prefix+"cache-postings", false, "Experimental. Whether TSDB should cache postings or not.") cfg.Config.RegisterFlagsWithPrefix(prefix, f) } From 99a4c834df0e5bf3922db24977b192a9d21001c0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 27 Jun 2023 11:54:47 -0300 Subject: [PATCH 60/84] remove unnecessary test --- .../stores/tsdb/cached_postings_index_test.go | 55 ------------------- 1 file changed, 55 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index ee46b4530097d..82dc1dc84d85d 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -386,61 +386,6 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { } } -func TestBla(t *testing.T) { - runTSDBIndexCache(t) - defer sharedCacheClient.Stop() - series := []LoadableSeries{ - { - Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), - Chunks: []index.ChunkMeta{ - { - MinTime: 0, - MaxTime: 10, - Checksum: 1, - Entries: 10, - KB: 10, - }, - { - MinTime: 10, - MaxTime: 20, - Checksum: 2, - Entries: 20, - KB: 20, - }, - }, - }, - { - Labels: mustParseLabels(`{foo="bar", ping="pong"}`), - Chunks: []index.ChunkMeta{ - { - MinTime: 0, - MaxTime: 10, - Checksum: 3, - Entries: 30, - KB: 30, - }, - { - MinTime: 10, - MaxTime: 20, - Checksum: 4, - Entries: 40, - KB: 40, - }, - }, - }, - } - tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) - acc := &stats.Stats{} - acc2 := []ChunkRef{} - - tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck - tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck - tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 15, acc2, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck - tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 15, acc2, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) -} - func BenchmarkSeriesRepetitive(b *testing.B) { runTSDBIndexCache(b) defer sharedCacheClient.Stop() From 8eb1308c257fd80495fd18beb3d183602f4c3b98 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 28 Jun 2023 13:56:20 -0300 Subject: [PATCH 61/84] Use the checksum as part of the key. --- .../stores/tsdb/cached_postings_index.go | 3 +- .../stores/tsdb/cached_postings_index_test.go | 60 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 6a0fecc3dec0d..6b75e4d6a2119 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -42,7 +42,8 @@ type cachedPostingsReader struct { } func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { - key := CanonicalLabelMatchersKey(matchers) + checksum := c.reader.Checksum() + key := fmt.Sprintf("%s:%d", CanonicalLabelMatchersKey(matchers), checksum) if postings, got := c.fetchPostings(ctx, key); got { return fn(postings) } diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index 82dc1dc84d85d..a54d7e6d3e17c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -437,3 +437,63 @@ func BenchmarkSeriesRepetitive(b *testing.B) { tsdbIndex.Stats(context.Background(), "fake", 5, 15, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck } } + +func TestMultipleIndexesFiles(t *testing.T) { + runTSDBIndexCache(t) + defer sharedCacheClient.Stop() + series := []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 1, + Entries: 10, + KB: 10, + }, + }, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: []index.ChunkMeta{ + { + MinTime: 0, + MaxTime: 10, + Checksum: 3, + Entries: 30, + KB: 30, + }, + }, + }, + } + tempDir := t.TempDir() + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + + refs, err := tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + require.NoError(t, err) + require.Len(t, refs, 2) + + // repeat the same index, it hits the cache. + refs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + require.NoError(t, err) + require.Len(t, refs, 2) + + // completely change the index now + series = []LoadableSeries{ + { + Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), + Chunks: []index.ChunkMeta{}, + }, + { + Labels: mustParseLabels(`{foo="bar", ping="pong"}`), + Chunks: []index.ChunkMeta{}, + }, + } + + tempDir = t.TempDir() + tsdbIndex = BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + refs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck + require.NoError(t, err) + require.Len(t, refs, 0) +} From c3736af5e82a631d8164e982a16a1f5c7606d596 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sun, 2 Jul 2023 14:38:41 -0300 Subject: [PATCH 62/84] Add changelog entry. --- CHANGELOG.md | 1 + pkg/storage/stores/tsdb/cached_postings_index.go | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bb718737b1084..235c3ef01cd60 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ ##### Enhancements +* [9621](https://github.com/grafana/loki/pull/9621) **DylanGuedes**: Introduces LRU cache and TSDB postings cache. * [9710](https://github.com/grafana/loki/pull/9710) **chaudum**: Add shuffle sharding to index gateway * [9573](https://github.com/grafana/loki/pull/9573) **CCOLLOT**: Lambda-Promtail: Add support for AWS CloudFront log ingestion. * [9497](https://github.com/grafana/loki/pull/9497) **CCOLLOT**: Lambda-Promtail: Add support for AWS CloudTrail log ingestion. diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 6b75e4d6a2119..7de15d885e8c1 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -25,6 +25,16 @@ type PostingsReader interface { var sharedCacheClient cache.Cache +// NewCachedPostingsReader uses the cache defined by `index_read_cache` to store and read Postings. +// +// The cache key is stored/read as `matchers:reader_checksum`. +// +// The cache value is stored as: `[n, refs...]`, where n is how many series references this entry has, and refs is +// a sequence of series references encoded as the diff between the current series and the previous one. +// +// Example: if the postings for stream "app=kubernetes,env=production" is `[1,7,30,50]` and its reader has `checksum=12345`: +// - The cache key for the entry will be: `app=kubernetes,env=production:12345` +// - The cache value for the entry will be: [4, 1, 6, 23, 20]. func NewCachedPostingsReader(reader IndexReader, logger log.Logger, cacheClient cache.Cache) PostingsReader { return &cachedPostingsReader{ reader: reader, From 8f556ed49c2233c58d8ccb41f1b668c31f1257fd Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sun, 2 Jul 2023 16:34:38 -0300 Subject: [PATCH 63/84] Add functional test. --- integration/loki_micro_services_test.go | 118 ++++++++++++++++++++++++ 1 file changed, 118 insertions(+) diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 491ef04c9bf46..581ecab7d3ad2 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -566,3 +566,121 @@ func TestSchedulerRing(t *testing.T) { assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines) }) } + +func TestQueryTSDB_WithCachedPostings(t *testing.T) { + clu := cluster.New(nil, cluster.SchemaWithTSDB) + + defer func() { + assert.NoError(t, clu.Cleanup()) + }() + + var ( + tDistributor = clu.AddComponent( + "distributor", + "-target=distributor", + ) + tIndexGateway = clu.AddComponent( + "index-gateway", + "-target=index-gateway", + "-store.index-cache-read.lrucache.enabled=true", + "-tsdb.cache-postings=true", + ) + ) + require.NoError(t, clu.Run()) + + var ( + tIngester = clu.AddComponent( + "ingester", + "-target=ingester", + "-ingester.flush-on-shutdown=true", + "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + tQueryScheduler = clu.AddComponent( + "query-scheduler", + "-target=query-scheduler", + "-query-scheduler.use-scheduler-ring=false", + "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + tCompactor = clu.AddComponent( + "compactor", + "-target=compactor", + "-boltdb.shipper.compactor.compaction-interval=1s", + "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + ) + require.NoError(t, clu.Run()) + + // finally, run the query-frontend and querier. + var ( + tQueryFrontend = clu.AddComponent( + "query-frontend", + "-target=query-frontend", + "-frontend.scheduler-address="+tQueryScheduler.GRPCURL(), + "-frontend.default-validity=0s", + "-common.compactor-address="+tCompactor.HTTPURL(), + "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + _ = clu.AddComponent( + "querier", + "-target=querier", + "-querier.scheduler-address="+tQueryScheduler.GRPCURL(), + "-common.compactor-address="+tCompactor.HTTPURL(), + "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), + ) + ) + require.NoError(t, clu.Run()) + + tenantID := randStringRunes() + + now := time.Now() + cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL()) + cliDistributor.Now = now + cliIngester := client.New(tenantID, "", tIngester.HTTPURL()) + cliIngester.Now = now + cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL()) + cliQueryFrontend.Now = now + + // end of setup. Ingestion and querying below. + + t.Run("ingest-logs", func(t *testing.T) { + require.NoError(t, cliDistributor.PushLogLineWithTimestamp("lineA", time.Now().Add(-72*time.Hour), map[string]string{"job": "fake"})) + require.NoError(t, cliDistributor.PushLogLineWithTimestamp("lineB", time.Now().Add(-48*time.Hour), map[string]string{"job": "fake"})) + }) + + // restart ingester which should flush the chunks and index + require.NoError(t, tIngester.Restart()) + + // Query lines + t.Run("query to verify logs being served from storage", func(t *testing.T) { + resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`) + require.NoError(t, err) + assert.Equal(t, "streams", resp.Data.ResultType) + + var lines []string + for _, stream := range resp.Data.Stream { + for _, val := range stream.Values { + lines = append(lines, val[1]) + } + } + + assert.ElementsMatch(t, []string{"lineA", "lineB"}, lines) + }) + + // ingest logs with ts=now. + require.NoError(t, cliDistributor.PushLogLine("lineC", map[string]string{"job": "fake"})) + require.NoError(t, cliDistributor.PushLogLine("lineD", map[string]string{"job": "fake"})) + + // default length is 7 days. + resp, err := cliQueryFrontend.RunRangeQuery(context.Background(), `{job="fake"}`) + require.NoError(t, err) + assert.Equal(t, "streams", resp.Data.ResultType) + + var lines []string + for _, stream := range resp.Data.Stream { + for _, val := range stream.Values { + lines = append(lines, val[1]) + } + } + // expect lines from both, ingesters memory and from the store. + assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines) +} From 18da973eb2d68e8c818ce129cac6f5800012b682 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 11 Jul 2023 09:55:56 -0300 Subject: [PATCH 64/84] Change "cache_postings" -> "enable_cache_postings" --- docs/sources/configuration/_index.md | 6 +++--- pkg/storage/chunk/cache/cache.go | 2 +- pkg/storage/chunk/cache/lru_cache.go | 2 +- pkg/storage/stores/tsdb/index.go | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/sources/configuration/_index.md b/docs/sources/configuration/_index.md index ca0f043ab2bc5..32aba798756f8 100644 --- a/docs/sources/configuration/_index.md +++ b/docs/sources/configuration/_index.md @@ -2034,8 +2034,8 @@ tsdb_shipper: [ingesterdbretainperiod: ] # Experimental. Whether TSDB should cache postings or not. - # CLI flag: -tsdb.cache-postings - [cache_postings: | default = false] + # CLI flag: -tsdb.enable-cache-postings + [enable_cache_postings: | default = false] ``` ### chunk_store_config @@ -3958,7 +3958,7 @@ lru_cache: # CLI flag: -.lrucache.max-items [max_items: | default = 50000] - # Whether LRUCache should be enabled or not. + # Whether a LRU cache (last-recently used) should be enabled or not. # CLI flag: -.lrucache.enabled [enabled: | default = false] diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index 4ee4db73d78e2..4bbf457ea2390 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -35,7 +35,7 @@ type Config struct { MemcacheClient MemcachedClientConfig `yaml:"memcached_client"` Redis RedisConfig `yaml:"redis"` EmbeddedCache EmbeddedCacheConfig `yaml:"embedded_cache"` - LRUCache LRUCacheConfig `yaml:"lru_cache"` + LRUCache LRUCacheConfig `yaml:"lru_cache" category:"experimental"` Fifocache FifoCacheConfig `yaml:"fifocache"` // deprecated // This is to name the cache metrics properly. diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 886f68735fa61..c34483d917b59 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -40,7 +40,7 @@ func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f f.IntVar(&cfg.MaxItems, prefix+".max-items", 50000, description+"Maximum items in the cache.") - f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether LRUCache should be enabled or not.") + f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether a LRU cache (last-recently used) should be enabled or not.") } func (cfg *LRUCacheConfig) Validate() error { diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 0601fe0ece7fe..a91d8274fa8d8 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -27,11 +27,11 @@ type ChunkRef struct { type IndexCfg struct { indexshipper.Config `yaml:",inline"` - CachePostings bool `yaml:"cache_postings" category:"experimental"` + CachePostings bool `yaml:"enable_cache_postings" category:"experimental"` } func (cfg *IndexCfg) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.BoolVar(&cfg.CachePostings, prefix+"cache-postings", false, "Experimental. Whether TSDB should cache postings or not.") + f.BoolVar(&cfg.CachePostings, prefix+"enable-cache-postings", false, "Experimental. Whether TSDB should cache postings or not.") cfg.Config.RegisterFlagsWithPrefix(prefix, f) } From 942d45023a5852255436fd0663726bdb2ef42425 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 11 Jul 2023 09:57:11 -0300 Subject: [PATCH 65/84] Apply Haudi suggestion (see https://github.com/grafana/loki/pull/9621#discussion_r1259269384). --- pkg/storage/stores/tsdb/cached_postings_index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 7de15d885e8c1..c0c6ee77533ed 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -72,7 +72,7 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe level.Error(c.log).Log("msg", "failed to cache postings", "err", err, "matchers", key) } - // because `index.ExpandedPostings` walks with the iterator, we have to reset it current index by instantiating a new ListPostings. + // because `index.ExpandPostings` walks the iterator, we have to reset it current index by instantiating a new ListPostings. return fn(index.NewListPostings(expandedPosts)) } From a4979c8fbe92e577f3aff0e057d7fe1da625bf99 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 11 Jul 2023 10:05:45 -0300 Subject: [PATCH 66/84] update flag used by e2e test --- integration/loki_micro_services_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 4dce13db48969..94f655be1e492 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -584,7 +584,7 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { "index-gateway", "-target=index-gateway", "-store.index-cache-read.lrucache.enabled=true", - "-tsdb.cache-postings=true", + "-tsdb.enable-cache-postings=true", ) ) require.NoError(t, clu.Run()) From 3e0c31dc5cbea101c80f8f47712f7ea03cad7dfe Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 11 Jul 2023 18:26:34 -0300 Subject: [PATCH 67/84] Refactor how the caching struct is passed --- pkg/storage/stores/tsdb/cached_postings_index.go | 2 -- .../stores/tsdb/cached_postings_index_test.go | 12 ++++++------ pkg/storage/stores/tsdb/compactor.go | 9 ++++----- pkg/storage/stores/tsdb/index_client_test.go | 12 ++++++------ pkg/storage/stores/tsdb/manager.go | 4 ++-- pkg/storage/stores/tsdb/multi_file_index_test.go | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 15 ++++++++------- pkg/storage/stores/tsdb/single_file_index_test.go | 8 ++++---- pkg/storage/stores/tsdb/store.go | 8 +++++--- tools/tsdb/index-analyzer/main.go | 2 +- tools/tsdb/migrate-versions/main_test.go | 2 +- 11 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index c0c6ee77533ed..0302f319eef51 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -23,8 +23,6 @@ type PostingsReader interface { ForPostings(ctx context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error } -var sharedCacheClient cache.Cache - // NewCachedPostingsReader uses the cache defined by `index_read_cache` to store and read Postings. // // The cache key is stored/read as `matchers:reader_checksum`. diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index a54d7e6d3e17c..d9c52a044d812 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -78,7 +78,7 @@ func TestSingleIdxCached(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases, IndexOpts{UsePostingsCache: true}) + return BuildIndex(t, t.TempDir(), cases, IndexOpts{}) }, }, { @@ -263,7 +263,7 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, IndexOpts{UsePostingsCache: true}) + }, IndexOpts{}) b.ResetTimer() b.ReportAllocs() @@ -377,7 +377,7 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{}) acc := &stats.Stats{} err := tsdbIndex.Stats(context.Background(), "fake", tc.from, tc.through, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Equal(t, tc.expectedErr, err) @@ -430,7 +430,7 @@ func BenchmarkSeriesRepetitive(b *testing.B) { }, } tempDir := b.TempDir() - tsdbIndex := BuildIndex(b, tempDir, series, IndexOpts{UsePostingsCache: true}) + tsdbIndex := BuildIndex(b, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) acc := &stats.Stats{} for i := 0; i < b.N; i++ { @@ -468,7 +468,7 @@ func TestMultipleIndexesFiles(t *testing.T) { }, } tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) refs, err := tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck require.NoError(t, err) @@ -492,7 +492,7 @@ func TestMultipleIndexesFiles(t *testing.T) { } tempDir = t.TempDir() - tsdbIndex = BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: true}) + tsdbIndex = BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) refs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck require.NoError(t, err) require.Len(t, refs, 0) diff --git a/pkg/storage/stores/tsdb/compactor.go b/pkg/storage/stores/tsdb/compactor.go index 746fa8bca66ba..4b19983e870ad 100644 --- a/pkg/storage/stores/tsdb/compactor.go +++ b/pkg/storage/stores/tsdb/compactor.go @@ -36,8 +36,7 @@ func (i indexProcessor) NewTableCompactor(ctx context.Context, commonIndexSet co } func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableName, userID, workingDir string, periodConfig config.PeriodConfig, logger log.Logger) (compactor.CompactedIndex, error) { - opts := IndexOpts{UsePostingsCache: false} - indexFile, err := OpenShippableTSDB(path, opts) + indexFile, err := OpenShippableTSDB(path, IndexOpts{}) if err != nil { return nil, err } @@ -101,7 +100,7 @@ func (t *tableCompactor) CompactTable() error { } downloadPaths[job] = downloadedAt - idx, err := OpenShippableTSDB(downloadedAt, IndexOpts{UsePostingsCache: false}) + idx, err := OpenShippableTSDB(downloadedAt, IndexOpts{}) if err != nil { return err } @@ -219,7 +218,7 @@ func setupBuilder(ctx context.Context, userID string, sourceIndexSet compactor.I } }() - indexFile, err := OpenShippableTSDB(path, IndexOpts{UsePostingsCache: false}) + indexFile, err := OpenShippableTSDB(path, IndexOpts{}) if err != nil { return nil, err } @@ -388,7 +387,7 @@ func (c *compactedIndex) ToIndexFile() (index_shipper.Index, error) { return nil, err } - return NewShippableTSDBFile(id, IndexOpts{UsePostingsCache: false}) + return NewShippableTSDBFile(id, IndexOpts{}) } func getUnsafeBytes(s string) []byte { diff --git a/pkg/storage/stores/tsdb/index_client_test.go b/pkg/storage/stores/tsdb/index_client_test.go index d860826b036ee..973e346c9ea4b 100644 --- a/pkg/storage/stores/tsdb/index_client_test.go +++ b/pkg/storage/stores/tsdb/index_client_test.go @@ -60,7 +60,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99)), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -69,7 +69,7 @@ func BenchmarkIndexClient_Stats(b *testing.B) { Labels: mustParseLabels(`{foo="bar"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99)), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, } @@ -118,7 +118,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -135,7 +135,7 @@ func TestIndexClient_Stats(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, } @@ -246,7 +246,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{fizz="buzz"}`), Chunks: buildChunkMetas(int64(indexStartToday), int64(indexStartToday+99), 10), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, tableRange.PeriodConfig.IndexTables.TableFor(indexStartYesterday): { @@ -263,7 +263,7 @@ func TestIndexClient_SeriesVolume(t *testing.T) { Labels: mustParseLabels(`{ping="pong"}`), Chunks: buildChunkMetas(int64(indexStartYesterday), int64(indexStartYesterday+99), 10), }, - }, IndexOpts{UsePostingsCache: false}), + }, IndexOpts{}), }, } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 9d70a8d846bd0..bbe25d7f73569 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -129,7 +129,7 @@ func (m *tsdbManager) Start() (err error) { indices++ prefixed := NewPrefixedIdentifier(id, filepath.Join(mulitenantDir, bucket), "") - loaded, err := NewShippableTSDBFile(prefixed, IndexOpts{UsePostingsCache: false}) + loaded, err := NewShippableTSDBFile(prefixed, IndexOpts{}) if err != nil { level.Warn(m.log).Log( @@ -221,7 +221,7 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, shipper indexshipper.Ind level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dst.Path(), "duration", time.Since(start)) - loaded, err := NewShippableTSDBFile(dst, IndexOpts{UsePostingsCache: false}) + loaded, err := NewShippableTSDBFile(dst, IndexOpts{}) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/multi_file_index_test.go b/pkg/storage/stores/tsdb/multi_file_index_test.go index 49ae1efff3d38..e82d158bd6dba 100644 --- a/pkg/storage/stores/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/tsdb/multi_file_index_test.go @@ -61,7 +61,7 @@ func TestMultiIndex(t *testing.T) { var indices []Index dir := t.TempDir() for i := 0; i < n; i++ { - indices = append(indices, BuildIndex(t, dir, cases, IndexOpts{UsePostingsCache: false})) + indices = append(indices, BuildIndex(t, dir, cases, IndexOpts{})) } idx := NewMultiIndex(IndexSlice(indices)) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 742f145fbef82..2987da519cd4d 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -16,6 +16,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/chunk/cache" index_shipper "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" util_log "github.com/grafana/loki/pkg/util/log" @@ -27,7 +28,7 @@ var ErrAlreadyOnDesiredVersion = errors.New("tsdb file already on desired versio type GetRawFileReaderFunc func() (io.ReadSeeker, error) type IndexOpts struct { - UsePostingsCache bool + PostingsCache cache.Cache } func OpenShippableTSDB(p string, opts IndexOpts) (index_shipper.Index, error) { @@ -40,7 +41,7 @@ func OpenShippableTSDB(p string, opts IndexOpts) (index_shipper.Index, error) { } func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index_shipper.Index, error) { - opts := IndexOpts{UsePostingsCache: false} + opts := IndexOpts{} indexFile, err := OpenShippableTSDB(path, opts) if err != nil { return nil, err @@ -80,7 +81,7 @@ func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (index if err != nil { return nil, err } - return NewShippableTSDBFile(id, IndexOpts{UsePostingsCache: false}) + return NewShippableTSDBFile(id, IndexOpts{}) } // nolint @@ -135,7 +136,7 @@ func NewTSDBIndexFromFile(location string, opts IndexOpts) (Index, GetRawFileRea return nil, nil, err } - postingsReader := getPostingsReader(reader, opts.UsePostingsCache) + postingsReader := getPostingsReader(reader, opts.PostingsCache) tsdbIdx := NewTSDBIndex(reader, postingsReader) return tsdbIdx, func() (io.ReadSeeker, error) { @@ -143,11 +144,11 @@ func NewTSDBIndexFromFile(location string, opts IndexOpts) (Index, GetRawFileRea }, nil } -func getPostingsReader(reader IndexReader, usePostingsCache bool) PostingsReader { +func getPostingsReader(reader IndexReader, postingsCache cache.Cache) PostingsReader { var pr PostingsReader - if usePostingsCache && sharedCacheClient != nil { - pr = NewCachedPostingsReader(reader, util_log.Logger, sharedCacheClient) + if postingsCache != nil { + pr = NewCachedPostingsReader(reader, util_log.Logger, postingsCache) } if pr == nil { diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index fa8f02a397346..474606d8e4a22 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -73,7 +73,7 @@ func TestSingleIdx(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases, IndexOpts{UsePostingsCache: false}) + return BuildIndex(t, t.TempDir(), cases, IndexOpts{}) }, }, { @@ -256,7 +256,7 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, IndexOpts{UsePostingsCache: true}) + }, IndexOpts{PostingsCache: sharedCacheClient}) b.ResetTimer() b.ReportAllocs() @@ -311,7 +311,7 @@ func TestTSDBIndex_Stats(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: false}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{}) // Create the test cases testCases := []struct { @@ -426,7 +426,7 @@ func TestTSDBIndex_SeriesVolume(t *testing.T) { // Create the TSDB index tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{UsePostingsCache: false}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{}) from := model.TimeFromUnixNano(t1.UnixNano()) through := model.TimeFromUnixNano(t2.UnixNano()) diff --git a/pkg/storage/stores/tsdb/store.go b/pkg/storage/stores/tsdb/store.go index 2fede452b99a2..a70c047ca77b4 100644 --- a/pkg/storage/stores/tsdb/store.go +++ b/pkg/storage/stores/tsdb/store.go @@ -75,11 +75,13 @@ func NewStore( func (s *store) init(name string, indexCfg IndexCfg, schemaCfg config.SchemaConfig, objectClient client.ObjectClient, limits downloads.Limits, tableRange config.TableRange, reg prometheus.Registerer, idxCache cache.Cache) error { - sharedCacheClient = idxCache + var sharedCache cache.Cache + if indexCfg.CachePostings && indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil { + sharedCache = idxCache + } - usePostingsCache := indexCfg.CachePostings && indexCfg.Mode == indexshipper.ModeReadOnly && idxCache != nil openFn := func(p string) (indexshipper_index.Index, error) { - return OpenShippableTSDB(p, IndexOpts{UsePostingsCache: usePostingsCache}) + return OpenShippableTSDB(p, IndexOpts{PostingsCache: sharedCache}) } var err error diff --git a/tools/tsdb/index-analyzer/main.go b/tools/tsdb/index-analyzer/main.go index 4171d29b4a946..b59d1ea22a4b5 100644 --- a/tools/tsdb/index-analyzer/main.go +++ b/tools/tsdb/index-analyzer/main.go @@ -35,7 +35,7 @@ func main() { tableRanges := getIndexStoreTableRanges(config.TSDBType, conf.SchemaConfig.Configs) openFn := func(p string) (indexshipper_index.Index, error) { - return tsdb.OpenShippableTSDB(p, tsdb.IndexOpts{UsePostingsCache: false}) + return tsdb.OpenShippableTSDB(p, tsdb.IndexOpts{}) } shipper, err := indexshipper.NewIndexShipper( diff --git a/tools/tsdb/migrate-versions/main_test.go b/tools/tsdb/migrate-versions/main_test.go index 6da2c92bc7977..5fa80cae526b3 100644 --- a/tools/tsdb/migrate-versions/main_test.go +++ b/tools/tsdb/migrate-versions/main_test.go @@ -88,7 +88,7 @@ func TestMigrateTables(t *testing.T) { require.NoError(t, err) tableName := fmt.Sprintf("%s%d", indexPrefix, i) - idx, err := tsdb.NewShippableTSDBFile(id, tsdb.IndexOpts{UsePostingsCache: false}) + idx, err := tsdb.NewShippableTSDBFile(id, tsdb.IndexOpts{}) require.NoError(t, err) require.NoError(t, uploadFile(idx, indexStorageClient, tableName, userID)) From fd8b411128475b6b5513c11cf262a3f2d0dc0104 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sun, 16 Jul 2023 11:17:45 -0300 Subject: [PATCH 68/84] fix lint. --- .../stores/tsdb/cached_postings_index_test.go | 52 ++++++++++++------- .../stores/tsdb/single_file_index_test.go | 10 +--- 2 files changed, 33 insertions(+), 29 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index d9c52a044d812..b8f285139475c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -18,16 +18,13 @@ import ( "github.com/grafana/loki/pkg/util/flagext" ) -func runTSDBIndexCache(t testing.TB) { +func TestSingleIdxCached(t *testing.T) { + // setup cache. cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") require.NoError(t, e) - sharedCacheClient = c -} + defer c.Stop() -func TestSingleIdxCached(t *testing.T) { - runTSDBIndexCache(t) - defer sharedCacheClient.Stop() cases := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar"}`), @@ -78,7 +75,7 @@ func TestSingleIdxCached(t *testing.T) { { desc: "file", fn: func() Index { - return BuildIndex(t, t.TempDir(), cases, IndexOpts{}) + return BuildIndex(t, t.TempDir(), cases, IndexOpts{PostingsCache: c}) }, }, { @@ -221,8 +218,11 @@ func TestSingleIdxCached(t *testing.T) { } func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { - runTSDBIndexCache(b) - defer sharedCacheClient.Stop() + // setup cache. + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(b, e) + defer c.Stop() now := model.Now() queryFrom, queryThrough := now.Add(3*time.Hour).Add(time.Millisecond), now.Add(5*time.Hour).Add(-time.Millisecond) @@ -263,7 +263,7 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, IndexOpts{}) + }, IndexOpts{PostingsCache: c}) b.ResetTimer() b.ReportAllocs() @@ -275,8 +275,12 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { } func TestCacheableTSDBIndex_Stats(t *testing.T) { - runTSDBIndexCache(t) - defer sharedCacheClient.Stop() + // setup cache. + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(t, e) + defer c.Stop() + series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -377,7 +381,7 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: c}) acc := &stats.Stats{} err := tsdbIndex.Stats(context.Background(), "fake", tc.from, tc.through, acc, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Equal(t, tc.expectedErr, err) @@ -387,8 +391,12 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { } func BenchmarkSeriesRepetitive(b *testing.B) { - runTSDBIndexCache(b) - defer sharedCacheClient.Stop() + // setup cache. + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(b, e) + defer c.Stop() + series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -430,7 +438,7 @@ func BenchmarkSeriesRepetitive(b *testing.B) { }, } tempDir := b.TempDir() - tsdbIndex := BuildIndex(b, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) + tsdbIndex := BuildIndex(b, tempDir, series, IndexOpts{PostingsCache: c}) acc := &stats.Stats{} for i := 0; i < b.N; i++ { @@ -439,8 +447,12 @@ func BenchmarkSeriesRepetitive(b *testing.B) { } func TestMultipleIndexesFiles(t *testing.T) { - runTSDBIndexCache(t) - defer sharedCacheClient.Stop() + // setup cache. + cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} + c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") + require.NoError(t, e) + defer c.Stop() + series := []LoadableSeries{ { Labels: mustParseLabels(`{foo="bar", fizz="buzz"}`), @@ -468,7 +480,7 @@ func TestMultipleIndexesFiles(t *testing.T) { }, } tempDir := t.TempDir() - tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) + tsdbIndex := BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: c}) refs, err := tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck require.NoError(t, err) @@ -492,7 +504,7 @@ func TestMultipleIndexesFiles(t *testing.T) { } tempDir = t.TempDir() - tsdbIndex = BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: sharedCacheClient}) + tsdbIndex = BuildIndex(t, tempDir, series, IndexOpts{PostingsCache: c}) refs, err = tsdbIndex.GetChunkRefs(context.Background(), "fake", 5, 10, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) //nolint:errcheck require.NoError(t, err) require.Len(t, refs, 0) diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 474606d8e4a22..2836f9eccd8f6 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -9,10 +9,8 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/storage/chunk" - "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/pkg/storage/stores/index/stats" - "github.com/grafana/loki/pkg/util/flagext" "github.com/go-kit/log" "github.com/prometheus/common/model" @@ -216,12 +214,6 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { queryFrom, queryThrough := now.Add(3*time.Hour).Add(time.Millisecond), now.Add(5*time.Hour).Add(-time.Millisecond) queryBounds := newBounds(queryFrom, queryThrough) numChunksToMatch := 0 - - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, err := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(b, err) - sharedCacheClient = c - var chunkMetas []index.ChunkMeta // build a chunk for every second with randomized chunk length for from, through := now, now.Add(24*time.Hour); from <= through; from = from.Add(time.Second) { @@ -256,7 +248,7 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { Labels: mustParseLabels(`{foo1="bar1", ping="pong"}`), Chunks: chunkMetas, }, - }, IndexOpts{PostingsCache: sharedCacheClient}) + }, IndexOpts{}) b.ResetTimer() b.ReportAllocs() From 871e3e4493f28af026a9f11dbdf284322c5d146b Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Sun, 16 Jul 2023 11:25:05 -0300 Subject: [PATCH 69/84] Use background writes for LRU cache. --- pkg/storage/chunk/cache/cache.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index 4bbf457ea2390..8aa21f01ab88d 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -148,14 +148,17 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger, cacheType sta } if cfg.LRUCache.Enabled { - cache, err := NewLRUCache(cfg.Prefix+"inmemory-lru-cache", cfg.LRUCache, reg, logger, cacheType) + cacheName := cfg.Prefix + "inmemory-lru-cache" + lruCache, err := NewLRUCache(cacheName, cfg.LRUCache, reg, logger, cacheType) if err != nil { level.Error(logger).Log("msg", "failed to initialize LRU cache", "err", err) return nil, err } - if cache != nil { - caches = append(caches, CollectStats(Instrument(cfg.Prefix+"inmemory-lru-cache", cache, reg))) + if lruCache != nil { + instrumentCache := Instrument(cacheName, lruCache, reg) + backgroundCache := NewBackground(cacheName, cfg.Background, instrumentCache, reg) + caches = append(caches, CollectStats(backgroundCache)) } } From 0c4e1416f99752d5bb7da7cc6052f7149444215f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 18 Jul 2023 09:01:35 -0300 Subject: [PATCH 70/84] Add length=0 bypass. --- pkg/storage/stores/tsdb/cached_postings_index.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 0302f319eef51..7bae454d80c04 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -79,6 +79,9 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe // Length argument is expected number of postings, used for preallocating buffer. func diffVarintEncodeNoHeader(p []storage.SeriesRef) ([]byte, error) { length := len(p) + if length == 0 { + return []byte{0}, nil + } buf := encoding.Encbuf{} buf.PutUvarint32(uint32(length)) @@ -112,6 +115,9 @@ func decodeToPostings(b []byte) index.Postings { decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) postingsLen := decoder.Uvarint32() + if postingsLen == 0 { + return index.EmptyPostings() + } refs := make([]storage.SeriesRef, 0, postingsLen) prev := storage.SeriesRef(0) From 67852d95cd8efb47c5d9c687933f103dacf0e638 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 18 Jul 2023 09:04:22 -0300 Subject: [PATCH 71/84] Change default max item size. --- pkg/storage/chunk/cache/lru_cache.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index c34483d917b59..49b0992e57e6b 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -18,6 +18,8 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" ) +const maxInt = int(^uint(0) >> 1) + type LRUCacheConfig struct { MaxSizeBytes flagext.ByteSize `yaml:"max_size_bytes"` MaxItemSizeBytes flagext.ByteSize `yaml:"max_item_size_bytes"` @@ -38,7 +40,7 @@ func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied. Default: 1GB.") cfg.MaxSizeBytes = flagext.ByteSize(1 * gigabyte) - f.IntVar(&cfg.MaxItems, prefix+".max-items", 50000, description+"Maximum items in the cache.") + f.IntVar(&cfg.MaxItems, prefix+".max-items", maxInt, description+"Maximum items in the cache.") f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether a LRU cache (last-recently used) should be enabled or not.") } From 715273cef22150fb4f0f29ddbdaf60fe3d2501f7 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 18 Jul 2023 09:20:13 -0300 Subject: [PATCH 72/84] Update docs --- docs/sources/configure/_index.md | 11 ++++++----- pkg/storage/chunk/cache/cache.go | 2 +- pkg/storage/chunk/cache/lru_cache.go | 11 ++++------- 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 03cc6e34f3978..d2ec872e46ffb 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -3950,20 +3950,21 @@ embedded_cache: # CLI flag: -.embedded-cache.ttl [ttl: | default = 1h] +# LRU (Last-recently Used) in-memory cache. lru_cache: # Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be - # applied. Default: 1GB. + # applied. # CLI flag: -.lrucache.max-size-bytes - [max_size_bytes: | default = 0B] + [max_size_bytes: | default = 1GB] # Maximum memory size of a single item in the cache. A unit suffix (KB, MB, - # GB) may be applied. Default: 100MB. + # GB) may be applied. # CLI flag: -.lrucache.max-item-size-bytes - [max_item_size_bytes: | default = 0B] + [max_item_size_bytes: | default = 100MB] # Maximum items in the cache. # CLI flag: -.lrucache.max-items - [max_items: | default = 50000] + [max_items: | default = 9223372036854775807] # Whether a LRU cache (last-recently used) should be enabled or not. # CLI flag: -.lrucache.enabled diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index 8aa21f01ab88d..d9de513eac340 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -35,7 +35,7 @@ type Config struct { MemcacheClient MemcachedClientConfig `yaml:"memcached_client"` Redis RedisConfig `yaml:"redis"` EmbeddedCache EmbeddedCacheConfig `yaml:"embedded_cache"` - LRUCache LRUCacheConfig `yaml:"lru_cache" category:"experimental"` + LRUCache LRUCacheConfig `yaml:"lru_cache" category:"experimental" doc:"description=LRU (Last-recently Used) in-memory cache."` Fifocache FifoCacheConfig `yaml:"fifocache"` // deprecated // This is to name the cache metrics properly. diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 49b0992e57e6b..7bd622c20204b 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -31,14 +31,11 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { - megabyte := 1000000 - gigabyte := megabyte * 1000 + cfg.MaxItemSizeBytes.Set("100MB") + f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") - f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied. Default: 100MB.") - cfg.MaxItemSizeBytes = flagext.ByteSize(100 * megabyte) - - f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied. Default: 1GB.") - cfg.MaxSizeBytes = flagext.ByteSize(1 * gigabyte) + cfg.MaxSizeBytes.Set("1GB") + f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") f.IntVar(&cfg.MaxItems, prefix+".max-items", maxInt, description+"Maximum items in the cache.") From d2fc8417db0838d5c50091ea3c535a7292ce92b0 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 18 Jul 2023 16:46:06 -0300 Subject: [PATCH 73/84] lint --- pkg/storage/chunk/cache/lru_cache.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go index 7bd622c20204b..09361ae580d54 100644 --- a/pkg/storage/chunk/cache/lru_cache.go +++ b/pkg/storage/chunk/cache/lru_cache.go @@ -31,10 +31,10 @@ type LRUCacheConfig struct { // RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { - cfg.MaxItemSizeBytes.Set("100MB") + cfg.MaxItemSizeBytes.Set("100MB") //nolint:errcheck f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") - cfg.MaxSizeBytes.Set("1GB") + cfg.MaxSizeBytes.Set("1GB") //nolint:errcheck f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") f.IntVar(&cfg.MaxItems, prefix+".max-items", maxInt, description+"Maximum items in the cache.") From 468914a6fb41df07c952995d8d511afef9694b12 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 20 Jul 2023 10:28:55 -0300 Subject: [PATCH 74/84] Implements snappy postings decoding/encoding --- .../stores/tsdb/cached_postings_index.go | 87 ++------ pkg/storage/stores/tsdb/postings_codec.go | 137 ++++++++++++ .../stores/tsdb/postings_codec_test.go | 205 ++++++++++++++++++ 3 files changed, 365 insertions(+), 64 deletions(-) create mode 100644 pkg/storage/stores/tsdb/postings_codec.go create mode 100644 pkg/storage/stores/tsdb/postings_codec_test.go diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index 7bae454d80c04..da4a701aeae26 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -1,22 +1,23 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/postings_codec.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + package tsdb import ( "context" - "encoding/binary" "fmt" "sort" "strings" "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" - promEncoding "github.com/prometheus/prometheus/tsdb/encoding" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/grafana/loki/pkg/util/encoding" ) type PostingsReader interface { @@ -74,69 +75,13 @@ func (c *cachedPostingsReader) ForPostings(ctx context.Context, matchers []*labe return fn(index.NewListPostings(expandedPosts)) } -// diffVarintEncodeNoHeader encodes postings into diff+varint representation. -// It doesn't add any header to the output bytes. -// Length argument is expected number of postings, used for preallocating buffer. -func diffVarintEncodeNoHeader(p []storage.SeriesRef) ([]byte, error) { - length := len(p) - if length == 0 { - return []byte{0}, nil - } - - buf := encoding.Encbuf{} - buf.PutUvarint32(uint32(length)) - - // This encoding uses around ~1 bytes per posting, but let's use - // conservative 1.25 bytes per posting to avoid extra allocations. - if length > 0 { - buf.B = make([]byte, 0, binary.MaxVarintLen32+5*length/4) - } - - buf.PutUvarint32(uint32(length)) // first we put the postings length used when decoding. - - prev := storage.SeriesRef(0) - for _, ref := range p { - if ref < prev { - return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", ref, prev) - } - - // This is the 'diff' part -- compute difference from previous value. - buf.PutUvarint32(uint32(ref - prev)) - prev = ref - } - - return buf.Get(), nil -} - -func decodeToPostings(b []byte) index.Postings { - if len(b) <= 0 { - return index.EmptyPostings() - } - - decoder := encoding.DecWrap(promEncoding.Decbuf{B: b}) - postingsLen := decoder.Uvarint32() - if postingsLen == 0 { - return index.EmptyPostings() - } - refs := make([]storage.SeriesRef, 0, postingsLen) - prev := storage.SeriesRef(0) - - for i := 0; i < int(postingsLen); i++ { - v := storage.SeriesRef(decoder.Uvarint32()) + prev - refs = append(refs, v) - prev = v - } - - return index.NewListPostings(refs) -} - func (c *cachedPostingsReader) storePostings(ctx context.Context, expandedPostings []storage.SeriesRef, canonicalMatchers string) error { - dataToCache, err := diffVarintEncodeNoHeader(expandedPostings) + buf, err := diffVarintSnappyEncode(index.NewListPostings(expandedPostings), len(expandedPostings)) if err != nil { - level.Warn(c.log).Log("msg", "couldn't encode postings", "err", err, "matchers", canonicalMatchers) + return fmt.Errorf("couldn't encode postings: %w", err) } - return c.cacheClient.Store(ctx, []string{canonicalMatchers}, [][]byte{dataToCache}) + return c.cacheClient.Store(ctx, []string{canonicalMatchers}, [][]byte{buf}) } func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (index.Postings, bool) { @@ -149,12 +94,26 @@ func (c *cachedPostingsReader) fetchPostings(ctx context.Context, key string) (i if len(found) > 0 { // we only use a single key so we only care about index=0. - return decodeToPostings(bufs[0]), true + p, err := decodeToPostings(bufs[0]) + if err != nil { + level.Error(c.log).Log("msg", "failed to fetch postings", "err", err) + return nil, false + } + return p, true } return nil, false } +func decodeToPostings(b []byte) (index.Postings, error) { + p, err := diffVarintSnappyDecode(b) + if err != nil { + return nil, fmt.Errorf("couldn't decode postings: %w", err) + } + + return p, nil +} + // CanonicalLabelMatchersKey creates a canonical version of LabelMatchersKey func CanonicalLabelMatchersKey(ms []*labels.Matcher) string { sorted := make([]labels.Matcher, len(ms)) diff --git a/pkg/storage/stores/tsdb/postings_codec.go b/pkg/storage/stores/tsdb/postings_codec.go new file mode 100644 index 0000000000000..a6517352213ee --- /dev/null +++ b/pkg/storage/stores/tsdb/postings_codec.go @@ -0,0 +1,137 @@ +package tsdb + +import ( + "bytes" + + "github.com/golang/snappy" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/storage" + promEncoding "github.com/prometheus/prometheus/tsdb/encoding" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/util/encoding" +) + +type codec string + +const ( + codecHeaderSnappy codec = "dvs" // As in "diff+varint+snappy". +) + +// isDiffVarintSnappyEncodedPostings returns true, if input looks like it has been encoded by diff+varint+snappy codec. +func isDiffVarintSnappyEncodedPostings(input []byte) bool { + return bytes.HasPrefix(input, []byte(codecHeaderSnappy)) +} + +func diffVarintSnappyDecode(input []byte) (index.Postings, error) { + if !isDiffVarintSnappyEncodedPostings(input) { + return nil, errors.New(string(codecHeaderSnappy) + " header not found") + } + + offset := len(codecHeaderSnappy) + + raw, err := snappy.Decode(nil, input[offset:]) + if err != nil { + return nil, errors.Wrap(err, "snappy decode") + } + + return newDiffVarintPostings(raw), nil +} + +func newDiffVarintPostings(input []byte) *diffVarintPostings { + return &diffVarintPostings{buf: &promEncoding.Decbuf{B: input}} +} + +// diffVarintPostings is an implementation of index.Postings based on diff+varint encoded data. +type diffVarintPostings struct { + buf *promEncoding.Decbuf + cur storage.SeriesRef +} + +func (it *diffVarintPostings) At() storage.SeriesRef { + return it.cur +} + +func (it *diffVarintPostings) Next() bool { + if it.buf.Err() != nil || it.buf.Len() == 0 { + return false + } + + val := it.buf.Uvarint64() + if it.buf.Err() != nil { + return false + } + + it.cur = it.cur + storage.SeriesRef(val) + return true +} + +func (it *diffVarintPostings) Seek(x storage.SeriesRef) bool { + if it.cur >= x { + return true + } + + // We cannot do any search due to how values are stored, + // so we simply advance until we find the right value. + for it.Next() { + if it.At() >= x { + return true + } + } + + return false +} + +func (it *diffVarintPostings) Err() error { + return it.buf.Err() +} + +// diffVarintSnappyEncode encodes postings into diff+varint representation, +// and applies snappy compression on the result. +// Returned byte slice starts with codecHeaderSnappy header. +// Length argument is expected number of postings, used for preallocating buffer. +func diffVarintSnappyEncode(p index.Postings, length int) ([]byte, error) { + buf, err := diffVarintEncodeNoHeader(p, length) + if err != nil { + return nil, err + } + + // Make result buffer large enough to hold our header and compressed block. + result := make([]byte, len(codecHeaderSnappy)+snappy.MaxEncodedLen(len(buf))) + copy(result, codecHeaderSnappy) + + compressed := snappy.Encode(result[len(codecHeaderSnappy):], buf) + + // Slice result buffer based on compressed size. + result = result[:len(codecHeaderSnappy)+len(compressed)] + return result, nil +} + +// diffVarintEncodeNoHeader encodes postings into diff+varint representation. +// It doesn't add any header to the output bytes. +func diffVarintEncodeNoHeader(p index.Postings, length int) ([]byte, error) { + buf := encoding.Encbuf{} + + // This encoding uses around ~1 bytes per posting, but let's use + // conservative 1.25 bytes per posting to avoid extra allocations. + if length > 0 { + buf.B = make([]byte, 0, 5*length/4) + } + + prev := storage.SeriesRef(0) + for p.Next() { + v := p.At() + if v < prev { + return nil, errors.Errorf("postings entries must be in increasing order, current: %d, previous: %d", v, prev) + } + + // This is the 'diff' part -- compute difference from previous value. + buf.PutUvarint64(uint64(v - prev)) + prev = v + } + if p.Err() != nil { + return nil, p.Err() + } + + return buf.B, nil +} diff --git a/pkg/storage/stores/tsdb/postings_codec_test.go b/pkg/storage/stores/tsdb/postings_codec_test.go new file mode 100644 index 0000000000000..3d0f01402fe5d --- /dev/null +++ b/pkg/storage/stores/tsdb/postings_codec_test.go @@ -0,0 +1,205 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/postings_codec_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/storepb/testutil/series.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package tsdb + +import ( + "os" + "testing" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +const ( + // labelLongSuffix is a label with ~50B in size, to emulate real-world high cardinality. + labelLongSuffix = "aaaaaaaaaabbbbbbbbbbccccccccccdddddddddd" +) + +func TestDiffVarintCodec(t *testing.T) { + chunksDir := t.TempDir() + + headOpts := tsdb.DefaultHeadOptions() + headOpts.ChunkDirRoot = chunksDir + headOpts.ChunkRange = 1000 + h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) + assert.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, h.Close()) + assert.NoError(t, os.RemoveAll(chunksDir)) + }) + + idx, err := h.Index() + assert.NoError(t, err) + t.Cleanup(func() { + assert.NoError(t, idx.Close()) + }) + + postingsMap := map[string]index.Postings{ + `n="1"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchEqual, "n", "1"+labelLongSuffix)), + `j="foo"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchEqual, "j", "foo")), + `j!="foo"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchNotEqual, "j", "foo")), + `i=~".*"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchRegexp, "i", ".*")), + `i=~".+"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchRegexp, "i", ".+")), + `i=~"1.+"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchRegexp, "i", "1.+")), + `i=~"^$"'`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchRegexp, "i", "^$")), + `i!=""`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchNotEqual, "i", "")), + `n!="2"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchNotEqual, "n", "2"+labelLongSuffix)), + `i!~"2.*"`: matchPostings(t, idx, labels.MustNewMatcher(labels.MatchNotRegexp, "i", "^2.*$")), + } + + codecs := map[string]struct { + codingFunction func(index.Postings, int) ([]byte, error) + decodingFunction func([]byte) (index.Postings, error) + }{ + "raw": {codingFunction: diffVarintEncodeNoHeader, decodingFunction: func(bytes []byte) (index.Postings, error) { + return newDiffVarintPostings(bytes), nil + }}, + "snappy": {codingFunction: diffVarintSnappyEncode, decodingFunction: diffVarintSnappyDecode}, + } + + for postingName, postings := range postingsMap { + p, err := toUint64Postings(postings) + require.NoError(t, err) + + for cname, codec := range codecs { + name := cname + "/" + postingName + + t.Run(name, func(t *testing.T) { + p.reset() // We reuse postings between runs, so we need to reset iterator. + + data, err := codec.codingFunction(p, p.len()) + require.NoError(t, err) + + t.Log("encoded size", len(data), "bytes") + t.Logf("ratio: %0.3f", float64(len(data))/float64(4*p.len())) + + decodedPostings, err := codec.decodingFunction(data) + require.NoError(t, err) + + p.reset() + comparePostings(t, p, decodedPostings) + }) + } + } +} + +func TestLabelMatchersTypeValues(t *testing.T) { + expectedValues := map[labels.MatchType]int{ + labels.MatchEqual: 0, + labels.MatchNotEqual: 1, + labels.MatchRegexp: 2, + labels.MatchNotRegexp: 3, + } + + for matcherType, val := range expectedValues { + require.Equal(t, int(labels.MustNewMatcher(matcherType, "", "").Type), val, + "diffVarintSnappyWithMatchersEncode relies on the number values of hte matchers not changing. "+ + "It caches each matcher type as these integer values. "+ + "If the integer values change, then the already cached values in the index cache will be improperly decoded.") + } +} + +func comparePostings(t *testing.T, p1, p2 index.Postings) { + for p1.Next() { + require.True(t, p2.Next()) + require.Equal(t, p1.At(), p2.At()) + } + + if p2.Next() { + t.Fatal("p2 has more values") + return + } + + require.NoError(t, p1.Err()) + require.NoError(t, p2.Err()) +} + +func matchPostings(t testing.TB, ix tsdb.IndexReader, m *labels.Matcher) index.Postings { + vals, err := ix.LabelValues(m.Name) + assert.NoError(t, err) + + matching := []string(nil) + for _, v := range vals { + if m.Matches(v) { + matching = append(matching, v) + } + } + + p, err := ix.Postings(m.Name, matching...) + assert.NoError(t, err) + return p +} + +func toUint64Postings(p index.Postings) (*uint64Postings, error) { + var vals []storage.SeriesRef + for p.Next() { + vals = append(vals, p.At()) + } + return &uint64Postings{vals: vals, ix: -1}, p.Err() +} + +// Postings with no decoding step. +type uint64Postings struct { + vals []storage.SeriesRef + ix int +} + +func (p *uint64Postings) At() storage.SeriesRef { + if p.ix < 0 || p.ix >= len(p.vals) { + return 0 + } + return p.vals[p.ix] +} + +func (p *uint64Postings) Next() bool { + if p.ix < len(p.vals)-1 { + p.ix++ + return true + } + return false +} + +func (p *uint64Postings) Seek(x storage.SeriesRef) bool { + if p.At() >= x { + return true + } + + // We cannot do any search due to how values are stored, + // so we simply advance until we find the right value. + for p.Next() { + if p.At() >= x { + return true + } + } + + return false +} + +func (p *uint64Postings) Err() error { + return nil +} + +func (p *uint64Postings) reset() { + p.ix = -1 +} + +func (p *uint64Postings) len() int { + return len(p.vals) +} + +func allPostings(t testing.TB, ix tsdb.IndexReader) index.Postings { + k, v := index.AllPostingsKey() + p, err := ix.Postings(k, v) + assert.NoError(t, err) + return p +} From ab45f97324f87ba6b8cccf34d3725cd7776501bf Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 20 Jul 2023 13:55:21 -0300 Subject: [PATCH 75/84] fix formatting --- pkg/storage/stores/tsdb/postings_codec_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/postings_codec_test.go b/pkg/storage/stores/tsdb/postings_codec_test.go index 3d0f01402fe5d..41f8c8efd107a 100644 --- a/pkg/storage/stores/tsdb/postings_codec_test.go +++ b/pkg/storage/stores/tsdb/postings_codec_test.go @@ -12,12 +12,13 @@ import ( "os" "testing" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) const ( From 61e1bec5062a50e771f624617f6665afd18c33c6 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 26 Jul 2023 17:58:22 -0300 Subject: [PATCH 76/84] Remove LRU cache. --- CHANGELOG.md | 2 +- docs/sources/configure/_index.md | 23 +- pkg/storage/chunk/cache/cache.go | 17 - pkg/storage/chunk/cache/lru_cache.go | 307 ------------------ pkg/storage/chunk/cache/lru_cache_test.go | 141 -------- .../stores/tsdb/cached_postings_index_test.go | 26 +- pkg/storage/stores/tsdb/index.go | 2 +- 7 files changed, 14 insertions(+), 504 deletions(-) delete mode 100644 pkg/storage/chunk/cache/lru_cache.go delete mode 100644 pkg/storage/chunk/cache/lru_cache_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 7b02b948b82d5..1325a684d2b97 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,7 +6,7 @@ ##### Enhancements -* [9621](https://github.com/grafana/loki/pull/9621) **DylanGuedes**: Introduces LRU cache and TSDB postings cache. +* [9621](https://github.com/grafana/loki/pull/9621) **DylanGuedes**: Introduces TSDB postings cache. * [9797](https://github.com/grafana/loki/pull/9797) **chaudum**: Add new `loki_index_gateway_requests_total` counter metric to observe per-tenant RPS * [9710](https://github.com/grafana/loki/pull/9710) **chaudum**: Add shuffle sharding to index gateway * [9573](https://github.com/grafana/loki/pull/9573) **CCOLLOT**: Lambda-Promtail: Add support for AWS CloudFront log ingestion. diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index d2ec872e46ffb..ba1fc88ffe82b 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -2036,7 +2036,8 @@ tsdb_shipper: [ingesterdbretainperiod: ] - # Experimental. Whether TSDB should cache postings or not. + # Experimental. Whether TSDB should cache postings or not. The + # index-read-cache will be used as the backend. # CLI flag: -tsdb.enable-cache-postings [enable_cache_postings: | default = false] ``` @@ -3950,26 +3951,6 @@ embedded_cache: # CLI flag: -.embedded-cache.ttl [ttl: | default = 1h] -# LRU (Last-recently Used) in-memory cache. -lru_cache: - # Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be - # applied. - # CLI flag: -.lrucache.max-size-bytes - [max_size_bytes: | default = 1GB] - - # Maximum memory size of a single item in the cache. A unit suffix (KB, MB, - # GB) may be applied. - # CLI flag: -.lrucache.max-item-size-bytes - [max_item_size_bytes: | default = 100MB] - - # Maximum items in the cache. - # CLI flag: -.lrucache.max-items - [max_items: | default = 9223372036854775807] - - # Whether a LRU cache (last-recently used) should be enabled or not. - # CLI flag: -.lrucache.enabled - [enabled: | default = false] - fifocache: # Maximum memory size of the cache in bytes. A unit suffix (KB, MB, GB) may be # applied. diff --git a/pkg/storage/chunk/cache/cache.go b/pkg/storage/chunk/cache/cache.go index d9de513eac340..d12240c87b091 100644 --- a/pkg/storage/chunk/cache/cache.go +++ b/pkg/storage/chunk/cache/cache.go @@ -35,7 +35,6 @@ type Config struct { MemcacheClient MemcachedClientConfig `yaml:"memcached_client"` Redis RedisConfig `yaml:"redis"` EmbeddedCache EmbeddedCacheConfig `yaml:"embedded_cache"` - LRUCache LRUCacheConfig `yaml:"lru_cache" category:"experimental" doc:"description=LRU (Last-recently Used) in-memory cache."` Fifocache FifoCacheConfig `yaml:"fifocache"` // deprecated // This is to name the cache metrics properly. @@ -57,7 +56,6 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, description string, f cfg.MemcacheClient.RegisterFlagsWithPrefix(prefix, description, f) cfg.Redis.RegisterFlagsWithPrefix(prefix, description, f) cfg.Fifocache.RegisterFlagsWithPrefix(prefix, description, f) - cfg.LRUCache.RegisterFlagsWithPrefix(prefix+"lrucache", description, f) cfg.EmbeddedCache.RegisterFlagsWithPrefix(prefix, description, f) f.IntVar(&cfg.AsyncCacheWriteBackConcurrency, prefix+"max-async-cache-write-back-concurrency", 16, "The maximum number of concurrent asynchronous writeback cache can occur.") f.IntVar(&cfg.AsyncCacheWriteBackBufferSize, prefix+"max-async-cache-write-back-buffer-size", 500, "The maximum number of enqueued asynchronous writeback cache allowed.") @@ -147,21 +145,6 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger, cacheType sta } } - if cfg.LRUCache.Enabled { - cacheName := cfg.Prefix + "inmemory-lru-cache" - lruCache, err := NewLRUCache(cacheName, cfg.LRUCache, reg, logger, cacheType) - if err != nil { - level.Error(logger).Log("msg", "failed to initialize LRU cache", "err", err) - return nil, err - } - - if lruCache != nil { - instrumentCache := Instrument(cacheName, lruCache, reg) - backgroundCache := NewBackground(cacheName, cfg.Background, instrumentCache, reg) - caches = append(caches, CollectStats(backgroundCache)) - } - } - if IsMemcacheSet(cfg) && IsRedisSet(cfg) { return nil, errors.New("use of multiple cache storage systems is not supported") } diff --git a/pkg/storage/chunk/cache/lru_cache.go b/pkg/storage/chunk/cache/lru_cache.go deleted file mode 100644 index 09361ae580d54..0000000000000 --- a/pkg/storage/chunk/cache/lru_cache.go +++ /dev/null @@ -1,307 +0,0 @@ -package cache - -import ( - "context" - "flag" - "fmt" - "sync" - "unsafe" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - lru "github.com/hashicorp/golang-lru/simplelru" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/grafana/loki/pkg/logqlmodel/stats" - "github.com/grafana/loki/pkg/util/flagext" - util_log "github.com/grafana/loki/pkg/util/log" -) - -const maxInt = int(^uint(0) >> 1) - -type LRUCacheConfig struct { - MaxSizeBytes flagext.ByteSize `yaml:"max_size_bytes"` - MaxItemSizeBytes flagext.ByteSize `yaml:"max_item_size_bytes"` - - MaxItems int `yaml:"max_items"` - - Enabled bool `yaml:"enabled"` -} - -// RegisterFlagsWithPrefix adds the flags required to config this to the given FlagSet -func (cfg *LRUCacheConfig) RegisterFlagsWithPrefix(prefix, description string, f *flag.FlagSet) { - cfg.MaxItemSizeBytes.Set("100MB") //nolint:errcheck - f.Var(&cfg.MaxItemSizeBytes, prefix+".max-item-size-bytes", description+"Maximum memory size of a single item in the cache. A unit suffix (KB, MB, GB) may be applied.") - - cfg.MaxSizeBytes.Set("1GB") //nolint:errcheck - f.Var(&cfg.MaxSizeBytes, prefix+".max-size-bytes", description+"Maximum memory size of the whole cache. A unit suffix (KB, MB, GB) may be applied.") - - f.IntVar(&cfg.MaxItems, prefix+".max-items", maxInt, description+"Maximum items in the cache.") - - f.BoolVar(&cfg.Enabled, prefix+".enabled", false, description+"Whether a LRU cache (last-recently used) should be enabled or not.") -} - -func (cfg *LRUCacheConfig) Validate() error { - return nil -} - -type LRUCache struct { - cacheType stats.CacheType - - done chan struct{} - - mtx sync.Mutex - - logger log.Logger - lru *lru.LRU - - maxCacheBytes uint64 - maxItemSizeBytes uint64 - maxItems int - curSize uint64 - - evicted prometheus.Counter - requests prometheus.Counter - hits prometheus.Counter - totalMisses prometheus.Counter - added prometheus.Counter - current prometheus.Gauge - bytesInUse prometheus.Gauge - overflow prometheus.Counter -} - -func NewLRUCache(name string, cfg LRUCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (*LRUCache, error) { - util_log.WarnExperimentalUse(fmt.Sprintf("In-memory (LRU) cache - %s", name), logger) - - c := &LRUCache{ - cacheType: cacheType, - - maxItemSizeBytes: uint64(cfg.MaxItemSizeBytes), - maxCacheBytes: uint64(cfg.MaxSizeBytes), - maxItems: cfg.MaxItems, - - logger: logger, - - done: make(chan struct{}), - } - - c.totalMisses = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "misses_total", - Help: "The total number of Get calls that had no valid entry", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.bytesInUse = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "memory_bytes", - Help: "The current cache size in bytes", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.evicted = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "evicted_total", - Help: "Total number of items that were evicted.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.added = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "added_total", - Help: "Total number of items that were added to the cache.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.requests = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "gets_total", - Help: "Total number of requests to the cache.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.hits = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "hits_total", - Help: "Total number of requests to the cache that were a hit.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.current = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "entries", - Help: "Current number of items in the cache.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - c.overflow = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Namespace: "loki", - Subsystem: "cache", - Name: "overflow", - Help: "Total number of items that could not be added to the cache due to being too big.", - ConstLabels: prometheus.Labels{"cache": name}, - }) - - // Initialize LRU cache with a high size limit since we will manage evictions ourselves - // based on stored size using `RemoveOldest` method. - l, err := lru.NewLRU(c.maxItems, c.onEvict) - if err != nil { - return nil, err - } - c.lru = l - - level.Info(logger).Log( - "msg", "created in-memory LRU cache", - ) - - return c, nil -} - -// Fetch implements Cache. -func (c *LRUCache) Fetch(_ context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) { - found, missing, bufs = make([]string, 0, len(keys)), make([]string, 0, len(keys)), make([][]byte, 0, len(keys)) - for _, key := range keys { - val, ok := c.get(key) - if !ok { - missing = append(missing, key) - continue - } - - found = append(found, key) - bufs = append(bufs, val) - } - return -} - -// Store implements Cache. -func (c *LRUCache) Store(_ context.Context, keys []string, values [][]byte) error { - for i := range keys { - c.set(keys[i], values[i]) - } - - return nil -} - -// Stop implements Cache. -func (c *LRUCache) Stop() { - c.mtx.Lock() - defer c.mtx.Unlock() - - close(c.done) - - c.reset() -} - -func (c *LRUCache) GetCacheType() stats.CacheType { - return c.cacheType -} - -func (c *LRUCache) onEvict(key, val interface{}) { - c.evicted.Inc() - c.current.Dec() - - size := entryMemoryUsage(key.(string), val.([]byte)) - c.bytesInUse.Sub(float64(size)) - c.curSize -= uint64(size) -} - -func (c *LRUCache) get(key string) ([]byte, bool) { - c.requests.Inc() - - c.mtx.Lock() - defer c.mtx.Unlock() - - v, ok := c.lru.Get(key) - if !ok { - c.totalMisses.Inc() - return nil, false - } - c.hits.Inc() - return v.([]byte), true -} - -func (c *LRUCache) set(key string, val []byte) { - c.mtx.Lock() - defer c.mtx.Unlock() - - wasUpdate := false - previousSize := 0 - if v, ok := c.lru.Get(key); ok { - wasUpdate = true - previousSize = entryMemoryUsage(key, v.([]byte)) - c.curSize -= uint64(previousSize) - } - - if !c.ensureFits(key, val) { - c.overflow.Inc() - return - } - - // The caller may be passing in a sub-slice of a huge array. Copy the data - // to ensure we don't waste huge amounts of space for something small. - v := make([]byte, len(val)) - copy(v, val) - c.lru.Add(key, v) - - size := entryMemoryUsage(key, val) - - c.bytesInUse.Add(float64(size - previousSize)) - c.curSize += uint64(size) - - if wasUpdate { - // it was an update - don't update other metrics. - return - } - - c.current.Inc() - c.added.Inc() -} - -func (c *LRUCache) ensureFits(key string, val []byte) bool { - size := entryMemoryUsage(key, val) - if size > int(c.maxItemSizeBytes) { - level.Debug(c.logger).Log( - "msg", "item bigger than maxItemSizeBytes. Ignoring..", - "max_item_size_bytes", c.maxItemSizeBytes, - "max_cache_bytes", c.maxCacheBytes, - "cur_size", c.curSize, - "item_size", size, - ) - return false - } - - for c.curSize+uint64(size) > c.maxCacheBytes { - if _, _, ok := c.lru.RemoveOldest(); !ok { - level.Error(c.logger).Log( - "msg", "LRU has nothing more to evict, but we still cannot allocate the item. Resetting cache.", - "max_item_size_bytes", c.maxItemSizeBytes, - "max_size_bytes", c.maxCacheBytes, - "cur_size", c.curSize, - "item_size", size, - ) - c.reset() - } - } - return true -} - -func entryMemoryUsage(key string, val []byte) int { - if len(val) == 0 { - return 0 - } - return int(unsafe.Sizeof(val)) + len(key) -} - -func (c *LRUCache) reset() { - c.lru.Purge() - c.current.Set(0) - c.bytesInUse.Set(0) -} diff --git a/pkg/storage/chunk/cache/lru_cache_test.go b/pkg/storage/chunk/cache/lru_cache_test.go deleted file mode 100644 index e051b2fe3f02f..0000000000000 --- a/pkg/storage/chunk/cache/lru_cache_test.go +++ /dev/null @@ -1,141 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/store/cache/inmemory_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Thanos Authors. - -// Tests out the LRU cache implementation. -package cache - -import ( - "context" - "fmt" - "testing" - - "github.com/go-kit/log" - "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/util/flagext" -) - -func TestLRUCacheEviction(t *testing.T) { - const ( - cnt = 10 - evicted = 5 - ) - itemTemplate := &cacheEntry{ - key: "00", - value: []byte("00"), - } - entrySize := entryMemoryUsage(itemTemplate.key, itemTemplate.value) - - cfg := LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(entrySize * cnt), MaxItems: cnt, MaxItemSizeBytes: flagext.ByteSize(entrySize + 1), Enabled: true} - - c, err := NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(t, err) - defer c.Stop() - ctx := context.Background() - - // Check put / get works. Put/get 10 different items. - keys := []string{} - values := [][]byte{} - for i := 0; i < cnt; i++ { - key := fmt.Sprintf("%02d", i) - value := make([]byte, len(key)) - copy(value, key) - keys = append(keys, key) - values = append(values, value) - } - require.NoError(t, c.Store(ctx, keys, values)) - require.Equal(t, cnt, c.lru.Len()) - - assert.Equal(t, float64(10), testutil.ToFloat64(c.added), float64(10)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.evicted), float64(0)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current), float64(cnt)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.requests), float64(0)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses), float64(0)) - assert.Equal(t, float64(cnt*entryMemoryUsage(itemTemplate.key, itemTemplate.value)), testutil.ToFloat64(c.bytesInUse)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.overflow)) - - for i := 0; i < cnt; i++ { - key := fmt.Sprintf("%02d", i) - value, ok := c.get(key) - require.True(t, ok) - require.Equal(t, []byte(key), value) - } - - assert.Equal(t, float64(10), testutil.ToFloat64(c.added)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.evicted)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.requests)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses)) - assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) - - // Check evictions - keys = []string{} - values = [][]byte{} - for i := cnt - evicted; i < cnt+evicted; i++ { - key := fmt.Sprintf("%02d", i) - value := make([]byte, len(key)) - copy(value, key) - keys = append(keys, key) - values = append(values, value) - } - require.NoError(t, c.Store(ctx, keys, values)) - require.Equal(t, cnt, c.lru.Len()) - - assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) - assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.requests)) - assert.Equal(t, float64(0), testutil.ToFloat64(c.totalMisses)) - assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) - - for i := 0; i < cnt-evicted; i++ { - _, ok := c.get(fmt.Sprintf("%02d", i)) - require.False(t, ok) - } - for i := cnt - evicted; i < cnt+evicted; i++ { - key := fmt.Sprintf("%02d", i) - value, ok := c.get(key) - require.True(t, ok) - require.Equal(t, []byte(key), value) - } - - assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) - assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) - assert.Equal(t, float64(cnt*2+evicted), testutil.ToFloat64(c.requests)) - assert.Equal(t, float64(cnt-evicted), testutil.ToFloat64(c.totalMisses)) - assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) - - // Check updates work - keys = []string{} - values = [][]byte{} - for i := cnt; i < cnt+evicted; i++ { - keys = append(keys, fmt.Sprintf("%02d", i)) - vstr := fmt.Sprintf("%02d", i*2) - value := make([]byte, len(vstr)) - copy(value, vstr) - values = append(values, value) - } - assert.Equal(t, cnt, c.lru.Len()) - require.NoError(t, c.Store(ctx, keys, values)) - assert.Equal(t, cnt, c.lru.Len()) - - assert.Equal(t, float64(0), testutil.ToFloat64(c.overflow)) - assert.Equal(t, float64(15), testutil.ToFloat64(c.added)) - assert.Equal(t, float64(evicted), testutil.ToFloat64(c.evicted)) - assert.Equal(t, float64(cnt), testutil.ToFloat64(c.current)) - assert.Equal(t, float64(cnt*2+evicted), testutil.ToFloat64(c.requests)) - assert.Equal(t, float64(cnt-evicted), testutil.ToFloat64(c.totalMisses)) - assert.Equal(t, float64(cnt*entrySize), testutil.ToFloat64(c.bytesInUse)) - - for i := cnt; i < cnt+evicted; i++ { - value, ok := c.get(fmt.Sprintf("%02d", i)) - require.True(t, ok) - require.Equal(t, []byte(fmt.Sprintf("%02d", i*2)), value) - } - -} diff --git a/pkg/storage/stores/tsdb/cached_postings_index_test.go b/pkg/storage/stores/tsdb/cached_postings_index_test.go index b8f285139475c..18ab43deccdef 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index_test.go +++ b/pkg/storage/stores/tsdb/cached_postings_index_test.go @@ -15,14 +15,12 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/stores/index/stats" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/grafana/loki/pkg/util/flagext" ) func TestSingleIdxCached(t *testing.T) { // setup cache. - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(t, e) + cfg := cache.FifoCacheConfig{MaxSizeBytes: "1MB"} + c := cache.NewFifoCache("test-cache", cfg, nil, log.NewNopLogger(), "test") defer c.Stop() cases := []LoadableSeries{ @@ -219,9 +217,8 @@ func TestSingleIdxCached(t *testing.T) { func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { // setup cache. - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(b, e) + cfg := cache.FifoCacheConfig{MaxSizeBytes: "1MB"} + c := cache.NewFifoCache("test-cache", cfg, nil, log.NewNopLogger(), "test") defer c.Stop() now := model.Now() @@ -276,9 +273,8 @@ func BenchmarkCacheableTSDBIndex_GetChunkRefs(b *testing.B) { func TestCacheableTSDBIndex_Stats(t *testing.T) { // setup cache. - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(t, e) + cfg := cache.FifoCacheConfig{MaxSizeBytes: "1MB"} + c := cache.NewFifoCache("test-cache", cfg, nil, log.NewNopLogger(), "test") defer c.Stop() series := []LoadableSeries{ @@ -392,9 +388,8 @@ func TestCacheableTSDBIndex_Stats(t *testing.T) { func BenchmarkSeriesRepetitive(b *testing.B) { // setup cache. - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(b, e) + cfg := cache.FifoCacheConfig{MaxSizeBytes: "1MB"} + c := cache.NewFifoCache("test-cache", cfg, nil, log.NewNopLogger(), "test") defer c.Stop() series := []LoadableSeries{ @@ -448,9 +443,8 @@ func BenchmarkSeriesRepetitive(b *testing.B) { func TestMultipleIndexesFiles(t *testing.T) { // setup cache. - cfg := cache.LRUCacheConfig{MaxSizeBytes: flagext.ByteSize(100000), MaxItems: 5000, MaxItemSizeBytes: flagext.ByteSize(10000), Enabled: true} - c, e := cache.NewLRUCache("test-cache", cfg, nil, log.NewNopLogger(), "test") - require.NoError(t, e) + cfg := cache.FifoCacheConfig{MaxSizeBytes: "1MB"} + c := cache.NewFifoCache("test-cache", cfg, nil, log.NewNopLogger(), "test") defer c.Stop() series := []LoadableSeries{ diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index a91d8274fa8d8..c7a5155261d56 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -31,7 +31,7 @@ type IndexCfg struct { } func (cfg *IndexCfg) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.BoolVar(&cfg.CachePostings, prefix+"enable-cache-postings", false, "Experimental. Whether TSDB should cache postings or not.") + f.BoolVar(&cfg.CachePostings, prefix+"enable-cache-postings", false, "Experimental. Whether TSDB should cache postings or not. The index-read-cache will be used as the backend.") cfg.Config.RegisterFlagsWithPrefix(prefix, f) } From 2be199b66d7ff89dcc69bae53237047a193d6d74 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 26 Jul 2023 18:16:04 -0300 Subject: [PATCH 77/84] fix microservices test --- integration/loki_micro_services_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 3f08c92af6232..2e9cd252e4442 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -583,7 +583,6 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { tIndexGateway = clu.AddComponent( "index-gateway", "-target=index-gateway", - "-store.index-cache-read.lrucache.enabled=true", "-tsdb.enable-cache-postings=true", ) ) From 07926d549ccb6a006dd65e19f9966ec50518d064 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 27 Jul 2023 14:20:23 -0300 Subject: [PATCH 78/84] Rename enable-postings-cache flag. --- docs/sources/configure/_index.md | 4 ++-- integration/loki_micro_services_test.go | 2 +- pkg/storage/stores/tsdb/index.go | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 49a92bb31d804..0815fd6afd8da 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -2040,8 +2040,8 @@ tsdb_shipper: # Experimental. Whether TSDB should cache postings or not. The # index-read-cache will be used as the backend. - # CLI flag: -tsdb.enable-cache-postings - [enable_cache_postings: | default = false] + # CLI flag: -tsdb.enable-postings-cache + [enable_postings_cache: | default = false] ``` ### chunk_store_config diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 2e9cd252e4442..02684d4fc251c 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -583,7 +583,7 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { tIndexGateway = clu.AddComponent( "index-gateway", "-target=index-gateway", - "-tsdb.enable-cache-postings=true", + "-tsdb.enable-postings-cache=true", ) ) require.NoError(t, clu.Run()) diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 18d58ab18ab9b..74e6259143521 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -27,11 +27,11 @@ type ChunkRef struct { type IndexCfg struct { indexshipper.Config `yaml:",inline"` - CachePostings bool `yaml:"enable_cache_postings" category:"experimental"` + CachePostings bool `yaml:"enable_postings_cache" category:"experimental"` } func (cfg *IndexCfg) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { - f.BoolVar(&cfg.CachePostings, prefix+"enable-cache-postings", false, "Experimental. Whether TSDB should cache postings or not. The index-read-cache will be used as the backend.") + f.BoolVar(&cfg.CachePostings, prefix+"enable-postings-cache", false, "Experimental. Whether TSDB should cache postings or not. The index-read-cache will be used as the backend.") cfg.Config.RegisterFlagsWithPrefix(prefix, f) } From c3e6e8b534fd131ab7d9a6f25b43d0e3c80c12ae Mon Sep 17 00:00:00 2001 From: Dylan Guedes Date: Fri, 28 Jul 2023 15:27:11 -0300 Subject: [PATCH 79/84] Apply suggestions from code review Co-authored-by: Christian Haudum --- CHANGELOG.md | 2 +- pkg/storage/stores/tsdb/single_file_index.go | 13 +++---------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9ee0391072911..607f7e7014208 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,7 +6,7 @@ ##### Enhancements -* [9621](https://github.com/grafana/loki/pull/9621) **DylanGuedes**: Introduces TSDB postings cache. +* [9621](https://github.com/grafana/loki/pull/9621) **DylanGuedes**: Introduce TSDB postings cache. * [10010](https://github.com/grafana/loki/pull/10010) **rasta-rocket**: feat(promtail): retrieve BotTags field from cloudflare * [9995](https://github.com/grafana/loki/pull/9995) **chaudum**: Add jitter to the flush interval to prevent multiple ingesters to flush at the same time. * [9797](https://github.com/grafana/loki/pull/9797) **chaudum**: Add new `loki_index_gateway_requests_total` counter metric to observe per-tenant RPS diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index f51370dcca4bd..94d4a771b087f 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -147,24 +147,17 @@ func NewTSDBIndexFromFile(location string, opts IndexOpts) (Index, GetRawFileRea } func getPostingsReader(reader IndexReader, postingsCache cache.Cache) PostingsReader { - var pr PostingsReader - if postingsCache != nil { - pr = NewCachedPostingsReader(reader, util_log.Logger, postingsCache) - } - - if pr == nil { - pr = NewPostingsReader(reader) + return NewCachedPostingsReader(reader, util_log.Logger, postingsCache) } - - return pr + return NewPostingsReader(reader) } func NewPostingsReader(reader IndexReader) PostingsReader { return &simplePostingsReader{reader: reader} } -type simplePostingsReader struct { +type defaultPostingsReader struct { reader IndexReader } From ca33e61c24ef4abc6000a89b7ca9e6787855d206 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 28 Jul 2023 15:33:07 -0300 Subject: [PATCH 80/84] fix test --- pkg/storage/stores/tsdb/single_file_index.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 94d4a771b087f..9cbd03d9ec8f0 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -154,14 +154,14 @@ func getPostingsReader(reader IndexReader, postingsCache cache.Cache) PostingsRe } func NewPostingsReader(reader IndexReader) PostingsReader { - return &simplePostingsReader{reader: reader} + return &defaultPostingsReader{reader: reader} } type defaultPostingsReader struct { reader IndexReader } -func (s *simplePostingsReader) ForPostings(_ context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { +func (s *defaultPostingsReader) ForPostings(_ context.Context, matchers []*labels.Matcher, fn func(index.Postings) error) error { p, err := PostingsForMatchers(s.reader, nil, matchers...) if err != nil { return err From bfc54c4e243dc4782a8e71715086fac6fdbb26b5 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 28 Jul 2023 15:38:11 -0300 Subject: [PATCH 81/84] add description docs for tsdbshipper --- docs/sources/configure/_index.md | 3 +++ pkg/storage/factory.go | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 0815fd6afd8da..69662a99b6905 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -1978,6 +1978,9 @@ boltdb_shipper: # CLI flag: -boltdb.shipper.build-per-tenant-index [build_per_tenant_index: | default = false] +# Configures storing index in an Object Store +# (GCS/S3/Azure/Swift/COS/Filesystem) in a prometheus TSDB-like format. Required +# fields only required when TSDB is defined in config. tsdb_shipper: # Directory where ingesters would write index files which would then be # uploaded by shipper to configured storage diff --git a/pkg/storage/factory.go b/pkg/storage/factory.go index 9a601560c6b7f..01403cbe375f4 100644 --- a/pkg/storage/factory.go +++ b/pkg/storage/factory.go @@ -286,7 +286,7 @@ type Config struct { MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper" doc:"description=Configures storing index in an Object Store (GCS/S3/Azure/Swift/COS/Filesystem) in the form of boltdb files. Required fields only required when boltdb-shipper is defined in config."` - TSDBShipperConfig tsdb.IndexCfg `yaml:"tsdb_shipper"` + TSDBShipperConfig tsdb.IndexCfg `yaml:"tsdb_shipper" doc:"description=Configures storing index in an Object Store (GCS/S3/Azure/Swift/COS/Filesystem) in a prometheus TSDB-like format. Required fields only required when TSDB is defined in config."` // Config for using AsyncStore when using async index stores like `boltdb-shipper`. // It is required for getting chunk ids of recently flushed chunks from the ingesters. From b87c48d480c32d617e85b380d94907ba0be54489 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 1 Aug 2023 16:04:41 -0300 Subject: [PATCH 82/84] Test caching behaviro on e2e test. --- integration/loki_micro_services_test.go | 71 ++++++++++++++++++++++++- 1 file changed, 70 insertions(+), 1 deletion(-) diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 02684d4fc251c..dbc991ea359cc 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -2,11 +2,15 @@ package integration import ( "context" + "strings" "testing" "time" + dto "github.com/prometheus/client_model/go" + "github.com/prometheus/common/expfmt" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/proto" "github.com/grafana/loki/integration/client" "github.com/grafana/loki/integration/cluster" @@ -584,6 +588,7 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { "index-gateway", "-target=index-gateway", "-tsdb.enable-postings-cache=true", + "-store.index-cache-read.cache.enable-fifocache=true", ) ) require.NoError(t, clu.Run()) @@ -639,8 +644,18 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { cliIngester.Now = now cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL()) cliQueryFrontend.Now = now + cliIndexGateway := client.New(tenantID, "", tIndexGateway.HTTPURL()) + cliIndexGateway.Now = now - // end of setup. Ingestion and querying below. + // initial cache state. + igwMetrics, err := cliIndexGateway.Metrics() + require.NoError(t, err) + assertCacheState(t, igwMetrics, &expectedCacheState{ + cacheName: "store.index-cache-read.embedded-cache", + gets: 0, + misses: 0, + added: 0, + }) t.Run("ingest-logs", func(t *testing.T) { require.NoError(t, cliDistributor.PushLogLineWithTimestamp("lineA", time.Now().Add(-72*time.Hour), map[string]string{"job": "fake"})) @@ -666,6 +681,15 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { assert.ElementsMatch(t, []string{"lineA", "lineB"}, lines) }) + igwMetrics, err = cliIndexGateway.Metrics() + require.NoError(t, err) + assertCacheState(t, igwMetrics, &expectedCacheState{ + cacheName: "store.index-cache-read.embedded-cache", + gets: 50, + misses: 1, + added: 1, + }) + // ingest logs with ts=now. require.NoError(t, cliDistributor.PushLogLine("lineC", map[string]string{"job": "fake"})) require.NoError(t, cliDistributor.PushLogLine("lineD", map[string]string{"job": "fake"})) @@ -683,4 +707,49 @@ func TestQueryTSDB_WithCachedPostings(t *testing.T) { } // expect lines from both, ingesters memory and from the store. assert.ElementsMatch(t, []string{"lineA", "lineB", "lineC", "lineD"}, lines) + +} + +func getValueFromMF(mf *dto.MetricFamily, lbs []*dto.LabelPair) float64 { + for _, m := range mf.Metric { + if !assert.ObjectsAreEqualValues(lbs, m.GetLabel()) { + continue + } + + return m.Counter.GetValue() + } + + return 0 +} + +func assertCacheState(t *testing.T, metrics string, e *expectedCacheState) { + var parser expfmt.TextParser + mfs, err := parser.TextToMetricFamilies(strings.NewReader(metrics)) + require.NoError(t, err) + + lbs := []*dto.LabelPair{ + { + Name: proto.String("cache"), + Value: proto.String(e.cacheName), + }, + } + + mf, found := mfs["querier_cache_added_new_total"] + require.True(t, found) + require.Equal(t, e.added, getValueFromMF(mf, lbs)) + + mf, found = mfs["querier_cache_gets_total"] + require.True(t, found) + require.Equal(t, e.gets, getValueFromMF(mf, lbs)) + + mf, found = mfs["querier_cache_misses_total"] + require.True(t, found) + require.Equal(t, e.misses, getValueFromMF(mf, lbs)) +} + +type expectedCacheState struct { + cacheName string + gets float64 + misses float64 + added float64 } From ee07d761442d0148717b8109d96dd8702b252bef Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 1 Aug 2023 16:25:16 -0300 Subject: [PATCH 83/84] update go.mod --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 26b8133808769..4ff91592be05c 100644 --- a/go.mod +++ b/go.mod @@ -129,6 +129,7 @@ require ( golang.org/x/exp v0.0.0-20230321023759-10a507213a29 golang.org/x/oauth2 v0.10.0 golang.org/x/text v0.11.0 + google.golang.org/protobuf v1.31.0 ) require ( @@ -309,7 +310,6 @@ require ( google.golang.org/genproto v0.0.0-20230530153820-e85fd2cbaebc // indirect google.golang.org/genproto/googleapis/api v0.0.0-20230530153820-e85fd2cbaebc // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20230530153820-e85fd2cbaebc // indirect - google.golang.org/protobuf v1.31.0 // indirect gopkg.in/fsnotify/fsnotify.v1 v1.4.7 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect From 61772400e35b938d88e48558617f137028b3a06a Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Thu, 3 Aug 2023 14:37:33 -0300 Subject: [PATCH 84/84] change to sorteablelabelmatchers. --- pkg/storage/stores/tsdb/cached_postings_index.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/storage/stores/tsdb/cached_postings_index.go b/pkg/storage/stores/tsdb/cached_postings_index.go index da4a701aeae26..ec5c99ebda20c 100644 --- a/pkg/storage/stores/tsdb/cached_postings_index.go +++ b/pkg/storage/stores/tsdb/cached_postings_index.go @@ -120,7 +120,7 @@ func CanonicalLabelMatchersKey(ms []*labels.Matcher) string { for i := range ms { sorted[i] = labels.Matcher{Type: ms[i].Type, Name: ms[i].Name, Value: ms[i].Value} } - sort.Sort(sortedLabelMatchers(sorted)) + sort.Sort(sorteableLabelMatchers(sorted)) const ( typeLen = 2 @@ -141,9 +141,9 @@ func CanonicalLabelMatchersKey(ms []*labels.Matcher) string { return sb.String() } -type sortedLabelMatchers []labels.Matcher +type sorteableLabelMatchers []labels.Matcher -func (c sortedLabelMatchers) Less(i, j int) bool { +func (c sorteableLabelMatchers) Less(i, j int) bool { if c[i].Name != c[j].Name { return c[i].Name < c[j].Name } @@ -153,5 +153,5 @@ func (c sortedLabelMatchers) Less(i, j int) bool { return c[i].Value < c[j].Value } -func (c sortedLabelMatchers) Len() int { return len(c) } -func (c sortedLabelMatchers) Swap(i, j int) { c[i], c[j] = c[j], c[i] } +func (c sorteableLabelMatchers) Len() int { return len(c) } +func (c sorteableLabelMatchers) Swap(i, j int) { c[i], c[j] = c[j], c[i] }