From 9df684c6a61874719da293b1ccefb96bd3bde7b8 Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Mon, 2 Dec 2024 17:23:38 +0530 Subject: [PATCH 01/21] Initial changes --- pkg/blockbuilder/builder/tsdb.go | 4 ++++ pkg/ingester/flush.go | 1 + pkg/storage/chunk/chunk.go | 3 ++- pkg/storage/stores/composite_store.go | 17 +++++++++++++++++ pkg/storage/stores/index/index.go | 1 + pkg/storage/stores/series_store_write.go | 4 ++++ .../stores/shipper/indexshipper/tsdb/head.go | 11 +++++++++++ .../shipper/indexshipper/tsdb/head_manager.go | 12 ++++++++++++ .../stores/shipper/indexshipper/tsdb/store.go | 6 ++++++ 9 files changed, 58 insertions(+), 1 deletion(-) diff --git a/pkg/blockbuilder/builder/tsdb.go b/pkg/blockbuilder/builder/tsdb.go index e90bedb3815ad..672d7e1134f2a 100644 --- a/pkg/blockbuilder/builder/tsdb.go +++ b/pkg/blockbuilder/builder/tsdb.go @@ -60,6 +60,10 @@ func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chk return nil } +func (t *TsdbCreator) seriesStats(userID string, fp uint64) []string { + return nil +} + type chunkInfo struct { chunkMetas index.ChunkMetas tsdbFormat int diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 592ec0690b6b3..2262d4db2d72b 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -433,6 +433,7 @@ func (i *Ingester) flushChunks(ctx context.Context, fp model.Fingerprint, labelP } firstTime, lastTime := util.RoundToMilliseconds(c.chunk.Bounds()) + // c.chunk already has memchunk that contains symbolizer and structured metadata (can add series stats) ch := chunk.NewChunk( userID, fp, metric, chunkenc.NewFacade(c.chunk, i.cfg.BlockSize, i.cfg.TargetChunkSize), diff --git a/pkg/storage/chunk/chunk.go b/pkg/storage/chunk/chunk.go index a4cb63a442cb5..8099dab902b70 100644 --- a/pkg/storage/chunk/chunk.go +++ b/pkg/storage/chunk/chunk.go @@ -44,12 +44,13 @@ type Chunk struct { logproto.ChunkRef Metric labels.Labels `json:"metric"` - // We never use Delta encoding (the zero value), so if this entry is // missing, we default to DoubleDelta. Encoding Encoding `json:"encoding"` Data Data `json:"-"` + // each stream populates this series stats for all chunks pointing to the same instance + //seriesStats *SeriesStats // The encoded version of the chunk, held so we don't need to re-encode it encoded []byte } diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 182a686c889b8..8f93013df6e50 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -21,6 +21,7 @@ import ( type ChunkWriter interface { Put(ctx context.Context, chunks []chunk.Chunk) error PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error + SeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64) ([]string, error) } type ChunkFetcherProvider interface { @@ -106,6 +107,22 @@ func (c CompositeStore) PutOne(ctx context.Context, from, through model.Time, ch }) } +func (c CompositeStore) SeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64) ([]string, error) { + var results []string + err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { + r, err := store.SeriesStats(innerCtx, from, through, userID, fp) + if err != nil { + return err + } + results = append(results, r...) + return nil + }) + if err != nil { + return nil, err + } + return results, nil +} + func (c CompositeStore) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { for _, store := range c.stores { store.Store.SetChunkFilterer(chunkFilter) diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 4014fca8c3e7b..ebfa6bf12a3d5 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -53,6 +53,7 @@ type Reader interface { type Writer interface { IndexChunk(ctx context.Context, from, through model.Time, chk chunk.Chunk) error + SeriesStats(userID string, fp uint64) []string } type ReaderWriter interface { diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index a36ae4510b8e3..0b1fd3d98594a 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -127,3 +127,7 @@ func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk return nil } + +func (c *Writer) SeriesStats(userID string, fp uint64) []string { + return c.indexWriter.SeriesStats(userID, fp) +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 00e4f86b9ad90..6df0725ac6121 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -176,6 +176,10 @@ func (h *Head) Append(ls labels.Labels, fprint uint64, chks index.ChunkMetas) (c return } +func (h *Head) seriesStats(fp uint64) []string { + return h.series.seriesStats(fp) +} + // seriesHashmap is a simple hashmap for memSeries by their label set. It is built // on top of a regular hashmap and holds a slice of series to resolve hash collisions. // Its methods require the hash to be submitted with it to avoid re-computations throughout @@ -284,7 +288,14 @@ func (s *stripeSeries) Append( return } +func (s *stripeSeries) seriesStats(fp uint64) []string { + // should return the set of struct metadata labels + // ideally should be a pointer to struct that contains the []string + return nil +} + type memSeries struct { + // (h11) potentially add stats? sync.RWMutex ref uint64 // The unique reference within a *Head ls labels.Labels diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 989982f268857..c384a1b8b3fae 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -254,10 +254,15 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, fprint uint64, chk m.mtx.RLock() defer m.mtx.RUnlock() + // (h11): appends to tsdb head here - coming from PutOne eventually from the ingester (flush loop) rec := m.activeHeads.Append(userID, ls, fprint, chks) return m.active.Log(rec) } +func (m *HeadManager) SeriesStats(userID string, fp uint64) []string { + return m.activeHeads.seriesStats(userID, fp) +} + func (m *HeadManager) Start() error { if err := os.RemoveAll(filepath.Join(m.dir, "scratch")); err != nil { return errors.Wrap(err, "removing tsdb scratch dir") @@ -655,6 +660,7 @@ func newTenantHeads(start time.Time, shards int, metrics *Metrics, logger log.Lo return res } +// (h11) appends from the ingester func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chks index.ChunkMetas) *WALRecord { var mint, maxt int64 for _, chk := range chks { @@ -690,6 +696,12 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chk return rec } +func (t *tenantHeads) seriesStats(userID string, fp uint64) []string { + // (h11) : don't create head.extract to a different function to just get + head := t.getOrCreateTenantHead(userID) + return head.seriesStats(fp) +} + func (t *tenantHeads) getOrCreateTenantHead(userID string) *Head { idx := t.shardForTenant(userID) mtx := &t.locks[idx] diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index 8ca8a2489e6d3..3e0ed6b085ffe 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -25,6 +25,7 @@ import ( type IndexWriter interface { Append(userID string, ls labels.Labels, fprint uint64, chks tsdbindex.ChunkMetas) error + SeriesStats(userID string, fp uint64) []string } type store struct { @@ -166,6 +167,7 @@ func (s *store) IndexChunk(_ context.Context, _ model.Time, _ model.Time, chk ch MaxTime: int64(chk.ChunkRef.Through), KB: uint32(approxKB), Entries: uint32(chk.Data.Entries()), + // set of fields chunk.data.structure metadata fields }, } if err := s.indexWriter.Append(chk.UserID, chk.Metric, chk.ChunkRef.Fingerprint, metas); err != nil { @@ -174,6 +176,10 @@ func (s *store) IndexChunk(_ context.Context, _ model.Time, _ model.Time, chk ch return nil } +func (s *store) SeriesStats(userID string, fp uint64) []string { + return s.indexWriter.SeriesStats(userID, fp) +} + type failingIndexWriter struct{} func (f failingIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbindex.ChunkMetas) error { From 89c5ec6c737c3b94e4b58ca84b56c1baa3107739 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 2 Dec 2024 13:22:49 +0100 Subject: [PATCH 02/21] add connection to ingester --- pkg/ingester/flush.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 2262d4db2d72b..a9468669592fa 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -270,6 +270,13 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. return nil } + // (h11) this should return the stats to pass down to the writer + // Each stream has a list of chunks (all closed but the last one. + // The stream should have two stream: + // 1. ClosedStats: stats ready to be flushed + // 2. UpdatableStats: stats with the most up to date data: includes stats for unclosed chunks. + // When a chunk is closed, we copy UpdatableStats into ClosedStats, and create a new UpdatableStats as a copy of ClosedStats + // That way, collectChunksToFlush can return ClosedStats, or we can use it right away chunks, labels, chunkMtx := i.collectChunksToFlush(instance, fp, immediate) if len(chunks) < 1 { return nil @@ -311,6 +318,18 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. return fmt.Errorf("failed to flush chunks: %w, num_chunks: %d, labels: %s", err, len(chunks), lbs) } + // (h11) Rename to UpdateSeriesStats passing the series stats object + s, exists := instance.streams.LoadByFP(fp) + if !exists { + level.Error(i.logger).Log("msg", "stream not found", "fp", fp) + return nil + } + + // GET the stats from the stream, and call UpdateSeriesStats() with the stats object + // Then reset the series stats + + i.store.SeriesStats(ctx, 0, 0, userID, uint64(fp)) + return nil } From 62058ed130f58602525bb074bd41dd4be3b7c24d Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Mon, 2 Dec 2024 19:04:50 +0530 Subject: [PATCH 03/21] Enhancements to write path --- pkg/ingester/flush.go | 14 ++-- pkg/ingester/stream.go | 20 +++++ pkg/storage/stores/composite_store.go | 19 ++--- pkg/storage/stores/index/index.go | 3 +- pkg/storage/stores/series_store_write.go | 5 +- .../stores/shipper/indexshipper/tsdb/head.go | 73 ++++++++++++++++--- .../shipper/indexshipper/tsdb/head_manager.go | 20 ++++- .../shipper/indexshipper/tsdb/manager.go | 2 + .../stores/shipper/indexshipper/tsdb/store.go | 6 +- 9 files changed, 122 insertions(+), 40 deletions(-) diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index a9468669592fa..46f5cd94b424a 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -15,6 +15,7 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/tenant" "github.com/grafana/dskit/user" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -272,12 +273,12 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. // (h11) this should return the stats to pass down to the writer // Each stream has a list of chunks (all closed but the last one. - // The stream should have two stream: + // The stream should have two stats: // 1. ClosedStats: stats ready to be flushed // 2. UpdatableStats: stats with the most up to date data: includes stats for unclosed chunks. // When a chunk is closed, we copy UpdatableStats into ClosedStats, and create a new UpdatableStats as a copy of ClosedStats // That way, collectChunksToFlush can return ClosedStats, or we can use it right away - chunks, labels, chunkMtx := i.collectChunksToFlush(instance, fp, immediate) + chunks, labels, stats, chunkMtx := i.collectChunksToFlush(instance, fp, immediate) if len(chunks) < 1 { return nil } @@ -328,18 +329,18 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. // GET the stats from the stream, and call UpdateSeriesStats() with the stats object // Then reset the series stats - i.store.SeriesStats(ctx, 0, 0, userID, uint64(fp)) + i.store.UpdateSeriesStats(ctx, 0, 0, userID, uint64(fp), stats) return nil } -func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, *sync.RWMutex) { +func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, tsdb.SeriesStats, *sync.RWMutex) { var stream *stream var ok bool stream, ok = instance.streams.LoadByFP(fp) if !ok { - return nil, nil, nil + return nil, nil, tsdb.SeriesStats{}, nil } stream.chunkMtx.Lock() @@ -356,6 +357,7 @@ func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint // Ensure no more writes happen to this chunk. if !stream.chunks[j].closed { stream.chunks[j].closed = true + stream.cutStats() } // Flush this chunk if it hasn't already been successfully flushed. if stream.chunks[j].flushed.IsZero() { @@ -368,7 +370,7 @@ func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint } } } - return result, stream.labels, &stream.chunkMtx + return result, stream.labels, stream.stats(), &stream.chunkMtx } func (i *Ingester) shouldFlushChunk(chunk *chunkDesc) (bool, string) { diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index b36cbb290db7e..5bf6533e5f06a 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -11,6 +11,7 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -82,6 +83,11 @@ type stream struct { chunkHeadBlockFormat chunkenc.HeadBlockFmt configs *runtime.TenantConfigs + + // closedChunksStats has the stats up to the most recent closed chunk. + closedChunksStats tsdb.SeriesStats + // openChunksStats has the stats up to the most recent open chunk (i.e. closedChunksStats + newer data). + openChunksStats tsdb.SeriesStats } type chunkDesc struct { @@ -354,6 +360,10 @@ func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry, usa s.handleLoggingOfDuplicateEntry(entries[i]) } + // h11: index fliushing and chunks flushing happen independently + // we need to have the stats in sync: when the index stats are flushed, we need to reset the stats + s.openChunksStats.AddStructuredMetadata(entries[i].StructuredMetadata) + s.entryCt++ s.lastLine.ts = entries[i].Timestamp s.lastLine.content = entries[i].Line @@ -506,6 +516,7 @@ func (s *stream) cutChunk(ctx context.Context) *chunkDesc { level.Error(util_log.WithContext(ctx, util_log.Logger)).Log("msg", "failed to Close chunk", "err", err) } chunk.closed = true + s.cutStats() s.metrics.samplesPerChunk.Observe(float64(chunk.chunk.Size())) s.metrics.blocksPerChunk.Observe(float64(chunk.chunk.BlockCount())) @@ -649,6 +660,15 @@ func (s *stream) addTailer(t *tailer) { s.tailers[t.getID()] = t } +func (s *stream) cutStats() { + s.closedChunksStats = s.openChunksStats + s.openChunksStats = s.closedChunksStats.Copy() +} + +func (s *stream) stats() tsdb.SeriesStats { + return s.closedChunksStats +} + func headBlockType(chunkfmt byte, unorderedWrites bool) chunkenc.HeadBlockFmt { if unorderedWrites { if chunkfmt >= chunkenc.ChunkFormatV3 { diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 8f93013df6e50..070e812deff08 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -4,6 +4,7 @@ import ( "context" "sort" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -21,7 +22,7 @@ import ( type ChunkWriter interface { Put(ctx context.Context, chunks []chunk.Chunk) error PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error - SeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64) ([]string, error) + UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats tsdb.SeriesStats) error } type ChunkFetcherProvider interface { @@ -107,20 +108,10 @@ func (c CompositeStore) PutOne(ctx context.Context, from, through model.Time, ch }) } -func (c CompositeStore) SeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64) ([]string, error) { - var results []string - err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { - r, err := store.SeriesStats(innerCtx, from, through, userID, fp) - if err != nil { - return err - } - results = append(results, r...) - return nil +func (c CompositeStore) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats tsdb.SeriesStats) error { + return c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { + return store.UpdateSeriesStats(innerCtx, from, through, userID, fp, stats) }) - if err != nil { - return nil, err - } - return results, nil } func (c CompositeStore) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index ebfa6bf12a3d5..40a1f62f15086 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -5,6 +5,7 @@ import ( "time" "github.com/grafana/dskit/instrument" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -53,7 +54,7 @@ type Reader interface { type Writer interface { IndexChunk(ctx context.Context, from, through model.Time, chk chunk.Chunk) error - SeriesStats(userID string, fp uint64) []string + UpdateSeriesStats(userID string, fp uint64, stats tsdb.SeriesStats) } type ReaderWriter interface { diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index 0b1fd3d98594a..699e338aee38d 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -4,6 +4,7 @@ import ( "context" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -128,6 +129,6 @@ func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk return nil } -func (c *Writer) SeriesStats(userID string, fp uint64) []string { - return c.indexWriter.SeriesStats(userID, fp) +func (c *Writer) UpdateSeriesStats(userID string, fp uint64, stats tsdb.SeriesStats) { + c.indexWriter.UpdateSeriesStats(userID, fp, stats) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 6df0725ac6121..2aab4da16224c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -16,6 +16,7 @@ import ( "sync" "github.com/go-kit/log" + "github.com/grafana/loki/pkg/push" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" @@ -176,8 +177,12 @@ func (h *Head) Append(ls labels.Labels, fprint uint64, chks index.ChunkMetas) (c return } -func (h *Head) seriesStats(fp uint64) []string { - return h.series.seriesStats(fp) +func (h *Head) updateSeriesStats(fp uint64, stats SeriesStats) { + h.series.updateSeriesStats(fp, stats) +} + +func (h *Head) ResetSeriesStats() { + h.series.resetSeriesStats() } // seriesHashmap is a simple hashmap for memSeries by their label set. It is built @@ -288,19 +293,67 @@ func (s *stripeSeries) Append( return } -func (s *stripeSeries) seriesStats(fp uint64) []string { - // should return the set of struct metadata labels - // ideally should be a pointer to struct that contains the []string - return nil +func (s *stripeSeries) updateSeriesStats(fp uint64, stats SeriesStats) { + series := s.getByID(fp) + + series.Lock() + defer series.Unlock() + series.stats.Merge(stats) +} + +func (s *stripeSeries) resetSeriesStats() { + for _, seriesMap := range s.series { + seriesMap.Lock() + for _, series := range seriesMap.m { + series.stats.Reset() + } + seriesMap.Unlock() + } +} + +// h11: add a mutex. +// Reset takes the muted to reset all the stats +// AddStructuredMetadata takes the mutes and updates the stats +// The index manager calls Reset when the index is written +// In the ingester push logic, we keep updating the index stats +type SeriesStats struct { + StructuredMetadataFieldNames map[string]struct{} +} + +func (s *SeriesStats) Copy() SeriesStats { + out := SeriesStats{ + StructuredMetadataFieldNames: make(map[string]struct{}, len(s.StructuredMetadataFieldNames)), + } + for k := range s.StructuredMetadataFieldNames { + out.StructuredMetadataFieldNames[k] = struct{}{} + } + return out +} + +func (s *SeriesStats) Merge(other SeriesStats) { + for k := range other.StructuredMetadataFieldNames { + s.StructuredMetadataFieldNames[k] = struct{}{} + } +} + +func (s *SeriesStats) AddStructuredMetadata(metadata push.LabelsAdapter) { + for _, l := range metadata { + s.StructuredMetadataFieldNames[l.Name] = struct{}{} + } +} + +func (s *SeriesStats) Reset() { + s.StructuredMetadataFieldNames = make(map[string]struct{}) } type memSeries struct { // (h11) potentially add stats? sync.RWMutex - ref uint64 // The unique reference within a *Head - ls labels.Labels - fp uint64 - chks index.ChunkMetas + ref uint64 // The unique reference within a *Head + ls labels.Labels + fp uint64 + chks index.ChunkMetas + stats SeriesStats } func newMemSeries(ref uint64, ls labels.Labels, fp uint64) *memSeries { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index c384a1b8b3fae..5ef7d4a54cac3 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -259,8 +259,8 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, fprint uint64, chk return m.active.Log(rec) } -func (m *HeadManager) SeriesStats(userID string, fp uint64) []string { - return m.activeHeads.seriesStats(userID, fp) +func (m *HeadManager) UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) { + m.activeHeads.updateSeriesStats(userID, fp, stats) } func (m *HeadManager) Start() error { @@ -696,10 +696,10 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chk return rec } -func (t *tenantHeads) seriesStats(userID string, fp uint64) []string { +func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats SeriesStats) { // (h11) : don't create head.extract to a different function to just get head := t.getOrCreateTenantHead(userID) - return head.seriesStats(fp) + head.updateSeriesStats(fp, stats) } func (t *tenantHeads) getOrCreateTenantHead(userID string) *Head { @@ -854,3 +854,15 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c return nil } + +func (t *tenantHeads) ResetSeriesStats() { + for i, shard := range t.tenants { + t.locks[i].RLock() + defer t.locks[i].RUnlock() + + for _, tenant := range shard { + tenant.ResetSeriesStats() + } + } + +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index 84c250eb7464c..9ee3f191fc668 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -204,6 +204,8 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe return err } + heads.ResetSeriesStats() + for p, b := range periods { dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) dst := NewPrefixedIdentifier( diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index 3e0ed6b085ffe..1dfeac62f0b17 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -25,7 +25,7 @@ import ( type IndexWriter interface { Append(userID string, ls labels.Labels, fprint uint64, chks tsdbindex.ChunkMetas) error - SeriesStats(userID string, fp uint64) []string + UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) } type store struct { @@ -176,8 +176,8 @@ func (s *store) IndexChunk(_ context.Context, _ model.Time, _ model.Time, chk ch return nil } -func (s *store) SeriesStats(userID string, fp uint64) []string { - return s.indexWriter.SeriesStats(userID, fp) +func (s *store) UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) { + s.indexWriter.UpdateSeriesStats(userID, fp, stats) } type failingIndexWriter struct{} From f8aef68380d8378374093601b4de2f85258e431c Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 2 Dec 2024 16:44:33 +0100 Subject: [PATCH 04/21] Encoding stats into TSDB --- pkg/ingester/flush.go | 24 ++++----- pkg/ingester/stream.go | 22 ++++---- pkg/storage/stores/composite_store.go | 4 +- pkg/storage/stores/index/index.go | 2 +- pkg/storage/stores/series_store_write.go | 3 +- .../shipper/indexshipper/tsdb/builder.go | 20 ++++++- .../stores/shipper/indexshipper/tsdb/head.go | 33 ++++++++---- .../shipper/indexshipper/tsdb/head_manager.go | 26 +++------- .../shipper/indexshipper/tsdb/index/index.go | 52 +++++++++++++++++-- .../shipper/indexshipper/tsdb/manager.go | 8 +-- .../stores/shipper/indexshipper/tsdb/store.go | 4 +- 11 files changed, 129 insertions(+), 69 deletions(-) diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 46f5cd94b424a..07f580f6b2c00 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -306,7 +306,9 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. "total_comp", humanize.Bytes(uint64(totalCompressedSize)), "avg_comp", humanize.Bytes(uint64(totalCompressedSize/len(chunks))), "total_uncomp", humanize.Bytes(uint64(totalUncompressedSize)), - "avg_uncomp", humanize.Bytes(uint64(totalUncompressedSize/len(chunks)))) + "avg_uncomp", humanize.Bytes(uint64(totalUncompressedSize/len(chunks))), + "metadata_fields", len(labels), + ) logValues = append(logValues, frc.Log()...) logValues = append(logValues, "labels", lbs) level.Info(i.logger).Log(logValues...) @@ -319,28 +321,20 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. return fmt.Errorf("failed to flush chunks: %w, num_chunks: %d, labels: %s", err, len(chunks), lbs) } - // (h11) Rename to UpdateSeriesStats passing the series stats object - s, exists := instance.streams.LoadByFP(fp) - if !exists { - level.Error(i.logger).Log("msg", "stream not found", "fp", fp) - return nil + if err := i.store.UpdateSeriesStats(ctx, 0, 0, userID, uint64(fp), stats); err != nil { + return fmt.Errorf("failed to update series stats: %w", err) } - // GET the stats from the stream, and call UpdateSeriesStats() with the stats object - // Then reset the series stats - - i.store.UpdateSeriesStats(ctx, 0, 0, userID, uint64(fp), stats) - return nil } -func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, tsdb.SeriesStats, *sync.RWMutex) { +func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, *tsdb.StreamStats, *sync.RWMutex) { var stream *stream var ok bool stream, ok = instance.streams.LoadByFP(fp) if !ok { - return nil, nil, tsdb.SeriesStats{}, nil + return nil, nil, nil, nil } stream.chunkMtx.Lock() @@ -357,7 +351,7 @@ func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint // Ensure no more writes happen to this chunk. if !stream.chunks[j].closed { stream.chunks[j].closed = true - stream.cutStats() + stream.cutSeriesStats() } // Flush this chunk if it hasn't already been successfully flushed. if stream.chunks[j].flushed.IsZero() { @@ -370,7 +364,7 @@ func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint } } } - return result, stream.labels, stream.stats(), &stream.chunkMtx + return result, stream.labels, stream.flushableSeriesStats(), &stream.chunkMtx } func (i *Ingester) shouldFlushChunk(chunk *chunkDesc) (bool, string) { diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 5bf6533e5f06a..15ac155188064 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -84,10 +84,10 @@ type stream struct { configs *runtime.TenantConfigs - // closedChunksStats has the stats up to the most recent closed chunk. - closedChunksStats tsdb.SeriesStats - // openChunksStats has the stats up to the most recent open chunk (i.e. closedChunksStats + newer data). - openChunksStats tsdb.SeriesStats + // closedStreamStats has the stats up to the most recent closed chunk. + closedStreamStats *tsdb.StreamStats + // openStreamStats has the stats up to the most recent open chunk (i.e. closedChunksStats + newer data). + openStreamStats *tsdb.StreamStats } type chunkDesc struct { @@ -362,7 +362,7 @@ func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry, usa // h11: index fliushing and chunks flushing happen independently // we need to have the stats in sync: when the index stats are flushed, we need to reset the stats - s.openChunksStats.AddStructuredMetadata(entries[i].StructuredMetadata) + s.openStreamStats.AddStructuredMetadata(entries[i].StructuredMetadata) s.entryCt++ s.lastLine.ts = entries[i].Timestamp @@ -516,7 +516,7 @@ func (s *stream) cutChunk(ctx context.Context) *chunkDesc { level.Error(util_log.WithContext(ctx, util_log.Logger)).Log("msg", "failed to Close chunk", "err", err) } chunk.closed = true - s.cutStats() + s.cutSeriesStats() s.metrics.samplesPerChunk.Observe(float64(chunk.chunk.Size())) s.metrics.blocksPerChunk.Observe(float64(chunk.chunk.BlockCount())) @@ -660,13 +660,13 @@ func (s *stream) addTailer(t *tailer) { s.tailers[t.getID()] = t } -func (s *stream) cutStats() { - s.closedChunksStats = s.openChunksStats - s.openChunksStats = s.closedChunksStats.Copy() +func (s *stream) cutSeriesStats() { + s.closedStreamStats = s.openStreamStats + s.openStreamStats = s.closedStreamStats.Copy() } -func (s *stream) stats() tsdb.SeriesStats { - return s.closedChunksStats +func (s *stream) flushableSeriesStats() *tsdb.StreamStats { + return s.closedStreamStats } func headBlockType(chunkfmt byte, unorderedWrites bool) chunkenc.HeadBlockFmt { diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 070e812deff08..a56b2ac551e44 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -22,7 +22,7 @@ import ( type ChunkWriter interface { Put(ctx context.Context, chunks []chunk.Chunk) error PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error - UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats tsdb.SeriesStats) error + UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error } type ChunkFetcherProvider interface { @@ -108,7 +108,7 @@ func (c CompositeStore) PutOne(ctx context.Context, from, through model.Time, ch }) } -func (c CompositeStore) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats tsdb.SeriesStats) error { +func (c CompositeStore) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error { return c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { return store.UpdateSeriesStats(innerCtx, from, through, userID, fp, stats) }) diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 40a1f62f15086..fd74fca4e00da 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -54,7 +54,7 @@ type Reader interface { type Writer interface { IndexChunk(ctx context.Context, from, through model.Time, chk chunk.Chunk) error - UpdateSeriesStats(userID string, fp uint64, stats tsdb.SeriesStats) + UpdateSeriesStats(userID string, fp uint64, stats *tsdb.StreamStats) } type ReaderWriter interface { diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index 699e338aee38d..e03f04d370ef4 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -129,6 +129,7 @@ func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk return nil } -func (c *Writer) UpdateSeriesStats(userID string, fp uint64, stats tsdb.SeriesStats) { +func (c *Writer) UpdateSeriesStats(_ context.Context, _, _ model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error { c.indexWriter.UpdateSeriesStats(userID, fp, stats) + return nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go index 815888c14586f..5051447600111 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go @@ -32,6 +32,7 @@ type stream struct { labels labels.Labels fp model.Fingerprint chunks index.ChunkMetas + stats *StreamStats } func NewBuilder(version int) *Builder { @@ -41,7 +42,8 @@ func NewBuilder(version int) *Builder { } } -func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { +// TODO(h11): stats should be variadic but I added it to avoid changinf it in many places +func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats ...*StreamStats) { id := ls.String() s, ok := b.streams[id] if !ok { @@ -52,7 +54,15 @@ func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index b.streams[id] = s } + // h11: Seeing how we append insteasd of setting the chunks points that AddSeries can be called multiple times. + // Doesn't look like we check for duplicated chunks so I think we keep adding chunks to the series until we rotate the index + // when we reach a new period (new day). If we can verify this, then, for stats we could either merge as we do here of just set. + // But I'm not sure if then we should only reset the stream stats when we rotate the index. s.chunks = append(s.chunks, chks...) + + if len(stats) > 0 { + s.stats.Merge(stats[0]) + } } func (b *Builder) FinalizeChunks() { @@ -175,10 +185,18 @@ func (b *Builder) build( // Build symbols symbolsMap := make(map[string]struct{}) for _, s := range streams { + // Add labels/names to symbols for _, l := range s.labels { symbolsMap[l.Name] = struct{}{} symbolsMap[l.Value] = struct{}{} } + + // Add SM stats to symbols + if s.stats != nil { + for smName := range s.stats.StructuredMetadataFieldNames { + symbolsMap[smName] = struct{}{} + } + } } // Sort symbols diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 2aab4da16224c..01198330521ea 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -177,7 +177,7 @@ func (h *Head) Append(ls labels.Labels, fprint uint64, chks index.ChunkMetas) (c return } -func (h *Head) updateSeriesStats(fp uint64, stats SeriesStats) { +func (h *Head) updateSeriesStats(fp uint64, stats *StreamStats) { h.series.updateSeriesStats(fp, stats) } @@ -293,7 +293,7 @@ func (s *stripeSeries) Append( return } -func (s *stripeSeries) updateSeriesStats(fp uint64, stats SeriesStats) { +func (s *stripeSeries) updateSeriesStats(fp uint64, stats *StreamStats) { series := s.getByID(fp) series.Lock() @@ -316,12 +316,16 @@ func (s *stripeSeries) resetSeriesStats() { // AddStructuredMetadata takes the mutes and updates the stats // The index manager calls Reset when the index is written // In the ingester push logic, we keep updating the index stats -type SeriesStats struct { +type StreamStats struct { + mu sync.RWMutex StructuredMetadataFieldNames map[string]struct{} } -func (s *SeriesStats) Copy() SeriesStats { - out := SeriesStats{ +func (s *StreamStats) Copy() *StreamStats { + s.mu.RLock() + defer s.mu.RUnlock() + + out := &StreamStats{ StructuredMetadataFieldNames: make(map[string]struct{}, len(s.StructuredMetadataFieldNames)), } for k := range s.StructuredMetadataFieldNames { @@ -330,20 +334,29 @@ func (s *SeriesStats) Copy() SeriesStats { return out } -func (s *SeriesStats) Merge(other SeriesStats) { +func (s *StreamStats) Merge(other *StreamStats) { + s.mu.Lock() + defer s.mu.Unlock() + for k := range other.StructuredMetadataFieldNames { s.StructuredMetadataFieldNames[k] = struct{}{} } } -func (s *SeriesStats) AddStructuredMetadata(metadata push.LabelsAdapter) { +func (s *StreamStats) AddStructuredMetadata(metadata push.LabelsAdapter) { + s.mu.Lock() + defer s.mu.Unlock() + for _, l := range metadata { s.StructuredMetadataFieldNames[l.Name] = struct{}{} } } -func (s *SeriesStats) Reset() { - s.StructuredMetadataFieldNames = make(map[string]struct{}) +func (s *StreamStats) Reset() { + s.mu.Lock() + defer s.mu.Unlock() + + s.StructuredMetadataFieldNames = make(map[string]struct{}, len(s.StructuredMetadataFieldNames)) } type memSeries struct { @@ -353,7 +366,7 @@ type memSeries struct { ls labels.Labels fp uint64 chks index.ChunkMetas - stats SeriesStats + stats StreamStats } func newMemSeries(ref uint64, ls labels.Labels, fp uint64) *memSeries { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 5ef7d4a54cac3..b35e614a60051 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -259,7 +259,7 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, fprint uint64, chk return m.active.Log(rec) } -func (m *HeadManager) UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) { +func (m *HeadManager) UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) { m.activeHeads.updateSeriesStats(userID, fp, stats) } @@ -696,7 +696,7 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chk return rec } -func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats SeriesStats) { +func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats *StreamStats) { // (h11) : don't create head.extract to a different function to just get head := t.getOrCreateTenantHead(userID) head.updateSeriesStats(fp, stats) @@ -821,13 +821,13 @@ func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter ind } // helper only used in building TSDBs -func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error) error { +func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, head *Head) error) error { for i, shard := range t.tenants { t.locks[i].RLock() defer t.locks[i].RUnlock() - for user, tenant := range shard { - idx := tenant.Index() + for tenant, head := range shard { + idx := head.Index() ps, err := postingsForMatcher(idx, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { return err @@ -842,10 +842,10 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks) if err != nil { - return errors.Wrapf(err, "iterating postings for tenant: %s", user) + return errors.Wrapf(err, "iterating postings for tenant: %s", tenant) } - if err := fn(user, ls, fp, chks); err != nil { + if err := fn(tenant, ls, fp, chks, head); err != nil { return err } } @@ -854,15 +854,3 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c return nil } - -func (t *tenantHeads) ResetSeriesStats() { - for i, shard := range t.tenants { - t.locks[i].RLock() - defer t.locks[i].RUnlock() - - for _, tenant := range shard { - tenant.ResetSeriesStats() - } - } - -} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 0e10f8648a375..6ef473b450c78 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -18,6 +18,7 @@ import ( "context" "encoding/binary" "fmt" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "hash" "hash/crc32" "io" @@ -403,7 +404,7 @@ func (w *Creator) writeMeta() error { // fingerprint differs from what labels.Hash() produces. For example, // multitenant TSDBs embed a tenant label, but the actual series has no such // label and so the derived fingerprint differs. -func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks ...ChunkMeta) error { +func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks []ChunkMeta, stats ...*tsdb.StreamStats) error { if err := w.ensureStage(idxStageSeries); err != nil { return err } @@ -465,6 +466,27 @@ func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model. w.buf2.PutUvarint32(valueIndex) } + // h11: write stream stats + var smFields map[string]struct{} + if len(stats) > 0 { + smFields = stats[0].StructuredMetadataFieldNames + } + + w.buf2.PutUvarint(len(smFields)) + for sm := range smFields { + var err error + cacheEntry, ok := w.symbolCache[sm] + nameIndex := cacheEntry.index + if !ok { + nameIndex, err = w.symbols.ReverseLookup(sm) + if err != nil { + return errors.Errorf("symbol entry for %q does not exist, %v", sm, err) + } + } + // h11: Should we add this to w.labelNames? + w.buf2.PutUvarint32(nameIndex) + } + w.addChunks(chunks, &w.buf2, &w.buf1, ChunkPageSize) w.buf1.Reset() @@ -1753,7 +1775,7 @@ func (r *Reader) LabelValueFor(id storage.SeriesRef, label string) (string, erro } // Series reads the series with the given ID and writes its labels and chunks into lbls and chks. -func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta) (uint64, error) { +func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *tsdb.StreamStats) (uint64, error) { offset := id // In version 2+ series IDs are no longer exact references but series are 16-byte padded // and the ID is the multiple of 16 of the actual position. @@ -1765,7 +1787,7 @@ func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *l return 0, d.Err() } - fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks) + fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks, stats) if err != nil { return 0, errors.Wrap(err, "read series") } @@ -2171,6 +2193,24 @@ func (dec *Decoder) prepSeries(b []byte, lbls *labels.Labels, chks *[]ChunkMeta) return &d, fprint, nil } +func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *tsdb.StreamStats) error { + nSMFieldNames := d.Uvarint() + + stats.StructuredMetadataFieldNames = make(map[string]struct{}, nSMFieldNames) + for i := 0; i < nSMFieldNames; i++ { + fieldName := uint32(d.Uvarint()) + + ln, err := dec.LookupSymbol(fieldName) + if err != nil { + return errors.Wrap(err, "lookup structured metadata field name") + } + + stats.StructuredMetadataFieldNames[ln] = struct{}{} + } + + return nil +} + // prepSeriesBy returns series labels and chunks for a series and only returning selected `by` label names. // If `by` is empty, it returns all labels for the series. func (dec *Decoder) prepSeriesBy(b []byte, lbls *labels.Labels, chks *[]ChunkMeta, by map[string]struct{}) (*encoding.Decbuf, uint64, error) { @@ -2357,12 +2397,16 @@ func (dec *Decoder) readChunkStatsPriorV3(d *encoding.Decbuf, seriesRef storage. } // Series decodes a series entry from the given byte slice into lset and chks. -func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta) (uint64, error) { +func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *tsdb.StreamStats) (uint64, error) { d, fprint, err := dec.prepSeries(b, lbls, chks) if err != nil { return 0, err } + if err := dec.readSeriesStats(version, d, stats); err != nil { + return 0, errors.Wrap(err, "series stats") + } + // read chunks based on fmt if err := dec.readChunks(version, d, seriesRef, from, through, chks); err != nil { return 0, errors.Wrapf(err, "series %s", lbls.String()) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index 9ee3f191fc668..dd9ea5aebb284 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -160,7 +160,7 @@ type chunkInfo struct { func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshipper.IndexShipper, tableRanges []config.TableRange) (err error) { periods := make(map[string]*Builder) - if err := heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error { + if err := heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, head *Head) error { // chunks may overlap index period bounds, in which case they're written to multiple pds := make(map[string]chunkInfo) @@ -198,14 +198,16 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe ) } + // The builder now has the up-to-date series data, we can now reset the stats + // TODO(h11): We need to write the stats to the builder here + head.ResetSeriesStats() + return nil }); err != nil { level.Error(m.log).Log("err", err.Error(), "msg", "building TSDB") return err } - heads.ResetSeriesStats() - for p, b := range periods { dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) dst := NewPrefixedIdentifier( diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index 1dfeac62f0b17..4266efa0d6bd7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -25,7 +25,7 @@ import ( type IndexWriter interface { Append(userID string, ls labels.Labels, fprint uint64, chks tsdbindex.ChunkMetas) error - UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) + UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) } type store struct { @@ -176,7 +176,7 @@ func (s *store) IndexChunk(_ context.Context, _ model.Time, _ model.Time, chk ch return nil } -func (s *store) UpdateSeriesStats(userID string, fp uint64, stats SeriesStats) { +func (s *store) UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) { s.indexWriter.UpdateSeriesStats(userID, fp, stats) } From f655fde95a2f16ffeda7ada3081db8a572f56c40 Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Tue, 3 Dec 2024 16:01:04 +0530 Subject: [PATCH 05/21] Read path --- pkg/bloombuild/common/tsdb.go | 47 +++++------- pkg/bloombuild/common/tsdb_test.go | 10 +-- .../planner/strategies/chunksize.go | 71 ++++++++---------- .../planner/strategies/splitkeyspace_test.go | 2 +- pkg/indexgateway/gateway.go | 75 +++++++++---------- pkg/storage/stores/composite_store.go | 2 +- pkg/storage/stores/index/index.go | 2 +- .../stores/shipper/indexshipper/tsdb/head.go | 16 ++++ .../shipper/indexshipper/tsdb/head_manager.go | 11 +-- .../indexshipper/tsdb/head_manager_test.go | 59 ++------------- .../shipper/indexshipper/tsdb/head_read.go | 8 +- .../stores/shipper/indexshipper/tsdb/index.go | 6 +- .../shipper/indexshipper/tsdb/index/index.go | 2 +- .../shipper/indexshipper/tsdb/index_client.go | 18 ++++- .../tsdb/index_shipper_querier.go | 8 +- .../shipper/indexshipper/tsdb/lazy_index.go | 8 +- .../indexshipper/tsdb/multi_file_index.go | 8 +- .../tsdb/multi_file_index_test.go | 2 +- .../shipper/indexshipper/tsdb/multitenant.go | 4 +- .../shipper/indexshipper/tsdb/querier.go | 2 +- .../indexshipper/tsdb/sharding/for_series.go | 28 +------ .../indexshipper/tsdb/single_file_index.go | 41 ++++++++-- .../tsdb/single_file_index_test.go | 6 +- 23 files changed, 203 insertions(+), 233 deletions(-) diff --git a/pkg/bloombuild/common/tsdb.go b/pkg/bloombuild/common/tsdb.go index a58b7cd6130f9..491f3882337e7 100644 --- a/pkg/bloombuild/common/tsdb.go +++ b/pkg/bloombuild/common/tsdb.go @@ -128,34 +128,27 @@ func NewTSDBSeriesIter(ctx context.Context, user string, f sharding.ForSeries, b // TODO(salvacorts): Create a pool series := make([]*v1.Series, 0, 100) - if err := f.ForSeries( - ctx, - user, - bounds, - 0, math.MaxInt64, - func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { - select { - case <-ctx.Done(): - return true - default: - res := &v1.Series{ - Fingerprint: fp, - Chunks: make(v1.ChunkRefs, 0, len(chks)), - } - for _, chk := range chks { - res.Chunks = append(res.Chunks, v1.ChunkRef{ - From: model.Time(chk.MinTime), - Through: model.Time(chk.MaxTime), - Checksum: chk.Checksum, - }) - } - - series = append(series, res) - return false + if err := f.ForSeries(ctx, user, bounds, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + select { + case <-ctx.Done(): + return true + default: + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), } - }, - labels.MustNewMatcher(labels.MatchEqual, "", ""), - ); err != nil { + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + From: model.Time(chk.MinTime), + Through: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + series = append(series, res) + return false + } + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")); err != nil { return nil, err } diff --git a/pkg/bloombuild/common/tsdb_test.go b/pkg/bloombuild/common/tsdb_test.go index b2df7982f4382..3dc9c75f645b3 100644 --- a/pkg/bloombuild/common/tsdb_test.go +++ b/pkg/bloombuild/common/tsdb_test.go @@ -16,15 +16,7 @@ import ( type forSeriesTestImpl []*v1.Series -func (f forSeriesTestImpl) ForSeries( - _ context.Context, - _ string, - _ index.FingerprintFilter, - _ model.Time, - _ model.Time, - fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) bool, - _ ...*labels.Matcher, -) error { +func (f forSeriesTestImpl) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { for i := range f { unmapped := make([]index.ChunkMeta, 0, len(f[i].Chunks)) for _, c := range f[i].Chunks { diff --git a/pkg/bloombuild/planner/strategies/chunksize.go b/pkg/bloombuild/planner/strategies/chunksize.go index b21c90b6f2e28..2b0e57720049d 100644 --- a/pkg/bloombuild/planner/strategies/chunksize.go +++ b/pkg/bloombuild/planner/strategies/chunksize.go @@ -243,46 +243,39 @@ func (s *ChunkSizeStrategy) sizedSeriesIter( currentBatch = newSeriesBatch(idx.tsdbIdentifier) for _, gap := range idx.gaps { - if err := idx.tsdb.ForSeries( - ctx, - tenant, - gap, - 0, math.MaxInt64, - func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { - select { - case <-ctx.Done(): - return true - default: - var seriesSize uint64 - for _, chk := range chks { - seriesSize += uint64(chk.KB * 1024) - } - - // Cut a new batch IF the current batch is not empty (so we add at least one series to the batch) - // AND Adding this series to the batch would exceed the target task size. - if currentBatch.Len() > 0 && currentBatch.Size()+seriesSize > targetTaskSizeBytes { - batches = append(batches, currentBatch) - currentBatch = newSeriesBatch(idx.tsdbIdentifier) - } - - res := &v1.Series{ - Fingerprint: fp, - Chunks: make(v1.ChunkRefs, 0, len(chks)), - } - for _, chk := range chks { - res.Chunks = append(res.Chunks, v1.ChunkRef{ - From: model.Time(chk.MinTime), - Through: model.Time(chk.MaxTime), - Checksum: chk.Checksum, - }) - } - - currentBatch.Append(res, seriesSize) - return false + if err := idx.tsdb.ForSeries(ctx, tenant, gap, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + select { + case <-ctx.Done(): + return true + default: + var seriesSize uint64 + for _, chk := range chks { + seriesSize += uint64(chk.KB * 1024) } - }, - labels.MustNewMatcher(labels.MatchEqual, "", ""), - ); err != nil { + + // Cut a new batch IF the current batch is not empty (so we add at least one series to the batch) + // AND Adding this series to the batch would exceed the target task size. + if currentBatch.Len() > 0 && currentBatch.Size()+seriesSize > targetTaskSizeBytes { + batches = append(batches, currentBatch) + currentBatch = newSeriesBatch(idx.tsdbIdentifier) + } + + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), + } + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + From: model.Time(chk.MinTime), + Through: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + currentBatch.Append(res, seriesSize) + return false + } + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")); err != nil { return nil, 0, err } diff --git a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go index 18480d74c98fc..25696c5c78a85 100644 --- a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go +++ b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go @@ -333,7 +333,7 @@ func newFakeForSeries(series []*v1.Series) *fakeForSeries { } } -func (f fakeForSeries) ForSeries(_ context.Context, _ string, ff index.FingerprintFilter, _ model.Time, _ model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), _ ...*labels.Matcher) error { +func (f fakeForSeries) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { overlapping := make([]*v1.Series, 0, len(f.series)) for _, s := range f.series { if ff.Match(s.Fingerprint) { diff --git a/pkg/indexgateway/gateway.go b/pkg/indexgateway/gateway.go index f535c7fce3776..b6a3200ee3a51 100644 --- a/pkg/indexgateway/gateway.go +++ b/pkg/indexgateway/gateway.go @@ -607,51 +607,44 @@ func accumulateChunksToShards( var mtx sync.Mutex - if err := forSeries.ForSeries( - ctx, - user, - v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), - req.From, req.Through, - func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { - mtx.Lock() - defer mtx.Unlock() - - // check if this is a fingerprint we need - if _, ok := filteredM[fp]; !ok { - return false - } + if err := forSeries.ForSeries(ctx, user, v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { + mtx.Lock() + defer mtx.Unlock() - filteredChks := filteredM[fp] - var j int - - outer: - for i := range filteredChks { - for j < len(chks) { - switch filteredChks[i].Cmp(chks[j]) { - case iter.Less: - // this chunk is not in the queried index, continue checking other chunks - continue outer - case iter.Greater: - // next chunk in index but didn't pass filter; continue - j++ - continue - case iter.Eq: - // a match; set the sizing info - filteredChks[i].KB = chks[j].KB - filteredChks[i].Entries = chks[j].Entries - j++ - continue outer - } - } + // check if this is a fingerprint we need + if _, ok := filteredM[fp]; !ok { + return false + } - // we've finished this index's chunks; no need to keep checking filtered chunks - break + filteredChks := filteredM[fp] + var j int + + outer: + for i := range filteredChks { + for j < len(chks) { + switch filteredChks[i].Cmp(chks[j]) { + case iter.Less: + // this chunk is not in the queried index, continue checking other chunks + continue outer + case iter.Greater: + // next chunk in index but didn't pass filter; continue + j++ + continue + case iter.Eq: + // a match; set the sizing info + filteredChks[i].KB = chks[j].KB + filteredChks[i].Entries = chks[j].Entries + j++ + continue outer + } } - return false - }, - p.Matchers..., - ); err != nil { + // we've finished this index's chunks; no need to keep checking filtered chunks + break + } + + return false + }, nil, p.Matchers...); err != nil { return nil, nil, err } diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index a56b2ac551e44..56a0cf1842651 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -292,7 +292,7 @@ func (c CompositeStore) HasForSeries(from, through model.Time) (sharding.ForSeri matchers ...*labels.Matcher, ) error { for _, impl := range impls { - if err := impl.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...); err != nil { + if err := impl.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...); err != nil { return err } } diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index fd74fca4e00da..0ea36f671e5b6 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -209,7 +209,7 @@ func (m MonitoredReaderWriter) HasForSeries(from, through model.Time) (sharding. matchers ...*labels.Matcher, ) error { return loki_instrument.TimeRequest(ctx, "for_series", instrument.NewHistogramCollector(m.metrics.indexQueryLatency), instrument.ErrorCode, func(ctx context.Context) error { - return impl.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + return impl.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) }) }, ) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 01198330521ea..22f0bfb484313 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -181,6 +181,10 @@ func (h *Head) updateSeriesStats(fp uint64, stats *StreamStats) { h.series.updateSeriesStats(fp, stats) } +func (h *Head) SeriesStats() (StreamStats, error) { + return h.series.SeriesStats() +} + func (h *Head) ResetSeriesStats() { h.series.resetSeriesStats() } @@ -261,6 +265,18 @@ func (s *stripeSeries) getByID(id uint64) *memSeries { return x.m[id] } +func (s *stripeSeries) SeriesStats() (StreamStats, error) { + var stats StreamStats + for _, seriesMap := range s.series { + seriesMap.RLock() + defer seriesMap.RUnlock() + for _, series := range seriesMap.m { + stats.Merge(series.stats.Copy()) + } + } + return stats, nil +} + // Append adds chunks to the correct series and returns whether a new series was added func (s *stripeSeries) Append( ls labels.Labels, diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index b35e614a60051..a96e9ac3830d2 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -759,12 +759,12 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } -func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, _ []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil, nil } - return idx.GetChunkRefs(ctx, userID, from, through, nil, fpFilter, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, nil, nil, fpFilter, matchers...) } @@ -812,12 +812,12 @@ func (t *tenantHeads) Volume(ctx context.Context, userID string, from, through m return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { +func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil } - return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) } // helper only used in building TSDBs @@ -839,7 +839,8 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c chks []index.ChunkMeta ) - fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks) + // h11: Pass stream stats + fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks, nil) if err != nil { return errors.Wrapf(err, "iterating postings for tenant: %s", tenant) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go index 400e827ec3f76..dc57537ee982f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go @@ -110,14 +110,7 @@ func Test_TenantHeads_Append(t *testing.T) { } _ = h.Append("fake", ls, ls.Hash(), chks) - found, err := h.GetChunkRefs( - context.Background(), - "fake", - 0, - 100, - nil, nil, - labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), - ) + found, err := h.GetChunkRefs(context.Background(), "fake", 0, 100, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs("fake", ls.Hash(), chks), found) @@ -165,14 +158,7 @@ func Test_TenantHeads_MultiRead(t *testing.T) { // ensure we're only returned the data from the correct tenant for _, tenant := range tenants { - found, err := h.GetChunkRefs( - context.Background(), - tenant.user, - 0, - 100, - nil, nil, - labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), - ) + found, err := h.GetChunkRefs(context.Background(), tenant.user, 0, 100, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(tenant.user, tenant.ls.Hash(), chks), found) } @@ -254,13 +240,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, recoverHead(mgr.name, mgr.dir, mgr.activeHeads, grp.wals, false)) for _, c := range cases { - refs, err := mgr.GetChunkRefs( - context.Background(), - c.User, - 0, math.MaxInt64, - nil, nil, - labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), - ) + refs, err := mgr.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(c.User, c.Fingerprint, c.Chunks), refs) } @@ -309,13 +289,7 @@ func Test_HeadManager_QueryAfterRotate(t *testing.T) { mgr.tick(nextPeriod) // synthetic tick to rotate head for _, c := range cases { - refs, err := mgr.GetChunkRefs( - context.Background(), - c.User, - 0, math.MaxInt64, - nil, nil, - labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), - ) + refs, err := mgr.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(c.User, c.Fingerprint, c.Chunks), refs) } @@ -385,13 +359,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { multiIndex := NewMultiIndex(IndexSlice{mgr, mgr.tsdbManager.(noopTSDBManager).tenantHeads}) for _, c := range cases { - refs, err := multiIndex.GetChunkRefs( - context.Background(), - c.User, - 0, math.MaxInt64, - nil, nil, - labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), - ) + refs, err := multiIndex.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) require.Nil(t, err) lbls := labels.NewBuilder(c.Labels) @@ -420,13 +388,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { // Ensure old + new data is queryable for _, c := range append(cases, newCase) { - refs, err := multiIndex.GetChunkRefs( - context.Background(), - c.User, - 0, math.MaxInt64, - nil, nil, - labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), - ) + refs, err := multiIndex.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) require.Nil(t, err) lbls := labels.NewBuilder(c.Labels) @@ -657,14 +619,7 @@ func BenchmarkTenantHeads(b *testing.B) { tenant := r % nTenants // nolint:ineffassign,staticcheck - res, _ = heads.GetChunkRefs( - context.Background(), - fmt.Sprint(tenant), - 0, math.MaxInt64, - res, - nil, - labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), - ) + res, _ = heads.GetChunkRefs(context.Background(), fmt.Sprint(tenant), 0, math.MaxInt64, nil, res, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) }(r) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index cf709e7bd97c0..ccfb8711716fc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -41,6 +41,10 @@ type headIndexReader struct { mint, maxt int64 } +func (h *headIndexReader) SeriesStats() (StreamStats, error) { + hd := h.head + return hd.SeriesStats() +} func (h *headIndexReader) Bounds() (int64, int64) { return h.head.MinTime(), h.head.MaxTime() } @@ -122,7 +126,7 @@ func (h *headIndexReader) Postings(name string, fpFilter index.FingerprintFilter } // Series returns the series for the given reference. -func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]index.ChunkMeta) (uint64, error) { +func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]index.ChunkMeta, stats **StreamStats) (uint64, error) { s := h.head.series.getByID(uint64(ref)) if s == nil { @@ -131,6 +135,8 @@ func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int6 } *lbls = append((*lbls)[:0], s.ls...) + *stats = &s.stats + queryBounds := newBounds(model.Time(from), model.Time(through)) *chks = (*chks)[:0] diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index a60b86b6a6e00..f7c006c93c721 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -58,7 +58,7 @@ type Index interface { // the requested shard. If it is nil, TSDB will return all results, // regardless of shard. // Note: any shard used must be a valid factor of two, meaning `0_of_2` and `3_of_4` are fine, but `0_of_3` is not. - GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) + GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) // Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) @@ -71,7 +71,7 @@ type NoopIndex struct{} func (NoopIndex) Close() error { return nil } func (NoopIndex) Bounds() (_, through model.Time) { return } -func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, _ []ChunkRef, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]ChunkRef, error) { +func (NoopIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { return nil, nil } @@ -96,6 +96,6 @@ func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAc return nil } -func (NoopIndex) ForSeries(_ context.Context, _ string, _ index.FingerprintFilter, _ model.Time, _ model.Time, _ func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), _ ...*labels.Matcher) error { +func (NoopIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 6ef473b450c78..97bd3700deee1 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -1775,7 +1775,7 @@ func (r *Reader) LabelValueFor(id storage.SeriesRef, label string) (string, erro } // Series reads the series with the given ID and writes its labels and chunks into lbls and chks. -func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *tsdb.StreamStats) (uint64, error) { +func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats **tsdb.StreamStats) (uint64, error) { offset := id // In version 2+ series IDs are no longer exact references but series are 16-byte padded // and the ID is the multiple of 16 of the actual position. diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index 7031a8488cafd..b400c9f6aa381 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -3,12 +3,15 @@ package tsdb import ( "context" "sort" + "strings" "sync" "time" + "github.com/go-kit/log/level" "github.com/grafana/loki/v3/pkg/logql" v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" + util_log "github.com/grafana/loki/v3/pkg/util/log" "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" @@ -122,8 +125,19 @@ func (c *IndexClient) GetChunkRefs(ctx context.Context, userID string, from, thr return nil, err } + // h11: somehow check if the predicates have structured metadata labels + var filterLabelNames []string + filters := syntax.ExtractLabelFiltersBeforeParser(predicate.Plan().AST) + for _, f := range filters { + filterLabelNames = append(filterLabelNames, f.RequiredLabelNames()...) + } + if len(filterLabelNames) > 0 { + level.Debug(util_log.Logger).Log("msg", "filtering by label names", "label_names", strings.Join(filterLabelNames, ",")) + // h11: dedup + } + // TODO(owen-d): use a pool to reduce allocs here - chks, err := c.idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) + chks, err := c.idx.GetChunkRefs(ctx, userID, from, through, filterLabelNames, nil, shard, matchers...) if err != nil { return nil, err } @@ -290,7 +304,7 @@ func (c *IndexClient) GetShards(ctx context.Context, userID string, from, throug m[fp] = append(m[fp], chks...) mtx.Unlock() return false - }, predicate.Matchers...); err != nil { + }, nil, predicate.Matchers...); err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index 6ca252770169e..b0bb007cf564e 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -84,12 +84,12 @@ func (i *indexShipperQuerier) Close() error { return nil } -func (i *indexShipperQuerier) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *indexShipperQuerier) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { idx, err := i.indices(ctx, from, through, userID) if err != nil { return nil, err } - return idx.GetChunkRefs(ctx, userID, from, through, res, fpFilter, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, nil, res, fpFilter, matchers...) } func (i *indexShipperQuerier) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { @@ -134,13 +134,13 @@ func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, t return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { +func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { idx, err := i.indices(ctx, from, through, userID) if err != nil { return err } - return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) } type resultAccumulator[T any] struct { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index ad3fb3b086200..17241994d70e8 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -36,12 +36,12 @@ func (f LazyIndex) Close() error { return i.Close() } -func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { i, err := f() if err != nil { return nil, err } - return i.GetChunkRefs(ctx, userID, from, through, res, fpFilter, matchers...) + return i.GetChunkRefs(ctx, userID, from, through, nil, res, fpFilter, matchers...) } func (f LazyIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { i, err := f() @@ -81,10 +81,10 @@ func (f LazyIndex) Volume(ctx context.Context, userID string, from, through mode return i.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { +func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { i, err := f() if err != nil { return err } - return i.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + return i.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index 23bab83e41708..f91a9217ccaae 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -132,7 +132,7 @@ func (i *MultiIndex) forMatchingIndices(ctx context.Context, from, through model } -func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { acc := newResultAccumulator(func(xs [][]ChunkRef) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() @@ -170,7 +170,7 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro from, through, func(ctx context.Context, idx Index) error { - got, err := idx.GetChunkRefs(ctx, userID, from, through, nil, fpFilter, matchers...) + got, err := idx.GetChunkRefs(ctx, userID, from, through, nil, nil, fpFilter, matchers...) if err != nil { return err } @@ -371,8 +371,8 @@ func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through mo }) } -func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { +func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { - return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) }) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go index 8139c52b39fc0..2d2c6fda24b56 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go @@ -67,7 +67,7 @@ func TestMultiIndex(t *testing.T) { idx := NewMultiIndex(IndexSlice(indices)) t.Run("GetChunkRefs", func(t *testing.T) { - refs, err := idx.GetChunkRefs(context.Background(), "fake", 2, 5, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + refs, err := idx.GetChunkRefs(context.Background(), "fake", 2, 5, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Nil(t, err) expected := []ChunkRef{ diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index 403443a805c0d..163d7c4ac2a8e 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -97,6 +97,6 @@ func (m *MultiTenantIndex) Volume(ctx context.Context, userID string, from, thro return m.idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { - return m.idx.ForSeries(ctx, userID, fpFilter, from, through, fn, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { + return m.idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, withTenantLabelMatcher(userID, matchers)...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go index 60ec32ee954b0..fa5ce9a2ff29c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go @@ -66,7 +66,7 @@ type IndexReader interface { // Series populates the given labels and chunk metas for the series identified // by the reference. // Returns storage.ErrNotFound if the ref does not resolve to a known series. - Series(ref storage.SeriesRef, from int64, through int64, lset *labels.Labels, chks *[]index.ChunkMeta) (uint64, error) + Series(ref storage.SeriesRef, from int64, through int64, lset *labels.Labels, chks *[]index.ChunkMeta, stats **StreamStats) (uint64, error) // ChunkStats returns the stats for the chunks in the given series. ChunkStats(ref storage.SeriesRef, from, through int64, lset *labels.Labels, by map[string]struct{}) (uint64, index.ChunkStats, error) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go index bca81214dc9ce..0500500a2331f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go @@ -19,19 +19,7 @@ import ( // such as when the Index is backed by multiple files with the same series present. // NB(owen-d): mainly in this package to avoid circular dependencies elsewhere type ForSeries interface { - ForSeries( - ctx context.Context, - userID string, - fpFilter index.FingerprintFilter, - from model.Time, - through model.Time, - fn func( - labels.Labels, - model.Fingerprint, - []index.ChunkMeta, - ) (stop bool), - matchers ...*labels.Matcher, - ) error + ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error } // function Adapter for ForSeries implementation @@ -49,18 +37,6 @@ type ForSeriesFunc func( matchers ...*labels.Matcher, ) error -func (f ForSeriesFunc) ForSeries( - ctx context.Context, - userID string, - fpFilter index.FingerprintFilter, - from model.Time, - through model.Time, - fn func( - labels.Labels, - model.Fingerprint, - []index.ChunkMeta, - ) (stop bool), - matchers ...*labels.Matcher, -) error { +func (f ForSeriesFunc) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return f(ctx, userID, fpFilter, from, through, fn, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 6bd7e6e79a251..5edf9657afbbc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -161,9 +161,10 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { // Iteration will stop if the callback returns true. // Accepts a userID argument in order to implement `Index` interface, but since this is a single tenant index, // it is ignored (it's enforced elsewhere in index selection) -func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), matchers ...*labels.Matcher) error { +func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { // TODO(owen-d): use pool + var stats *StreamStats var ls labels.Labels chks := ChunkMetasPool.Get() defer ChunkMetasPool.Put(chks) @@ -175,11 +176,41 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.Fing return i.forPostings(ctx, fpFilter, from, through, matchers, func(p index.Postings) error { for p.Next() { - hash, err := i.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) + hash, err := i.reader.Series(p.At(), int64(from), int64(through), &ls, &chks, &stats) if err != nil { return err } + if stats != nil { + // Looking at the structured metadata fields names, discard all series for which we don't have the required filter label names + allFound := false + for _, ln := range filterLabelNames { + var found bool + // Search label name in stream labels + for _, lbs := range ls { + if lbs.Name == ln { + found = true + break + } + } + // If not a stream label, search in structured metadata fields + if !found { + if _, ok := stats.StructuredMetadataFieldNames[ln]; ok { + found = true + } + } + // If label name not found at all, we can skip this series + if !found { + allFound = false + break + } + } + + if !allFound { + continue + } + } + // skip series that belong to different shards if fpFilter != nil && !fpFilter.Match(model.Fingerprint(hash)) { continue @@ -212,7 +243,7 @@ func (i *TSDBIndex) forPostings( return fn(p) } -func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, _ []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() } @@ -230,7 +261,7 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu }) } return false - }, matchers...); err != nil { + }, nil, matchers...); err != nil { return nil, err } @@ -252,7 +283,7 @@ func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Ti Fingerprint: fp, }) return false - }, matchers...); err != nil { + }, nil, matchers...); err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go index 9784475091bf8..a78fed4d3815c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go @@ -89,7 +89,7 @@ func TestSingleIdx(t *testing.T) { t.Run(variant.desc, func(t *testing.T) { idx := variant.fn() t.Run("GetChunkRefs", func(t *testing.T) { - refs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + refs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Nil(t, err) expected := []ChunkRef{ @@ -130,7 +130,7 @@ func TestSingleIdx(t *testing.T) { Shard: 1, Of: 2, } - shardedRefs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + shardedRefs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.Nil(t, err) @@ -251,7 +251,7 @@ func BenchmarkTSDBIndex_GetChunkRefs(b *testing.B) { 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")) + chkRefs, err := tsdbIndex.GetChunkRefs(context.Background(), "fake", queryFrom, queryThrough, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) require.NoError(b, err) require.Len(b, chkRefs, numChunksToMatch*2) } From cc5ba777e486d117a6da8effe9abc51c079fc19c Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Tue, 3 Dec 2024 16:55:14 +0530 Subject: [PATCH 06/21] Compiler fixes --- pkg/ingester/flush.go | 4 +- pkg/ingester/stream.go | 28 ++++---- pkg/storage/store.go | 5 ++ pkg/storage/stores/composite_store.go | 5 +- pkg/storage/stores/index/index.go | 7 +- pkg/storage/stores/series/index/index.go | 1 + .../series/series_index_gateway_store.go | 5 ++ .../stores/series/series_index_store.go | 5 ++ pkg/storage/stores/series_store_write.go | 4 +- pkg/storage/stores/series_store_write_test.go | 5 ++ .../shipper/indexshipper/tsdb/builder.go | 23 +++--- .../shipper/indexshipper/tsdb/compactor.go | 6 +- .../stores/shipper/indexshipper/tsdb/head.go | 71 +------------------ .../shipper/indexshipper/tsdb/head_manager.go | 4 +- .../shipper/indexshipper/tsdb/head_read.go | 6 +- .../shipper/indexshipper/tsdb/index/index.go | 67 +++++++++++++++-- .../shipper/indexshipper/tsdb/multitenant.go | 4 +- .../shipper/indexshipper/tsdb/querier.go | 2 +- .../indexshipper/tsdb/single_file_index.go | 5 +- .../stores/shipper/indexshipper/tsdb/store.go | 12 +++- 20 files changed, 145 insertions(+), 124 deletions(-) diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 07f580f6b2c00..63c10057e5f93 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -15,7 +15,7 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/tenant" "github.com/grafana/dskit/user" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -328,7 +328,7 @@ func (i *Ingester) flushUserSeries(ctx context.Context, userID string, fp model. return nil } -func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, *tsdb.StreamStats, *sync.RWMutex) { +func (i *Ingester) collectChunksToFlush(instance *instance, fp model.Fingerprint, immediate bool) ([]*chunkDesc, labels.Labels, *index.StreamStats, *sync.RWMutex) { var stream *stream var ok bool stream, ok = instance.streams.LoadByFP(fp) diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 15ac155188064..325dcb33d4a23 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -11,7 +11,7 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -85,9 +85,9 @@ type stream struct { configs *runtime.TenantConfigs // closedStreamStats has the stats up to the most recent closed chunk. - closedStreamStats *tsdb.StreamStats + closedStreamStats *index.StreamStats // openStreamStats has the stats up to the most recent open chunk (i.e. closedChunksStats + newer data). - openStreamStats *tsdb.StreamStats + openStreamStats *index.StreamStats } type chunkDesc struct { @@ -139,6 +139,8 @@ func newStream( chunkHeadBlockFormat: headBlockFmt, configs: configs, + + openStreamStats: index.NewStreamStats(), } } @@ -182,18 +184,18 @@ func (s *stream) NewChunk() *chunkenc.MemChunk { func (s *stream) Push( ctx context.Context, entries []logproto.Entry, - // WAL record to add push contents to. - // May be nil to disable this functionality. +// WAL record to add push contents to. +// May be nil to disable this functionality. record *wal.Record, - // Counter used in WAL replay to avoid duplicates. - // If this is non-zero, the stream will reject entries - // with a counter value less than or equal to it's own. - // It is set to zero and thus bypassed outside of WAL replays. +// Counter used in WAL replay to avoid duplicates. +// If this is non-zero, the stream will reject entries +// with a counter value less than or equal to it's own. +// It is set to zero and thus bypassed outside of WAL replays. counter int64, - // Lock chunkMtx while pushing. - // If this is false, chunkMtx must be held outside Push. +// Lock chunkMtx while pushing. +// If this is false, chunkMtx must be held outside Push. lockChunk bool, - // Whether nor not to ingest all at once or not. It is a per-tenant configuration. +// Whether nor not to ingest all at once or not. It is a per-tenant configuration. rateLimitWholeStream bool, usageTracker push.UsageTracker, @@ -665,7 +667,7 @@ func (s *stream) cutSeriesStats() { s.openStreamStats = s.closedStreamStats.Copy() } -func (s *stream) flushableSeriesStats() *tsdb.StreamStats { +func (s *stream) flushableSeriesStats() *index.StreamStats { return s.closedStreamStats } diff --git a/pkg/storage/store.go b/pkg/storage/store.go index a8e6a1add3239..657acd1171746 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -6,6 +6,7 @@ import ( "math" "time" + index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/types" "github.com/grafana/loki/v3/pkg/util/httpreq" @@ -595,6 +596,10 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. type failingChunkWriter struct{} +func (f failingChunkWriter) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *index2.StreamStats) error { + return nil +} + func (f failingChunkWriter) Put(_ context.Context, _ []chunk.Chunk) error { return errWritingChunkUnsupported } diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 56a0cf1842651..2eef0c34083ec 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -4,7 +4,6 @@ import ( "context" "sort" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -22,7 +21,7 @@ import ( type ChunkWriter interface { Put(ctx context.Context, chunks []chunk.Chunk) error PutOne(ctx context.Context, from, through model.Time, chunk chunk.Chunk) error - UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error + UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb_index.StreamStats) error } type ChunkFetcherProvider interface { @@ -108,7 +107,7 @@ func (c CompositeStore) PutOne(ctx context.Context, from, through model.Time, ch }) } -func (c CompositeStore) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error { +func (c CompositeStore) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *tsdb_index.StreamStats) error { return c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { return store.UpdateSeriesStats(innerCtx, from, through, userID, fp, stats) }) diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 0ea36f671e5b6..89cc6a0d9df55 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -5,7 +5,6 @@ import ( "time" "github.com/grafana/dskit/instrument" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -54,7 +53,7 @@ type Reader interface { type Writer interface { IndexChunk(ctx context.Context, from, through model.Time, chk chunk.Chunk) error - UpdateSeriesStats(userID string, fp uint64, stats *tsdb.StreamStats) + UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) } type ReaderWriter interface { @@ -67,6 +66,10 @@ type MonitoredReaderWriter struct { metrics *metrics } +func (m MonitoredReaderWriter) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { + m.rw.UpdateSeriesStats(userID, fp, stats) +} + func NewMonitoredReaderWriter(rw ReaderWriter, reg prometheus.Registerer) *MonitoredReaderWriter { return &MonitoredReaderWriter{ rw: rw, diff --git a/pkg/storage/stores/series/index/index.go b/pkg/storage/stores/series/index/index.go index 5011c7e8f0815..bfc4e385ba2f8 100644 --- a/pkg/storage/stores/series/index/index.go +++ b/pkg/storage/stores/series/index/index.go @@ -16,6 +16,7 @@ type ReadClient interface { type WriteClient interface { NewWriteBatch() WriteBatch BatchWrite(context.Context, WriteBatch) error + // h11: UpdateSeriesStats(userID string, fp uint64, stats *StreamStats)?? } // Client is a client for the storage of the index (e.g. DynamoDB or Bigtable). diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index 992762d8c5b1b..bc729bbecd879 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -6,6 +6,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -36,6 +37,10 @@ type IndexGatewayClientStore struct { logger log.Logger } +func (c *IndexGatewayClientStore) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { + level.Warn(c.logger).Log("msg", "UpdateSeriesStats called on index gateway client store, but it does not support it") +} + func NewIndexGatewayClientStore(client GatewayClient, logger log.Logger) *IndexGatewayClientStore { return &IndexGatewayClientStore{ client: client, diff --git a/pkg/storage/stores/series/series_index_store.go b/pkg/storage/stores/series/series_index_store.go index c88e15e0cf1c7..3490fd520e378 100644 --- a/pkg/storage/stores/series/series_index_store.go +++ b/pkg/storage/stores/series/series_index_store.go @@ -8,6 +8,7 @@ import ( "sync" "github.com/go-kit/log/level" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" @@ -75,6 +76,10 @@ type IndexReaderWriter struct { writeDedupeCache cache.Cache } +func (c *IndexReaderWriter) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { + level.Warn(util_log.Logger).Log("msg", "UpdateSeriesStats called on index gateway client store, but it does not support it") +} + func NewIndexReaderWriter(schemaCfg config.SchemaConfig, schema series_index.SeriesStoreSchema, index series_index.Client, fetcher *fetcher.Fetcher, chunkBatchSize int, writeDedupeCache cache.Cache) *IndexReaderWriter { return &IndexReaderWriter{ diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index e03f04d370ef4..b120f16dace7f 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -4,7 +4,7 @@ import ( "context" "github.com/go-kit/log/level" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" + index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -129,7 +129,7 @@ func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk return nil } -func (c *Writer) UpdateSeriesStats(_ context.Context, _, _ model.Time, userID string, fp uint64, stats *tsdb.StreamStats) error { +func (c *Writer) UpdateSeriesStats(_ context.Context, _, _ model.Time, userID string, fp uint64, stats *index2.StreamStats) error { c.indexWriter.UpdateSeriesStats(userID, fp, stats) return nil } diff --git a/pkg/storage/stores/series_store_write_test.go b/pkg/storage/stores/series_store_write_test.go index 5ff8a00d99706..a8785c9753daa 100644 --- a/pkg/storage/stores/series_store_write_test.go +++ b/pkg/storage/stores/series_store_write_test.go @@ -4,6 +4,7 @@ import ( "context" "testing" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -47,6 +48,10 @@ type mockIndexWriter struct { called int } +func (m *mockIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { + return +} + func (m *mockIndexWriter) IndexChunk(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { m.called++ return nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go index 5051447600111..34c271d771061 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go @@ -32,7 +32,7 @@ type stream struct { labels labels.Labels fp model.Fingerprint chunks index.ChunkMetas - stats *StreamStats + stats *index.StreamStats } func NewBuilder(version int) *Builder { @@ -43,13 +43,14 @@ func NewBuilder(version int) *Builder { } // TODO(h11): stats should be variadic but I added it to avoid changinf it in many places -func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats ...*StreamStats) { +func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats ...*index.StreamStats) { id := ls.String() s, ok := b.streams[id] if !ok { s = &stream{ labels: ls, fp: fp, + stats: index.NewStreamStats(), } b.streams[id] = s } @@ -104,10 +105,10 @@ func (b *Builder) DropChunk(streamID string, chk index.ChunkMeta) (bool, error) func (b *Builder) Build( ctx context.Context, scratchDir string, - // Determines how to create the resulting Identifier and file name. - // This is variable as we use Builder for multiple reasons, - // such as building multi-tenant tsdbs on the ingester - // and per tenant ones during compaction +// Determines how to create the resulting Identifier and file name. +// This is variable as we use Builder for multiple reasons, +// such as building multi-tenant tsdbs on the ingester +// and per tenant ones during compaction createFn func(from, through model.Time, checksum uint32) Identifier, ) (id Identifier, err error) { // Ensure the parent dir exists (i.e. index///) @@ -218,7 +219,7 @@ func (b *Builder) build( if !b.chunksFinalized { s.chunks = s.chunks.Finalize() } - if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.fp, s.chunks...); err != nil { + if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.fp, s.chunks, s.stats); err != nil { return nil, err } } @@ -228,10 +229,10 @@ func (b *Builder) build( func (b *Builder) BuildInMemory( ctx context.Context, - // Determines how to create the resulting Identifier and file name. - // This is variable as we use Builder for multiple reasons, - // such as building multi-tenant tsdbs on the ingester - // and per tenant ones during compaction +// Determines how to create the resulting Identifier and file name. +// This is variable as we use Builder for multiple reasons, +// such as building multi-tenant tsdbs on the ingester +// and per tenant ones during compaction createFn func(from, through model.Time, checksum uint32) Identifier, ) (id Identifier, data []byte, err error) { writer, err := index.NewMemWriterWithVersion(ctx, b.version) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go index 06028ce1d63d2..9ee3920c3c254 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go @@ -56,7 +56,7 @@ func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableN err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { builder.AddSeries(lbls.Copy(), fp, chks) return false - }, labels.MustNewMatcher(labels.MatchEqual, "", "")) + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { return nil, err } @@ -216,7 +216,7 @@ func setupBuilder(ctx context.Context, indexType int, userID string, sourceIndex err := idx.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { builder.AddSeries(withoutTenantLabel(lbls.Copy()), fp, chks) return false - }, withTenantLabelMatcher(userID, []*labels.Matcher{})...) + }, nil, withTenantLabelMatcher(userID, []*labels.Matcher{})...) if err != nil { return nil, err } @@ -249,7 +249,7 @@ func setupBuilder(ctx context.Context, indexType int, userID string, sourceIndex err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { builder.AddSeries(lbls.Copy(), fp, chks) return false - }, labels.MustNewMatcher(labels.MatchEqual, "", "")) + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 22f0bfb484313..c8e978b42125a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -16,7 +16,6 @@ import ( "sync" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/push" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" @@ -177,14 +176,10 @@ func (h *Head) Append(ls labels.Labels, fprint uint64, chks index.ChunkMetas) (c return } -func (h *Head) updateSeriesStats(fp uint64, stats *StreamStats) { +func (h *Head) updateSeriesStats(fp uint64, stats *index.StreamStats) { h.series.updateSeriesStats(fp, stats) } -func (h *Head) SeriesStats() (StreamStats, error) { - return h.series.SeriesStats() -} - func (h *Head) ResetSeriesStats() { h.series.resetSeriesStats() } @@ -265,18 +260,6 @@ func (s *stripeSeries) getByID(id uint64) *memSeries { return x.m[id] } -func (s *stripeSeries) SeriesStats() (StreamStats, error) { - var stats StreamStats - for _, seriesMap := range s.series { - seriesMap.RLock() - defer seriesMap.RUnlock() - for _, series := range seriesMap.m { - stats.Merge(series.stats.Copy()) - } - } - return stats, nil -} - // Append adds chunks to the correct series and returns whether a new series was added func (s *stripeSeries) Append( ls labels.Labels, @@ -309,7 +292,7 @@ func (s *stripeSeries) Append( return } -func (s *stripeSeries) updateSeriesStats(fp uint64, stats *StreamStats) { +func (s *stripeSeries) updateSeriesStats(fp uint64, stats *index.StreamStats) { series := s.getByID(fp) series.Lock() @@ -327,54 +310,6 @@ func (s *stripeSeries) resetSeriesStats() { } } -// h11: add a mutex. -// Reset takes the muted to reset all the stats -// AddStructuredMetadata takes the mutes and updates the stats -// The index manager calls Reset when the index is written -// In the ingester push logic, we keep updating the index stats -type StreamStats struct { - mu sync.RWMutex - StructuredMetadataFieldNames map[string]struct{} -} - -func (s *StreamStats) Copy() *StreamStats { - s.mu.RLock() - defer s.mu.RUnlock() - - out := &StreamStats{ - StructuredMetadataFieldNames: make(map[string]struct{}, len(s.StructuredMetadataFieldNames)), - } - for k := range s.StructuredMetadataFieldNames { - out.StructuredMetadataFieldNames[k] = struct{}{} - } - return out -} - -func (s *StreamStats) Merge(other *StreamStats) { - s.mu.Lock() - defer s.mu.Unlock() - - for k := range other.StructuredMetadataFieldNames { - s.StructuredMetadataFieldNames[k] = struct{}{} - } -} - -func (s *StreamStats) AddStructuredMetadata(metadata push.LabelsAdapter) { - s.mu.Lock() - defer s.mu.Unlock() - - for _, l := range metadata { - s.StructuredMetadataFieldNames[l.Name] = struct{}{} - } -} - -func (s *StreamStats) Reset() { - s.mu.Lock() - defer s.mu.Unlock() - - s.StructuredMetadataFieldNames = make(map[string]struct{}, len(s.StructuredMetadataFieldNames)) -} - type memSeries struct { // (h11) potentially add stats? sync.RWMutex @@ -382,7 +317,7 @@ type memSeries struct { ls labels.Labels fp uint64 chks index.ChunkMetas - stats StreamStats + stats index.StreamStats } func newMemSeries(ref uint64, ls labels.Labels, fp uint64) *memSeries { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index a96e9ac3830d2..70f938a56a233 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -259,7 +259,7 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, fprint uint64, chk return m.active.Log(rec) } -func (m *HeadManager) UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) { +func (m *HeadManager) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { m.activeHeads.updateSeriesStats(userID, fp, stats) } @@ -696,7 +696,7 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chk return rec } -func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats *StreamStats) { +func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { // (h11) : don't create head.extract to a different function to just get head := t.getOrCreateTenantHead(userID) head.updateSeriesStats(fp, stats) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index ccfb8711716fc..633b00a8fe615 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -41,10 +41,6 @@ type headIndexReader struct { mint, maxt int64 } -func (h *headIndexReader) SeriesStats() (StreamStats, error) { - hd := h.head - return hd.SeriesStats() -} func (h *headIndexReader) Bounds() (int64, int64) { return h.head.MinTime(), h.head.MaxTime() } @@ -126,7 +122,7 @@ func (h *headIndexReader) Postings(name string, fpFilter index.FingerprintFilter } // Series returns the series for the given reference. -func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]index.ChunkMeta, stats **StreamStats) (uint64, error) { +func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]index.ChunkMeta, stats **index.StreamStats) (uint64, error) { s := h.head.series.getByID(uint64(ref)) if s == nil { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 97bd3700deee1..6df91d17324b6 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -18,7 +18,6 @@ import ( "context" "encoding/binary" "fmt" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" "hash" "hash/crc32" "io" @@ -30,6 +29,8 @@ import ( "time" "unsafe" + "github.com/grafana/loki/pkg/push" + "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -404,7 +405,7 @@ func (w *Creator) writeMeta() error { // fingerprint differs from what labels.Hash() produces. For example, // multitenant TSDBs embed a tenant label, but the actual series has no such // label and so the derived fingerprint differs. -func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks []ChunkMeta, stats ...*tsdb.StreamStats) error { +func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks []ChunkMeta, stats ...*StreamStats) error { if err := w.ensureStage(idxStageSeries); err != nil { return err } @@ -1775,7 +1776,7 @@ func (r *Reader) LabelValueFor(id storage.SeriesRef, label string) (string, erro } // Series reads the series with the given ID and writes its labels and chunks into lbls and chks. -func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats **tsdb.StreamStats) (uint64, error) { +func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats **StreamStats) (uint64, error) { offset := id // In version 2+ series IDs are no longer exact references but series are 16-byte padded // and the ID is the multiple of 16 of the actual position. @@ -1787,7 +1788,7 @@ func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *l return 0, d.Err() } - fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks, stats) + fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks, *stats) if err != nil { return 0, errors.Wrap(err, "read series") } @@ -2193,7 +2194,7 @@ func (dec *Decoder) prepSeries(b []byte, lbls *labels.Labels, chks *[]ChunkMeta) return &d, fprint, nil } -func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *tsdb.StreamStats) error { +func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *StreamStats) error { nSMFieldNames := d.Uvarint() stats.StructuredMetadataFieldNames = make(map[string]struct{}, nSMFieldNames) @@ -2397,7 +2398,7 @@ func (dec *Decoder) readChunkStatsPriorV3(d *encoding.Decbuf, seriesRef storage. } // Series decodes a series entry from the given byte slice into lset and chks. -func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *tsdb.StreamStats) (uint64, error) { +func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *StreamStats) (uint64, error) { d, fprint, err := dec.prepSeries(b, lbls, chks) if err != nil { return 0, err @@ -2572,3 +2573,57 @@ func overlap(from, through, chkFrom, chkThrough int64) bool { // sample timestamp in the chunk, whereas through is exclusive return from <= chkThrough && through > chkFrom } + +// h11: add a mutex. +// Reset takes the muted to reset all the stats +// AddStructuredMetadata takes the mutes and updates the stats +// The index manager calls Reset when the index is written +// In the ingester push logic, we keep updating the index stats +type StreamStats struct { + mu sync.RWMutex + StructuredMetadataFieldNames map[string]struct{} +} + +func NewStreamStats() *StreamStats { + return &StreamStats{ + StructuredMetadataFieldNames: make(map[string]struct{}), + } +} + +func (s *StreamStats) Copy() *StreamStats { + s.mu.RLock() + defer s.mu.RUnlock() + + out := &StreamStats{ + StructuredMetadataFieldNames: make(map[string]struct{}, len(s.StructuredMetadataFieldNames)), + } + for k := range s.StructuredMetadataFieldNames { + out.StructuredMetadataFieldNames[k] = struct{}{} + } + return out +} + +func (s *StreamStats) Merge(other *StreamStats) { + s.mu.Lock() + defer s.mu.Unlock() + + for k := range other.StructuredMetadataFieldNames { + s.StructuredMetadataFieldNames[k] = struct{}{} + } +} + +func (s *StreamStats) AddStructuredMetadata(metadata push.LabelsAdapter) { + s.mu.Lock() + defer s.mu.Unlock() + + for _, l := range metadata { + s.StructuredMetadataFieldNames[l.Name] = struct{}{} + } +} + +func (s *StreamStats) Reset() { + s.mu.Lock() + defer s.mu.Unlock() + + s.StructuredMetadataFieldNames = make(map[string]struct{}, len(s.StructuredMetadataFieldNames)) +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index 163d7c4ac2a8e..915d7038d7231 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -51,8 +51,8 @@ func (m *MultiTenantIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilter func (m *MultiTenantIndex) Close() error { return m.idx.Close() } -func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { - return m.idx.GetChunkRefs(ctx, userID, from, through, res, fpFilter, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, ln []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { + return m.idx.GetChunkRefs(ctx, userID, from, through, ln, res, fpFilter, withTenantLabelMatcher(userID, matchers)...) } func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go index fa5ce9a2ff29c..bee9e88529327 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go @@ -66,7 +66,7 @@ type IndexReader interface { // Series populates the given labels and chunk metas for the series identified // by the reference. // Returns storage.ErrNotFound if the ref does not resolve to a known series. - Series(ref storage.SeriesRef, from int64, through int64, lset *labels.Labels, chks *[]index.ChunkMeta, stats **StreamStats) (uint64, error) + Series(ref storage.SeriesRef, from int64, through int64, lset *labels.Labels, chks *[]index.ChunkMeta, stats **index.StreamStats) (uint64, error) // ChunkStats returns the stats for the chunks in the given series. ChunkStats(ref storage.SeriesRef, from, through int64, lset *labels.Labels, by map[string]struct{}) (uint64, index.ChunkStats, error) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 5edf9657afbbc..b525ce250967c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -58,7 +58,7 @@ func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (shipp err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { builder.AddSeries(lbls.Copy(), fp, chks) return false - }, labels.MustNewMatcher(labels.MatchEqual, "", "")) + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { return nil, err } @@ -164,7 +164,7 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { // TODO(owen-d): use pool - var stats *StreamStats + var stats *index.StreamStats var ls labels.Labels chks := ChunkMetasPool.Get() defer ChunkMetasPool.Put(chks) @@ -207,6 +207,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.Fing } if !allFound { + level.Debug(util_log.Logger).Log("msg", "skipping series as it does not have all required labels") continue } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index 4266efa0d6bd7..d65e4ce60d7b5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -25,7 +25,7 @@ import ( type IndexWriter interface { Append(userID string, ls labels.Labels, fprint uint64, chks tsdbindex.ChunkMetas) error - UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) + UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) } type store struct { @@ -176,7 +176,7 @@ func (s *store) IndexChunk(_ context.Context, _ model.Time, _ model.Time, chk ch return nil } -func (s *store) UpdateSeriesStats(userID string, fp uint64, stats *StreamStats) { +func (s *store) UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) { s.indexWriter.UpdateSeriesStats(userID, fp, stats) } @@ -186,8 +186,16 @@ func (f failingIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbin return fmt.Errorf("index writer is not initialized due to tsdb store being initialized in read-only mode") } +func (f failingIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) { + return +} + type noopIndexWriter struct{} +func (f noopIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) { + return +} + func (f noopIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbindex.ChunkMetas) error { return nil } From 9b33f62c206d4e5a9c60712b714bb39b4dce9c31 Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Tue, 3 Dec 2024 19:20:02 +0530 Subject: [PATCH 07/21] Read and Write working --- pkg/ingester/stream.go | 18 ++++----- .../shipper/indexshipper/tsdb/builder.go | 16 ++++---- .../indexshipper/tsdb/compactor_test.go | 4 +- .../shipper/indexshipper/tsdb/head_manager.go | 9 +++-- .../shipper/indexshipper/tsdb/index/index.go | 39 ++++++++++++------- .../tsdb/index_shipper_querier.go | 2 +- .../shipper/indexshipper/tsdb/lazy_index.go | 2 +- .../indexshipper/tsdb/multi_file_index.go | 2 +- .../shipper/indexshipper/tsdb/querier_test.go | 4 +- .../indexshipper/tsdb/single_file_index.go | 8 ++-- 10 files changed, 58 insertions(+), 46 deletions(-) diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 325dcb33d4a23..2bafd9c53c316 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -184,18 +184,18 @@ func (s *stream) NewChunk() *chunkenc.MemChunk { func (s *stream) Push( ctx context.Context, entries []logproto.Entry, -// WAL record to add push contents to. -// May be nil to disable this functionality. + // WAL record to add push contents to. + // May be nil to disable this functionality. record *wal.Record, -// Counter used in WAL replay to avoid duplicates. -// If this is non-zero, the stream will reject entries -// with a counter value less than or equal to it's own. -// It is set to zero and thus bypassed outside of WAL replays. + // Counter used in WAL replay to avoid duplicates. + // If this is non-zero, the stream will reject entries + // with a counter value less than or equal to it's own. + // It is set to zero and thus bypassed outside of WAL replays. counter int64, -// Lock chunkMtx while pushing. -// If this is false, chunkMtx must be held outside Push. + // Lock chunkMtx while pushing. + // If this is false, chunkMtx must be held outside Push. lockChunk bool, -// Whether nor not to ingest all at once or not. It is a per-tenant configuration. + // Whether nor not to ingest all at once or not. It is a per-tenant configuration. rateLimitWholeStream bool, usageTracker push.UsageTracker, diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go index 34c271d771061..a368ce98fa9a6 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go @@ -105,10 +105,10 @@ func (b *Builder) DropChunk(streamID string, chk index.ChunkMeta) (bool, error) func (b *Builder) Build( ctx context.Context, scratchDir string, -// Determines how to create the resulting Identifier and file name. -// This is variable as we use Builder for multiple reasons, -// such as building multi-tenant tsdbs on the ingester -// and per tenant ones during compaction + // Determines how to create the resulting Identifier and file name. + // This is variable as we use Builder for multiple reasons, + // such as building multi-tenant tsdbs on the ingester + // and per tenant ones during compaction createFn func(from, through model.Time, checksum uint32) Identifier, ) (id Identifier, err error) { // Ensure the parent dir exists (i.e. index///) @@ -229,10 +229,10 @@ func (b *Builder) build( func (b *Builder) BuildInMemory( ctx context.Context, -// Determines how to create the resulting Identifier and file name. -// This is variable as we use Builder for multiple reasons, -// such as building multi-tenant tsdbs on the ingester -// and per tenant ones during compaction + // Determines how to create the resulting Identifier and file name. + // This is variable as we use Builder for multiple reasons, + // such as building multi-tenant tsdbs on the ingester + // and per tenant ones during compaction createFn func(from, through model.Time, checksum uint32) Identifier, ) (id Identifier, data []byte, err error) { writer, err := index.NewMemWriterWithVersion(ctx, b.version) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index be0a343309c5c..ed7f3d1ee0dbf 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -612,7 +612,7 @@ func TestCompactor_Compact(t *testing.T) { err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) (stop bool) { actualChunks[lbls.String()] = chks return false - }, labels.MustNewMatcher(labels.MatchEqual, "", "")) + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) require.NoError(t, err) require.Equal(t, expectedChunks, actualChunks) @@ -827,7 +827,7 @@ func TestCompactedIndex(t *testing.T) { err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) (stop bool) { foundChunks[lbls.String()] = append(index.ChunkMetas{}, chks...) return false - }, labels.MustNewMatcher(labels.MatchEqual, "", "")) + }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) require.NoError(t, err) require.Equal(t, tc.finalExpectedChunks, foundChunks) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 70f938a56a233..9df9f8020835d 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -764,7 +764,7 @@ func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, thr if !ok { return nil, nil } - return idx.GetChunkRefs(ctx, userID, from, through, nil, nil, fpFilter, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, filterLabelNames, nil, fpFilter, matchers...) } @@ -835,12 +835,13 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c for ps.Next() { var ( - ls labels.Labels - chks []index.ChunkMeta + ls labels.Labels + chks []index.ChunkMeta + stats *index.StreamStats ) // h11: Pass stream stats - fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks, nil) + fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks, &stats) if err != nil { return errors.Wrapf(err, "iterating postings for tenant: %s", tenant) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 6df91d17324b6..0347bbad7c77b 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -140,6 +140,7 @@ type Creator struct { symbols *Symbols lastSymbol string symbolCache map[string]symbolCacheEntry + //smSymbolCache map[string]uint32 labelIndexes []labelIndexHashEntry // Label index offsets. labelNames map[string]uint64 // Label names, and their usage. @@ -1788,7 +1789,7 @@ func (r *Reader) Series(id storage.SeriesRef, from int64, through int64, lbls *l return 0, d.Err() } - fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks, *stats) + fprint, err := r.dec.Series(r.version, d.Get(), id, from, through, lbls, chks, stats) if err != nil { return 0, errors.Wrap(err, "read series") } @@ -2161,7 +2162,7 @@ func buildChunkSamples(d encoding.Decbuf, numChunks int, info *chunkSamples) err return d.Err() } -func (dec *Decoder) prepSeries(b []byte, lbls *labels.Labels, chks *[]ChunkMeta) (*encoding.Decbuf, uint64, error) { +func (dec *Decoder) prepSeries(version int, b []byte, lbls *labels.Labels, chks *[]ChunkMeta, stats **StreamStats) (*encoding.Decbuf, uint64, error) { *lbls = (*lbls)[:0] if chks != nil { *chks = (*chks)[:0] @@ -2191,13 +2192,18 @@ func (dec *Decoder) prepSeries(b []byte, lbls *labels.Labels, chks *[]ChunkMeta) *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } + + if err := dec.readSeriesStats(version, &d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") + } + return &d, fprint, nil } -func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *StreamStats) error { +func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **StreamStats) error { nSMFieldNames := d.Uvarint() - stats.StructuredMetadataFieldNames = make(map[string]struct{}, nSMFieldNames) + fields := make(map[string]struct{}, nSMFieldNames) for i := 0; i < nSMFieldNames; i++ { fieldName := uint32(d.Uvarint()) @@ -2206,7 +2212,11 @@ func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *Stre return errors.Wrap(err, "lookup structured metadata field name") } - stats.StructuredMetadataFieldNames[ln] = struct{}{} + fields[ln] = struct{}{} + } + + if stats != nil { + (*stats).StructuredMetadataFieldNames = fields } return nil @@ -2214,9 +2224,9 @@ func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats *Stre // prepSeriesBy returns series labels and chunks for a series and only returning selected `by` label names. // If `by` is empty, it returns all labels for the series. -func (dec *Decoder) prepSeriesBy(b []byte, lbls *labels.Labels, chks *[]ChunkMeta, by map[string]struct{}) (*encoding.Decbuf, uint64, error) { +func (dec *Decoder) prepSeriesBy(version int, b []byte, lbls *labels.Labels, chks *[]ChunkMeta, stats **StreamStats, by map[string]struct{}) (*encoding.Decbuf, uint64, error) { if by == nil { - return dec.prepSeries(b, lbls, chks) + return dec.prepSeries(version, b, lbls, chks, stats) } *lbls = (*lbls)[:0] if chks != nil { @@ -2251,11 +2261,16 @@ func (dec *Decoder) prepSeriesBy(b []byte, lbls *labels.Labels, chks *[]ChunkMet *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } + + if err := dec.readSeriesStats(version, &d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") + } + return &d, fprint, nil } func (dec *Decoder) ChunkStats(version int, b []byte, seriesRef storage.SeriesRef, from, through int64, lbls *labels.Labels, by map[string]struct{}) (uint64, ChunkStats, error) { - d, fp, err := dec.prepSeriesBy(b, lbls, nil, by) + d, fp, err := dec.prepSeriesBy(version, b, lbls, nil, nil, by) if err != nil { return 0, ChunkStats{}, err } @@ -2398,16 +2413,12 @@ func (dec *Decoder) readChunkStatsPriorV3(d *encoding.Decbuf, seriesRef storage. } // Series decodes a series entry from the given byte slice into lset and chks. -func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats *StreamStats) (uint64, error) { - d, fprint, err := dec.prepSeries(b, lbls, chks) +func (dec *Decoder) Series(version int, b []byte, seriesRef storage.SeriesRef, from int64, through int64, lbls *labels.Labels, chks *[]ChunkMeta, stats **StreamStats) (uint64, error) { + d, fprint, err := dec.prepSeries(version, b, lbls, chks, stats) if err != nil { return 0, err } - if err := dec.readSeriesStats(version, d, stats); err != nil { - return 0, errors.Wrap(err, "series stats") - } - // read chunks based on fmt if err := dec.readChunks(version, d, seriesRef, from, through, chks); err != nil { return 0, errors.Wrapf(err, "series %s", lbls.String()) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index b0bb007cf564e..cb17256afe114 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -89,7 +89,7 @@ func (i *indexShipperQuerier) GetChunkRefs(ctx context.Context, userID string, f if err != nil { return nil, err } - return idx.GetChunkRefs(ctx, userID, from, through, nil, res, fpFilter, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, filterLabelNames, res, fpFilter, matchers...) } func (i *indexShipperQuerier) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index 17241994d70e8..c9ce27750a967 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -41,7 +41,7 @@ func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, throug if err != nil { return nil, err } - return i.GetChunkRefs(ctx, userID, from, through, nil, res, fpFilter, matchers...) + return i.GetChunkRefs(ctx, userID, from, through, filterLabelNames, res, fpFilter, matchers...) } func (f LazyIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { i, err := f() diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index f91a9217ccaae..76eddc525490b 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -170,7 +170,7 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro from, through, func(ctx context.Context, idx Index) error { - got, err := idx.GetChunkRefs(ctx, userID, from, through, nil, nil, fpFilter, matchers...) + got, err := idx.GetChunkRefs(ctx, userID, from, through, filterLabelNames, nil, fpFilter, matchers...) if err != nil { return err } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go index a3c5caf5b81c4..93c7d3fc14bd4 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go @@ -114,12 +114,12 @@ func TestQueryIndex(t *testing.T) { ) require.True(t, p.Next()) - _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks) + _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, nil) require.Nil(t, err) require.Equal(t, cases[0].labels.String(), ls.String()) require.Equal(t, cases[0].chunks, chks) require.True(t, p.Next()) - _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks) + _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, nil) require.Nil(t, err) require.Equal(t, cases[1].labels.String(), ls.String()) require.Equal(t, cases[1].chunks, chks) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index b525ce250967c..588ab24f1640d 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -164,7 +164,7 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { // TODO(owen-d): use pool - var stats *index.StreamStats + stats := index.NewStreamStats() var ls labels.Labels chks := ChunkMetasPool.Get() defer ChunkMetasPool.Put(chks) @@ -181,7 +181,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.Fing return err } - if stats != nil { + if stats != nil && len(filterLabelNames) > 0 { // Looking at the structured metadata fields names, discard all series for which we don't have the required filter label names allFound := false for _, ln := range filterLabelNames { @@ -244,7 +244,7 @@ func (i *TSDBIndex) forPostings( return fn(p) } -func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, _ []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabels []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() } @@ -262,7 +262,7 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu }) } return false - }, nil, matchers...); err != nil { + }, filterLabels, matchers...); err != nil { return nil, err } From d8bfc7e57e0f92736eee6ed9dc66b3d84609c935 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Tue, 3 Dec 2024 16:03:58 +0100 Subject: [PATCH 08/21] fix tests --- .../planner/strategies/splitkeyspace_test.go | 2 +- pkg/ingester/flush_test.go | 5 +++ pkg/ingester/ingester_test.go | 5 +++ pkg/querier/querier_mock_test.go | 6 ++++ pkg/storage/stores/composite_store_test.go | 5 +++ .../shipper/indexshipper/tsdb/head_read.go | 4 ++- .../shipper/indexshipper/tsdb/index/index.go | 3 ++ .../indexshipper/tsdb/index/index_test.go | 31 ++++++++++++------- .../indexshipper/tsdb/single_file_index.go | 1 + pkg/storage/util_test.go | 4 +++ tools/tsdb/index-analyzer/analytics.go | 2 ++ tools/tsdb/tsdb-map/main_test.go | 4 +-- 12 files changed, 56 insertions(+), 16 deletions(-) diff --git a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go index 25696c5c78a85..28666116e37e4 100644 --- a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go +++ b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go @@ -333,7 +333,7 @@ func newFakeForSeries(series []*v1.Series) *fakeForSeries { } } -func (f fakeForSeries) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f fakeForSeries) ForSeries(_ context.Context, _ string, ff index.FingerprintFilter, _ model.Time, _ model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), _ []string, _ ...*labels.Matcher) error { overlapping := make([]*v1.Series, 0, len(f.series)) for _, s := range f.series { if ff.Match(s.Fingerprint) { diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index f4251747115a2..13115c6c45c48 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -3,6 +3,7 @@ package ingester import ( "errors" "fmt" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "os" "sort" "sync" @@ -461,6 +462,10 @@ func (s *testStore) Put(ctx context.Context, chunks []chunk.Chunk) error { return nil } +func (s *testStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index.StreamStats) error { + return nil +} + func (s *testStore) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { return nil } diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 9074580b4eb40..32de976c8f92e 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -2,6 +2,7 @@ package ingester import ( "fmt" + index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "math" "net" "net/http" @@ -435,6 +436,10 @@ type mockStore struct { chunks map[string][]chunk.Chunk } +func (s *mockStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index2.StreamStats) error { + return nil +} + func (s *mockStore) Put(ctx context.Context, chunks []chunk.Chunk) error { s.mtx.Lock() defer s.mtx.Unlock() diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 0fd9b421de000..5418889b9a4d0 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "math" "time" @@ -329,6 +330,11 @@ type storeMock struct { func newStoreMock() *storeMock { return &storeMock{} } + +func (s *storeMock) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index.StreamStats) error { + return nil +} + func (s *storeMock) SetChunkFilterer(chunk.RequestChunkFilterer) {} func (s *storeMock) SetExtractorWrapper(log.SampleExtractorWrapper) {} func (s *storeMock) SetPipelineWrapper(log.PipelineWrapper) {} diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index 90062add1552d..c50cc0c4cb896 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -3,6 +3,7 @@ package stores import ( "context" "fmt" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "reflect" "testing" @@ -31,6 +32,10 @@ func (m mockStore) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) err return nil } +func (m mockStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index.StreamStats) error { + return nil +} + func (m mockStore) LabelValuesForMetricName(_ context.Context, _ string, _, _ model.Time, _ string, _ string, _ ...*labels.Matcher) ([]string, error) { return nil, nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index 633b00a8fe615..4a64893e22d3c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -131,7 +131,9 @@ func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int6 } *lbls = append((*lbls)[:0], s.ls...) - *stats = &s.stats + if stats != nil { + *stats = &s.stats + } queryBounds := newBounds(model.Time(from), model.Time(through)) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 0347bbad7c77b..78f1c38c2df0b 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -2216,6 +2216,9 @@ func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **Str } if stats != nil { + if *stats == nil { + *stats = NewStreamStats() + } (*stats).StructuredMetadataFieldNames = fields } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go index 8b2ac832a5797..d0b7f528df2d7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go @@ -174,10 +174,10 @@ func TestIndexRW_Postings(t *testing.T) { // Postings lists are only written if a series with the respective // reference was added before. - require.NoError(t, iw.AddSeries(1, series[0], model.Fingerprint(series[0].Hash()))) - require.NoError(t, iw.AddSeries(2, series[1], model.Fingerprint(series[1].Hash()))) - require.NoError(t, iw.AddSeries(3, series[2], model.Fingerprint(series[2].Hash()))) - require.NoError(t, iw.AddSeries(4, series[3], model.Fingerprint(series[3].Hash()))) + require.NoError(t, iw.AddSeries(1, series[0], model.Fingerprint(series[0].Hash()), nil)) + require.NoError(t, iw.AddSeries(2, series[1], model.Fingerprint(series[1].Hash()), nil)) + require.NoError(t, iw.AddSeries(3, series[2], model.Fingerprint(series[2].Hash()), nil)) + require.NoError(t, iw.AddSeries(4, series[3], model.Fingerprint(series[3].Hash()), nil)) _, err = iw.Close(false) require.NoError(t, err) @@ -192,7 +192,7 @@ func TestIndexRW_Postings(t *testing.T) { var c []ChunkMeta for i := 0; p.Next(); i++ { - _, err := ir.Series(p.At(), 0, math.MaxInt64, &l, &c) + _, err := ir.Series(p.At(), 0, math.MaxInt64, &l, &c, nil) require.NoError(t, err) require.Equal(t, 0, len(c)) @@ -266,7 +266,7 @@ func TestPostingsMany(t *testing.T) { }) for i, s := range series { - require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s, model.Fingerprint(s.Hash()))) + require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s, model.Fingerprint(s.Hash()), nil)) } _, err = iw.Close(false) require.NoError(t, err) @@ -314,7 +314,7 @@ func TestPostingsMany(t *testing.T) { var lbls labels.Labels var metas []ChunkMeta for it.Next() { - _, err := ir.Series(it.At(), 0, math.MaxInt64, &lbls, &metas) + _, err := ir.Series(it.At(), 0, math.MaxInt64, &lbls, &metas, nil) require.NoError(t, err) got = append(got, lbls.Get("i")) } @@ -394,7 +394,7 @@ func TestPersistence_index_e2e(t *testing.T) { mi := newMockIndex() for i, s := range input { - err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks...) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) require.NoError(t, err) require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...)) @@ -430,7 +430,7 @@ func TestPersistence_index_e2e(t *testing.T) { ref := gotp.At() - _, err := ir.Series(ref, 0, math.MaxInt64, &lset, &chks) + _, err := ir.Series(ref, 0, math.MaxInt64, &lset, &chks, nil) require.NoError(t, err) err = mi.Series(expp.At(), &explset, &expchks) @@ -740,7 +740,7 @@ func TestDecoder_ChunkSamples(t *testing.T) { } for i, l := range lbls { - err = iw.AddSeries(storage.SeriesRef(i), l, model.Fingerprint(l.Hash()), tc.chunkMetas...) + err = iw.AddSeries(storage.SeriesRef(i), l, model.Fingerprint(l.Hash()), tc.chunkMetas) require.NoError(t, err) } @@ -761,7 +761,7 @@ func TestDecoder_ChunkSamples(t *testing.T) { require.Nil(t, ir.dec.chunksSample[postings.At()]) // read series so that chunk samples get built - _, err = ir.Series(postings.At(), 0, math.MaxInt64, &lset, &chks) + _, err = ir.Series(postings.At(), 0, math.MaxInt64, &lset, &chks, nil) require.NoError(t, err) require.Equal(t, tc.chunkMetas, chks) @@ -785,6 +785,13 @@ func TestDecoder_ChunkSamples(t *testing.T) { d.Uvarint() d.Uvarint() } + + // Read structured metadata names stats + k = d.Uvarint() + for i := 0; i < k; i++ { + d.Uvarint() + } + require.Equal(t, len(tc.chunkMetas), d.Uvarint()) for i, cs := range ir.dec.chunksSample[postings.At()].chunks { require.Equal(t, tc.expectedChunkSamples[i].idx, cs.idx) @@ -996,7 +1003,7 @@ func BenchmarkInitReader_ReadOffsetTable(b *testing.B) { } for i, s := range input { - err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks...) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) require.NoError(b, err) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 588ab24f1640d..a8975e9426767 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -181,6 +181,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.Fing return err } + // Try to filter out series that don't have the required filter label names in their stream labels or structured metadata fields if stats != nil && len(filterLabelNames) > 0 { // Looking at the structured metadata fields names, discard all series for which we don't have the required filter label names allFound := false diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index a0dc75999692f..bca1526a59ca7 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -2,6 +2,7 @@ package storage import ( "context" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "sort" "testing" "time" @@ -190,6 +191,9 @@ func (m *mockChunkStore) Put(_ context.Context, _ []chunk.Chunk) error { return func (m *mockChunkStore) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { return nil } +func (m *mockChunkStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index.StreamStats) error { + return nil +} func (m *mockChunkStore) GetSeries(ctx context.Context, _ string, _, _ model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) { result := make([]labels.Labels, 0, len(m.chunks)) diff --git a/tools/tsdb/index-analyzer/analytics.go b/tools/tsdb/index-analyzer/analytics.go index b574a58341924..ac97779c920f8 100644 --- a/tools/tsdb/index-analyzer/analytics.go +++ b/tools/tsdb/index-analyzer/analytics.go @@ -58,6 +58,7 @@ func analyze(indexShipper indexshipper.IndexShipper, tableName string, tenants [ tenant, model.Earliest, model.Latest, + nil, chunkRes, nil, labels.MustNewMatcher(labels.MatchEqual, "", ""), ) @@ -82,6 +83,7 @@ func analyze(indexShipper indexshipper.IndexShipper, tableName string, tenants [ } return false }, + nil, labels.MustNewMatcher(labels.MatchEqual, "", ""), ) diff --git a/tools/tsdb/tsdb-map/main_test.go b/tools/tsdb/tsdb-map/main_test.go index 56fdcdbc3b255..2e8f1ce4d2646 100644 --- a/tools/tsdb/tsdb-map/main_test.go +++ b/tools/tsdb/tsdb-map/main_test.go @@ -99,7 +99,7 @@ func BenchmarkQuery_GetChunkRefs(b *testing.B) { refs := tsdb.ChunkRefsPool.Get() for i := 0; i < b.N; i++ { var err error - refs, err = idx.GetChunkRefs(context.Background(), "fake", 0, math.MaxInt64, refs, nil, bm.matchers...) + refs, err = idx.GetChunkRefs(context.Background(), "fake", 0, math.MaxInt64, nil, refs, nil, bm.matchers...) if err != nil { panic(err) } @@ -132,7 +132,7 @@ func BenchmarkQuery_GetChunkRefsSharded(b *testing.B) { } var err error - refs, err = idx.GetChunkRefs(context.Background(), "fake", 0, math.MaxInt64, refs, &shard, bm.matchers...) + refs, err = idx.GetChunkRefs(context.Background(), "fake", 0, math.MaxInt64, nil, refs, &shard, bm.matchers...) if err != nil { panic(err) } From 2c2402d7a68a18b2aa4a631f096cbcee1d70b46f Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Tue, 3 Dec 2024 17:13:32 +0100 Subject: [PATCH 09/21] WIP: Support structured metadata in labels API endpoint --- pkg/logproto/logproto.pb.go | 421 ++++++++++-------- pkg/logproto/logproto.proto | 1 + pkg/querier/handler.go | 7 +- pkg/querier/queryrange/queryrange.pb.go | 331 ++++++++------ pkg/querier/queryrange/queryrange.proto | 1 + .../shipper/indexshipper/tsdb/head_manager.go | 4 +- .../shipper/indexshipper/tsdb/head_read.go | 59 ++- .../stores/shipper/indexshipper/tsdb/index.go | 6 +- .../shipper/indexshipper/tsdb/index/index.go | 16 +- .../tsdb/index_shipper_querier.go | 4 +- .../shipper/indexshipper/tsdb/lazy_index.go | 4 +- .../indexshipper/tsdb/multi_file_index.go | 12 +- .../tsdb/multi_file_index_test.go | 4 +- .../shipper/indexshipper/tsdb/multitenant.go | 10 +- .../shipper/indexshipper/tsdb/querier.go | 10 +- .../indexshipper/tsdb/single_file_index.go | 2 +- .../tsdb/single_file_index_test.go | 4 +- 17 files changed, 534 insertions(+), 362 deletions(-) diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index 011d0b28f6c01..941f424b905db 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -777,7 +777,8 @@ func (m *LabelRequest) GetQuery() string { } type LabelResponse struct { - Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` + StructuredMetadata []string `protobuf:"bytes,2,rep,name=structuredMetadata,proto3" json:"structuredMetadata,omitempty"` } func (m *LabelResponse) Reset() { *m = LabelResponse{} } @@ -819,6 +820,13 @@ func (m *LabelResponse) GetValues() []string { return nil } +func (m *LabelResponse) GetStructuredMetadata() []string { + if m != nil { + return m.StructuredMetadata + } + return nil +} + type Sample struct { Timestamp int64 `protobuf:"varint,1,opt,name=timestamp,proto3" json:"ts"` Value float64 `protobuf:"fixed64,2,opt,name=value,proto3" json:"value"` @@ -3162,181 +3170,183 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) } var fileDescriptor_c28a5f14f1f4c79a = []byte{ - // 2774 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3a, 0xcd, 0x6f, 0x1b, 0xc7, - 0xf5, 0x5c, 0x7e, 0xf3, 0x91, 0x92, 0xe5, 0x11, 0x6d, 0x13, 0xb2, 0xc3, 0x55, 0x06, 0xbf, 0x5f, - 0xa2, 0xc4, 0x8e, 0x68, 0x3b, 0x4d, 0x9a, 0x38, 0x4d, 0x53, 0x53, 0x8a, 0x1d, 0x3b, 0x8a, 0xed, - 0x8c, 0x1c, 0x27, 0x2d, 0x1a, 0x04, 0x6b, 0x72, 0x44, 0x2e, 0x4c, 0xee, 0xd2, 0xbb, 0xc3, 0x38, - 0xbc, 0xf5, 0x1f, 0x28, 0x1a, 0xa0, 0x28, 0xda, 0x5e, 0x0a, 0x14, 0x28, 0xd0, 0xa2, 0x40, 0x2e, - 0x45, 0x0f, 0x3d, 0x14, 0xed, 0xa5, 0x40, 0xd3, 0x5b, 0x8e, 0x41, 0x0e, 0x6c, 0xa3, 0x5c, 0x0a, - 0x01, 0x05, 0x72, 0x6a, 0x81, 0x9c, 0x8a, 0xf9, 0xda, 0x9d, 0x5d, 0x51, 0x55, 0xe8, 0xba, 0x48, - 0x72, 0x21, 0x67, 0xde, 0xbc, 0x79, 0x33, 0xef, 0x63, 0xde, 0x17, 0x09, 0x27, 0x47, 0x77, 0x7a, - 0xad, 0x81, 0xdf, 0x1b, 0x05, 0x3e, 0xf3, 0xa3, 0xc1, 0xba, 0xf8, 0x44, 0x65, 0x3d, 0x5f, 0xa9, - 0xf7, 0xfc, 0x9e, 0x2f, 0x71, 0xf8, 0x48, 0xae, 0xaf, 0xd8, 0x3d, 0xdf, 0xef, 0x0d, 0x68, 0x4b, - 0xcc, 0x6e, 0x8f, 0x77, 0x5a, 0xcc, 0x1d, 0xd2, 0x90, 0x39, 0xc3, 0x91, 0x42, 0x58, 0x55, 0xd4, - 0xef, 0x0e, 0x86, 0x7e, 0x97, 0x0e, 0x5a, 0x21, 0x73, 0x58, 0x28, 0x3f, 0x15, 0xc6, 0x32, 0xc7, - 0x18, 0x8d, 0xc3, 0xbe, 0xf8, 0x50, 0xc0, 0xb3, 0x1c, 0x18, 0x32, 0x3f, 0x70, 0x7a, 0xb4, 0xd5, - 0xe9, 0x8f, 0xbd, 0x3b, 0xad, 0x8e, 0xd3, 0xe9, 0xd3, 0x56, 0x40, 0xc3, 0xf1, 0x80, 0x85, 0x72, - 0xc2, 0x26, 0x23, 0xaa, 0xc8, 0xe0, 0xdf, 0x5a, 0x70, 0x6c, 0xcb, 0xb9, 0x4d, 0x07, 0x37, 0xfd, - 0x5b, 0xce, 0x60, 0x4c, 0x43, 0x42, 0xc3, 0x91, 0xef, 0x85, 0x14, 0x6d, 0x40, 0x71, 0xc0, 0x17, - 0xc2, 0x86, 0xb5, 0x9a, 0x5b, 0xab, 0x9e, 0x3f, 0xbd, 0x1e, 0x31, 0x39, 0x73, 0x83, 0x84, 0x86, - 0x2f, 0x7a, 0x2c, 0x98, 0x10, 0xb5, 0x75, 0xe5, 0x16, 0x54, 0x0d, 0x30, 0x5a, 0x82, 0xdc, 0x1d, - 0x3a, 0x69, 0x58, 0xab, 0xd6, 0x5a, 0x85, 0xf0, 0x21, 0x3a, 0x07, 0x85, 0xb7, 0x39, 0x99, 0x46, - 0x76, 0xd5, 0x5a, 0xab, 0x9e, 0x3f, 0x19, 0x1f, 0xf2, 0x9a, 0xe7, 0xde, 0x1d, 0x53, 0xb1, 0x5b, - 0x1d, 0x24, 0x31, 0x2f, 0x64, 0x9f, 0xb1, 0xf0, 0x69, 0x38, 0xba, 0x6f, 0x1d, 0x1d, 0x87, 0xa2, - 0xc0, 0x90, 0x37, 0xae, 0x10, 0x35, 0xc3, 0x75, 0x40, 0xdb, 0x2c, 0xa0, 0xce, 0x90, 0x38, 0x8c, - 0xdf, 0xf7, 0xee, 0x98, 0x86, 0x0c, 0xbf, 0x02, 0xcb, 0x09, 0xa8, 0x62, 0xfb, 0x69, 0xa8, 0x86, - 0x31, 0x58, 0xf1, 0x5e, 0x8f, 0xaf, 0x15, 0xef, 0x21, 0x26, 0x22, 0xfe, 0x99, 0x05, 0x10, 0xaf, - 0xa1, 0x26, 0x80, 0x5c, 0x7d, 0xc9, 0x09, 0xfb, 0x82, 0xe1, 0x3c, 0x31, 0x20, 0xe8, 0x0c, 0x1c, - 0x8d, 0x67, 0xd7, 0xfc, 0xed, 0xbe, 0x13, 0x74, 0x85, 0x0c, 0xf2, 0x64, 0xff, 0x02, 0x42, 0x90, - 0x0f, 0x1c, 0x46, 0x1b, 0xb9, 0x55, 0x6b, 0x2d, 0x47, 0xc4, 0x98, 0x73, 0xcb, 0xa8, 0xe7, 0x78, - 0xac, 0x91, 0x17, 0xe2, 0x54, 0x33, 0x0e, 0xe7, 0x16, 0x41, 0xc3, 0x46, 0x61, 0xd5, 0x5a, 0x5b, - 0x20, 0x6a, 0x86, 0xff, 0x99, 0x83, 0xda, 0xab, 0x63, 0x1a, 0x4c, 0x94, 0x00, 0x50, 0x13, 0xca, - 0x21, 0x1d, 0xd0, 0x0e, 0xf3, 0x03, 0xa9, 0x91, 0x76, 0xb6, 0x61, 0x91, 0x08, 0x86, 0xea, 0x50, - 0x18, 0xb8, 0x43, 0x97, 0x89, 0x6b, 0x2d, 0x10, 0x39, 0x41, 0x17, 0xa0, 0x10, 0x32, 0x27, 0x60, - 0xe2, 0x2e, 0xd5, 0xf3, 0x2b, 0xeb, 0xd2, 0x94, 0xd7, 0xb5, 0x29, 0xaf, 0xdf, 0xd4, 0xa6, 0xdc, - 0x2e, 0xbf, 0x3f, 0xb5, 0x33, 0xef, 0xfe, 0xd5, 0xb6, 0x88, 0xdc, 0x82, 0x9e, 0x86, 0x1c, 0xf5, - 0xba, 0xe2, 0xbe, 0x9f, 0x77, 0x27, 0xdf, 0x80, 0xce, 0x41, 0xa5, 0xeb, 0x06, 0xb4, 0xc3, 0x5c, - 0xdf, 0x13, 0x5c, 0x2d, 0x9e, 0x5f, 0x8e, 0x35, 0xb2, 0xa9, 0x97, 0x48, 0x8c, 0x85, 0xce, 0x40, - 0x31, 0xe4, 0xa2, 0x0b, 0x1b, 0x25, 0x6e, 0x0b, 0xed, 0xfa, 0xde, 0xd4, 0x5e, 0x92, 0x90, 0x33, - 0xfe, 0xd0, 0x65, 0x74, 0x38, 0x62, 0x13, 0xa2, 0x70, 0xd0, 0xe3, 0x50, 0xea, 0xd2, 0x01, 0xe5, - 0x0a, 0x2f, 0x0b, 0x85, 0x2f, 0x19, 0xe4, 0xc5, 0x02, 0xd1, 0x08, 0xe8, 0x4d, 0xc8, 0x8f, 0x06, - 0x8e, 0xd7, 0xa8, 0x08, 0x2e, 0x16, 0x63, 0xc4, 0x1b, 0x03, 0xc7, 0x6b, 0x3f, 0xfb, 0xd1, 0xd4, - 0x7e, 0xaa, 0xe7, 0xb2, 0xfe, 0xf8, 0xf6, 0x7a, 0xc7, 0x1f, 0xb6, 0x7a, 0x81, 0xb3, 0xe3, 0x78, - 0x4e, 0x6b, 0xe0, 0xdf, 0x71, 0x5b, 0x6f, 0x3f, 0xd9, 0xe2, 0x0f, 0xf4, 0xee, 0x98, 0x06, 0x2e, - 0x0d, 0x5a, 0x9c, 0xcc, 0xba, 0x50, 0x09, 0xdf, 0x4a, 0x04, 0x59, 0x74, 0x95, 0xdb, 0x9f, 0x1f, - 0xd0, 0x0d, 0xfe, 0x7a, 0xc3, 0x06, 0x88, 0x53, 0x4e, 0xc4, 0xa7, 0x08, 0x38, 0xa1, 0x3b, 0x97, - 0x03, 0x7f, 0x3c, 0x6a, 0x1f, 0xd9, 0x9b, 0xda, 0x26, 0x3e, 0x31, 0x27, 0x57, 0xf3, 0xe5, 0xe2, - 0x52, 0x09, 0xbf, 0x97, 0x03, 0xb4, 0xed, 0x0c, 0x47, 0x03, 0x3a, 0x97, 0xfa, 0x23, 0x45, 0x67, - 0xef, 0x5b, 0xd1, 0xb9, 0x79, 0x15, 0x1d, 0x6b, 0x2d, 0x3f, 0x9f, 0xd6, 0x0a, 0x9f, 0x57, 0x6b, - 0xc5, 0x2f, 0xbd, 0xd6, 0x70, 0x03, 0xf2, 0x9c, 0x32, 0x77, 0x96, 0x81, 0x73, 0x4f, 0xe8, 0xa6, - 0x46, 0xf8, 0x10, 0x6f, 0x41, 0x51, 0xf2, 0x85, 0x56, 0xd2, 0xca, 0x4b, 0xbe, 0xdb, 0x58, 0x71, - 0x39, 0xad, 0x92, 0xa5, 0x58, 0x25, 0x39, 0x21, 0x6c, 0xfc, 0x7b, 0x0b, 0x16, 0x94, 0x45, 0x28, - 0xdf, 0x77, 0x1b, 0x4a, 0xd2, 0xf7, 0x68, 0xbf, 0x77, 0x22, 0xed, 0xf7, 0x2e, 0x76, 0x9d, 0x11, - 0xa3, 0x41, 0xbb, 0xf5, 0xfe, 0xd4, 0xb6, 0x3e, 0x9a, 0xda, 0x8f, 0x1e, 0x24, 0x34, 0x1d, 0x9d, - 0xb4, 0xbf, 0xd4, 0x84, 0xd1, 0x69, 0x71, 0x3b, 0x16, 0x2a, 0xb3, 0x3a, 0xb2, 0x2e, 0x83, 0xda, - 0x15, 0xaf, 0x47, 0x43, 0x4e, 0x39, 0xcf, 0x2d, 0x82, 0x48, 0x1c, 0xce, 0xe6, 0x3d, 0x27, 0xf0, - 0x5c, 0xaf, 0x17, 0x36, 0x72, 0xc2, 0xa7, 0x47, 0x73, 0xfc, 0x13, 0x0b, 0x96, 0x13, 0x66, 0xad, - 0x98, 0x78, 0x06, 0x8a, 0x21, 0xd7, 0x94, 0xe6, 0xc1, 0x30, 0x8a, 0x6d, 0x01, 0x6f, 0x2f, 0xaa, - 0xcb, 0x17, 0xe5, 0x9c, 0x28, 0xfc, 0x07, 0x77, 0xb5, 0x3f, 0x59, 0x50, 0x13, 0x81, 0x49, 0xbf, - 0x35, 0x04, 0x79, 0xcf, 0x19, 0x52, 0xa5, 0x2a, 0x31, 0x36, 0xa2, 0x15, 0x3f, 0xae, 0xac, 0xa3, - 0xd5, 0xbc, 0x0e, 0xd6, 0xba, 0x6f, 0x07, 0x6b, 0xc5, 0xef, 0xae, 0x0e, 0x05, 0x6e, 0xde, 0x13, - 0xe1, 0x5c, 0x2b, 0x44, 0x4e, 0xf0, 0xa3, 0xb0, 0xa0, 0xb8, 0x50, 0xa2, 0x3d, 0x28, 0xc0, 0x0e, - 0xa1, 0x28, 0x35, 0x81, 0xfe, 0x0f, 0x2a, 0x51, 0x2a, 0x23, 0xb8, 0xcd, 0xb5, 0x8b, 0x7b, 0x53, - 0x3b, 0xcb, 0x42, 0x12, 0x2f, 0x20, 0xdb, 0x0c, 0xfa, 0x56, 0xbb, 0xb2, 0x37, 0xb5, 0x25, 0x40, - 0x85, 0x78, 0x74, 0x0a, 0xf2, 0x7d, 0x1e, 0x37, 0xb9, 0x08, 0xf2, 0xed, 0xf2, 0xde, 0xd4, 0x16, - 0x73, 0x22, 0x3e, 0xf1, 0x65, 0xa8, 0x6d, 0xd1, 0x9e, 0xd3, 0x99, 0xa8, 0x43, 0xeb, 0x9a, 0x1c, - 0x3f, 0xd0, 0xd2, 0x34, 0x1e, 0x86, 0x5a, 0x74, 0xe2, 0x5b, 0xc3, 0x50, 0xbd, 0x86, 0x6a, 0x04, - 0x7b, 0x25, 0xc4, 0x3f, 0xb5, 0x40, 0xd9, 0x00, 0xc2, 0x46, 0xb6, 0xc3, 0x7d, 0x21, 0xec, 0x4d, - 0x6d, 0x05, 0xd1, 0xc9, 0x0c, 0x7a, 0x0e, 0x4a, 0xa1, 0x38, 0x91, 0x13, 0x4b, 0x9b, 0x96, 0x58, - 0x68, 0x1f, 0xe1, 0x26, 0xb2, 0x37, 0xb5, 0x35, 0x22, 0xd1, 0x03, 0xb4, 0x9e, 0x48, 0x08, 0x24, - 0x63, 0x8b, 0x7b, 0x53, 0xdb, 0x80, 0x9a, 0x09, 0x02, 0xfe, 0xcc, 0x82, 0xea, 0x4d, 0xc7, 0x8d, - 0x4c, 0xa8, 0xa1, 0x55, 0x14, 0xfb, 0x6a, 0x09, 0xe0, 0x96, 0xd8, 0xa5, 0x03, 0x67, 0x72, 0xc9, - 0x0f, 0x04, 0xdd, 0x05, 0x12, 0xcd, 0xe3, 0x18, 0x9e, 0x9f, 0x19, 0xc3, 0x0b, 0xf3, 0xbb, 0xf6, - 0xff, 0xad, 0x23, 0xbd, 0x9a, 0x2f, 0x67, 0x97, 0x72, 0xf8, 0x3d, 0x0b, 0x6a, 0x92, 0x79, 0x65, - 0x79, 0xdf, 0x85, 0xa2, 0x94, 0x8d, 0x60, 0xff, 0x3f, 0x38, 0xa6, 0xd3, 0xf3, 0x38, 0x25, 0x45, - 0x13, 0xbd, 0x00, 0x8b, 0xdd, 0xc0, 0x1f, 0x8d, 0x68, 0x77, 0x5b, 0xb9, 0xbf, 0x6c, 0xda, 0xfd, - 0x6d, 0x9a, 0xeb, 0x24, 0x85, 0x8e, 0xff, 0x62, 0xc1, 0x82, 0x72, 0x26, 0x4a, 0x5d, 0x91, 0x88, - 0xad, 0xfb, 0x8e, 0x9e, 0xd9, 0x79, 0xa3, 0xe7, 0x71, 0x28, 0xf6, 0x78, 0x7c, 0xd1, 0x0e, 0x49, - 0xcd, 0xe6, 0x8b, 0xaa, 0xf8, 0x2a, 0x2c, 0x6a, 0x56, 0x0e, 0xf0, 0xa8, 0x2b, 0x69, 0x8f, 0x7a, - 0xa5, 0x4b, 0x3d, 0xe6, 0xee, 0xb8, 0x91, 0x8f, 0x54, 0xf8, 0xf8, 0x07, 0x16, 0x2c, 0xa5, 0x51, - 0xd0, 0x66, 0xaa, 0xb0, 0x78, 0xe4, 0x60, 0x72, 0x66, 0x4d, 0xa1, 0x49, 0xab, 0xca, 0xe2, 0xa9, - 0xc3, 0x2a, 0x8b, 0xba, 0xe9, 0x64, 0x2a, 0xca, 0x2b, 0xe0, 0x1f, 0x5b, 0xb0, 0x90, 0xd0, 0x25, - 0x7a, 0x06, 0xf2, 0x3b, 0x81, 0x3f, 0x9c, 0x4b, 0x51, 0x62, 0x07, 0xfa, 0x1a, 0x64, 0x99, 0x3f, - 0x97, 0x9a, 0xb2, 0xcc, 0xe7, 0x5a, 0x52, 0xec, 0xe7, 0x64, 0xde, 0x2e, 0x67, 0xf8, 0x29, 0xa8, - 0x08, 0x86, 0x6e, 0x38, 0x6e, 0x30, 0x33, 0x60, 0xcc, 0x66, 0xe8, 0x39, 0x38, 0x22, 0x9d, 0xe1, - 0xec, 0xcd, 0xb5, 0x59, 0x9b, 0x6b, 0x7a, 0xf3, 0x49, 0x28, 0x88, 0xa4, 0x83, 0x6f, 0xe9, 0x3a, - 0xcc, 0xd1, 0x5b, 0xf8, 0x18, 0x1f, 0x83, 0x65, 0xfe, 0x06, 0x69, 0x10, 0x6e, 0xf8, 0x63, 0x8f, - 0xe9, 0xba, 0xe9, 0x0c, 0xd4, 0x93, 0x60, 0x65, 0x25, 0x75, 0x28, 0x74, 0x38, 0x40, 0xd0, 0x58, - 0x20, 0x72, 0x82, 0x7f, 0x61, 0x01, 0xba, 0x4c, 0x99, 0x38, 0xe5, 0xca, 0x66, 0xf4, 0x3c, 0x56, - 0xa0, 0x3c, 0x74, 0x58, 0xa7, 0x4f, 0x83, 0x50, 0xe7, 0x2f, 0x7a, 0xfe, 0x45, 0x24, 0x9e, 0xf8, - 0x1c, 0x2c, 0x27, 0x6e, 0xa9, 0x78, 0x5a, 0x81, 0x72, 0x47, 0xc1, 0x54, 0xc8, 0x8b, 0xe6, 0xf8, - 0x37, 0x59, 0x28, 0xeb, 0xb4, 0x0e, 0x9d, 0x83, 0xea, 0x8e, 0xeb, 0xf5, 0x68, 0x30, 0x0a, 0x5c, - 0x25, 0x82, 0xbc, 0x4c, 0xf3, 0x0c, 0x30, 0x31, 0x27, 0xe8, 0x09, 0x28, 0x8d, 0x43, 0x1a, 0xbc, - 0xe5, 0xca, 0x97, 0x5e, 0x69, 0xd7, 0x77, 0xa7, 0x76, 0xf1, 0xb5, 0x90, 0x06, 0x57, 0x36, 0x79, - 0xf0, 0x19, 0x8b, 0x11, 0x91, 0xdf, 0x5d, 0xf4, 0xb2, 0x32, 0x53, 0x91, 0xc0, 0xb5, 0xbf, 0xce, - 0xaf, 0x9f, 0x72, 0x75, 0xa3, 0xc0, 0x1f, 0x52, 0xd6, 0xa7, 0xe3, 0xb0, 0xd5, 0xf1, 0x87, 0x43, - 0xdf, 0x6b, 0x89, 0xde, 0x81, 0x60, 0x9a, 0x47, 0x50, 0xbe, 0x5d, 0x59, 0xee, 0x4d, 0x28, 0xb1, - 0x7e, 0xe0, 0x8f, 0x7b, 0x7d, 0x11, 0x18, 0x72, 0xed, 0x0b, 0xf3, 0xd3, 0xd3, 0x14, 0x88, 0x1e, - 0xa0, 0x87, 0xb9, 0xb4, 0x68, 0xe7, 0x4e, 0x38, 0x1e, 0xca, 0xda, 0xb3, 0x5d, 0xd8, 0x9b, 0xda, - 0xd6, 0x13, 0x24, 0x02, 0xe3, 0x8b, 0xb0, 0x90, 0x48, 0x85, 0xd1, 0x59, 0xc8, 0x07, 0x74, 0x47, - 0xbb, 0x02, 0xb4, 0x3f, 0x63, 0x96, 0xd1, 0x9f, 0xe3, 0x10, 0xf1, 0x89, 0xbf, 0x9f, 0x05, 0xdb, - 0xa8, 0xfa, 0x2f, 0xf9, 0xc1, 0x2b, 0x94, 0x05, 0x6e, 0xe7, 0x9a, 0x33, 0xa4, 0xda, 0xbc, 0x6c, - 0xa8, 0x0e, 0x05, 0xf0, 0x2d, 0xe3, 0x15, 0xc1, 0x30, 0xc2, 0x43, 0x0f, 0x01, 0x88, 0x67, 0x27, - 0xd7, 0xe5, 0x83, 0xaa, 0x08, 0x88, 0x58, 0xde, 0x48, 0x08, 0xbb, 0x35, 0xa7, 0x70, 0x94, 0x90, - 0xaf, 0xa4, 0x85, 0x3c, 0x37, 0x9d, 0x48, 0xb2, 0xe6, 0x73, 0x29, 0x24, 0x9f, 0x0b, 0xfe, 0x87, - 0x05, 0xcd, 0x2d, 0x7d, 0xf3, 0xfb, 0x14, 0x87, 0xe6, 0x37, 0xfb, 0x80, 0xf8, 0xcd, 0x3d, 0x40, - 0x7e, 0xf3, 0x29, 0x7e, 0x9b, 0x00, 0x5b, 0xae, 0x47, 0x2f, 0xb9, 0x03, 0x46, 0x83, 0x19, 0x45, - 0xd2, 0x0f, 0x73, 0xb1, 0xc7, 0x21, 0x74, 0x47, 0xcb, 0x60, 0xc3, 0x70, 0xf3, 0x0f, 0x82, 0xc5, - 0xec, 0x03, 0x64, 0x31, 0x97, 0xf2, 0x80, 0x1e, 0x94, 0x76, 0x04, 0x7b, 0x32, 0x62, 0x27, 0xfa, - 0x4f, 0x31, 0xef, 0xed, 0x6f, 0xaa, 0xc3, 0x9f, 0x3e, 0x24, 0xe1, 0x12, 0x7d, 0xc4, 0x56, 0x38, - 0xf1, 0x98, 0xf3, 0x8e, 0xb1, 0x9f, 0xe8, 0x43, 0x90, 0xa3, 0x72, 0xba, 0xc2, 0xcc, 0x9c, 0xee, - 0x79, 0x75, 0xcc, 0x7f, 0x93, 0xd7, 0xe1, 0x5e, 0xec, 0x60, 0x85, 0x52, 0x94, 0x83, 0x7d, 0xe4, - 0xb0, 0xe7, 0x2f, 0x1f, 0x3d, 0x5a, 0x4b, 0x96, 0x66, 0xb5, 0xa8, 0x34, 0xeb, 0xd2, 0x77, 0x12, - 0x75, 0x19, 0xfe, 0x83, 0x05, 0x4b, 0x97, 0x29, 0x4b, 0x66, 0x63, 0x5f, 0x21, 0xe5, 0xe3, 0x97, - 0xe0, 0xa8, 0x71, 0x7f, 0x25, 0xa7, 0x27, 0x53, 0x29, 0xd8, 0xb1, 0x58, 0x52, 0x42, 0x06, 0xaa, - 0xb2, 0x4d, 0x66, 0x5f, 0x37, 0xa0, 0x6a, 0x2c, 0xa2, 0x8b, 0xa9, 0xbc, 0x6b, 0x39, 0xd5, 0xd0, - 0xe5, 0xb9, 0x43, 0xbb, 0xae, 0x78, 0x92, 0xf5, 0xab, 0xca, 0xaa, 0xa3, 0x1c, 0x65, 0x1b, 0x90, - 0x50, 0xac, 0x20, 0x6b, 0x46, 0x49, 0x01, 0x7d, 0x39, 0x4a, 0xc0, 0xa2, 0x39, 0x7a, 0x18, 0xf2, - 0x81, 0x7f, 0x4f, 0x27, 0xd4, 0x0b, 0xf1, 0x91, 0xc4, 0xbf, 0x47, 0xc4, 0x12, 0x7e, 0x0e, 0x72, - 0xc4, 0xbf, 0x87, 0x9a, 0x00, 0x81, 0xe3, 0xf5, 0xe8, 0xad, 0xa8, 0x94, 0xab, 0x11, 0x03, 0x72, - 0x40, 0x06, 0xb3, 0x01, 0x47, 0xcd, 0x1b, 0x49, 0x75, 0xaf, 0x43, 0xe9, 0xd5, 0xb1, 0x29, 0xae, - 0x7a, 0x4a, 0x5c, 0xb2, 0x63, 0xa0, 0x91, 0xb8, 0xcd, 0x40, 0x0c, 0x47, 0xa7, 0xa0, 0xc2, 0x9c, - 0xdb, 0x03, 0x7a, 0x2d, 0x76, 0x96, 0x31, 0x80, 0xaf, 0xf2, 0x2a, 0xf4, 0x96, 0x91, 0x8a, 0xc5, - 0x00, 0xf4, 0x38, 0x2c, 0xc5, 0x77, 0xbe, 0x11, 0xd0, 0x1d, 0xf7, 0x1d, 0xa1, 0xe1, 0x1a, 0xd9, - 0x07, 0x47, 0x6b, 0x70, 0x24, 0x86, 0x6d, 0x8b, 0x94, 0x27, 0x2f, 0x50, 0xd3, 0x60, 0x2e, 0x1b, - 0xc1, 0xee, 0x8b, 0x77, 0xc7, 0xce, 0x40, 0x3c, 0xd3, 0x1a, 0x31, 0x20, 0xf8, 0x8f, 0x16, 0x1c, - 0x95, 0xaa, 0xe6, 0x6f, 0xe0, 0xab, 0x68, 0xf5, 0xbf, 0xb4, 0x00, 0x99, 0x1c, 0x28, 0xd3, 0xfa, - 0x7f, 0xb3, 0x23, 0xc5, 0x73, 0xaa, 0xaa, 0x28, 0xae, 0x25, 0x28, 0x6e, 0x2a, 0x61, 0x28, 0x76, - 0x64, 0xe7, 0x4d, 0xb4, 0xd0, 0x65, 0xf5, 0x2e, 0x21, 0x44, 0x7d, 0x23, 0x1b, 0x0a, 0xb7, 0x27, - 0x8c, 0x86, 0xaa, 0xf6, 0x16, 0x4d, 0x07, 0x01, 0x20, 0xf2, 0x8b, 0x9f, 0x45, 0x3d, 0x26, 0xac, - 0x26, 0x1f, 0x9f, 0xa5, 0x40, 0x44, 0x0f, 0xf0, 0xbf, 0xb2, 0xb0, 0x70, 0xcb, 0x1f, 0x8c, 0xe3, - 0xf0, 0xfa, 0x55, 0x0a, 0x2d, 0x89, 0x86, 0x40, 0x41, 0x37, 0x04, 0x10, 0xe4, 0x43, 0x46, 0x47, - 0xc2, 0xb2, 0x72, 0x44, 0x8c, 0x11, 0x86, 0x1a, 0x73, 0x82, 0x1e, 0x65, 0xb2, 0xcc, 0x6a, 0x14, - 0x45, 0xfe, 0x9b, 0x80, 0xa1, 0x55, 0xa8, 0x3a, 0xbd, 0x5e, 0x40, 0x7b, 0x0e, 0xa3, 0xed, 0x49, - 0xa3, 0x24, 0x0e, 0x33, 0x41, 0xe8, 0x2a, 0x2c, 0x76, 0x9c, 0x4e, 0xdf, 0xf5, 0x7a, 0xd7, 0x47, - 0xcc, 0xf5, 0xbd, 0xb0, 0x51, 0x16, 0x1e, 0xfc, 0xd4, 0xba, 0xf9, 0x93, 0xd4, 0xfa, 0x46, 0x02, - 0x47, 0xf9, 0xb1, 0xd4, 0x4e, 0xfc, 0x06, 0x2c, 0x6a, 0xc1, 0x2b, 0xf3, 0x38, 0x0b, 0xa5, 0xb7, - 0x05, 0x64, 0x46, 0xb3, 0x4f, 0xa2, 0x2a, 0x52, 0x1a, 0x2d, 0xf9, 0xa3, 0x86, 0xe6, 0x1f, 0x5f, - 0x85, 0xa2, 0x44, 0x47, 0xa7, 0xcc, 0xc2, 0x4b, 0xe6, 0x9e, 0x7c, 0xae, 0xaa, 0x28, 0x0c, 0x45, - 0x49, 0x48, 0x19, 0x91, 0xb0, 0x33, 0x09, 0x21, 0xea, 0x1b, 0xff, 0x28, 0x0b, 0xc7, 0x36, 0x29, - 0xa3, 0x1d, 0x46, 0xbb, 0x97, 0x5c, 0x3a, 0xe8, 0x7e, 0xa1, 0x3d, 0x81, 0xa8, 0xb3, 0x97, 0x33, - 0x3a, 0x7b, 0xdc, 0x87, 0x0d, 0x5c, 0x8f, 0x6e, 0x19, 0xad, 0xa1, 0x18, 0x10, 0xcb, 0xa8, 0x60, - 0x36, 0x8d, 0xb4, 0x8d, 0x14, 0x0d, 0x1b, 0x89, 0x1b, 0x82, 0xa5, 0x44, 0x0f, 0x53, 0x57, 0xa0, - 0xe5, 0xb8, 0x7c, 0xc5, 0xbf, 0xb3, 0xe0, 0x78, 0x5a, 0x2e, 0x4a, 0x8d, 0x2f, 0x42, 0x71, 0x47, - 0x40, 0xf6, 0xb7, 0x9d, 0x13, 0x3b, 0x64, 0xe7, 0x42, 0xa2, 0x9a, 0x9d, 0x0b, 0x09, 0x41, 0x8f, - 0x25, 0x7e, 0xb0, 0x6a, 0x2f, 0xef, 0x4d, 0xed, 0x23, 0x02, 0x60, 0xe0, 0x2a, 0x66, 0xce, 0x44, - 0x17, 0xcf, 0xc5, 0x2d, 0x11, 0x09, 0x31, 0x09, 0xab, 0xfe, 0xe6, 0x9f, 0x2d, 0x58, 0x48, 0x5c, - 0x44, 0x88, 0x88, 0x3f, 0x01, 0x15, 0x1e, 0xe4, 0x04, 0x3d, 0x06, 0x79, 0x36, 0x19, 0xa9, 0xa8, - 0xd0, 0x3e, 0xf6, 0xd9, 0xd4, 0x3e, 0x9a, 0xd8, 0x76, 0x73, 0x32, 0xa2, 0x44, 0xa0, 0xf0, 0x97, - 0xd3, 0x71, 0x82, 0xae, 0xeb, 0x39, 0x03, 0x97, 0x49, 0xed, 0xe4, 0x89, 0x09, 0xe2, 0xee, 0x68, - 0xe4, 0x04, 0xa1, 0x4e, 0x02, 0x2b, 0xd2, 0x1d, 0x29, 0x10, 0xd1, 0x03, 0xd1, 0xdc, 0xb9, 0x43, - 0x59, 0xa7, 0x2f, 0xc3, 0x82, 0x6a, 0xee, 0x08, 0x48, 0xa2, 0xb9, 0x23, 0x20, 0xf8, 0xe7, 0x56, - 0x6c, 0x9c, 0xf2, 0x0d, 0x7f, 0xe9, 0x8c, 0x13, 0x7f, 0x3b, 0xb6, 0x13, 0x7d, 0x45, 0x65, 0x27, - 0x2f, 0xc0, 0x62, 0x37, 0xb1, 0x72, 0xb0, 0xbd, 0xc8, 0xc6, 0x75, 0x0a, 0x1d, 0x8f, 0x63, 0x3d, - 0x0a, 0xc8, 0x01, 0x7a, 0x4c, 0x29, 0x27, 0xbb, 0x5f, 0x39, 0xb1, 0xd4, 0x73, 0x87, 0x4b, 0xfd, - 0xf1, 0x47, 0xa0, 0x12, 0xfd, 0x48, 0x89, 0xaa, 0x50, 0xba, 0x74, 0x9d, 0xbc, 0x7e, 0x91, 0x6c, - 0x2e, 0x65, 0x50, 0x0d, 0xca, 0xed, 0x8b, 0x1b, 0x2f, 0x8b, 0x99, 0x75, 0xfe, 0xd7, 0x45, 0x9d, - 0xb8, 0x04, 0xe8, 0x1b, 0x50, 0x90, 0xd9, 0xc8, 0xf1, 0x98, 0x39, 0xf3, 0xf7, 0xbb, 0x95, 0x13, - 0xfb, 0xe0, 0x52, 0x4a, 0x38, 0x73, 0xd6, 0x42, 0xd7, 0xa0, 0x2a, 0x80, 0xaa, 0x43, 0x7e, 0x2a, - 0xdd, 0xa8, 0x4e, 0x50, 0x7a, 0xe8, 0x80, 0x55, 0x83, 0xde, 0x05, 0x28, 0x48, 0x81, 0x1d, 0x4f, - 0x25, 0x8d, 0x33, 0x6e, 0x93, 0xf8, 0xcd, 0x00, 0x67, 0xd0, 0xb3, 0x90, 0xbf, 0xe9, 0xb8, 0x03, - 0x64, 0xe4, 0xac, 0x46, 0x63, 0x7b, 0xe5, 0x78, 0x1a, 0x6c, 0x1c, 0xfb, 0x7c, 0xd4, 0x9f, 0x3f, - 0x91, 0x6e, 0x12, 0xea, 0xed, 0x8d, 0xfd, 0x0b, 0xd1, 0xc9, 0xd7, 0x65, 0x17, 0x59, 0xb7, 0xaa, - 0xd0, 0x43, 0xc9, 0xa3, 0x52, 0x9d, 0xad, 0x95, 0xe6, 0x41, 0xcb, 0x11, 0xc1, 0x2d, 0xa8, 0x1a, - 0x6d, 0x22, 0x53, 0xac, 0xfb, 0x7b, 0x5c, 0xa6, 0x58, 0x67, 0xf4, 0x96, 0x70, 0x06, 0x5d, 0x86, - 0x32, 0xcf, 0xf4, 0xc5, 0xcf, 0x49, 0x27, 0xd3, 0x09, 0xbd, 0x91, 0xc8, 0xad, 0x9c, 0x9a, 0xbd, - 0x18, 0x11, 0xfa, 0x16, 0x54, 0x2e, 0x53, 0xa6, 0x22, 0xd8, 0x89, 0x74, 0x08, 0x9c, 0x21, 0xa9, - 0x64, 0x18, 0xc5, 0x19, 0xf4, 0x86, 0x28, 0x3a, 0x92, 0xee, 0x19, 0xd9, 0x07, 0xb8, 0xe1, 0xe8, - 0x5e, 0xab, 0x07, 0x23, 0x44, 0x94, 0x5f, 0x4f, 0x50, 0x56, 0x79, 0x83, 0x7d, 0xc0, 0x83, 0x8d, - 0x28, 0xdb, 0x87, 0xfc, 0xd9, 0x04, 0x67, 0xce, 0xbf, 0xa9, 0xff, 0x6f, 0xb1, 0xe9, 0x30, 0x07, - 0x5d, 0x87, 0x45, 0x21, 0xcb, 0xe8, 0x0f, 0x19, 0x09, 0x9b, 0xdf, 0xf7, 0xef, 0x8f, 0x84, 0xcd, - 0xef, 0xff, 0x17, 0x08, 0xce, 0xb4, 0xdf, 0xfc, 0xe0, 0xe3, 0x66, 0xe6, 0xc3, 0x8f, 0x9b, 0x99, - 0x4f, 0x3f, 0x6e, 0x5a, 0xdf, 0xdb, 0x6d, 0x5a, 0xbf, 0xda, 0x6d, 0x5a, 0xef, 0xef, 0x36, 0xad, - 0x0f, 0x76, 0x9b, 0xd6, 0xdf, 0x76, 0x9b, 0xd6, 0xdf, 0x77, 0x9b, 0x99, 0x4f, 0x77, 0x9b, 0xd6, - 0xbb, 0x9f, 0x34, 0x33, 0x1f, 0x7c, 0xd2, 0xcc, 0x7c, 0xf8, 0x49, 0x33, 0xf3, 0x9d, 0x47, 0x0f, - 0x2f, 0xc5, 0xa5, 0x5b, 0x2c, 0x8a, 0xaf, 0x27, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xc0, 0xc9, - 0x6a, 0x9e, 0x47, 0x24, 0x00, 0x00, + // 2801 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0xcb, 0x8f, 0x1b, 0xc7, + 0xd1, 0xe7, 0xf0, 0xb5, 0x64, 0x91, 0xbb, 0x5a, 0xf5, 0x52, 0x12, 0xb1, 0x92, 0x39, 0xeb, 0xc6, + 0xf7, 0xd9, 0x6b, 0x4b, 0x26, 0x25, 0x39, 0x76, 0x6c, 0x39, 0x8e, 0x23, 0xee, 0x5a, 0xb2, 0xe4, + 0xd5, 0xc3, 0xbd, 0xb2, 0xec, 0x04, 0x31, 0x8c, 0x11, 0xd9, 0x4b, 0x0e, 0x44, 0xce, 0x50, 0x33, + 0x4d, 0xcb, 0xbc, 0xe5, 0x1f, 0x08, 0x62, 0x20, 0x08, 0x92, 0x5c, 0x02, 0x04, 0x08, 0x90, 0x20, + 0x80, 0x2f, 0x41, 0x0e, 0x39, 0x04, 0xc9, 0x25, 0x40, 0x9c, 0x9b, 0x8f, 0x86, 0x0f, 0x4c, 0xbc, + 0xbe, 0x04, 0x0b, 0x04, 0xf0, 0x29, 0x01, 0x7c, 0x0a, 0xfa, 0x35, 0xd3, 0x33, 0xcb, 0x8d, 0x4c, + 0x45, 0x81, 0xad, 0x0b, 0x39, 0xfd, 0xeb, 0xea, 0x47, 0x3d, 0xba, 0xaa, 0xba, 0x66, 0xe0, 0xf8, + 0xe8, 0x76, 0xaf, 0x35, 0xf0, 0x7b, 0xa3, 0xc0, 0x67, 0x7e, 0xf4, 0xd0, 0x14, 0xbf, 0xa8, 0xa4, + 0xdb, 0xab, 0xb5, 0x9e, 0xdf, 0xf3, 0x25, 0x0d, 0x7f, 0x92, 0xfd, 0xab, 0x76, 0xcf, 0xf7, 0x7b, + 0x03, 0xda, 0x12, 0xad, 0x5b, 0xe3, 0x9d, 0x16, 0x73, 0x87, 0x34, 0x64, 0xce, 0x70, 0xa4, 0x08, + 0xd6, 0xd4, 0xec, 0x77, 0x06, 0x43, 0xbf, 0x4b, 0x07, 0xad, 0x90, 0x39, 0x2c, 0x94, 0xbf, 0x8a, + 0x62, 0x85, 0x53, 0x8c, 0xc6, 0x61, 0x5f, 0xfc, 0x28, 0xf0, 0x34, 0x07, 0x43, 0xe6, 0x07, 0x4e, + 0x8f, 0xb6, 0x3a, 0xfd, 0xb1, 0x77, 0xbb, 0xd5, 0x71, 0x3a, 0x7d, 0xda, 0x0a, 0x68, 0x38, 0x1e, + 0xb0, 0x50, 0x36, 0xd8, 0x64, 0x44, 0xd5, 0x34, 0xf8, 0xb7, 0x16, 0x1c, 0xd9, 0x72, 0x6e, 0xd1, + 0xc1, 0x0d, 0xff, 0xa6, 0x33, 0x18, 0xd3, 0x90, 0xd0, 0x70, 0xe4, 0x7b, 0x21, 0x45, 0x1b, 0x50, + 0x1c, 0xf0, 0x8e, 0xb0, 0x6e, 0xad, 0xe5, 0xd6, 0x2b, 0x67, 0x4f, 0x36, 0x23, 0x26, 0x67, 0x0e, + 0x90, 0x68, 0xf8, 0xb2, 0xc7, 0x82, 0x09, 0x51, 0x43, 0x57, 0x6f, 0x42, 0xc5, 0x80, 0xd1, 0x32, + 0xe4, 0x6e, 0xd3, 0x49, 0xdd, 0x5a, 0xb3, 0xd6, 0xcb, 0x84, 0x3f, 0xa2, 0x33, 0x50, 0x78, 0x87, + 0x4f, 0x53, 0xcf, 0xae, 0x59, 0xeb, 0x95, 0xb3, 0xc7, 0xe3, 0x45, 0x5e, 0xf7, 0xdc, 0x3b, 0x63, + 0x2a, 0x46, 0xab, 0x85, 0x24, 0xe5, 0xb9, 0xec, 0x73, 0x16, 0x3e, 0x09, 0x87, 0xf7, 0xf5, 0xa3, + 0xa3, 0x50, 0x14, 0x14, 0x72, 0xc7, 0x65, 0xa2, 0x5a, 0xb8, 0x06, 0x68, 0x9b, 0x05, 0xd4, 0x19, + 0x12, 0x87, 0xf1, 0xfd, 0xde, 0x19, 0xd3, 0x90, 0xe1, 0x2b, 0xb0, 0x92, 0x40, 0x15, 0xdb, 0xcf, + 0x42, 0x25, 0x8c, 0x61, 0xc5, 0x7b, 0x2d, 0xde, 0x56, 0x3c, 0x86, 0x98, 0x84, 0xf8, 0x67, 0x16, + 0x40, 0xdc, 0x87, 0x1a, 0x00, 0xb2, 0xf7, 0x15, 0x27, 0xec, 0x0b, 0x86, 0xf3, 0xc4, 0x40, 0xd0, + 0x29, 0x38, 0x1c, 0xb7, 0xae, 0xfa, 0xdb, 0x7d, 0x27, 0xe8, 0x0a, 0x19, 0xe4, 0xc9, 0xfe, 0x0e, + 0x84, 0x20, 0x1f, 0x38, 0x8c, 0xd6, 0x73, 0x6b, 0xd6, 0x7a, 0x8e, 0x88, 0x67, 0xce, 0x2d, 0xa3, + 0x9e, 0xe3, 0xb1, 0x7a, 0x5e, 0x88, 0x53, 0xb5, 0x38, 0xce, 0x2d, 0x82, 0x86, 0xf5, 0xc2, 0x9a, + 0xb5, 0xbe, 0x48, 0x54, 0x0b, 0xff, 0x33, 0x07, 0xd5, 0xd7, 0xc6, 0x34, 0x98, 0x28, 0x01, 0xa0, + 0x06, 0x94, 0x42, 0x3a, 0xa0, 0x1d, 0xe6, 0x07, 0x52, 0x23, 0xed, 0x6c, 0xdd, 0x22, 0x11, 0x86, + 0x6a, 0x50, 0x18, 0xb8, 0x43, 0x97, 0x89, 0x6d, 0x2d, 0x12, 0xd9, 0x40, 0xe7, 0xa0, 0x10, 0x32, + 0x27, 0x60, 0x62, 0x2f, 0x95, 0xb3, 0xab, 0x4d, 0x69, 0xca, 0x4d, 0x6d, 0xca, 0xcd, 0x1b, 0xda, + 0x94, 0xdb, 0xa5, 0x0f, 0xa6, 0x76, 0xe6, 0xbd, 0xbf, 0xda, 0x16, 0x91, 0x43, 0xd0, 0xb3, 0x90, + 0xa3, 0x5e, 0x57, 0xec, 0xf7, 0x8b, 0x8e, 0xe4, 0x03, 0xd0, 0x19, 0x28, 0x77, 0xdd, 0x80, 0x76, + 0x98, 0xeb, 0x7b, 0x82, 0xab, 0xa5, 0xb3, 0x2b, 0xb1, 0x46, 0x36, 0x75, 0x17, 0x89, 0xa9, 0xd0, + 0x29, 0x28, 0x86, 0x5c, 0x74, 0x61, 0x7d, 0x81, 0xdb, 0x42, 0xbb, 0xb6, 0x37, 0xb5, 0x97, 0x25, + 0x72, 0xca, 0x1f, 0xba, 0x8c, 0x0e, 0x47, 0x6c, 0x42, 0x14, 0x0d, 0x7a, 0x12, 0x16, 0xba, 0x74, + 0x40, 0xb9, 0xc2, 0x4b, 0x42, 0xe1, 0xcb, 0xc6, 0xf4, 0xa2, 0x83, 0x68, 0x02, 0xf4, 0x16, 0xe4, + 0x47, 0x03, 0xc7, 0xab, 0x97, 0x05, 0x17, 0x4b, 0x31, 0xe1, 0xf5, 0x81, 0xe3, 0xb5, 0x9f, 0xff, + 0x78, 0x6a, 0x3f, 0xd3, 0x73, 0x59, 0x7f, 0x7c, 0xab, 0xd9, 0xf1, 0x87, 0xad, 0x5e, 0xe0, 0xec, + 0x38, 0x9e, 0xd3, 0x1a, 0xf8, 0xb7, 0xdd, 0xd6, 0x3b, 0x4f, 0xb7, 0xf8, 0x01, 0xbd, 0x33, 0xa6, + 0x81, 0x4b, 0x83, 0x16, 0x9f, 0xa6, 0x29, 0x54, 0xc2, 0x87, 0x12, 0x31, 0x2d, 0xba, 0xcc, 0xed, + 0xcf, 0x0f, 0xe8, 0x06, 0x3f, 0xbd, 0x61, 0x1d, 0xc4, 0x2a, 0xc7, 0xe2, 0x55, 0x04, 0x4e, 0xe8, + 0xce, 0xc5, 0xc0, 0x1f, 0x8f, 0xda, 0x87, 0xf6, 0xa6, 0xb6, 0x49, 0x4f, 0xcc, 0xc6, 0xe5, 0x7c, + 0xa9, 0xb8, 0xbc, 0x80, 0xdf, 0xcf, 0x01, 0xda, 0x76, 0x86, 0xa3, 0x01, 0x9d, 0x4b, 0xfd, 0x91, + 0xa2, 0xb3, 0xf7, 0xad, 0xe8, 0xdc, 0xbc, 0x8a, 0x8e, 0xb5, 0x96, 0x9f, 0x4f, 0x6b, 0x85, 0x2f, + 0xaa, 0xb5, 0xe2, 0x57, 0x5e, 0x6b, 0xb8, 0x0e, 0x79, 0x3e, 0x33, 0x77, 0x96, 0x81, 0x73, 0x57, + 0xe8, 0xa6, 0x4a, 0xf8, 0x23, 0xde, 0x82, 0xa2, 0xe4, 0x0b, 0xad, 0xa6, 0x95, 0x97, 0x3c, 0xb7, + 0xb1, 0xe2, 0x72, 0x5a, 0x25, 0xcb, 0xb1, 0x4a, 0x72, 0x42, 0xd8, 0xf8, 0xf7, 0x16, 0x2c, 0x2a, + 0x8b, 0x50, 0xbe, 0xef, 0x16, 0x2c, 0x48, 0xdf, 0xa3, 0xfd, 0xde, 0xb1, 0xb4, 0xdf, 0x3b, 0xdf, + 0x75, 0x46, 0x8c, 0x06, 0xed, 0xd6, 0x07, 0x53, 0xdb, 0xfa, 0x78, 0x6a, 0x3f, 0x7e, 0x90, 0xd0, + 0x74, 0x74, 0xd2, 0xfe, 0x52, 0x4f, 0x8c, 0x4e, 0x8a, 0xdd, 0xb1, 0x50, 0x99, 0xd5, 0xa1, 0xa6, + 0x0c, 0x6a, 0x97, 0xbc, 0x1e, 0x0d, 0xf9, 0xcc, 0x79, 0x6e, 0x11, 0x44, 0xd2, 0x70, 0x36, 0xef, + 0x3a, 0x81, 0xe7, 0x7a, 0xbd, 0xb0, 0x9e, 0x13, 0x3e, 0x3d, 0x6a, 0xe3, 0x9f, 0x58, 0xb0, 0x92, + 0x30, 0x6b, 0xc5, 0xc4, 0x73, 0x50, 0x0c, 0xb9, 0xa6, 0x34, 0x0f, 0x86, 0x51, 0x6c, 0x0b, 0xbc, + 0xbd, 0xa4, 0x36, 0x5f, 0x94, 0x6d, 0xa2, 0xe8, 0x1f, 0xdc, 0xd6, 0xfe, 0x64, 0x41, 0x55, 0x04, + 0x26, 0x7d, 0xd6, 0x10, 0xe4, 0x3d, 0x67, 0x48, 0x95, 0xaa, 0xc4, 0xb3, 0x11, 0xad, 0xf8, 0x72, + 0x25, 0x1d, 0xad, 0xe6, 0x75, 0xb0, 0xd6, 0x7d, 0x3b, 0x58, 0x2b, 0x3e, 0x77, 0x35, 0x28, 0x70, + 0xf3, 0x9e, 0x08, 0xe7, 0x5a, 0x26, 0xb2, 0x81, 0xdf, 0x80, 0x45, 0xc5, 0x85, 0x12, 0xed, 0x01, + 0x01, 0x16, 0x35, 0x01, 0x85, 0x2c, 0x18, 0x77, 0xd8, 0x38, 0xa0, 0xdd, 0x2b, 0x94, 0x39, 0x5d, + 0x87, 0x39, 0xf5, 0xac, 0xa0, 0x99, 0xd1, 0x83, 0x87, 0x50, 0x94, 0x9a, 0x43, 0xff, 0x07, 0xe5, + 0x28, 0xf5, 0x11, 0xd2, 0xc9, 0xb5, 0x8b, 0x7b, 0x53, 0x3b, 0xcb, 0x42, 0x12, 0x77, 0x20, 0xdb, + 0x4c, 0x12, 0xac, 0x76, 0x79, 0x6f, 0x6a, 0x4b, 0x40, 0xa5, 0x04, 0xe8, 0x04, 0xe4, 0xfb, 0x3c, + 0xce, 0x72, 0x91, 0xe5, 0xdb, 0xa5, 0xbd, 0xa9, 0x2d, 0xda, 0x44, 0xfc, 0xe2, 0x8b, 0x50, 0xdd, + 0xa2, 0x3d, 0xa7, 0x33, 0x51, 0x8b, 0xd6, 0xf4, 0x74, 0x7c, 0x41, 0x4b, 0xcf, 0xf1, 0x28, 0x54, + 0xa3, 0x15, 0xdf, 0x1e, 0x86, 0xea, 0xf4, 0x54, 0x22, 0xec, 0x4a, 0x88, 0x7f, 0x6a, 0x81, 0xb2, + 0x19, 0x84, 0x8d, 0xec, 0x88, 0xfb, 0x4e, 0xd8, 0x9b, 0xda, 0x0a, 0xd1, 0xc9, 0x0f, 0x7a, 0x01, + 0x16, 0x42, 0xb1, 0x62, 0x28, 0x64, 0x91, 0x34, 0x45, 0xd1, 0xd1, 0x3e, 0xc4, 0x4d, 0x6a, 0x6f, + 0x6a, 0x6b, 0x42, 0xa2, 0x1f, 0x50, 0x33, 0x91, 0x40, 0x48, 0xc6, 0x96, 0xf6, 0xa6, 0xb6, 0x81, + 0x9a, 0x09, 0x05, 0xfe, 0xdc, 0x82, 0xca, 0x0d, 0xc7, 0x8d, 0x4c, 0xae, 0xae, 0x55, 0x1a, 0xfb, + 0x76, 0x09, 0x70, 0xcb, 0xed, 0xd2, 0x81, 0x33, 0xb9, 0xe0, 0x07, 0x62, 0xde, 0x45, 0x12, 0xb5, + 0xe3, 0x98, 0x9f, 0x9f, 0x19, 0xf3, 0x0b, 0xf3, 0x87, 0x82, 0xff, 0xad, 0xe3, 0xbd, 0x9c, 0x2f, + 0x65, 0x97, 0x73, 0xf8, 0x7d, 0x0b, 0xaa, 0x92, 0x79, 0x65, 0xa9, 0xdf, 0x85, 0xa2, 0x94, 0x8d, + 0x60, 0xff, 0x3f, 0x38, 0xb2, 0x93, 0xf3, 0x38, 0x31, 0x35, 0x27, 0x7a, 0x09, 0x96, 0xba, 0x81, + 0x3f, 0x1a, 0xd1, 0xee, 0xb6, 0x72, 0x97, 0xd9, 0xb4, 0xbb, 0xdc, 0x34, 0xfb, 0x49, 0x8a, 0x1c, + 0xff, 0xc5, 0x82, 0x45, 0xe5, 0x7c, 0x94, 0xba, 0x22, 0x11, 0x5b, 0xf7, 0x1d, 0x6d, 0xb3, 0xf3, + 0x46, 0xdb, 0xa3, 0x50, 0xec, 0xf1, 0x78, 0xa4, 0x1d, 0x98, 0x6a, 0xcd, 0x17, 0x85, 0xf1, 0x65, + 0x58, 0xd2, 0xac, 0x1c, 0xe0, 0x81, 0x57, 0xd3, 0x1e, 0xf8, 0x52, 0x97, 0x7a, 0xcc, 0xdd, 0x71, + 0x23, 0x9f, 0xaa, 0xe8, 0xf1, 0x0f, 0x2c, 0x58, 0x4e, 0x93, 0xa0, 0xcd, 0xd4, 0x45, 0xe4, 0xb1, + 0x83, 0xa7, 0x33, 0xef, 0x20, 0x7a, 0x6a, 0x75, 0x13, 0x79, 0xe6, 0x5e, 0x37, 0x91, 0x9a, 0xe9, + 0x64, 0xca, 0xca, 0x2b, 0xe0, 0x1f, 0x5b, 0xb0, 0x98, 0xd0, 0x25, 0x7a, 0x0e, 0xf2, 0x3b, 0x81, + 0x3f, 0x9c, 0x4b, 0x51, 0x62, 0x04, 0xfa, 0x1a, 0x64, 0x99, 0x3f, 0x97, 0x9a, 0xb2, 0xcc, 0xe7, + 0x5a, 0x52, 0xec, 0xe7, 0x64, 0x9e, 0x2f, 0x5b, 0xf8, 0x19, 0x28, 0x0b, 0x86, 0xae, 0x3b, 0x6e, + 0x30, 0x33, 0xc0, 0xcc, 0x66, 0xe8, 0x05, 0x38, 0x24, 0x9d, 0xe1, 0xec, 0xc1, 0xd5, 0x59, 0x83, + 0xab, 0x7a, 0xf0, 0x71, 0x28, 0x88, 0x24, 0x85, 0x0f, 0x11, 0x3e, 0x5e, 0x0d, 0x11, 0x5e, 0xfd, + 0x08, 0xac, 0xf0, 0x33, 0x48, 0x83, 0x70, 0xc3, 0x1f, 0x7b, 0x4c, 0xdf, 0xb3, 0x4e, 0x41, 0x2d, + 0x09, 0x2b, 0x2b, 0xa9, 0x41, 0xa1, 0xc3, 0x01, 0x31, 0xc7, 0x22, 0x91, 0x0d, 0xfc, 0x0b, 0x0b, + 0xd0, 0x45, 0xca, 0xc4, 0x2a, 0x97, 0x36, 0xa3, 0xe3, 0xb1, 0x0a, 0xa5, 0xa1, 0xc3, 0x3a, 0x7d, + 0x1a, 0x84, 0x3a, 0xdf, 0xd1, 0xed, 0x2f, 0x23, 0x51, 0xc5, 0x67, 0x60, 0x25, 0xb1, 0x4b, 0xc5, + 0xd3, 0x2a, 0x94, 0x3a, 0x0a, 0x53, 0x21, 0x32, 0x6a, 0xe3, 0xdf, 0x64, 0xa1, 0xa4, 0xd3, 0x40, + 0x74, 0x06, 0x2a, 0x3b, 0xae, 0xd7, 0xa3, 0xc1, 0x28, 0x70, 0x95, 0x08, 0xf2, 0x32, 0x2d, 0x34, + 0x60, 0x62, 0x36, 0xd0, 0x53, 0xb0, 0x30, 0x0e, 0x69, 0xf0, 0xb6, 0x2b, 0x4f, 0x7a, 0xb9, 0x5d, + 0xdb, 0x9d, 0xda, 0xc5, 0xd7, 0x43, 0x1a, 0x5c, 0xda, 0xe4, 0xc1, 0x67, 0x2c, 0x9e, 0x88, 0xfc, + 0xef, 0xa2, 0x57, 0x95, 0x99, 0x8a, 0x84, 0xaf, 0xfd, 0x75, 0xbe, 0xfd, 0x94, 0xab, 0x1b, 0x05, + 0xfe, 0x90, 0xb2, 0x3e, 0x1d, 0x87, 0xad, 0x8e, 0x3f, 0x1c, 0xfa, 0x5e, 0x4b, 0xd4, 0x1a, 0x04, + 0xd3, 0x3c, 0x82, 0xf2, 0xe1, 0xca, 0x72, 0x6f, 0xc0, 0x02, 0xeb, 0x07, 0xfe, 0xb8, 0xd7, 0x17, + 0x81, 0x21, 0xd7, 0x3e, 0x37, 0xff, 0x7c, 0x7a, 0x06, 0xa2, 0x1f, 0xd0, 0xa3, 0x5c, 0x5a, 0xb4, + 0x73, 0x3b, 0x1c, 0x0f, 0xe5, 0x5d, 0xb5, 0x5d, 0xd8, 0x9b, 0xda, 0xd6, 0x53, 0x24, 0x82, 0xf1, + 0x79, 0x58, 0x4c, 0xa4, 0xce, 0xe8, 0x34, 0xe4, 0x03, 0xba, 0xa3, 0x5d, 0x01, 0xda, 0x9f, 0x61, + 0xcb, 0xe8, 0xcf, 0x69, 0x88, 0xf8, 0xc5, 0xdf, 0xcf, 0x82, 0x6d, 0x54, 0x09, 0x2e, 0xf8, 0xc1, + 0x15, 0xca, 0x02, 0xb7, 0x73, 0xd5, 0x19, 0x52, 0x6d, 0x5e, 0x36, 0x54, 0x86, 0x02, 0x7c, 0xdb, + 0x38, 0x45, 0x30, 0x8c, 0xe8, 0xd0, 0x23, 0x00, 0xe2, 0xd8, 0xc9, 0x7e, 0x79, 0xa0, 0xca, 0x02, + 0x11, 0xdd, 0x1b, 0x09, 0x61, 0xb7, 0xe6, 0x14, 0x8e, 0x12, 0xf2, 0xa5, 0xb4, 0x90, 0xe7, 0x9e, + 0x27, 0x92, 0xac, 0x79, 0x5c, 0x0a, 0xc9, 0xe3, 0x82, 0xff, 0x61, 0x41, 0x63, 0x4b, 0xef, 0xfc, + 0x3e, 0xc5, 0xa1, 0xf9, 0xcd, 0x3e, 0x20, 0x7e, 0x73, 0x0f, 0x90, 0xdf, 0x7c, 0x8a, 0xdf, 0x06, + 0xc0, 0x96, 0xeb, 0xd1, 0x0b, 0xee, 0x80, 0xd1, 0x60, 0xc6, 0xa5, 0xea, 0x87, 0xb9, 0xd8, 0xe3, + 0x10, 0xba, 0xa3, 0x65, 0xb0, 0x61, 0xb8, 0xf9, 0x07, 0xc1, 0x62, 0xf6, 0x01, 0xb2, 0x98, 0x4b, + 0x79, 0x40, 0x0f, 0x16, 0x76, 0x04, 0x7b, 0x32, 0x62, 0x27, 0xea, 0x55, 0x31, 0xef, 0xed, 0x6f, + 0xaa, 0xc5, 0x9f, 0xbd, 0x47, 0xc2, 0x25, 0xea, 0x8e, 0xad, 0x70, 0xe2, 0x31, 0xe7, 0x5d, 0x63, + 0x3c, 0xd1, 0x8b, 0x20, 0x47, 0xe5, 0x74, 0x85, 0x99, 0x39, 0xdd, 0x8b, 0x6a, 0x99, 0xff, 0x26, + 0xaf, 0xc3, 0xbd, 0xd8, 0xc1, 0x0a, 0xa5, 0x28, 0x07, 0xfb, 0xd8, 0xbd, 0x8e, 0xbf, 0x3c, 0xf4, + 0x68, 0x3d, 0x79, 0x95, 0xab, 0x46, 0x57, 0xb9, 0x2e, 0x7d, 0x37, 0x71, 0x8f, 0xc3, 0x7f, 0xb0, + 0x60, 0xf9, 0x22, 0x65, 0xc9, 0x6c, 0xec, 0x21, 0x52, 0x3e, 0x7e, 0x05, 0x0e, 0x1b, 0xfb, 0x57, + 0x72, 0x7a, 0x3a, 0x95, 0x82, 0x1d, 0x89, 0x25, 0x25, 0x64, 0xa0, 0x6e, 0xc2, 0xc9, 0xec, 0xeb, + 0x3a, 0x54, 0x8c, 0x4e, 0x74, 0x3e, 0x95, 0x77, 0xad, 0xa4, 0x0a, 0xc0, 0x3c, 0x77, 0x68, 0xd7, + 0x14, 0x4f, 0xf2, 0xbe, 0xab, 0xb2, 0xea, 0x28, 0x47, 0xd9, 0x06, 0x24, 0x14, 0x2b, 0xa6, 0x35, + 0xa3, 0xa4, 0x40, 0x5f, 0x8d, 0x12, 0xb0, 0xa8, 0x8d, 0x1e, 0x85, 0x7c, 0xe0, 0xdf, 0xd5, 0x09, + 0xf5, 0x62, 0xbc, 0x24, 0xf1, 0xef, 0x12, 0xd1, 0x85, 0x5f, 0x80, 0x1c, 0xf1, 0xef, 0xa2, 0x06, + 0x40, 0xe0, 0x78, 0x3d, 0x7a, 0x33, 0xba, 0xca, 0x55, 0x89, 0x81, 0x1c, 0x90, 0xc1, 0x6c, 0xc0, + 0x61, 0x73, 0x47, 0x52, 0xdd, 0x4d, 0x58, 0x78, 0x6d, 0x6c, 0x8a, 0xab, 0x96, 0x12, 0x97, 0xac, + 0x30, 0x68, 0x22, 0x6e, 0x33, 0x10, 0xe3, 0xe8, 0x04, 0x94, 0x99, 0x73, 0x6b, 0x40, 0xaf, 0xc6, + 0xce, 0x32, 0x06, 0x78, 0x2f, 0xbf, 0x85, 0xde, 0x34, 0x52, 0xb1, 0x18, 0x40, 0x4f, 0xc2, 0x72, + 0xbc, 0xe7, 0xeb, 0x01, 0xdd, 0x71, 0xdf, 0x15, 0x1a, 0xae, 0x92, 0x7d, 0x38, 0x5a, 0x87, 0x43, + 0x31, 0xb6, 0x2d, 0x52, 0x9e, 0xbc, 0x20, 0x4d, 0xc3, 0x5c, 0x36, 0x82, 0xdd, 0x97, 0xef, 0x8c, + 0x9d, 0x81, 0x38, 0xa6, 0x55, 0x62, 0x20, 0xf8, 0x8f, 0x16, 0x1c, 0x96, 0xaa, 0xe6, 0x67, 0xe0, + 0x61, 0xb4, 0xfa, 0x5f, 0x5a, 0x80, 0x4c, 0x0e, 0x94, 0x69, 0xfd, 0xbf, 0x59, 0xc1, 0xe2, 0x39, + 0x55, 0x45, 0x5c, 0xae, 0x25, 0x14, 0x17, 0xa1, 0x30, 0x14, 0x3b, 0xb2, 0x52, 0x27, 0x4a, 0xee, + 0xf2, 0xf6, 0x2e, 0x11, 0xa2, 0xfe, 0x91, 0x0d, 0x85, 0x5b, 0x13, 0x46, 0x43, 0x75, 0xf7, 0x16, + 0x45, 0x07, 0x01, 0x10, 0xf9, 0xc7, 0xd7, 0xa2, 0x1e, 0x13, 0x56, 0x93, 0x8f, 0xd7, 0x52, 0x10, + 0xd1, 0x0f, 0xf8, 0x5f, 0x59, 0x58, 0xbc, 0xe9, 0x0f, 0xc6, 0x71, 0x78, 0x7d, 0x98, 0x42, 0x4b, + 0xa2, 0x20, 0x50, 0xd0, 0x05, 0x01, 0x04, 0xf9, 0x90, 0xd1, 0x91, 0xb0, 0xac, 0x1c, 0x11, 0xcf, + 0x08, 0x43, 0x95, 0x39, 0x41, 0x8f, 0x32, 0x79, 0xcd, 0xaa, 0x17, 0x45, 0xfe, 0x9b, 0xc0, 0xd0, + 0x1a, 0x54, 0x9c, 0x5e, 0x2f, 0xa0, 0x3d, 0x87, 0xd1, 0xf6, 0xa4, 0xbe, 0x20, 0x16, 0x33, 0x21, + 0x74, 0x19, 0x96, 0x3a, 0x4e, 0xa7, 0xef, 0x7a, 0xbd, 0x6b, 0x23, 0xe6, 0xfa, 0x5e, 0x58, 0x2f, + 0x09, 0x0f, 0x7e, 0xa2, 0x69, 0xbe, 0xc2, 0x6a, 0x6e, 0x24, 0x68, 0x94, 0x1f, 0x4b, 0x8d, 0xc4, + 0x6f, 0xc2, 0x92, 0x16, 0xbc, 0x32, 0x8f, 0xd3, 0xb0, 0xf0, 0x8e, 0x40, 0x66, 0x14, 0x07, 0x25, + 0xa9, 0x9a, 0x4a, 0x93, 0x25, 0x5f, 0x82, 0x68, 0xfe, 0xf1, 0x65, 0x28, 0x4a, 0x72, 0x74, 0xc2, + 0xbc, 0x78, 0xc9, 0xdc, 0x93, 0xb7, 0xd5, 0x2d, 0x0a, 0x43, 0x51, 0x4e, 0xa4, 0x8c, 0x48, 0xd8, + 0x99, 0x44, 0x88, 0xfa, 0xc7, 0x3f, 0xca, 0xc2, 0x91, 0x4d, 0xca, 0x68, 0x87, 0xd1, 0xee, 0x05, + 0x97, 0x0e, 0xba, 0x5f, 0x6a, 0x4d, 0x20, 0xaa, 0x04, 0xe6, 0x8c, 0x4a, 0x20, 0xf7, 0x61, 0x03, + 0xd7, 0xa3, 0x5b, 0x46, 0x69, 0x28, 0x06, 0x62, 0x19, 0x15, 0xcc, 0xa2, 0x91, 0xb6, 0x91, 0xa2, + 0x61, 0x23, 0x71, 0x01, 0x71, 0x21, 0x51, 0xf3, 0xd4, 0x37, 0xd0, 0x52, 0x7c, 0x7d, 0xc5, 0xbf, + 0xb3, 0xe0, 0x68, 0x5a, 0x2e, 0x4a, 0x8d, 0x2f, 0x43, 0x71, 0x47, 0x20, 0xfb, 0xcb, 0xd4, 0x89, + 0x11, 0xb2, 0x72, 0x21, 0x49, 0xcd, 0xca, 0x85, 0x44, 0xd0, 0x13, 0x89, 0x17, 0x5c, 0xed, 0x95, + 0xbd, 0xa9, 0x7d, 0x48, 0x00, 0x06, 0xad, 0x62, 0xe6, 0x54, 0xb4, 0xf1, 0x5c, 0x5c, 0x12, 0x91, + 0x88, 0x39, 0xb1, 0x7a, 0xe1, 0xf8, 0x67, 0x0b, 0x16, 0x13, 0x1b, 0x11, 0x22, 0xe2, 0x47, 0x40, + 0x85, 0x07, 0xd9, 0x40, 0x4f, 0x40, 0x9e, 0x4d, 0x46, 0x2a, 0x2a, 0xb4, 0x8f, 0x7c, 0x3e, 0xb5, + 0x0f, 0x27, 0x86, 0xdd, 0x98, 0x8c, 0x28, 0x11, 0x24, 0xfc, 0xe4, 0x74, 0x9c, 0xa0, 0xeb, 0x7a, + 0xce, 0xc0, 0x65, 0x52, 0x3b, 0x79, 0x62, 0x42, 0xdc, 0x1d, 0x8d, 0x9c, 0x20, 0xd4, 0x49, 0x60, + 0x59, 0xba, 0x23, 0x05, 0x11, 0xfd, 0x20, 0x8a, 0x3b, 0xb7, 0x29, 0xeb, 0xf4, 0x65, 0x58, 0x50, + 0xc5, 0x1d, 0x81, 0x24, 0x8a, 0x3b, 0x02, 0xc1, 0x3f, 0xb7, 0x62, 0xe3, 0x94, 0x67, 0xf8, 0x2b, + 0x67, 0x9c, 0xf8, 0xdb, 0xb1, 0x9d, 0xe8, 0x2d, 0x2a, 0x3b, 0x79, 0x09, 0x96, 0xba, 0x89, 0x9e, + 0x83, 0xed, 0x45, 0x16, 0xba, 0x53, 0xe4, 0x78, 0x1c, 0xeb, 0x51, 0x20, 0x07, 0xe8, 0x31, 0xa5, + 0x9c, 0xec, 0x7e, 0xe5, 0xc4, 0x52, 0xcf, 0xdd, 0x5b, 0xea, 0x4f, 0x3e, 0x06, 0xe5, 0xe8, 0xa5, + 0x26, 0xaa, 0xc0, 0xc2, 0x85, 0x6b, 0xe4, 0x8d, 0xf3, 0x64, 0x73, 0x39, 0x83, 0xaa, 0x50, 0x6a, + 0x9f, 0xdf, 0x78, 0x55, 0xb4, 0xac, 0xb3, 0xbf, 0x2e, 0xea, 0xc4, 0x25, 0x40, 0xdf, 0x80, 0x82, + 0xcc, 0x46, 0x8e, 0xc6, 0xcc, 0x99, 0xef, 0xfb, 0x56, 0x8f, 0xed, 0xc3, 0xa5, 0x94, 0x70, 0xe6, + 0xb4, 0x85, 0xae, 0x42, 0x45, 0x80, 0xaa, 0x42, 0x7e, 0x22, 0x5d, 0xa8, 0x4e, 0xcc, 0xf4, 0xc8, + 0x01, 0xbd, 0xc6, 0x7c, 0xe7, 0xa0, 0x20, 0x05, 0x76, 0x34, 0x95, 0x34, 0xce, 0xd8, 0x4d, 0xe2, + 0x1d, 0x03, 0xce, 0xa0, 0xe7, 0x21, 0x7f, 0xc3, 0x71, 0x07, 0xc8, 0xc8, 0x59, 0x8d, 0xc2, 0xf6, + 0xea, 0xd1, 0x34, 0x6c, 0x2c, 0xfb, 0x62, 0x54, 0x9f, 0x3f, 0x96, 0x2e, 0x12, 0xea, 0xe1, 0xf5, + 0xfd, 0x1d, 0xd1, 0xca, 0xd7, 0x64, 0x15, 0x59, 0x97, 0xaa, 0xd0, 0x23, 0xc9, 0xa5, 0x52, 0x95, + 0xad, 0xd5, 0xc6, 0x41, 0xdd, 0xd1, 0x84, 0x5b, 0x50, 0x31, 0xca, 0x44, 0xa6, 0x58, 0xf7, 0xd7, + 0xb8, 0x4c, 0xb1, 0xce, 0xa8, 0x2d, 0xe1, 0x0c, 0xba, 0x08, 0x25, 0x9e, 0xe9, 0x8b, 0xd7, 0x4f, + 0xc7, 0xd3, 0x09, 0xbd, 0x91, 0xc8, 0xad, 0x9e, 0x98, 0xdd, 0x19, 0x4d, 0xf4, 0x2d, 0x28, 0x5f, + 0xa4, 0x4c, 0x45, 0xb0, 0x63, 0xe9, 0x10, 0x38, 0x43, 0x52, 0xc9, 0x30, 0x8a, 0x33, 0xe8, 0x4d, + 0x71, 0xe9, 0x48, 0xba, 0x67, 0x64, 0x1f, 0xe0, 0x86, 0xa3, 0x7d, 0xad, 0x1d, 0x4c, 0x10, 0xcd, + 0xfc, 0x46, 0x62, 0x66, 0x95, 0x37, 0xd8, 0x07, 0x1c, 0xd8, 0x68, 0x66, 0xfb, 0x1e, 0x1f, 0xa7, + 0xe0, 0xcc, 0xd9, 0xb7, 0xf4, 0xf7, 0x19, 0x9b, 0x0e, 0x73, 0xd0, 0x35, 0x58, 0x12, 0xb2, 0x8c, + 0x3e, 0xe0, 0x48, 0xd8, 0xfc, 0xbe, 0xaf, 0x45, 0x12, 0x36, 0xbf, 0xff, 0xab, 0x11, 0x9c, 0x69, + 0xbf, 0xf5, 0xe1, 0x27, 0x8d, 0xcc, 0x47, 0x9f, 0x34, 0x32, 0x9f, 0x7d, 0xd2, 0xb0, 0xbe, 0xb7, + 0xdb, 0xb0, 0x7e, 0xb5, 0xdb, 0xb0, 0x3e, 0xd8, 0x6d, 0x58, 0x1f, 0xee, 0x36, 0xac, 0xbf, 0xed, + 0x36, 0xac, 0xbf, 0xef, 0x36, 0x32, 0x9f, 0xed, 0x36, 0xac, 0xf7, 0x3e, 0x6d, 0x64, 0x3e, 0xfc, + 0xb4, 0x91, 0xf9, 0xe8, 0xd3, 0x46, 0xe6, 0x3b, 0x8f, 0xdf, 0xfb, 0x2a, 0x2e, 0xdd, 0x62, 0x51, + 0xfc, 0x3d, 0xfd, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc6, 0x7b, 0x80, 0x47, 0x77, 0x24, 0x00, + 0x00, } func (x Direction) String() string { @@ -3813,6 +3823,14 @@ func (this *LabelResponse) Equal(that interface{}) bool { return false } } + if len(this.StructuredMetadata) != len(that1.StructuredMetadata) { + return false + } + for i := range this.StructuredMetadata { + if this.StructuredMetadata[i] != that1.StructuredMetadata[i] { + return false + } + } return true } func (this *Sample) Equal(that interface{}) bool { @@ -5326,9 +5344,10 @@ func (this *LabelResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 5) + s := make([]string, 0, 6) s = append(s, "&logproto.LabelResponse{") s = append(s, "Values: "+fmt.Sprintf("%#v", this.Values)+",\n") + s = append(s, "StructuredMetadata: "+fmt.Sprintf("%#v", this.StructuredMetadata)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -7137,6 +7156,15 @@ func (m *LabelResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.StructuredMetadata) > 0 { + for iNdEx := len(m.StructuredMetadata) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.StructuredMetadata[iNdEx]) + copy(dAtA[i:], m.StructuredMetadata[iNdEx]) + i = encodeVarintLogproto(dAtA, i, uint64(len(m.StructuredMetadata[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } if len(m.Values) > 0 { for iNdEx := len(m.Values) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.Values[iNdEx]) @@ -9289,6 +9317,12 @@ func (m *LabelResponse) Size() (n int) { n += 1 + l + sovLogproto(uint64(l)) } } + if len(m.StructuredMetadata) > 0 { + for _, s := range m.StructuredMetadata { + l = len(s) + n += 1 + l + sovLogproto(uint64(l)) + } + } return n } @@ -10290,6 +10324,7 @@ func (this *LabelResponse) String() string { } s := strings.Join([]string{`&LabelResponse{`, `Values:` + fmt.Sprintf("%v", this.Values) + `,`, + `StructuredMetadata:` + fmt.Sprintf("%v", this.StructuredMetadata) + `,`, `}`, }, "") return s @@ -12824,6 +12859,38 @@ func (m *LabelResponse) Unmarshal(dAtA []byte) error { } m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StructuredMetadata", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StructuredMetadata = append(m.StructuredMetadata, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 543a346cc5c7d..e75865d1385cd 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -152,6 +152,7 @@ message LabelRequest { message LabelResponse { repeated string values = 1; + repeated string structuredMetadata = 2; } message Sample { diff --git a/pkg/querier/handler.go b/pkg/querier/handler.go index bb01a0e3754b5..7788b94b09a47 100644 --- a/pkg/querier/handler.go +++ b/pkg/querier/handler.go @@ -73,9 +73,10 @@ func (h *Handler) Do(ctx context.Context, req queryrangebase.Request) (queryrang } return &queryrange.LokiLabelNamesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: res.Values, + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: res.Values, + StructuredMetadata: res.StructuredMetadata, }, nil case *logproto.IndexStatsRequest: request := loghttp.NewRangeQueryWithDefaults() diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index 43dc2f50cd306..348970e53d438 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -565,11 +565,12 @@ func (m *LokiSeriesResponse) GetStatistics() stats.Result { } type LokiLabelNamesResponse struct { - Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` - Data []string `protobuf:"bytes,2,rep,name=Data,proto3" json:"data,omitempty"` - Version uint32 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` - Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,4,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"` - Statistics stats.Result `protobuf:"bytes,5,opt,name=statistics,proto3" json:"statistics"` + Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` + Data []string `protobuf:"bytes,2,rep,name=Data,proto3" json:"data,omitempty"` + Version uint32 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` + Headers []github_com_grafana_loki_v3_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,4,rep,name=Headers,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"` + Statistics stats.Result `protobuf:"bytes,5,opt,name=statistics,proto3" json:"statistics"` + StructuredMetadata []string `protobuf:"bytes,6,rep,name=StructuredMetadata,proto3" json:"structuredMetadata,omitempty"` } func (m *LokiLabelNamesResponse) Reset() { *m = LokiLabelNamesResponse{} } @@ -632,6 +633,13 @@ func (m *LokiLabelNamesResponse) GetStatistics() stats.Result { return stats.Result{} } +func (m *LokiLabelNamesResponse) GetStructuredMetadata() []string { + if m != nil { + return m.StructuredMetadata + } + return nil +} + type LokiData struct { ResultType string `protobuf:"bytes,1,opt,name=ResultType,proto3" json:"resultType"` Result []github_com_grafana_loki_pkg_push.Stream `protobuf:"bytes,2,rep,name=Result,proto3,customtype=github.com/grafana/loki/pkg/push.Stream" json:"result"` @@ -1571,132 +1579,134 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1991 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcf, 0x6f, 0x1b, 0xc7, - 0x15, 0xe6, 0xf2, 0xa7, 0x38, 0x92, 0x68, 0x75, 0xac, 0x2a, 0x5b, 0xc5, 0xe1, 0xb2, 0x04, 0x9a, - 0xa8, 0x45, 0x4b, 0xc6, 0x54, 0xe2, 0x26, 0xaa, 0x6b, 0xc4, 0x6b, 0xd9, 0x95, 0x5d, 0xbb, 0x71, - 0x56, 0x42, 0x0e, 0xbd, 0x14, 0x23, 0x72, 0x44, 0x6e, 0x45, 0xee, 0xae, 0x77, 0x86, 0xb2, 0x05, - 0x14, 0x45, 0xfe, 0x81, 0xa0, 0xb9, 0xf7, 0x5e, 0xf4, 0x56, 0x14, 0x28, 0x7a, 0xe8, 0xa9, 0xc7, - 0xe4, 0x50, 0xc0, 0xc7, 0x80, 0x40, 0xd9, 0x9a, 0xbe, 0x14, 0x3a, 0x05, 0xe8, 0x3f, 0x50, 0xcc, - 0x8f, 0x5d, 0xce, 0x70, 0x57, 0x35, 0xe9, 0x16, 0x05, 0x54, 0xf8, 0x42, 0xee, 0xcc, 0xbc, 0x6f, - 0x76, 0xe6, 0x7b, 0xdf, 0x9b, 0x37, 0x33, 0x0b, 0xde, 0x0a, 0x8e, 0xbb, 0xcd, 0x47, 0x43, 0x1c, - 0xba, 0x38, 0xe4, 0xff, 0xa7, 0x21, 0xf2, 0xba, 0x58, 0x79, 0x6c, 0x04, 0xa1, 0x4f, 0x7d, 0x08, - 0xa6, 0x35, 0x9b, 0xad, 0xae, 0x4b, 0x7b, 0xc3, 0xc3, 0x46, 0xdb, 0x1f, 0x34, 0xbb, 0x7e, 0xd7, - 0x6f, 0x76, 0x7d, 0xbf, 0xdb, 0xc7, 0x28, 0x70, 0x89, 0x7c, 0x6c, 0x86, 0x41, 0xbb, 0x49, 0x28, - 0xa2, 0x43, 0x22, 0xf0, 0x9b, 0xeb, 0xcc, 0x90, 0x3f, 0x72, 0x88, 0xac, 0xb5, 0xa4, 0x39, 0x2f, - 0x1d, 0x0e, 0x8f, 0x9a, 0xd4, 0x1d, 0x60, 0x42, 0xd1, 0x20, 0x88, 0x0c, 0xd8, 0xf8, 0xfa, 0x7e, - 0x57, 0x20, 0x5d, 0xaf, 0x83, 0x9f, 0x74, 0x11, 0xc5, 0x8f, 0xd1, 0xa9, 0x34, 0x78, 0x5d, 0x33, - 0x88, 0x1e, 0x64, 0xe3, 0xa6, 0xd6, 0x18, 0x20, 0x4a, 0x71, 0xe8, 0xc9, 0xb6, 0x6f, 0x68, 0x6d, - 0xe4, 0x18, 0xd3, 0x76, 0x4f, 0x36, 0xd5, 0x64, 0xd3, 0xa3, 0xfe, 0xc0, 0xef, 0xe0, 0x3e, 0x9f, - 0x08, 0x11, 0xbf, 0xd2, 0xe2, 0x32, 0xb3, 0x08, 0x86, 0xa4, 0xc7, 0x7f, 0x64, 0xe5, 0xad, 0x17, - 0x72, 0x79, 0x88, 0x08, 0x6e, 0x76, 0xf0, 0x91, 0xeb, 0xb9, 0xd4, 0xf5, 0x3d, 0xa2, 0x3e, 0xcb, - 0x4e, 0xae, 0xcd, 0xd7, 0xc9, 0xac, 0x7f, 0x36, 0xdf, 0x66, 0x38, 0x42, 0xfd, 0x10, 0x75, 0x71, - 0xb3, 0xdd, 0x1b, 0x7a, 0xc7, 0xcd, 0x36, 0x6a, 0xf7, 0x70, 0x33, 0xc4, 0x64, 0xd8, 0xa7, 0x44, - 0x14, 0xe8, 0x69, 0x80, 0xe5, 0x9b, 0xea, 0x5f, 0xe4, 0xc1, 0xf2, 0x7d, 0xff, 0xd8, 0x75, 0xf0, - 0xa3, 0x21, 0x26, 0x14, 0xae, 0x83, 0x02, 0xef, 0xd5, 0x34, 0x6a, 0xc6, 0x56, 0xd9, 0x11, 0x05, - 0x56, 0xdb, 0x77, 0x07, 0x2e, 0x35, 0xb3, 0x35, 0x63, 0x6b, 0xd5, 0x11, 0x05, 0x08, 0x41, 0x9e, - 0x50, 0x1c, 0x98, 0xb9, 0x9a, 0xb1, 0x95, 0x73, 0xf8, 0x33, 0xdc, 0x04, 0x4b, 0xae, 0x47, 0x71, - 0x78, 0x82, 0xfa, 0x66, 0x99, 0xd7, 0xc7, 0x65, 0x78, 0x03, 0x94, 0x08, 0x45, 0x21, 0x3d, 0x20, - 0x66, 0xbe, 0x66, 0x6c, 0x2d, 0xb7, 0x36, 0x1b, 0xc2, 0xf3, 0x8d, 0xc8, 0xf3, 0x8d, 0x83, 0xc8, - 0xf3, 0xf6, 0xd2, 0xe7, 0x63, 0x2b, 0xf3, 0xd9, 0xdf, 0x2c, 0xc3, 0x89, 0x40, 0x70, 0x07, 0x14, - 0xb0, 0xd7, 0x39, 0x20, 0x66, 0x61, 0x01, 0xb4, 0x80, 0xc0, 0xab, 0xa0, 0xdc, 0x71, 0x43, 0xdc, - 0x66, 0x2c, 0x9b, 0xc5, 0x9a, 0xb1, 0x55, 0x69, 0x5d, 0x6e, 0xc4, 0x42, 0xd9, 0x8d, 0x9a, 0x9c, - 0xa9, 0x15, 0x9b, 0x5e, 0x80, 0x68, 0xcf, 0x2c, 0x71, 0x26, 0xf8, 0x33, 0xac, 0x83, 0x22, 0xe9, - 0xa1, 0xb0, 0x43, 0xcc, 0xa5, 0x5a, 0x6e, 0xab, 0x6c, 0x83, 0xb3, 0xb1, 0x25, 0x6b, 0x1c, 0xf9, - 0x0f, 0x7f, 0x06, 0xf2, 0x41, 0x1f, 0x79, 0x26, 0xe0, 0xa3, 0x5c, 0x6b, 0x28, 0x5e, 0x7a, 0xd8, - 0x47, 0x9e, 0xfd, 0xfe, 0x68, 0x6c, 0xbd, 0xab, 0x06, 0x4f, 0x88, 0x8e, 0x90, 0x87, 0x9a, 0x7d, - 0xff, 0xd8, 0x6d, 0x9e, 0x6c, 0x37, 0x55, 0xdf, 0xb3, 0x8e, 0x1a, 0x1f, 0xb1, 0x0e, 0x18, 0xd4, - 0xe1, 0x1d, 0xc3, 0x7b, 0x60, 0x99, 0xf9, 0x18, 0xdf, 0x62, 0x0e, 0x26, 0xe6, 0x32, 0x7f, 0xcf, - 0x6b, 0xd3, 0xd9, 0xf0, 0x7a, 0x07, 0x1f, 0xfd, 0x28, 0xf4, 0x87, 0x81, 0x7d, 0xe9, 0x6c, 0x6c, - 0xa9, 0xf6, 0x8e, 0x5a, 0x80, 0xf7, 0x40, 0x85, 0x89, 0xc2, 0xf5, 0xba, 0x1f, 0x06, 0x5c, 0x81, - 0xe6, 0x0a, 0xef, 0xee, 0x4a, 0x43, 0x95, 0x4c, 0xe3, 0x96, 0x66, 0x63, 0xe7, 0x19, 0xbd, 0xce, - 0x0c, 0xb2, 0x3e, 0xc9, 0x01, 0xc8, 0xb4, 0x74, 0xd7, 0x23, 0x14, 0x79, 0xf4, 0x65, 0x24, 0x75, - 0x1d, 0x14, 0x59, 0xf0, 0x1f, 0x10, 0x2e, 0xaa, 0x79, 0x7d, 0x2c, 0x31, 0xba, 0x93, 0xf3, 0x0b, - 0x39, 0xb9, 0x90, 0xea, 0xe4, 0xe2, 0x0b, 0x9d, 0x5c, 0xfa, 0x1f, 0x39, 0x79, 0xe9, 0xbf, 0xeb, - 0xe4, 0xf2, 0x4b, 0x3b, 0xd9, 0x04, 0x79, 0x36, 0x4a, 0xb8, 0x06, 0x72, 0x21, 0x7a, 0xcc, 0x7d, - 0xba, 0xe2, 0xb0, 0xc7, 0xfa, 0x24, 0x0f, 0x56, 0xc4, 0x52, 0x42, 0x02, 0xdf, 0x23, 0x98, 0xf1, - 0xb8, 0xcf, 0x57, 0x7f, 0xe1, 0x79, 0xc9, 0x23, 0xaf, 0x71, 0x64, 0x0b, 0xfc, 0x00, 0xe4, 0x77, - 0x11, 0x45, 0x5c, 0x05, 0xcb, 0xad, 0x75, 0x95, 0x47, 0xd6, 0x17, 0x6b, 0xb3, 0x37, 0xd8, 0x40, - 0xce, 0xc6, 0x56, 0xa5, 0x83, 0x28, 0xfa, 0xae, 0x3f, 0x70, 0x29, 0x1e, 0x04, 0xf4, 0xd4, 0xe1, - 0x48, 0xf8, 0x2e, 0x28, 0xdf, 0x0e, 0x43, 0x3f, 0x3c, 0x38, 0x0d, 0x30, 0x57, 0x4d, 0xd9, 0x7e, - 0xed, 0x6c, 0x6c, 0x5d, 0xc6, 0x51, 0xa5, 0x82, 0x98, 0x5a, 0xc2, 0x6f, 0x83, 0x02, 0x2f, 0x70, - 0x9d, 0x94, 0xed, 0xcb, 0x67, 0x63, 0xeb, 0x12, 0x87, 0x28, 0xe6, 0xc2, 0x42, 0x97, 0x55, 0x61, - 0x2e, 0x59, 0xc5, 0xea, 0x2e, 0xaa, 0xea, 0x36, 0x41, 0xe9, 0x04, 0x87, 0x84, 0x75, 0x53, 0xe2, - 0xf5, 0x51, 0x11, 0xde, 0x04, 0x80, 0x11, 0xe3, 0x12, 0xea, 0xb6, 0x23, 0x67, 0xaf, 0x36, 0x44, - 0xb2, 0x71, 0xb8, 0x8f, 0x6c, 0x28, 0x59, 0x50, 0x0c, 0x1d, 0xe5, 0x19, 0xfe, 0xce, 0x00, 0xa5, - 0x3d, 0x8c, 0x3a, 0x38, 0x64, 0xee, 0xcd, 0x6d, 0x2d, 0xb7, 0xbe, 0xd5, 0x50, 0x33, 0xcb, 0xc3, - 0xd0, 0x1f, 0x60, 0xda, 0xc3, 0x43, 0x12, 0x39, 0x48, 0x58, 0xdb, 0xde, 0x68, 0x6c, 0xe1, 0x39, - 0xa5, 0x3a, 0x57, 0x42, 0x3b, 0xf7, 0x55, 0x67, 0x63, 0xcb, 0xf8, 0x9e, 0x13, 0x8d, 0x12, 0xb6, - 0xc0, 0xd2, 0x63, 0x14, 0x7a, 0xae, 0xd7, 0x25, 0x26, 0xe0, 0x91, 0xb6, 0x71, 0x36, 0xb6, 0x60, - 0x54, 0xa7, 0x38, 0x22, 0xb6, 0xab, 0xff, 0xd5, 0x00, 0x5f, 0x63, 0xc2, 0xd8, 0x67, 0xe3, 0x21, - 0xca, 0x12, 0x33, 0x40, 0xb4, 0xdd, 0x33, 0x0d, 0xd6, 0x8d, 0x23, 0x0a, 0x6a, 0xbe, 0xc9, 0xfe, - 0x47, 0xf9, 0x26, 0xb7, 0x78, 0xbe, 0x89, 0xd6, 0x95, 0x7c, 0xea, 0xba, 0x52, 0x38, 0x6f, 0x5d, - 0xa9, 0xff, 0x4a, 0xae, 0xa1, 0xd1, 0xfc, 0x16, 0x08, 0xa5, 0x3b, 0x71, 0x28, 0xe5, 0xf8, 0x68, - 0x63, 0x85, 0x8a, 0xbe, 0xee, 0x76, 0xb0, 0x47, 0xdd, 0x23, 0x17, 0x87, 0x2f, 0x08, 0x28, 0x45, - 0xa5, 0x39, 0x5d, 0xa5, 0xaa, 0xc4, 0xf2, 0x17, 0x42, 0x62, 0x7a, 0x5c, 0x15, 0x5e, 0x22, 0xae, - 0xea, 0xff, 0xcc, 0x82, 0x0d, 0xe6, 0x91, 0xfb, 0xe8, 0x10, 0xf7, 0x7f, 0x82, 0x06, 0x0b, 0x7a, - 0xe5, 0x4d, 0xc5, 0x2b, 0x65, 0x1b, 0xbe, 0x62, 0x7d, 0x3e, 0xd6, 0x7f, 0x63, 0x80, 0xa5, 0x28, - 0x01, 0xc0, 0x06, 0x00, 0x02, 0xc6, 0xd7, 0x78, 0xc1, 0x75, 0x85, 0x81, 0xc3, 0xb8, 0xd6, 0x51, - 0x2c, 0xe0, 0xcf, 0x41, 0x51, 0x94, 0x64, 0x2c, 0x28, 0x69, 0x73, 0x9f, 0x86, 0x18, 0x0d, 0x6e, - 0x76, 0x50, 0x40, 0x71, 0x68, 0xbf, 0xcf, 0x46, 0x31, 0x1a, 0x5b, 0x6f, 0x9d, 0xc7, 0x52, 0xb4, - 0xc3, 0x97, 0x38, 0xe6, 0x5f, 0xf1, 0x4e, 0x47, 0xbe, 0xa1, 0xfe, 0xa9, 0x01, 0xd6, 0xd8, 0x40, - 0x19, 0x35, 0xb1, 0x30, 0x76, 0xc1, 0x52, 0x28, 0x9f, 0xf9, 0x70, 0x97, 0x5b, 0xf5, 0x86, 0x4e, - 0x6b, 0x0a, 0x95, 0x3c, 0xe1, 0x1a, 0x4e, 0x8c, 0x84, 0xdb, 0x1a, 0x8d, 0xd9, 0x34, 0x1a, 0x45, - 0x8e, 0x56, 0x89, 0xfb, 0x73, 0x16, 0xc0, 0xbb, 0xec, 0x84, 0xc4, 0xf4, 0x37, 0x95, 0xea, 0x93, - 0xc4, 0x88, 0xae, 0x4c, 0x49, 0x49, 0xda, 0xdb, 0x37, 0x46, 0x63, 0x6b, 0xe7, 0x05, 0xda, 0xf9, - 0x37, 0x78, 0x65, 0x16, 0xaa, 0x7c, 0xb3, 0x17, 0x41, 0xbe, 0xf5, 0x3f, 0x64, 0x41, 0xe5, 0x63, - 0xbf, 0x3f, 0x1c, 0xe0, 0x98, 0xbe, 0x20, 0x41, 0x9f, 0x39, 0xa5, 0x4f, 0xb7, 0xb5, 0x77, 0x46, - 0x63, 0xeb, 0xda, 0xbc, 0xd4, 0xe9, 0xd8, 0x0b, 0x4d, 0xdb, 0xaf, 0x73, 0x60, 0xfd, 0xc0, 0x0f, - 0x7e, 0xbc, 0xcf, 0x4f, 0xd1, 0xca, 0x32, 0xd9, 0x4b, 0x90, 0xb7, 0x3e, 0x25, 0x8f, 0x21, 0x1e, - 0x20, 0x1a, 0xba, 0x4f, 0xec, 0x6b, 0xa3, 0xb1, 0xd5, 0x9a, 0x97, 0xb8, 0x29, 0xee, 0x22, 0x93, - 0xa6, 0xed, 0x81, 0x72, 0xf3, 0xed, 0x81, 0x66, 0xd6, 0x85, 0xfc, 0x7c, 0xeb, 0xc2, 0xef, 0x73, - 0x60, 0xe3, 0xa3, 0x21, 0xf2, 0xa8, 0xdb, 0xc7, 0xc2, 0x43, 0xb1, 0x7f, 0x7e, 0x91, 0xf0, 0x4f, - 0x75, 0xea, 0x1f, 0x1d, 0x23, 0x3d, 0xf5, 0xc1, 0x68, 0x6c, 0x5d, 0x9f, 0xd7, 0x53, 0x69, 0x3d, - 0xbc, 0xf2, 0xd9, 0xbc, 0x3e, 0xbb, 0xe5, 0x0f, 0x3d, 0xfa, 0xc0, 0xf5, 0x16, 0xf1, 0x99, 0x8e, - 0xf9, 0x18, 0xb7, 0xa9, 0x1f, 0x2e, 0xe6, 0xb3, 0xb4, 0x1e, 0x5e, 0xf9, 0x6c, 0x1e, 0x9f, 0xfd, - 0x29, 0x0b, 0x2a, 0xfb, 0x62, 0x4f, 0x1f, 0xb1, 0x75, 0x92, 0xe2, 0x2b, 0xf5, 0x12, 0x33, 0x38, - 0x6c, 0xe8, 0x88, 0xc5, 0x52, 0x88, 0x8e, 0xbd, 0xd0, 0x29, 0xe4, 0x2f, 0x59, 0xb0, 0xb1, 0x8b, - 0x29, 0x6e, 0x53, 0xdc, 0xb9, 0xe3, 0xe2, 0xbe, 0x42, 0xe2, 0x27, 0x46, 0x82, 0xc5, 0x9a, 0x72, - 0x08, 0x4f, 0x05, 0xd9, 0xf6, 0x68, 0x6c, 0xdd, 0x98, 0x97, 0xc7, 0xf4, 0x3e, 0x2e, 0x34, 0x9f, - 0x5f, 0x64, 0xc1, 0xd7, 0xc5, 0xc5, 0x92, 0xb8, 0xf5, 0x9e, 0xd2, 0xf9, 0xcb, 0x04, 0x9b, 0x96, - 0xba, 0xe6, 0xa7, 0x40, 0xec, 0x9b, 0xa3, 0xb1, 0xf5, 0xc3, 0xf9, 0x17, 0xfd, 0x94, 0x2e, 0xfe, - 0x6f, 0xb4, 0xc9, 0xcf, 0x82, 0x8b, 0x6a, 0x53, 0x07, 0xbd, 0x9c, 0x36, 0xf5, 0x3e, 0x2e, 0x34, - 0x9f, 0x7f, 0x2c, 0x81, 0x55, 0xae, 0x92, 0x98, 0xc6, 0xef, 0x00, 0x79, 0x78, 0x96, 0x1c, 0xc2, - 0xe8, 0xc2, 0x25, 0x0c, 0xda, 0x8d, 0x7d, 0x79, 0xac, 0x16, 0x16, 0xf0, 0x3d, 0x50, 0x24, 0xfc, - 0x5a, 0x43, 0x9e, 0x8b, 0xaa, 0xb3, 0x37, 0x87, 0xfa, 0x05, 0xca, 0x5e, 0xc6, 0x91, 0xf6, 0xf0, - 0x3a, 0x28, 0xf6, 0x39, 0x8b, 0xf2, 0x5a, 0xa7, 0x3e, 0x8b, 0x4c, 0x1e, 0xf4, 0x19, 0x5a, 0x60, - 0xe0, 0x35, 0x50, 0xe0, 0x09, 0x40, 0xa6, 0x03, 0xed, 0xb5, 0xc9, 0x63, 0xd0, 0x5e, 0xc6, 0x11, - 0xe6, 0xb0, 0x05, 0xf2, 0x41, 0xe8, 0x0f, 0xe4, 0x61, 0xf8, 0xca, 0xec, 0x3b, 0xd5, 0xd3, 0xe3, - 0x5e, 0xc6, 0xe1, 0xb6, 0xf0, 0x1d, 0x50, 0x22, 0xfc, 0xd8, 0x49, 0xf8, 0x35, 0x22, 0x3b, 0x73, - 0xcc, 0xc0, 0x14, 0x48, 0x64, 0x0a, 0xdf, 0x01, 0xc5, 0x13, 0x7e, 0xa8, 0x90, 0x77, 0xd3, 0x9b, - 0x2a, 0x48, 0x3f, 0x6e, 0xb0, 0x79, 0x09, 0x5b, 0x78, 0x07, 0xac, 0x50, 0x3f, 0x38, 0x8e, 0xf6, - 0xee, 0xf2, 0x0a, 0xb2, 0xa6, 0x62, 0xd3, 0xf6, 0xf6, 0x7b, 0x19, 0x47, 0xc3, 0xc1, 0x87, 0x60, - 0xed, 0x91, 0xb6, 0xdf, 0xc3, 0xd1, 0x65, 0xb3, 0xc6, 0x73, 0xfa, 0x4e, 0x74, 0x2f, 0xe3, 0x24, - 0xd0, 0x70, 0x17, 0x54, 0x88, 0x96, 0xe1, 0xe4, 0x87, 0x15, 0x6d, 0x5e, 0x7a, 0x0e, 0xdc, 0xcb, - 0x38, 0x33, 0x18, 0x78, 0x1f, 0x54, 0x3a, 0xda, 0xfa, 0x2e, 0x3f, 0x9b, 0x68, 0xa3, 0x4a, 0xcf, - 0x00, 0xac, 0x37, 0x1d, 0x0b, 0x3f, 0x04, 0x6b, 0xc1, 0xcc, 0xda, 0x26, 0xbf, 0x9b, 0x7c, 0x53, - 0x9f, 0x65, 0xca, 0x22, 0xc8, 0x26, 0x39, 0x0b, 0x56, 0x87, 0x27, 0x42, 0xdc, 0x5c, 0x3d, 0x7f, - 0x78, 0xfa, 0x22, 0xa0, 0x0e, 0x4f, 0xb4, 0x30, 0x27, 0xb4, 0xb5, 0x0d, 0x1c, 0x26, 0x66, 0x25, - 0xd9, 0x5f, 0xfa, 0xd6, 0x92, 0x8d, 0x6f, 0x16, 0x6d, 0x83, 0xe9, 0x02, 0x57, 0xff, 0xb4, 0x08, - 0x56, 0x64, 0xe0, 0x8a, 0xdb, 0xd7, 0xef, 0xc7, 0xb1, 0x28, 0xe2, 0xf6, 0x8d, 0xf3, 0x62, 0x91, - 0x9b, 0x2b, 0xa1, 0xf8, 0x76, 0x1c, 0x8a, 0x22, 0x88, 0x37, 0xa6, 0x8b, 0x26, 0x9f, 0x89, 0x82, - 0x90, 0xe1, 0xb7, 0x1d, 0x85, 0x9f, 0x88, 0xdd, 0xd7, 0xd3, 0xef, 0x30, 0x22, 0x94, 0x8c, 0xbd, - 0x1d, 0x50, 0x72, 0xc5, 0x27, 0xa9, 0xb4, 0xa8, 0x4d, 0x7e, 0xb1, 0x62, 0xd1, 0x24, 0x01, 0x70, - 0x7b, 0x1a, 0x83, 0x05, 0xf9, 0x09, 0x26, 0x11, 0x83, 0x31, 0x28, 0x0a, 0xc1, 0xab, 0x71, 0x08, - 0x16, 0x67, 0x3f, 0xdb, 0x44, 0x01, 0x18, 0x4f, 0x4c, 0xc6, 0xdf, 0x6d, 0xb0, 0x1a, 0x29, 0x96, - 0x37, 0xc9, 0x00, 0x7c, 0xe3, 0xbc, 0x8d, 0x62, 0x84, 0xd7, 0x51, 0xf0, 0x6e, 0x42, 0xe6, 0xe5, - 0xd9, 0xe4, 0x3e, 0x2b, 0xf2, 0xa8, 0xa7, 0x59, 0x8d, 0xdf, 0x03, 0x97, 0xa6, 0x32, 0x15, 0x63, - 0x02, 0xc9, 0xc3, 0xa1, 0x26, 0xf0, 0xa8, 0xab, 0x59, 0xa0, 0x3a, 0x2c, 0x29, 0xef, 0xe5, 0xf3, - 0x86, 0x15, 0x89, 0x3b, 0x31, 0x2c, 0xa9, 0xed, 0x3d, 0xb0, 0x34, 0xc0, 0x14, 0x75, 0x10, 0x45, - 0x66, 0x89, 0x27, 0xba, 0x37, 0x13, 0x21, 0x27, 0xd1, 0x8d, 0x07, 0xd2, 0xf0, 0xb6, 0x47, 0xc3, - 0x53, 0xb9, 0x57, 0x8f, 0xd1, 0x9b, 0x3f, 0x00, 0xab, 0x9a, 0x01, 0x5c, 0x03, 0xb9, 0x63, 0x1c, - 0x7d, 0xa6, 0x64, 0x8f, 0x70, 0x1d, 0x14, 0x4e, 0x50, 0x7f, 0x88, 0xb9, 0x3e, 0xcb, 0x8e, 0x28, - 0xec, 0x64, 0xdf, 0x33, 0xec, 0x32, 0x28, 0x85, 0xe2, 0x2d, 0x76, 0xf7, 0xe9, 0xb3, 0x6a, 0xe6, - 0xcb, 0x67, 0xd5, 0xcc, 0x57, 0xcf, 0xaa, 0xc6, 0x27, 0x93, 0xaa, 0xf1, 0xdb, 0x49, 0xd5, 0xf8, - 0x7c, 0x52, 0x35, 0x9e, 0x4e, 0xaa, 0xc6, 0xdf, 0x27, 0x55, 0xe3, 0x1f, 0x93, 0x6a, 0xe6, 0xab, - 0x49, 0xd5, 0xf8, 0xec, 0x79, 0x35, 0xf3, 0xf4, 0x79, 0x35, 0xf3, 0xe5, 0xf3, 0x6a, 0xe6, 0xa7, - 0x57, 0x17, 0xce, 0xb9, 0x87, 0x45, 0xce, 0xd4, 0xf6, 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xaa, - 0x37, 0x05, 0xb8, 0xaf, 0x21, 0x00, 0x00, + // 2019 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x1b, 0xc7, + 0x15, 0xe7, 0xf2, 0x53, 0x1c, 0x7d, 0x58, 0x1d, 0xab, 0xca, 0x56, 0x71, 0xb8, 0x2c, 0x81, 0x26, + 0x6a, 0xd1, 0x92, 0xb1, 0x94, 0xb8, 0x89, 0xea, 0x1a, 0xf1, 0x5a, 0x76, 0x65, 0xd7, 0x6e, 0x9c, + 0x95, 0x90, 0x43, 0x2f, 0xc5, 0x88, 0x1c, 0x91, 0x5b, 0x91, 0xbb, 0xeb, 0x9d, 0xa1, 0x6c, 0x01, + 0x45, 0x91, 0x7f, 0x20, 0x68, 0xee, 0xbd, 0x17, 0xbd, 0x15, 0x05, 0x8a, 0x1e, 0x7a, 0xea, 0xa5, + 0x40, 0x72, 0x28, 0xe0, 0x63, 0x40, 0xa0, 0xdb, 0x9a, 0xbe, 0x14, 0x3a, 0xe5, 0x4f, 0x28, 0xe6, + 0x63, 0x97, 0x33, 0xdc, 0x55, 0x4d, 0xba, 0x45, 0x01, 0x15, 0xbe, 0x90, 0xf3, 0xf1, 0x7e, 0x6f, + 0xdf, 0xbc, 0xdf, 0x7b, 0xf3, 0x76, 0x66, 0xc1, 0x5b, 0xc1, 0x71, 0xb7, 0xf5, 0x68, 0x88, 0x43, + 0x17, 0x87, 0xfc, 0xff, 0x34, 0x44, 0x5e, 0x17, 0x2b, 0xcd, 0x66, 0x10, 0xfa, 0xd4, 0x87, 0x60, + 0x32, 0xb2, 0xb1, 0xd5, 0x75, 0x69, 0x6f, 0x78, 0xd8, 0x6c, 0xfb, 0x83, 0x56, 0xd7, 0xef, 0xfa, + 0xad, 0xae, 0xef, 0x77, 0xfb, 0x18, 0x05, 0x2e, 0x91, 0xcd, 0x56, 0x18, 0xb4, 0x5b, 0x84, 0x22, + 0x3a, 0x24, 0x02, 0xbf, 0xb1, 0xc6, 0x04, 0x79, 0x93, 0x43, 0xe4, 0xa8, 0x25, 0xc5, 0x79, 0xef, + 0x70, 0x78, 0xd4, 0xa2, 0xee, 0x00, 0x13, 0x8a, 0x06, 0x41, 0x2c, 0xc0, 0xec, 0xeb, 0xfb, 0x5d, + 0x81, 0x74, 0xbd, 0x0e, 0x7e, 0xd2, 0x45, 0x14, 0x3f, 0x46, 0xa7, 0x52, 0xe0, 0x75, 0x4d, 0x20, + 0x6e, 0xc8, 0xc9, 0x0d, 0x6d, 0x32, 0x40, 0x94, 0xe2, 0xd0, 0x93, 0x73, 0xdf, 0xd0, 0xe6, 0xc8, + 0x31, 0xa6, 0xed, 0x9e, 0x9c, 0xaa, 0xcb, 0xa9, 0x47, 0xfd, 0x81, 0xdf, 0xc1, 0x7d, 0xbe, 0x10, + 0x22, 0x7e, 0xa5, 0xc4, 0x65, 0x26, 0x11, 0x0c, 0x49, 0x8f, 0xff, 0xc8, 0xc1, 0x5b, 0x2f, 0xf4, + 0xe5, 0x21, 0x22, 0xb8, 0xd5, 0xc1, 0x47, 0xae, 0xe7, 0x52, 0xd7, 0xf7, 0x88, 0xda, 0x96, 0x4a, + 0xae, 0xcd, 0xa6, 0x64, 0x9a, 0x9f, 0x8d, 0xb7, 0x19, 0x8e, 0x50, 0x3f, 0x44, 0x5d, 0xdc, 0x6a, + 0xf7, 0x86, 0xde, 0x71, 0xab, 0x8d, 0xda, 0x3d, 0xdc, 0x0a, 0x31, 0x19, 0xf6, 0x29, 0x11, 0x1d, + 0x7a, 0x1a, 0x60, 0xf9, 0xa4, 0xc6, 0x17, 0x45, 0xb0, 0x78, 0xdf, 0x3f, 0x76, 0x1d, 0xfc, 0x68, + 0x88, 0x09, 0x85, 0x6b, 0xa0, 0xc4, 0xb5, 0x9a, 0x46, 0xdd, 0xd8, 0xac, 0x3a, 0xa2, 0xc3, 0x46, + 0xfb, 0xee, 0xc0, 0xa5, 0x66, 0xbe, 0x6e, 0x6c, 0x2e, 0x3b, 0xa2, 0x03, 0x21, 0x28, 0x12, 0x8a, + 0x03, 0xb3, 0x50, 0x37, 0x36, 0x0b, 0x0e, 0x6f, 0xc3, 0x0d, 0xb0, 0xe0, 0x7a, 0x14, 0x87, 0x27, + 0xa8, 0x6f, 0x56, 0xf9, 0x78, 0xd2, 0x87, 0x37, 0x40, 0x85, 0x50, 0x14, 0xd2, 0x03, 0x62, 0x16, + 0xeb, 0xc6, 0xe6, 0xe2, 0xd6, 0x46, 0x53, 0x30, 0xdf, 0x8c, 0x99, 0x6f, 0x1e, 0xc4, 0xcc, 0xdb, + 0x0b, 0x9f, 0x47, 0x56, 0xee, 0xb3, 0xbf, 0x5b, 0x86, 0x13, 0x83, 0xe0, 0x0e, 0x28, 0x61, 0xaf, + 0x73, 0x40, 0xcc, 0xd2, 0x1c, 0x68, 0x01, 0x81, 0x57, 0x41, 0xb5, 0xe3, 0x86, 0xb8, 0xcd, 0xbc, + 0x6c, 0x96, 0xeb, 0xc6, 0xe6, 0xca, 0xd6, 0xe5, 0x66, 0x12, 0x28, 0xbb, 0xf1, 0x94, 0x33, 0x91, + 0x62, 0xcb, 0x0b, 0x10, 0xed, 0x99, 0x15, 0xee, 0x09, 0xde, 0x86, 0x0d, 0x50, 0x26, 0x3d, 0x14, + 0x76, 0x88, 0xb9, 0x50, 0x2f, 0x6c, 0x56, 0x6d, 0x70, 0x16, 0x59, 0x72, 0xc4, 0x91, 0xff, 0xf0, + 0x67, 0xa0, 0x18, 0xf4, 0x91, 0x67, 0x02, 0x6e, 0xe5, 0x6a, 0x53, 0x61, 0xe9, 0x61, 0x1f, 0x79, + 0xf6, 0xfb, 0xa3, 0xc8, 0x7a, 0x57, 0x4d, 0x9e, 0x10, 0x1d, 0x21, 0x0f, 0xb5, 0xfa, 0xfe, 0xb1, + 0xdb, 0x3a, 0xd9, 0x6e, 0xa9, 0xdc, 0x33, 0x45, 0xcd, 0x8f, 0x98, 0x02, 0x06, 0x75, 0xb8, 0x62, + 0x78, 0x0f, 0x2c, 0x32, 0x8e, 0xf1, 0x2d, 0x46, 0x30, 0x31, 0x17, 0xf9, 0x73, 0x5e, 0x9b, 0xac, + 0x86, 0x8f, 0x3b, 0xf8, 0xe8, 0x47, 0xa1, 0x3f, 0x0c, 0xec, 0x4b, 0x67, 0x91, 0xa5, 0xca, 0x3b, + 0x6a, 0x07, 0xde, 0x03, 0x2b, 0x2c, 0x28, 0x5c, 0xaf, 0xfb, 0x61, 0xc0, 0x23, 0xd0, 0x5c, 0xe2, + 0xea, 0xae, 0x34, 0xd5, 0x90, 0x69, 0xde, 0xd2, 0x64, 0xec, 0x22, 0x73, 0xaf, 0x33, 0x85, 0x6c, + 0x8c, 0x0b, 0x00, 0xb2, 0x58, 0xba, 0xeb, 0x11, 0x8a, 0x3c, 0xfa, 0x32, 0x21, 0x75, 0x1d, 0x94, + 0x59, 0xf2, 0x1f, 0x10, 0x1e, 0x54, 0xb3, 0x72, 0x2c, 0x31, 0x3a, 0xc9, 0xc5, 0xb9, 0x48, 0x2e, + 0x65, 0x92, 0x5c, 0x7e, 0x21, 0xc9, 0x95, 0xff, 0x11, 0xc9, 0x0b, 0xff, 0x5d, 0x92, 0xab, 0x2f, + 0x4d, 0xb2, 0x09, 0x8a, 0xcc, 0x4a, 0xb8, 0x0a, 0x0a, 0x21, 0x7a, 0xcc, 0x39, 0x5d, 0x72, 0x58, + 0xb3, 0x31, 0x2e, 0x82, 0x25, 0xb1, 0x95, 0x90, 0xc0, 0xf7, 0x08, 0x66, 0x7e, 0xdc, 0xe7, 0xbb, + 0xbf, 0x60, 0x5e, 0xfa, 0x91, 0x8f, 0x38, 0x72, 0x06, 0x7e, 0x00, 0x8a, 0xbb, 0x88, 0x22, 0x1e, + 0x05, 0x8b, 0x5b, 0x6b, 0xaa, 0x1f, 0x99, 0x2e, 0x36, 0x67, 0xaf, 0x33, 0x43, 0xce, 0x22, 0x6b, + 0xa5, 0x83, 0x28, 0xfa, 0xae, 0x3f, 0x70, 0x29, 0x1e, 0x04, 0xf4, 0xd4, 0xe1, 0x48, 0xf8, 0x2e, + 0xa8, 0xde, 0x0e, 0x43, 0x3f, 0x3c, 0x38, 0x0d, 0x30, 0x8f, 0x9a, 0xaa, 0xfd, 0xda, 0x59, 0x64, + 0x5d, 0xc6, 0xf1, 0xa0, 0x82, 0x98, 0x48, 0xc2, 0x6f, 0x83, 0x12, 0xef, 0xf0, 0x38, 0xa9, 0xda, + 0x97, 0xcf, 0x22, 0xeb, 0x12, 0x87, 0x28, 0xe2, 0x42, 0x42, 0x0f, 0xab, 0xd2, 0x4c, 0x61, 0x95, + 0x44, 0x77, 0x59, 0x8d, 0x6e, 0x13, 0x54, 0x4e, 0x70, 0x48, 0x98, 0x9a, 0x0a, 0x1f, 0x8f, 0xbb, + 0xf0, 0x26, 0x00, 0xcc, 0x31, 0x2e, 0xa1, 0x6e, 0x3b, 0x26, 0x7b, 0xb9, 0x29, 0x8a, 0x8d, 0xc3, + 0x39, 0xb2, 0xa1, 0xf4, 0x82, 0x22, 0xe8, 0x28, 0x6d, 0xf8, 0x3b, 0x03, 0x54, 0xf6, 0x30, 0xea, + 0xe0, 0x90, 0xd1, 0x5b, 0xd8, 0x5c, 0xdc, 0xfa, 0x56, 0x53, 0xad, 0x2c, 0x0f, 0x43, 0x7f, 0x80, + 0x69, 0x0f, 0x0f, 0x49, 0x4c, 0x90, 0x90, 0xb6, 0xbd, 0x51, 0x64, 0xe1, 0x19, 0x43, 0x75, 0xa6, + 0x82, 0x76, 0xee, 0xa3, 0xce, 0x22, 0xcb, 0xf8, 0x9e, 0x13, 0x5b, 0x09, 0xb7, 0xc0, 0xc2, 0x63, + 0x14, 0x7a, 0xae, 0xd7, 0x25, 0x26, 0xe0, 0x99, 0xb6, 0x7e, 0x16, 0x59, 0x30, 0x1e, 0x53, 0x88, + 0x48, 0xe4, 0x1a, 0x7f, 0x33, 0xc0, 0xd7, 0x58, 0x60, 0xec, 0x33, 0x7b, 0x88, 0xb2, 0xc5, 0x0c, + 0x10, 0x6d, 0xf7, 0x4c, 0x83, 0xa9, 0x71, 0x44, 0x47, 0xad, 0x37, 0xf9, 0xff, 0xa8, 0xde, 0x14, + 0xe6, 0xaf, 0x37, 0xf1, 0xbe, 0x52, 0xcc, 0xdc, 0x57, 0x4a, 0xe7, 0xed, 0x2b, 0x8d, 0x5f, 0xc9, + 0x3d, 0x34, 0x5e, 0xdf, 0x1c, 0xa9, 0x74, 0x27, 0x49, 0xa5, 0x02, 0xb7, 0x36, 0x89, 0x50, 0xa1, + 0xeb, 0x6e, 0x07, 0x7b, 0xd4, 0x3d, 0x72, 0x71, 0xf8, 0x82, 0x84, 0x52, 0xa2, 0xb4, 0xa0, 0x47, + 0xa9, 0x1a, 0x62, 0xc5, 0x0b, 0x11, 0x62, 0x7a, 0x5e, 0x95, 0x5e, 0x22, 0xaf, 0x1a, 0x7f, 0x29, + 0x80, 0x75, 0xc6, 0xc8, 0x7d, 0x74, 0x88, 0xfb, 0x3f, 0x41, 0x83, 0x39, 0x59, 0x79, 0x53, 0x61, + 0xa5, 0x6a, 0xc3, 0x57, 0x5e, 0x9f, 0x6d, 0x37, 0x7b, 0x08, 0xe0, 0x3e, 0x0d, 0x87, 0x6d, 0x3a, + 0x0c, 0x71, 0xe7, 0x01, 0xa6, 0x88, 0x39, 0x4d, 0xd6, 0xe3, 0xfa, 0x59, 0x64, 0x5d, 0x21, 0xa9, + 0x59, 0xc5, 0xa5, 0x19, 0xd8, 0xc6, 0x6f, 0x0c, 0xb0, 0x10, 0x97, 0x14, 0xd8, 0x04, 0x40, 0x18, + 0xc2, 0xab, 0x86, 0x60, 0x6f, 0x85, 0x99, 0x13, 0x26, 0xa3, 0x8e, 0x22, 0x01, 0x7f, 0x0e, 0xca, + 0xa2, 0x27, 0xb3, 0x4b, 0x29, 0xc4, 0xfb, 0x34, 0xc4, 0x68, 0x70, 0xb3, 0x83, 0x02, 0x8a, 0x43, + 0xfb, 0x7d, 0xb6, 0xae, 0x51, 0x64, 0xbd, 0x75, 0x9e, 0xdf, 0xe3, 0x33, 0x83, 0xc4, 0xb1, 0x88, + 0x11, 0xcf, 0x74, 0xe4, 0x13, 0x1a, 0x9f, 0x1a, 0x60, 0x95, 0x19, 0xca, 0x9c, 0x9d, 0x84, 0xda, + 0x2e, 0x58, 0x08, 0x65, 0x9b, 0x9b, 0xbb, 0xb8, 0xd5, 0x68, 0xea, 0x44, 0x65, 0x90, 0xc3, 0x4b, + 0xb8, 0xe1, 0x24, 0x48, 0xb8, 0xad, 0x11, 0x93, 0xcf, 0x22, 0x46, 0x54, 0x7d, 0x35, 0x01, 0xfe, + 0x9c, 0x07, 0xf0, 0x2e, 0x3b, 0x73, 0xb1, 0x88, 0x9e, 0x04, 0xff, 0x93, 0x94, 0x45, 0x57, 0x26, + 0x4e, 0x49, 0xcb, 0xdb, 0x37, 0x46, 0x91, 0xb5, 0xf3, 0x82, 0x68, 0xfc, 0x37, 0x78, 0x65, 0x15, + 0x6a, 0x42, 0xe4, 0x2f, 0x42, 0x42, 0x34, 0xfe, 0x90, 0x07, 0x2b, 0x1f, 0xfb, 0xfd, 0xe1, 0x00, + 0x27, 0xee, 0x0b, 0x52, 0xee, 0x33, 0x27, 0xee, 0xd3, 0x65, 0xed, 0x9d, 0x51, 0x64, 0x5d, 0x9b, + 0xd5, 0x75, 0x3a, 0xf6, 0x42, 0xbb, 0xed, 0xd7, 0x05, 0xb0, 0x76, 0xe0, 0x07, 0x3f, 0xde, 0xe7, + 0xe7, 0x72, 0x65, 0xe3, 0xed, 0xa5, 0x9c, 0xb7, 0x36, 0x71, 0x1e, 0x43, 0x3c, 0x40, 0x34, 0x74, + 0x9f, 0xd8, 0xd7, 0x46, 0x91, 0xb5, 0x35, 0xab, 0xe3, 0x26, 0xb8, 0x8b, 0xec, 0x34, 0xed, 0xad, + 0xaa, 0x30, 0xdb, 0x5b, 0xd5, 0xd4, 0xbe, 0x50, 0x9c, 0x6d, 0x5f, 0xf8, 0x7d, 0x01, 0xac, 0x7f, + 0x34, 0x44, 0x1e, 0x75, 0xfb, 0x58, 0x30, 0x94, 0xf0, 0xf3, 0x8b, 0x14, 0x3f, 0xb5, 0x09, 0x3f, + 0x3a, 0x46, 0x32, 0xf5, 0xc1, 0x28, 0xb2, 0xae, 0xcf, 0xca, 0x54, 0x96, 0x86, 0x57, 0x9c, 0xcd, + 0xca, 0xd9, 0x2d, 0x7f, 0xe8, 0xd1, 0x07, 0xae, 0x37, 0x0f, 0x67, 0x3a, 0xe6, 0x63, 0xdc, 0xa6, + 0x7e, 0x38, 0x1f, 0x67, 0x59, 0x1a, 0x5e, 0x71, 0x36, 0x0b, 0x67, 0x7f, 0xca, 0x83, 0x95, 0x7d, + 0x71, 0x4a, 0x88, 0xbd, 0x75, 0x92, 0xc1, 0x95, 0x7a, 0x2d, 0x1a, 0x1c, 0x36, 0x75, 0xc4, 0x7c, + 0x25, 0x44, 0xc7, 0x5e, 0xe8, 0x12, 0xf2, 0xd7, 0x3c, 0x58, 0xdf, 0xc5, 0x14, 0xb7, 0x29, 0xee, + 0xdc, 0x71, 0x71, 0x5f, 0x71, 0xe2, 0x27, 0x46, 0xca, 0x8b, 0x75, 0xe5, 0x58, 0x9f, 0x09, 0xb2, + 0xed, 0x51, 0x64, 0xdd, 0x98, 0xd5, 0x8f, 0xd9, 0x3a, 0x2e, 0xb4, 0x3f, 0xbf, 0xc8, 0x83, 0xaf, + 0x8b, 0xab, 0x2a, 0x71, 0x8f, 0x3e, 0x71, 0xe7, 0x2f, 0x53, 0xde, 0xb4, 0xd4, 0x3d, 0x3f, 0x03, + 0x62, 0xdf, 0x1c, 0x45, 0xd6, 0x0f, 0x67, 0xdf, 0xf4, 0x33, 0x54, 0xfc, 0xdf, 0xc4, 0x26, 0x3f, + 0x5d, 0xce, 0x1b, 0x9b, 0x3a, 0xe8, 0xe5, 0x62, 0x53, 0xd7, 0x71, 0xa1, 0xfd, 0xf9, 0xc7, 0x0a, + 0x58, 0xe6, 0x51, 0x92, 0xb8, 0xf1, 0x3b, 0x40, 0x1e, 0xc7, 0xa5, 0x0f, 0x61, 0x7c, 0x85, 0x13, + 0x06, 0xed, 0xe6, 0xbe, 0x3c, 0xa8, 0x0b, 0x09, 0xf8, 0x1e, 0x28, 0x13, 0x7e, 0x51, 0x22, 0xcf, + 0x45, 0xb5, 0xe9, 0xbb, 0x48, 0xfd, 0x4a, 0x66, 0x2f, 0xe7, 0x48, 0x79, 0x78, 0x1d, 0x94, 0xfb, + 0xdc, 0x8b, 0xf2, 0xa2, 0xa8, 0x31, 0x8d, 0x4c, 0x5f, 0x1d, 0x30, 0xb4, 0xc0, 0xc0, 0x6b, 0xa0, + 0xc4, 0x0b, 0x80, 0x2c, 0x07, 0xda, 0x63, 0xd3, 0xc7, 0xa0, 0xbd, 0x9c, 0x23, 0xc4, 0xe1, 0x16, + 0x28, 0x06, 0xa1, 0x3f, 0x90, 0xc7, 0xeb, 0x2b, 0xd3, 0xcf, 0x54, 0x4f, 0x8f, 0x7b, 0x39, 0x87, + 0xcb, 0xc2, 0x77, 0x40, 0x85, 0xf0, 0x63, 0x27, 0xe1, 0x17, 0x93, 0xec, 0xcc, 0x31, 0x05, 0x53, + 0x20, 0xb1, 0x28, 0x7c, 0x07, 0x94, 0x4f, 0xf8, 0xa1, 0x42, 0xde, 0x76, 0x6f, 0xa8, 0x20, 0xfd, + 0xb8, 0xc1, 0xd6, 0x25, 0x64, 0xe1, 0x1d, 0xb0, 0x44, 0xfd, 0xe0, 0x38, 0x7e, 0x77, 0x97, 0x97, + 0x9a, 0x75, 0x15, 0x9b, 0xf5, 0x6e, 0xbf, 0x97, 0x73, 0x34, 0x1c, 0x7c, 0x08, 0x56, 0x1f, 0x69, + 0xef, 0x7b, 0x38, 0xbe, 0xbe, 0xd6, 0xfc, 0x9c, 0xfd, 0x26, 0xba, 0x97, 0x73, 0x52, 0x68, 0xb8, + 0x0b, 0x56, 0x88, 0x56, 0xe1, 0xe4, 0xa7, 0x1a, 0x6d, 0x5d, 0x7a, 0x0d, 0xdc, 0xcb, 0x39, 0x53, + 0x18, 0x78, 0x1f, 0xac, 0x74, 0xb4, 0xfd, 0x5d, 0x7e, 0x88, 0xd1, 0xac, 0xca, 0xae, 0x00, 0x4c, + 0x9b, 0x8e, 0x85, 0x1f, 0x82, 0xd5, 0x60, 0x6a, 0x6f, 0x93, 0x5f, 0x62, 0xbe, 0xa9, 0xaf, 0x32, + 0x63, 0x13, 0x64, 0x8b, 0x9c, 0x06, 0xab, 0xe6, 0x89, 0x14, 0x37, 0x97, 0xcf, 0x37, 0x4f, 0xdf, + 0x04, 0x54, 0xf3, 0xc4, 0x0c, 0x23, 0xa1, 0xad, 0xbd, 0xc0, 0x61, 0x62, 0xae, 0xa4, 0xf5, 0x65, + 0xbf, 0x5a, 0x32, 0xfb, 0xa6, 0xd1, 0x36, 0x98, 0x6c, 0x70, 0x8d, 0x4f, 0xcb, 0x60, 0x49, 0x26, + 0xae, 0xb8, 0xcf, 0xfd, 0x7e, 0x92, 0x8b, 0x22, 0x6f, 0xdf, 0x38, 0x2f, 0x17, 0xb9, 0xb8, 0x92, + 0x8a, 0x6f, 0x27, 0xa9, 0x28, 0x92, 0x78, 0x7d, 0xb2, 0x69, 0xf2, 0x95, 0x28, 0x08, 0x99, 0x7e, + 0xdb, 0x71, 0xfa, 0x89, 0xdc, 0x7d, 0x3d, 0xfb, 0x0e, 0x23, 0x46, 0xc9, 0xdc, 0xdb, 0x01, 0x15, + 0x57, 0x7c, 0xe4, 0xca, 0xca, 0xda, 0xf4, 0x37, 0x30, 0x96, 0x4d, 0x12, 0x00, 0xb7, 0x27, 0x39, + 0x58, 0x92, 0x1f, 0x75, 0x52, 0x39, 0x98, 0x80, 0xe2, 0x14, 0xbc, 0x9a, 0xa4, 0x60, 0x79, 0xfa, + 0x43, 0x50, 0x9c, 0x80, 0xc9, 0xc2, 0x64, 0xfe, 0xdd, 0x06, 0xcb, 0x71, 0xc4, 0xf2, 0x29, 0x99, + 0x80, 0x6f, 0x9c, 0xf7, 0xa2, 0x18, 0xe3, 0x75, 0x14, 0xbc, 0x9b, 0x0a, 0xf3, 0xea, 0x74, 0x71, + 0x9f, 0x0e, 0xf2, 0x58, 0xd3, 0x74, 0x8c, 0xdf, 0x03, 0x97, 0x26, 0x61, 0x2a, 0x6c, 0x02, 0xe9, + 0xc3, 0xa1, 0x16, 0xe0, 0xb1, 0xaa, 0x69, 0xa0, 0x6a, 0x96, 0x0c, 0xef, 0xc5, 0xf3, 0xcc, 0x8a, + 0x83, 0x3b, 0x65, 0x96, 0x8c, 0xed, 0x3d, 0xb0, 0x30, 0x88, 0x2f, 0x18, 0x2b, 0xbc, 0xd0, 0xbd, + 0x99, 0x4a, 0x39, 0x89, 0x6e, 0xc6, 0xb7, 0x89, 0xb7, 0x3d, 0x1a, 0x9e, 0xca, 0x77, 0xf5, 0x04, + 0xbd, 0xf1, 0x03, 0xb0, 0xac, 0x09, 0xc0, 0x55, 0x50, 0x38, 0xc6, 0xf1, 0x87, 0x4f, 0xd6, 0x84, + 0x6b, 0xa0, 0x74, 0x82, 0xfa, 0x43, 0xcc, 0xe3, 0xb3, 0xea, 0x88, 0xce, 0x4e, 0xfe, 0x3d, 0xc3, + 0xae, 0x82, 0x4a, 0x28, 0x9e, 0x62, 0x77, 0x9f, 0x3e, 0xab, 0xe5, 0xbe, 0x7c, 0x56, 0xcb, 0x7d, + 0xf5, 0xac, 0x66, 0x7c, 0x32, 0xae, 0x19, 0xbf, 0x1d, 0xd7, 0x8c, 0xcf, 0xc7, 0x35, 0xe3, 0xe9, + 0xb8, 0x66, 0xfc, 0x63, 0x5c, 0x33, 0xfe, 0x39, 0xae, 0xe5, 0xbe, 0x1a, 0xd7, 0x8c, 0xcf, 0x9e, + 0xd7, 0x72, 0x4f, 0x9f, 0xd7, 0x72, 0x5f, 0x3e, 0xaf, 0xe5, 0x7e, 0x7a, 0x75, 0xee, 0x9a, 0x7b, + 0x58, 0xe6, 0x9e, 0xda, 0xfe, 0x57, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcb, 0x77, 0x42, 0x10, 0x01, + 0x22, 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -2043,6 +2053,14 @@ func (this *LokiLabelNamesResponse) Equal(that interface{}) bool { if !this.Statistics.Equal(&that1.Statistics) { return false } + if len(this.StructuredMetadata) != len(that1.StructuredMetadata) { + return false + } + for i := range this.StructuredMetadata { + if this.StructuredMetadata[i] != that1.StructuredMetadata[i] { + return false + } + } return true } func (this *LokiData) Equal(that interface{}) bool { @@ -3194,13 +3212,14 @@ func (this *LokiLabelNamesResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 10) s = append(s, "&queryrange.LokiLabelNamesResponse{") s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") s = append(s, "Data: "+fmt.Sprintf("%#v", this.Data)+",\n") s = append(s, "Version: "+fmt.Sprintf("%#v", this.Version)+",\n") s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") s = append(s, "Statistics: "+strings.Replace(this.Statistics.GoString(), `&`, ``, 1)+",\n") + s = append(s, "StructuredMetadata: "+fmt.Sprintf("%#v", this.StructuredMetadata)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -4068,6 +4087,15 @@ func (m *LokiLabelNamesResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) _ = i var l int _ = l + if len(m.StructuredMetadata) > 0 { + for iNdEx := len(m.StructuredMetadata) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.StructuredMetadata[iNdEx]) + copy(dAtA[i:], m.StructuredMetadata[iNdEx]) + i = encodeVarintQueryrange(dAtA, i, uint64(len(m.StructuredMetadata[iNdEx]))) + i-- + dAtA[i] = 0x32 + } + } { size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -5504,6 +5532,12 @@ func (m *LokiLabelNamesResponse) Size() (n int) { } l = m.Statistics.Size() n += 1 + l + sovQueryrange(uint64(l)) + if len(m.StructuredMetadata) > 0 { + for _, s := range m.StructuredMetadata { + l = len(s) + n += 1 + l + sovQueryrange(uint64(l)) + } + } return n } @@ -6166,6 +6200,7 @@ func (this *LokiLabelNamesResponse) String() string { `Version:` + fmt.Sprintf("%v", this.Version) + `,`, `Headers:` + fmt.Sprintf("%v", this.Headers) + `,`, `Statistics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statistics), "Result", "stats.Result", 1), `&`, ``, 1) + `,`, + `StructuredMetadata:` + fmt.Sprintf("%v", this.StructuredMetadata) + `,`, `}`, }, "") return s @@ -8312,6 +8347,38 @@ func (m *LokiLabelNamesResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StructuredMetadata", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StructuredMetadata = append(m.StructuredMetadata, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) diff --git a/pkg/querier/queryrange/queryrange.proto b/pkg/querier/queryrange/queryrange.proto index fe234939465d3..bfd4778ce3f79 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -129,6 +129,7 @@ message LokiLabelNamesResponse { (gogoproto.nullable) = false, (gogoproto.jsontag) = "statistics" ]; + repeated string StructuredMetadata = 6 [(gogoproto.jsontag) = "structuredMetadata,omitempty"]; } message LokiData { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 9df9f8020835d..3b6c06d171a94 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -778,10 +778,10 @@ func (t *tenantHeads) Series(ctx context.Context, userID string, from, through m } -func (t *tenantHeads) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { +func (t *tenantHeads) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { idx, ok := t.tenantIndex(userID, from, through) if !ok { - return nil, nil + return nil, nil, nil } return idx.LabelNames(ctx, userID, from, through, matchers...) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index 4a64893e22d3c..90c795b67dfa5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -85,15 +85,35 @@ func (h *headIndexReader) LabelValues(name string, matchers ...*labels.Matcher) // LabelNames returns all the unique label names present in the head // that are within the time range mint to maxt. -func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { +func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, []string, error) { if h.maxt < h.head.MinTime() || h.mint > h.head.MaxTime() { - return []string{}, nil + return []string{}, nil, nil } if len(matchers) == 0 { labelNames := h.head.postings.LabelNames() sort.Strings(labelNames) - return labelNames, nil + + SMFieldNames := make(map[string]struct{}) + for _, s := range h.head.series.series { + s.RLock() + for _, ms := range s.m { + ms.RLock() + for ln := range ms.stats.StructuredMetadataFieldNames { + SMFieldNames[ln] = struct{}{} + } + ms.Unlock() + } + s.Unlock() + } + + structuredMetadataFieldNames := make([]string, 0, len(SMFieldNames)) + for name := range SMFieldNames { + structuredMetadataFieldNames = append(structuredMetadataFieldNames, name) + } + sort.Strings(structuredMetadataFieldNames) + + return labelNames, structuredMetadataFieldNames, nil } return labelNamesWithMatchers(h, matchers...) @@ -200,21 +220,36 @@ func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (str // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. -func (h *headIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { - namesMap := make(map[string]struct{}) +func (h *headIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, error) { + SMNames := make(map[string]struct{}) + streamLabelNamesMap := make(map[string]struct{}) for _, id := range ids { memSeries := h.head.series.getByID(uint64(id)) if memSeries == nil { - return nil, storage.ErrNotFound + return nil, nil, storage.ErrNotFound } + + memSeries.RLock() for _, lbl := range memSeries.ls { - namesMap[lbl.Name] = struct{}{} + streamLabelNamesMap[lbl.Name] = struct{}{} + } + for ln := range memSeries.stats.StructuredMetadataFieldNames { + SMNames[ln] = struct{}{} } + memSeries.RUnlock() + } + + streamLabelNames := make([]string, 0, len(streamLabelNamesMap)) + for name := range streamLabelNamesMap { + streamLabelNames = append(streamLabelNames, name) } - names := make([]string, 0, len(namesMap)) - for name := range namesMap { - names = append(names, name) + sort.Strings(streamLabelNames) + + structuredMetadataFieldNames := make([]string, 0, len(SMNames)) + for name := range SMNames { + structuredMetadataFieldNames = append(structuredMetadataFieldNames, name) } - sort.Strings(names) - return names, nil + sort.Strings(structuredMetadataFieldNames) + + return streamLabelNames, structuredMetadataFieldNames, nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index f7c006c93c721..e3741c2020e91 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -61,7 +61,7 @@ type Index interface { GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) // Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) - LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) + LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error @@ -79,8 +79,8 @@ func (NoopIndex) GetChunkRefs(ctx context.Context, userID string, from, through func (NoopIndex) Series(_ context.Context, _ string, _, _ model.Time, _ []Series, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]Series, error) { return nil, nil } -func (NoopIndex) LabelNames(_ context.Context, _ string, _, _ model.Time, _ ...*labels.Matcher) ([]string, error) { - return nil, nil +func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { + return nil, nil, nil } func (NoopIndex) LabelValues(_ context.Context, _ string, _, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, error) { return nil, nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 78f1c38c2df0b..54445c775a2c3 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -1709,7 +1709,7 @@ func (r *Reader) LabelValues(name string, matchers ...*labels.Matcher) ([]string // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. -func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { +func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, error) { // Gather offsetsMap the name offsetsMap in the symbol table first offsetsMap := make(map[uint32]struct{}) for _, id := range ids { @@ -1723,12 +1723,12 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) buf := d.Get() if d.Err() != nil { - return nil, errors.Wrap(d.Err(), "get buffer for series") + return nil, nil, errors.Wrap(d.Err(), "get buffer for series") } offsets, err := r.dec.LabelNamesOffsetsFor(buf) if err != nil { - return nil, errors.Wrap(err, "get label name offsets") + return nil, nil, errors.Wrap(err, "get label name offsets") } for _, off := range offsets { offsetsMap[off] = struct{}{} @@ -1740,14 +1740,14 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { for off := range offsetsMap { name, err := r.lookupSymbol(off) if err != nil { - return nil, errors.Wrap(err, "lookup symbol in LabelNamesFor") + return nil, nil, errors.Wrap(err, "lookup symbol in LabelNamesFor") } names = append(names, name) } sort.Strings(names) - return names, nil + return names, nil, nil } // LabelValueFor returns label value for the given label name in the series referred to by ID. @@ -1923,9 +1923,9 @@ func (r *Reader) Size() int64 { // LabelNames returns all the unique label names present in the index. // TODO(twilkie) implement support for matchers -func (r *Reader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { +func (r *Reader) LabelNames(matchers ...*labels.Matcher) ([]string, []string, error) { if len(matchers) > 0 { - return nil, errors.Errorf("matchers parameter is not implemented: %+v", matchers) + return nil, nil, errors.Errorf("matchers parameter is not implemented: %+v", matchers) } labelNames := make([]string, 0, len(r.postings)) @@ -1937,7 +1937,7 @@ func (r *Reader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { labelNames = append(labelNames, name) } sort.Strings(labelNames) - return labelNames, nil + return labelNames, nil, nil } // NewStringListIter returns a StringIter for the given sorted list of strings. diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index cb17256afe114..7d69cef6bbe2f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -100,10 +100,10 @@ func (i *indexShipperQuerier) Series(ctx context.Context, userID string, from, t return idx.Series(ctx, userID, from, through, res, fpFilter, matchers...) } -func (i *indexShipperQuerier) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { +func (i *indexShipperQuerier) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { idx, err := i.indices(ctx, from, through, userID) if err != nil { - return nil, err + return nil, nil, err } return idx.LabelNames(ctx, userID, from, through, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index c9ce27750a967..b0c3b4d916fef 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -50,10 +50,10 @@ func (f LazyIndex) Series(ctx context.Context, userID string, from, through mode } return i.Series(ctx, userID, from, through, res, fpFilter, matchers...) } -func (f LazyIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { +func (f LazyIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { i, err := f() if err != nil { - return nil, err + return nil, nil, err } return i.LabelNames(ctx, userID, from, through, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index 76eddc525490b..f4bb97ae8316f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -243,7 +243,7 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo return merged, nil } -func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { +func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { acc := newResultAccumulator(func(xs [][]string) ([]string, error) { var ( maxLn int // maximum number of lNames, assuming no duplicates @@ -280,7 +280,7 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug from, through, func(ctx context.Context, idx Index) error { - got, err := idx.LabelNames(ctx, userID, from, through, matchers...) + got, _, err := idx.LabelNames(ctx, userID, from, through, matchers...) if err != nil { return err } @@ -288,17 +288,17 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug return nil }, ); err != nil { - return nil, err + return nil, nil, err } merged, err := acc.Merge() if err != nil { if err == ErrEmptyAccumulator { - return nil, nil + return nil, nil, nil } - return nil, err + return nil, nil, err } - return merged, nil + return merged, nil, nil } func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go index 2d2c6fda24b56..0a193a6b4ae4c 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index_test.go @@ -122,7 +122,7 @@ func TestMultiIndex(t *testing.T) { t.Run("LabelNames", func(t *testing.T) { // request data at the end of the tsdb range, but it should return all labels present - xs, err := idx.LabelNames(context.Background(), "fake", 8, 10) + xs, _, err := idx.LabelNames(context.Background(), "fake", 8, 10) require.Nil(t, err) expected := []string{"bazz", "bonk", "foo"} @@ -131,7 +131,7 @@ func TestMultiIndex(t *testing.T) { t.Run("LabelNamesWithMatchers", func(t *testing.T) { // request data at the end of the tsdb range, but it should return all labels present - xs, err := idx.LabelNames(context.Background(), "fake", 8, 10, labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) + xs, _, err := idx.LabelNames(context.Background(), "fake", 8, 10, labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) require.Nil(t, err) expected := []string{"bazz", "foo"} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index 915d7038d7231..8e2e39386c122 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -66,19 +66,19 @@ func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, thro return xs, nil } -func (m *MultiTenantIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - res, err := m.idx.LabelNames(ctx, userID, from, through, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { + res, _, err := m.idx.LabelNames(ctx, userID, from, through, withTenantLabelMatcher(userID, matchers)...) if err != nil { - return nil, err + return nil, nil, err } // Strip out the tenant label in response. i := sort.SearchStrings(res, TenantLabel) if i == len(res) || res[i] != TenantLabel { - return res, nil + return res, nil, nil } - return append(res[:i], res[i+1:]...), nil + return append(res[:i], res[i+1:]...), nil, nil } func (m *MultiTenantIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go index bee9e88529327..30afd5ab327f7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go @@ -72,7 +72,7 @@ type IndexReader interface { ChunkStats(ref storage.SeriesRef, from, through int64, lset *labels.Labels, by map[string]struct{}) (uint64, index.ChunkStats, error) // LabelNames returns all the unique label names present in the index in sorted order. - LabelNames(matchers ...*labels.Matcher) ([]string, error) + LabelNames(matchers ...*labels.Matcher) ([]string, []string, error) // LabelValueFor returns label value for the given label name in the series referred to by ID. // If the series couldn't be found or the series doesn't have the requested label a @@ -81,7 +81,7 @@ type IndexReader interface { // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. - LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) + LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, error) // Close releases the underlying resources of the reader. Close() error @@ -336,10 +336,10 @@ func labelValuesWithMatchers(r IndexReader, name string, matchers ...*labels.Mat return values, nil } -func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]string, error) { +func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]string, []string, error) { p, err := PostingsForMatchers(r, nil, matchers...) if err != nil { - return nil, err + return nil, nil, err } var postings []storage.SeriesRef @@ -347,7 +347,7 @@ func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]strin postings = append(postings, p.At()) } if p.Err() != nil { - return nil, errors.Wrapf(p.Err(), "postings for label names with matchers") + return nil, nil, errors.Wrapf(p.Err(), "postings for label names with matchers") } return r.LabelNamesFor(postings...) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index a8975e9426767..54f6d88091ee4 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -292,7 +292,7 @@ func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Ti return res, nil } -func (i *TSDBIndex) LabelNames(_ context.Context, _ string, _, _ model.Time, matchers ...*labels.Matcher) ([]string, error) { +func (i *TSDBIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { if len(matchers) == 0 { return i.reader.LabelNames() } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go index a78fed4d3815c..d269cafbf0441 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go @@ -176,7 +176,7 @@ func TestSingleIdx(t *testing.T) { 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) + ls, _, err := idx.LabelNames(context.Background(), "fake", 9, 10) require.Nil(t, err) sort.Strings(ls) require.Equal(t, []string{"bazz", "bonk", "foo"}, ls) @@ -184,7 +184,7 @@ func TestSingleIdx(t *testing.T) { 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")) + 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) From f8177399c4e209f73f4fcd30c68e929586c7b0e1 Mon Sep 17 00:00:00 2001 From: shantanualshi Date: Wed, 4 Dec 2024 16:06:54 +0530 Subject: [PATCH 10/21] Get SM on labels endpoint # Conflicts: # pkg/querier/ingester_querier.go --- pkg/bloombuild/common/tsdb.go | 2 +- pkg/bloombuild/common/tsdb_test.go | 2 +- .../planner/strategies/chunksize.go | 2 +- .../planner/strategies/splitkeyspace_test.go | 2 +- pkg/indexgateway/gateway.go | 4 +- pkg/ingester/flush_test.go | 3 +- pkg/ingester/index/bitprefix.go | 6 +- pkg/ingester/index/index.go | 8 +-- pkg/ingester/index/multi.go | 6 +- pkg/ingester/ingester.go | 7 ++- pkg/ingester/ingester_test.go | 3 +- pkg/ingester/instance.go | 19 +++++-- pkg/loghttp/labels.go | 5 +- pkg/querier/ingester_querier.go | 11 ++-- pkg/querier/querier.go | 14 +++-- pkg/querier/querier_mock_test.go | 4 +- pkg/querier/queryrange/codec.go | 39 +++++++++---- pkg/storage/stores/composite_store.go | 10 ++-- pkg/storage/stores/composite_store_entry.go | 6 +- pkg/storage/stores/composite_store_test.go | 13 +++-- pkg/storage/stores/index/index.go | 13 +++-- .../series/series_index_gateway_store.go | 6 +- .../stores/series/series_index_store.go | 20 ++++--- .../stores/series/series_store_test.go | 6 +- .../shipper/indexshipper/tsdb/compactor.go | 12 ++-- .../indexshipper/tsdb/compactor_test.go | 4 +- .../shipper/indexshipper/tsdb/head_manager.go | 2 +- .../shipper/indexshipper/tsdb/head_read.go | 4 +- .../stores/shipper/indexshipper/tsdb/index.go | 2 +- .../shipper/indexshipper/tsdb/index/index.go | 55 +++++++++++++++---- .../shipper/indexshipper/tsdb/index_client.go | 5 +- .../tsdb/index_shipper_querier.go | 2 +- .../shipper/indexshipper/tsdb/lazy_index.go | 2 +- .../indexshipper/tsdb/multi_file_index.go | 49 +++++++++++------ .../shipper/indexshipper/tsdb/multitenant.go | 9 +-- .../indexshipper/tsdb/sharding/for_series.go | 5 +- .../indexshipper/tsdb/single_file_index.go | 12 ++-- pkg/storage/util_test.go | 4 +- pkg/util/marshal/marshal.go | 9 +-- 39 files changed, 239 insertions(+), 148 deletions(-) diff --git a/pkg/bloombuild/common/tsdb.go b/pkg/bloombuild/common/tsdb.go index 491f3882337e7..d32ef435def2a 100644 --- a/pkg/bloombuild/common/tsdb.go +++ b/pkg/bloombuild/common/tsdb.go @@ -128,7 +128,7 @@ func NewTSDBSeriesIter(ctx context.Context, user string, f sharding.ForSeries, b // TODO(salvacorts): Create a pool series := make([]*v1.Series, 0, 100) - if err := f.ForSeries(ctx, user, bounds, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + if err := f.ForSeries(ctx, user, bounds, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { select { case <-ctx.Done(): return true diff --git a/pkg/bloombuild/common/tsdb_test.go b/pkg/bloombuild/common/tsdb_test.go index 3dc9c75f645b3..7cfabd52ad7b7 100644 --- a/pkg/bloombuild/common/tsdb_test.go +++ b/pkg/bloombuild/common/tsdb_test.go @@ -16,7 +16,7 @@ import ( type forSeriesTestImpl []*v1.Series -func (f forSeriesTestImpl) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f forSeriesTestImpl) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { for i := range f { unmapped := make([]index.ChunkMeta, 0, len(f[i].Chunks)) for _, c := range f[i].Chunks { diff --git a/pkg/bloombuild/planner/strategies/chunksize.go b/pkg/bloombuild/planner/strategies/chunksize.go index 2b0e57720049d..907cee957a94b 100644 --- a/pkg/bloombuild/planner/strategies/chunksize.go +++ b/pkg/bloombuild/planner/strategies/chunksize.go @@ -243,7 +243,7 @@ func (s *ChunkSizeStrategy) sizedSeriesIter( currentBatch = newSeriesBatch(idx.tsdbIdentifier) for _, gap := range idx.gaps { - if err := idx.tsdb.ForSeries(ctx, tenant, gap, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + if err := idx.tsdb.ForSeries(ctx, tenant, gap, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { select { case <-ctx.Done(): return true diff --git a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go index 28666116e37e4..5e2ebbc78264d 100644 --- a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go +++ b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go @@ -333,7 +333,7 @@ func newFakeForSeries(series []*v1.Series) *fakeForSeries { } } -func (f fakeForSeries) ForSeries(_ context.Context, _ string, ff index.FingerprintFilter, _ model.Time, _ model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), _ []string, _ ...*labels.Matcher) error { +func (f fakeForSeries) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { overlapping := make([]*v1.Series, 0, len(f.series)) for _, s := range f.series { if ff.Match(s.Fingerprint) { diff --git a/pkg/indexgateway/gateway.go b/pkg/indexgateway/gateway.go index b6a3200ee3a51..7a7a6618ffd6f 100644 --- a/pkg/indexgateway/gateway.go +++ b/pkg/indexgateway/gateway.go @@ -329,7 +329,7 @@ func (g *Gateway) LabelNamesForMetricName(ctx context.Context, req *logproto.Lab } matchers = matcherExpr.Mts } - names, err := g.indexQuerier.LabelNamesForMetricName(ctx, instanceID, req.From, req.Through, req.MetricName, matchers...) + names, _, err := g.indexQuerier.LabelNamesForMetricName(ctx, instanceID, req.From, req.Through, req.MetricName, matchers...) if err != nil { return nil, err } @@ -607,7 +607,7 @@ func accumulateChunksToShards( var mtx sync.Mutex - if err := forSeries.ForSeries(ctx, user, v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { + if err := forSeries.ForSeries(ctx, user, v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, stats *tsdb_index.StreamStats) (stop bool) { mtx.Lock() defer mtx.Unlock() diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index 13115c6c45c48..e64f858a5b50f 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -3,7 +3,6 @@ package ingester import ( "errors" "fmt" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "os" "sort" "sync" @@ -11,6 +10,8 @@ import ( "testing" "time" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + gokitlog "github.com/go-kit/log" "github.com/grafana/dskit/flagext" "github.com/grafana/dskit/kv" diff --git a/pkg/ingester/index/bitprefix.go b/pkg/ingester/index/bitprefix.go index 38df9381e49d0..631f0febfb027 100644 --- a/pkg/ingester/index/bitprefix.go +++ b/pkg/ingester/index/bitprefix.go @@ -169,9 +169,9 @@ func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *logq } // LabelNames returns all label names. -func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, error) { +func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, []string, error) { if err := ii.validateShard(shard); err != nil { - return nil, err + return nil, nil, err } var extractor func(unlockIndex) []string @@ -206,7 +206,7 @@ func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, erro results = append(results, shardResult) } - return mergeStringSlices(results), nil + return mergeStringSlices(results), nil, nil } // LabelValues returns the values for the given label. diff --git a/pkg/ingester/index/index.go b/pkg/ingester/index/index.go index 67246b081c544..70ca09c109e64 100644 --- a/pkg/ingester/index/index.go +++ b/pkg/ingester/index/index.go @@ -30,7 +30,7 @@ var ErrInvalidShardQuery = errors.New("incompatible index shard query") type Interface interface { Add(labels []logproto.LabelAdapter, fp model.Fingerprint) labels.Labels Lookup(matchers []*labels.Matcher, shard *logql.Shard) ([]model.Fingerprint, error) - LabelNames(shard *logql.Shard) ([]string, error) + LabelNames(shard *logql.Shard) ([]string, []string, error) LabelValues(name string, shard *logql.Shard) ([]string, error) Delete(labels labels.Labels, fp model.Fingerprint) } @@ -183,10 +183,10 @@ func (ii *InvertedIndex) Lookup(matchers []*labels.Matcher, s *logql.Shard) ([]m } // LabelNames returns all label names. -func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, error) { +func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, []string, error) { shard, err := ii.validateShard(s) if err != nil { - return nil, err + return nil, nil, err } shards := ii.getShards(shard) results := make([][]string, 0, len(shards)) @@ -195,7 +195,7 @@ func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, error) { results = append(results, shardResult) } - return mergeStringSlices(results), nil + return mergeStringSlices(results), nil, nil } // LabelValues returns the values for the given label. diff --git a/pkg/ingester/index/multi.go b/pkg/ingester/index/multi.go index 293be18016b16..321a7b1b2cc76 100644 --- a/pkg/ingester/index/multi.go +++ b/pkg/ingester/index/multi.go @@ -85,7 +85,7 @@ func (m *Multi) Lookup(t time.Time, matchers []*labels.Matcher, shard *logql.Sha return m.indexFor(t).Lookup(matchers, shard) } -func (m *Multi) LabelNames(t time.Time, shard *logql.Shard) ([]string, error) { +func (m *Multi) LabelNames(t time.Time, shard *logql.Shard) ([]string, []string, error) { return m.indexFor(t).LabelNames(shard) } @@ -116,8 +116,8 @@ func (noopInvertedIndex) Lookup(_ []*labels.Matcher, _ *logql.Shard) ([]model.Fi return nil, nil } -func (noopInvertedIndex) LabelNames(_ *logql.Shard) ([]string, error) { - return nil, nil +func (noopInvertedIndex) LabelNames(shard *logql.Shard) ([]string, []string, error) { + return nil, nil, nil } func (noopInvertedIndex) LabelValues(_ string, _ *logql.Shard) ([]string, error) { diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 5659c3b056a39..fb77ce7b3fcef 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -1311,21 +1311,22 @@ func (i *Ingester) Label(ctx context.Context, req *logproto.LabelRequest) (*logp return resp, nil } from, through := model.TimeFromUnixNano(start.UnixNano()), model.TimeFromUnixNano(req.End.UnixNano()) - var storeValues []string + var storeValues, smValues []string if req.Values { storeValues, err = cs.LabelValuesForMetricName(ctx, userID, from, through, "logs", req.Name, matchers...) if err != nil { return nil, err } } else { - storeValues, err = cs.LabelNamesForMetricName(ctx, userID, from, through, "logs", matchers...) + storeValues, smValues, err = cs.LabelNamesForMetricName(ctx, userID, from, through, "logs", matchers...) if err != nil { return nil, err } } return &logproto.LabelResponse{ - Values: util.MergeStringLists(resp.Values, storeValues), + Values: util.MergeStringLists(resp.Values, storeValues), + StructuredMetadata: util.MergeStringLists(resp.StructuredMetadata, smValues), }, nil } diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 32de976c8f92e..38fa6816031cc 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -2,7 +2,6 @@ package ingester import ( "fmt" - index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "math" "net" "net/http" @@ -12,6 +11,8 @@ import ( "testing" "time" + index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/backoff" diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 62182c6b9daef..dac6b33366be3 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -581,18 +581,22 @@ func (i *instance) label(ctx context.Context, req *logproto.LabelRequest, matche Values: labels, }, nil } - names, err := i.index.LabelNames(*req.Start, nil) + names, smNames, err := i.index.LabelNames(*req.Start, nil) if err != nil { return nil, err } labels = make([]string, len(names)) copy(labels, names) + smLabels := make([]string, len(smNames)) + copy(smLabels, smNames) return &logproto.LabelResponse{ - Values: labels, + Values: labels, + StructuredMetadata: smLabels, }, nil } labels := util.NewUniqueStrings(0) + smLabels := util.NewUniqueStrings(0) err := i.forMatchingStreams(ctx, *req.Start, matchers, nil, func(s *stream) error { for _, label := range s.labels { if req.Values && label.Name == req.Name { @@ -601,6 +605,12 @@ func (i *instance) label(ctx context.Context, req *logproto.LabelRequest, matche } if !req.Values { labels.Add(label.Name) + + if s.openStreamStats != nil { + for ln := range s.openStreamStats.StructuredMetadataFieldNames { + smLabels.Add(ln) + } + } } } return nil @@ -610,7 +620,8 @@ func (i *instance) label(ctx context.Context, req *logproto.LabelRequest, matche } return &logproto.LabelResponse{ - Values: labels.Strings(), + Values: labels.Strings(), + StructuredMetadata: smLabels.Strings(), }, nil } @@ -620,7 +631,7 @@ type UniqueValues map[string]struct{} func (i *instance) LabelsWithValues(ctx context.Context, startTime time.Time, matchers ...*labels.Matcher) (map[string]UniqueValues, error) { labelMap := make(map[string]UniqueValues) if len(matchers) == 0 { - labelsFromIndex, err := i.index.LabelNames(startTime, nil) + labelsFromIndex, _, err := i.index.LabelNames(startTime, nil) if err != nil { return nil, err } diff --git a/pkg/loghttp/labels.go b/pkg/loghttp/labels.go index 360c750048a5e..0a3a6c1b8f973 100644 --- a/pkg/loghttp/labels.go +++ b/pkg/loghttp/labels.go @@ -15,8 +15,9 @@ import ( // LabelResponse represents the http json response to a label query type LabelResponse struct { - Status string `json:"status"` - Data []string `json:"data,omitempty"` + Status string `json:"status"` + Data []string `json:"data,omitempty"` + StructuredMetadata []string `json:"structured_metadata,omitempty"` } // LabelSet is a key/value pair mapping of labels diff --git a/pkg/querier/ingester_querier.go b/pkg/querier/ingester_querier.go index 76c8b8bfc42db..1ec7176e167e6 100644 --- a/pkg/querier/ingester_querier.go +++ b/pkg/querier/ingester_querier.go @@ -263,20 +263,23 @@ func (q *IngesterQuerier) SelectSample(ctx context.Context, params logql.SelectS return iterators, nil } -func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) ([][]string, error) { +func (q *IngesterQuerier) Label(ctx context.Context, req *logproto.LabelRequest) ([][]string, [][]string, error) { resps, err := q.forAllIngesters(ctx, func(ctx context.Context, client logproto.QuerierClient) (interface{}, error) { return client.Label(ctx, req) }) if err != nil { - return nil, err + return nil, nil, err } results := make([][]string, 0, len(resps)) + smResults := make([][]string, 0, len(resps)) for _, resp := range resps { - results = append(results, resp.response.(*logproto.LabelResponse).Values) + r := resp.response.(*logproto.LabelResponse) + results = append(results, r.Values) + smResults = append(smResults, r.StructuredMetadata) } - return results, nil + return results, smResults, nil } func (q *IngesterQuerier) Tail(ctx context.Context, req *logproto.TailRequest) (map[string]logproto.Querier_TailClient, error) { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 3f03d1e037aad..398f25715e01b 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -420,7 +420,7 @@ func (q *SingleTenantQuerier) Label(ctx context.Context, req *logproto.LabelRequ ingesterQueryInterval, storeQueryInterval := q.buildQueryIntervals(*req.Start, *req.End) - var ingesterValues [][]string + var ingesterValues, ingesterMetaValues [][]string if !q.cfg.QueryStoreOnly && ingesterQueryInterval != nil { g.Go(func() error { var err error @@ -428,12 +428,12 @@ func (q *SingleTenantQuerier) Label(ctx context.Context, req *logproto.LabelRequ timeFramedReq.Start = &ingesterQueryInterval.start timeFramedReq.End = &ingesterQueryInterval.end - ingesterValues, err = q.ingesterQuerier.Label(ctx, &timeFramedReq) + ingesterValues, ingesterMetaValues, err = q.ingesterQuerier.Label(ctx, &timeFramedReq) return err }) } - var storeValues []string + var storeValues, storeMetaValues []string if !q.cfg.QueryIngesterOnly && storeQueryInterval != nil { g.Go(func() error { var ( @@ -445,7 +445,7 @@ func (q *SingleTenantQuerier) Label(ctx context.Context, req *logproto.LabelRequ if req.Values { storeValues, err = q.store.LabelValuesForMetricName(ctx, userID, from, through, "logs", req.Name, matchers...) } else { - storeValues, err = q.store.LabelNamesForMetricName(ctx, userID, from, through, "logs", matchers...) + storeValues, storeMetaValues, err = q.store.LabelNamesForMetricName(ctx, userID, from, through, "logs", matchers...) } return err }) @@ -456,8 +456,10 @@ func (q *SingleTenantQuerier) Label(ctx context.Context, req *logproto.LabelRequ } results := append(ingesterValues, storeValues) + smResults := append(ingesterMetaValues, storeMetaValues) return &logproto.LabelResponse{ - Values: listutil.MergeStringLists(results...), + Values: listutil.MergeStringLists(results...), + StructuredMetadata: listutil.MergeStringLists(smResults...), }, nil } @@ -972,7 +974,7 @@ func (q *SingleTenantQuerier) DetectedLabels(ctx context.Context, req *logproto. var err error start := model.TimeFromUnixNano(storeQueryInterval.start.UnixNano()) end := model.TimeFromUnixNano(storeQueryInterval.end.UnixNano()) - storeLabels, err := q.store.LabelNamesForMetricName(ctx, userID, start, end, "logs", matchers...) + storeLabels, _, err := q.store.LabelNamesForMetricName(ctx, userID, start, end, "logs", matchers...) for _, label := range storeLabels { values, err := q.store.LabelValuesForMetricName(ctx, userID, start, end, "logs", label, matchers...) if err != nil { diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 5418889b9a4d0..e290ce516ac6d 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -378,9 +378,9 @@ func (s *storeMock) LabelValuesForMetricName(ctx context.Context, userID string, return args.Get(0).([]string), args.Error(1) } -func (s *storeMock) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, m ...*labels.Matcher) ([]string, error) { +func (s *storeMock) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { args := s.Called(ctx, userID, from, through, metricName, m) - return args.Get(0).([]string), args.Error(1) + return args.Get(0).([]string), nil, args.Error(1) } func (s *storeMock) GetChunkFetcher(_ model.Time) *fetcher.Fetcher { diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 4c586815426fa..80b5ccd7f5bb1 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -1091,10 +1091,11 @@ func decodeResponseJSONFrom(buf []byte, req queryrangebase.Request, headers http return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } return &LokiLabelNamesResponse{ - Status: resp.Status, - Version: uint32(loghttp.GetVersion(req.Path())), - Data: resp.Data, - Headers: httpResponseHeadersToPromResponseHeaders(headers), + Status: resp.Status, + Version: uint32(loghttp.GetVersion(req.Path())), + Data: resp.Data, + StructuredMetadata: resp.StructuredMetadata, + Headers: httpResponseHeadersToPromResponseHeaders(headers), }, nil case *logproto.IndexStatsRequest: var resp logproto.IndexStatsResponse @@ -1348,11 +1349,14 @@ func encodeResponseJSONTo(version loghttp.Version, res queryrangebase.Response, } case *LokiLabelNamesResponse: if loghttp.Version(response.Version) == loghttp.VersionLegacy { - if err := marshal_legacy.WriteLabelResponseJSON(logproto.LabelResponse{Values: response.Data}, w); err != nil { + if err := marshal_legacy.WriteLabelResponseJSON(logproto.LabelResponse{ + Values: response.Data, + StructuredMetadata: response.StructuredMetadata, + }, w); err != nil { return err } } else { - if err := marshal.WriteLabelResponseJSON(response.Data, w); err != nil { + if err := marshal.WriteLabelResponseJSON(response.Data, response.StructuredMetadata, w); err != nil { return err } } @@ -1503,25 +1507,38 @@ func (Codec) MergeResponse(responses ...queryrangebase.Response) (queryrangebase labelNameRes := responses[0].(*LokiLabelNamesResponse) uniqueNames := make(map[string]struct{}) names := []string{} + uniqueSmNames := make(map[string]struct{}) + smNames := []string{} // only unique name should be merged for _, res := range responses { lokiResult := res.(*LokiLabelNamesResponse) mergedStats.MergeSplit(lokiResult.Statistics) + + // stream label names for _, labelName := range lokiResult.Data { if _, ok := uniqueNames[labelName]; !ok { names = append(names, labelName) uniqueNames[labelName] = struct{}{} } } + + // Structured metadata label names + for _, labelName := range lokiResult.StructuredMetadata { + if _, ok := uniqueSmNames[labelName]; !ok { + smNames = append(smNames, labelName) + uniqueSmNames[labelName] = struct{}{} + } + } } return &LokiLabelNamesResponse{ - Status: labelNameRes.Status, - Version: labelNameRes.Version, - Headers: labelNameRes.Headers, - Data: names, - Statistics: mergedStats, + Status: labelNameRes.Status, + Version: labelNameRes.Version, + Headers: labelNameRes.Headers, + Data: names, + StructuredMetadata: smNames, + Statistics: mergedStats, }, nil case *IndexStatsResponse: headers := responses[0].(*IndexStatsResponse).Headers diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 2eef0c34083ec..187ef7ee40df4 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -156,17 +156,18 @@ func (c CompositeStore) LabelValuesForMetricName(ctx context.Context, userID str } // LabelNamesForMetricName retrieves all label names for a metric name. -func (c CompositeStore) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) { - var result util.UniqueStrings +func (c CompositeStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { + var result, smLabelResult util.UniqueStrings err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { - labelNames, err := store.LabelNamesForMetricName(innerCtx, userID, from, through, metricName, matchers...) + labelNames, smLabels, err := store.LabelNamesForMetricName(innerCtx, userID, from, through, metricName, matchers...) if err != nil { return err } result.Add(labelNames...) + smLabelResult.Add(smLabels...) return nil }) - return result.Strings(), err + return result.Strings(), smLabelResult.Strings(), err } func (c CompositeStore) GetChunks( @@ -287,6 +288,7 @@ func (c CompositeStore) HasForSeries(from, through model.Time) (sharding.ForSeri labels.Labels, model.Fingerprint, []tsdb_index.ChunkMeta, + *tsdb_index.StreamStats, ) (stop bool), matchers ...*labels.Matcher, ) error { diff --git a/pkg/storage/stores/composite_store_entry.go b/pkg/storage/stores/composite_store_entry.go index 2a376e45259de..c3d9dfc9ccd05 100644 --- a/pkg/storage/stores/composite_store_entry.go +++ b/pkg/storage/stores/composite_store_entry.go @@ -110,15 +110,15 @@ func (c *storeEntry) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { } // LabelNamesForMetricName retrieves all label names for a metric name. -func (c *storeEntry) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) { +func (c *storeEntry) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelNamesForMetricName") defer sp.Finish() shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through) if err != nil { - return nil, err + return nil, nil, err } else if shortcut { - return nil, nil + return nil, nil, nil } sp.LogKV("metric", metricName) diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index c50cc0c4cb896..cb7722cf5c58e 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -3,10 +3,11 @@ package stores import ( "context" "fmt" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "reflect" "testing" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/pkg/errors" "github.com/grafana/loki/v3/pkg/logproto" @@ -50,8 +51,8 @@ func (m mockStore) GetSeries(_ context.Context, _ string, _, _ model.Time, _ ... return nil, nil } -func (m mockStore) LabelNamesForMetricName(_ context.Context, _ string, _, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, error) { - return nil, nil +func (m mockStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { + return nil, nil, nil } func (m mockStore) GetChunkFetcher(_ model.Time) *fetcher.Fetcher { @@ -215,8 +216,8 @@ func (m mockStoreLabel) LabelValuesForMetricName(_ context.Context, _ string, _, return m.values, nil } -func (m mockStoreLabel) LabelNamesForMetricName(_ context.Context, _ string, _, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, error) { - return m.values, nil +func (m mockStoreLabel) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { + return m.values, nil, nil } func TestCompositeStoreLabels(t *testing.T) { @@ -248,7 +249,7 @@ func TestCompositeStoreLabels(t *testing.T) { }, } { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { - have, err := cs.LabelNamesForMetricName(context.Background(), "", model.TimeFromUnix(tc.from), model.TimeFromUnix(tc.through), "") + have, _, err := cs.LabelNamesForMetricName(context.Background(), "", model.TimeFromUnix(tc.from), model.TimeFromUnix(tc.through), "") require.NoError(t, err) if !reflect.DeepEqual(tc.want, have) { t.Fatalf("wrong label names - %s", test.Diff(tc.want, have)) diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 89cc6a0d9df55..737cf2919cfc5 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -25,7 +25,7 @@ type Filterable interface { type BaseReader interface { GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) - LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) + LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) } type StatsReader interface { @@ -117,17 +117,17 @@ func (m MonitoredReaderWriter) LabelValuesForMetricName(ctx context.Context, use return values, nil } -func (m MonitoredReaderWriter) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) { - var values []string +func (m MonitoredReaderWriter) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { + var values, smLabels []string if err := loki_instrument.TimeRequest(ctx, "label_names", instrument.NewHistogramCollector(m.metrics.indexQueryLatency), instrument.ErrorCode, func(ctx context.Context) error { var err error - values, err = m.rw.LabelNamesForMetricName(ctx, userID, from, through, metricName, matchers...) + values, smLabels, err = m.rw.LabelNamesForMetricName(ctx, userID, from, through, metricName, matchers...) return err }); err != nil { - return nil, err + return nil, nil, err } - return values, nil + return values, smLabels, nil } func (m MonitoredReaderWriter) Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) { @@ -208,6 +208,7 @@ func (m MonitoredReaderWriter) HasForSeries(from, through model.Time) (sharding. labels.Labels, model.Fingerprint, []index.ChunkMeta, + *index.StreamStats, ) (stop bool), matchers ...*labels.Matcher, ) error { diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index bc729bbecd879..fa69ee50baafd 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -89,7 +89,7 @@ func (c *IndexGatewayClientStore) GetSeries(ctx context.Context, _ string, from, } // LabelNamesForMetricName retrieves all label names for a metric name. -func (c *IndexGatewayClientStore) LabelNamesForMetricName(ctx context.Context, _ string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) { +func (c *IndexGatewayClientStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { resp, err := c.client.LabelNamesForMetricName(ctx, &logproto.LabelNamesForMetricNameRequest{ MetricName: metricName, From: from, @@ -97,9 +97,9 @@ func (c *IndexGatewayClientStore) LabelNamesForMetricName(ctx context.Context, _ Matchers: (&syntax.MatchersExpr{Mts: matchers}).String(), }) if err != nil { - return nil, err + return nil, nil, err } - return resp.Values, nil + return resp.Values, resp.StructuredMetadata, nil } func (c *IndexGatewayClientStore) LabelValuesForMetricName(ctx context.Context, _ string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) { diff --git a/pkg/storage/stores/series/series_index_store.go b/pkg/storage/stores/series/series_index_store.go index 3490fd520e378..18cf564aa14b1 100644 --- a/pkg/storage/stores/series/series_index_store.go +++ b/pkg/storage/stores/series/series_index_store.go @@ -320,14 +320,15 @@ func (c *IndexReaderWriter) chunksToSeries(ctx context.Context, in []logproto.Ch } // LabelNamesForMetricName retrieves all label names for a metric name. -func (c *IndexReaderWriter) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, error) { +// todo(h11): Still incomplete +func (c *IndexReaderWriter) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelNamesForMetricName") defer sp.Finish() // Fetch the series IDs from the index seriesIDs, err := c.lookupSeriesByMetricNameMatchers(ctx, from, through, userID, metricName, matchers) if err != nil { - return nil, err + return nil, nil, err } sp.LogKV("series-ids", len(seriesIDs)) @@ -339,11 +340,11 @@ func (c *IndexReaderWriter) LabelNamesForMetricName(ctx context.Context, userID return c.lookupLabelNamesByChunks(ctx, from, through, userID, seriesIDs) } sp.LogKV("msg", "lookupLabelNamesBySeries", "err", err) - return nil, err + return nil, nil, err } sp.LogKV("labelNames", len(labelNames)) - return labelNames, nil + return labelNames, nil, nil } func (c *IndexReaderWriter) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) { @@ -666,7 +667,7 @@ func (c *IndexReaderWriter) lookupLabelNamesBySeries(ctx context.Context, from, return result.Strings(), nil } -func (c *IndexReaderWriter) lookupLabelNamesByChunks(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) { +func (c *IndexReaderWriter) lookupLabelNamesByChunks(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, []string, error) { sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.lookupLabelNamesByChunks") defer sp.Finish() @@ -674,14 +675,14 @@ func (c *IndexReaderWriter) lookupLabelNamesByChunks(ctx context.Context, from, chunkIDs, err := c.lookupChunksBySeries(ctx, from, through, userID, seriesIDs) if err != nil { sp.LogKV("msg", "lookupChunksBySeries", "err", err) - return nil, err + return nil, nil, err } sp.LogKV("chunk-ids", len(chunkIDs)) chunks, err := c.convertChunkIDsToChunks(ctx, userID, chunkIDs) if err != nil { sp.LogKV("err", "convertChunkIDsToChunks", "err", err) - return nil, err + return nil, nil, err } // Filter out chunks that are not in the selected time range and keep a single chunk per fingerprint @@ -695,9 +696,10 @@ func (c *IndexReaderWriter) lookupLabelNamesByChunks(ctx context.Context, from, allChunks, err := c.fetcher.FetchChunks(ctx, filtered) if err != nil { sp.LogKV("msg", "FetchChunks", "err", err) - return nil, err + return nil, nil, err } - return labelNamesFromChunks(allChunks), nil + // todo(h11): Needs to be modified to return metadata labels + return labelNamesFromChunks(allChunks), nil, nil } func (c *IndexReaderWriter) lookupChunksBySeries(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) { diff --git a/pkg/storage/stores/series/series_store_test.go b/pkg/storage/stores/series/series_store_test.go index d64fd70b25b59..0a3ab107fcd9c 100644 --- a/pkg/storage/stores/series/series_store_test.go +++ b/pkg/storage/stores/series/series_store_test.go @@ -297,7 +297,7 @@ func TestChunkStore_LabelNamesForMetricName(t *testing.T) { } // Query with ordinary time-range - labelNames1, err := store.LabelNamesForMetricName(ctx, userID, now.Add(-time.Hour), now, tc.metricName, tc.matchers...) + labelNames1, _, err := store.LabelNamesForMetricName(ctx, userID, now.Add(-time.Hour), now, tc.metricName, tc.matchers...) require.NoError(t, err) if !reflect.DeepEqual(tc.expect, labelNames1) { @@ -305,7 +305,7 @@ func TestChunkStore_LabelNamesForMetricName(t *testing.T) { } // Pushing end of time-range into future should yield exact same resultset - labelNames2, err := store.LabelNamesForMetricName(ctx, userID, now.Add(-time.Hour), now.Add(time.Hour*24*10), tc.metricName, tc.matchers...) + labelNames2, _, err := store.LabelNamesForMetricName(ctx, userID, now.Add(-time.Hour), now.Add(time.Hour*24*10), tc.metricName, tc.matchers...) require.NoError(t, err) if !reflect.DeepEqual(tc.expect, labelNames2) { @@ -313,7 +313,7 @@ func TestChunkStore_LabelNamesForMetricName(t *testing.T) { } // Query with both begin & end of time-range in future should yield empty resultset - labelNames3, err := store.LabelNamesForMetricName(ctx, userID, now.Add(time.Hour), now.Add(time.Hour*2), tc.metricName, tc.matchers...) + labelNames3, _, err := store.LabelNamesForMetricName(ctx, userID, now.Add(time.Hour), now.Add(time.Hour*2), tc.metricName, tc.matchers...) require.NoError(t, err) if len(labelNames3) != 0 { t.Fatalf("%s: future query should yield empty resultset ... actually got %v label names: %#v", diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go index 9ee3920c3c254..b9f6d5c6cf3d7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go @@ -53,8 +53,8 @@ func (i indexProcessor) OpenCompactedIndexFile(ctx context.Context, path, tableN } builder := NewBuilder(indexFormat) - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { - builder.AddSeries(lbls.Copy(), fp, chks) + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta, stats *tsdbindex.StreamStats) (stop bool) { + builder.AddSeries(lbls.Copy(), fp, chks, stats) return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { @@ -213,8 +213,8 @@ func setupBuilder(ctx context.Context, indexType int, userID string, sourceIndex // add users index from multi-tenant indexes to the builder for _, idx := range multiTenantIndexes { - err := idx.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { - builder.AddSeries(withoutTenantLabel(lbls.Copy()), fp, chks) + err := idx.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta, stats *tsdbindex.StreamStats) (stop bool) { + builder.AddSeries(withoutTenantLabel(lbls.Copy()), fp, chks, stats) return false }, nil, withTenantLabelMatcher(userID, []*labels.Matcher{})...) if err != nil { @@ -246,8 +246,8 @@ func setupBuilder(ctx context.Context, indexType int, userID string, sourceIndex } }() - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { - builder.AddSeries(lbls.Copy(), fp, chks) + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta, stats *tsdbindex.StreamStats) (stop bool) { + builder.AddSeries(lbls.Copy(), fp, chks, stats) return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index ed7f3d1ee0dbf..aae56643c7770 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -609,7 +609,7 @@ func TestCompactor_Compact(t *testing.T) { require.NoError(t, err) actualChunks = map[string]index.ChunkMetas{} - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) (stop bool) { actualChunks[lbls.String()] = chks return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) @@ -824,7 +824,7 @@ func TestCompactedIndex(t *testing.T) { require.NoError(t, err) foundChunks := map[string]index.ChunkMetas{} - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) (stop bool) { foundChunks[lbls.String()] = append(index.ChunkMetas{}, chks...) return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 3b6c06d171a94..97929c2afc029 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -812,7 +812,7 @@ func (t *tenantHeads) Volume(ctx context.Context, userID string, from, through m return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index 90c795b67dfa5..415bb83d2c6ae 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -102,9 +102,9 @@ func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, []s for ln := range ms.stats.StructuredMetadataFieldNames { SMFieldNames[ln] = struct{}{} } - ms.Unlock() + ms.RUnlock() } - s.Unlock() + s.RUnlock() } structuredMetadataFieldNames := make([]string, 0, len(SMFieldNames)) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index e3741c2020e91..04a22068a6a10 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -96,6 +96,6 @@ func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAc return nil } -func (NoopIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (NoopIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 54445c775a2c3..df745db58dcc3 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -1712,6 +1712,7 @@ func (r *Reader) LabelValues(name string, matchers ...*labels.Matcher) ([]string func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, error) { // Gather offsetsMap the name offsetsMap in the symbol table first offsetsMap := make(map[uint32]struct{}) + smOffsetsMap := make(map[uint32]struct{}) for _, id := range ids { offset := id // In version 2+ series IDs are no longer exact references but series are 16-byte padded @@ -1721,18 +1722,22 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, er } d := encoding.DecWrap(tsdb_enc.NewDecbufUvarintAt(r.b, int(offset), castagnoliTable)) - buf := d.Get() - if d.Err() != nil { - return nil, nil, errors.Wrap(d.Err(), "get buffer for series") - } - offsets, err := r.dec.LabelNamesOffsetsFor(buf) + offsets, err := r.dec.LabelNamesOffsetsFor(&d) if err != nil { return nil, nil, errors.Wrap(err, "get label name offsets") } for _, off := range offsets { offsetsMap[off] = struct{}{} } + + smOffset, err := r.dec.SmNamesOffsetsFor(&d) + if err != nil { + return nil, nil, errors.Wrap(err, "get structured metadata offsets") + } + for _, off := range smOffset { + smOffsetsMap[off] = struct{}{} + } } // Lookup the unique symbols. @@ -1740,14 +1745,24 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, er for off := range offsetsMap { name, err := r.lookupSymbol(off) if err != nil { - return nil, nil, errors.Wrap(err, "lookup symbol in LabelNamesFor") + return nil, nil, errors.Wrap(err, "lookup stream symbol in LabelNamesFor") } names = append(names, name) } - sort.Strings(names) - return names, nil, nil + // Lookup the unique structured metadata symbols. + smNames := make([]string, 0, len(smOffsetsMap)) + for o := range smOffsetsMap { + name, err := r.lookupSymbol(o) + if err != nil { + return nil, nil, errors.Wrap(err, "lookup structured metadata symbol in LabelNamesFor") + } + smNames = append(smNames, name) + } + sort.Strings(smNames) + + return names, smNames, nil } // LabelValueFor returns label value for the given label name in the series referred to by ID. @@ -2041,15 +2056,14 @@ func (dec *Decoder) Postings(b []byte) (int, Postings, error) { // LabelNamesOffsetsFor decodes the offsets of the name symbols for a given series. // They are returned in the same order they're stored, which should be sorted lexicographically. -func (dec *Decoder) LabelNamesOffsetsFor(b []byte) ([]uint32, error) { - d := encoding.DecWrap(tsdb_enc.Decbuf{B: b}) +func (dec *Decoder) LabelNamesOffsetsFor(d *encoding.Decbuf) ([]uint32, error) { _ = d.Be64() // skip fingerprint k := d.Uvarint() offsets := make([]uint32, k) for i := 0; i < k; i++ { - offsets[i] = uint32(d.Uvarint()) - _ = d.Uvarint() // skip the label value + offsets[i] = d.Uvarint32() + _ = d.Uvarint32() // skip the label value if d.Err() != nil { return nil, errors.Wrap(d.Err(), "read series label offsets") @@ -2059,6 +2073,23 @@ func (dec *Decoder) LabelNamesOffsetsFor(b []byte) ([]uint32, error) { return offsets, d.Err() } +// LabelNamesOffsetsFor decodes the offsets of the name symbols for a given series. +// They are returned in the same order they're stored, which should be sorted lexicographically. +func (dec *Decoder) SmNamesOffsetsFor(d *encoding.Decbuf) ([]uint32, error) { + k := d.Uvarint() + + offsets := make([]uint32, k) + for i := 0; i < k; i++ { + offsets[i] = d.Uvarint32() + + if d.Err() != nil { + return nil, errors.Wrap(d.Err(), "read structured metadata label offsets") + } + } + + return offsets, d.Err() +} + // LabelValueFor decodes a label for a given series. func (dec *Decoder) LabelValueFor(b []byte, label string) (string, error) { d := encoding.DecWrap(tsdb_enc.Decbuf{B: b}) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index b400c9f6aa381..d600b186877ee 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -184,7 +184,8 @@ func (c *IndexClient) LabelValuesForMetricName(ctx context.Context, userID strin } // tsdb no longer uses the __metric_name__="logs" hack, so we can ignore metric names! -func (c *IndexClient) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, _ string, matchers ...*labels.Matcher) ([]string, error) { +// h11: Stopped implementing the interface, needs fix +func (c *IndexClient) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, _ string, matchers ...*labels.Matcher) ([]string, []string, error) { return c.idx.LabelNames(ctx, userID, from, through, matchers...) } @@ -299,7 +300,7 @@ func (c *IndexClient) GetShards(ctx context.Context, userID string, from, throug var mtx sync.Mutex m := make(map[model.Fingerprint]index.ChunkMetas, 1024) - if err := c.idx.ForSeries(ctx, userID, v1.FullBounds, from, through, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + if err := c.idx.ForSeries(ctx, userID, v1.FullBounds, from, through, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { mtx.Lock() m[fp] = append(m[fp], chks...) mtx.Unlock() diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index 7d69cef6bbe2f..c37f4867719df 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -134,7 +134,7 @@ func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, t return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta, *tsdbindex.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { idx, err := i.indices(ctx, from, through, userID) if err != nil { return err diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index b0c3b4d916fef..adfd060ff6e5b 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -81,7 +81,7 @@ func (f LazyIndex) Volume(ctx context.Context, userID string, from, through mode return i.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { i, err := f() if err != nil { return err diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index f4bb97ae8316f..c93a561ccacca 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -244,31 +244,44 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo } func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { - acc := newResultAccumulator(func(xs [][]string) ([]string, error) { + type labelsResult struct { + labels []string + smLabels []string + } + + acc := newResultAccumulator(func(xs []labelsResult) (labelsResult, error) { var ( maxLn int // maximum number of lNames, assuming no duplicates - lists [][]string + lists []labelsResult ) - for _, group := range xs { - x := group - maxLn += len(x) + for _, x := range xs { + maxLn += len(x.labels) lists = append(lists, x) } // optimistically allocate the maximum length slice // to avoid growing incrementally - // TODO(owen-d): use pool - results := make([]string, 0, maxLn) - seen := make(map[string]struct{}) + results := labelsResult{ + labels: make([]string, 0, maxLn), + smLabels: make([]string, 0, maxLn), // Same capacity for sm + } + seenNames := make(map[string]struct{}) + seenSM := make(map[string]struct{}) for _, ls := range lists { - for _, l := range ls { - _, ok := seen[l] - if ok { - continue + // Handle label names + for _, l := range ls.labels { + if _, ok := seenNames[l]; !ok { + seenNames[l] = struct{}{} + results.labels = append(results.labels, l) + } + } + // Handle structured metadata names + for _, s := range ls.smLabels { + if _, ok := seenSM[s]; !ok { + seenSM[s] = struct{}{} + results.smLabels = append(results.smLabels, s) } - seen[l] = struct{}{} - results = append(results, l) } } @@ -280,11 +293,11 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug from, through, func(ctx context.Context, idx Index) error { - got, _, err := idx.LabelNames(ctx, userID, from, through, matchers...) + got, sm, err := idx.LabelNames(ctx, userID, from, through, matchers...) if err != nil { return err } - acc.Add(got) + acc.Add(labelsResult{labels: got, smLabels: sm}) return nil }, ); err != nil { @@ -298,7 +311,7 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug } return nil, nil, err } - return merged, nil, nil + return merged.labels, merged.smLabels, nil } func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { @@ -371,7 +384,7 @@ func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through mo }) } -func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) }) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index 8e2e39386c122..f6dec53a76203 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -67,7 +67,7 @@ func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, thro } func (m *MultiTenantIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { - res, _, err := m.idx.LabelNames(ctx, userID, from, through, withTenantLabelMatcher(userID, matchers)...) + res, sm, err := m.idx.LabelNames(ctx, userID, from, through, withTenantLabelMatcher(userID, matchers)...) if err != nil { return nil, nil, err } @@ -75,10 +75,11 @@ func (m *MultiTenantIndex) LabelNames(ctx context.Context, userID string, from, // Strip out the tenant label in response. i := sort.SearchStrings(res, TenantLabel) if i == len(res) || res[i] != TenantLabel { - return res, nil, nil + return res, sm, nil } + res = append(res[:i], res[i+1:]...) // Skip tenant label - return append(res[:i], res[i+1:]...), nil, nil + return res, sm, nil } func (m *MultiTenantIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { @@ -97,6 +98,6 @@ func (m *MultiTenantIndex) Volume(ctx context.Context, userID string, from, thro return m.idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return m.idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, withTenantLabelMatcher(userID, matchers)...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go index 0500500a2331f..11230c1dabd28 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go @@ -19,7 +19,7 @@ import ( // such as when the Index is backed by multiple files with the same series present. // NB(owen-d): mainly in this package to avoid circular dependencies elsewhere type ForSeries interface { - ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error + ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error } // function Adapter for ForSeries implementation @@ -33,10 +33,11 @@ type ForSeriesFunc func( labels.Labels, model.Fingerprint, []index.ChunkMeta, + *index.StreamStats, ) (stop bool), matchers ...*labels.Matcher, ) error -func (f ForSeriesFunc) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f ForSeriesFunc) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { return f(ctx, userID, fpFilter, from, through, fn, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 54f6d88091ee4..27d777dc62d27 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -55,8 +55,8 @@ func RebuildWithVersion(ctx context.Context, path string, desiredVer int) (shipp } builder := NewBuilder(desiredVer) - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { - builder.AddSeries(lbls.Copy(), fp, chks) + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(ctx, "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) (stop bool) { + builder.AddSeries(lbls.Copy(), fp, chks, stats) return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) if err != nil { @@ -161,7 +161,7 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { // Iteration will stop if the callback returns true. // Accepts a userID argument in order to implement `Index` interface, but since this is a single tenant index, // it is ignored (it's enforced elsewhere in index selection) -func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { // TODO(owen-d): use pool stats := index.NewStreamStats() @@ -222,7 +222,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, _ string, fpFilter index.Fing continue } - if stop := fn(ls, model.Fingerprint(hash), chks); stop { + if stop := fn(ls, model.Fingerprint(hash), chks, stats); stop { break } } @@ -251,7 +251,7 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu } res = res[:0] - if err := i.ForSeries(ctx, "", fpFilter, from, through, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + if err := i.ForSeries(ctx, "", fpFilter, from, through, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { for _, chk := range chks { res = append(res, ChunkRef{ @@ -276,7 +276,7 @@ func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Ti } res = res[:0] - if err := i.ForSeries(ctx, "", fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + if err := i.ForSeries(ctx, "", fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { if len(chks) == 0 { return } diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index bca1526a59ca7..3d5fe8255a2bc 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -225,8 +225,8 @@ func (m *mockChunkStore) LabelValuesForMetricName(_ context.Context, _ string, _ return nil, nil } -func (m *mockChunkStore) LabelNamesForMetricName(_ context.Context, _ string, _, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, error) { - return nil, nil +func (m *mockChunkStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { + return nil, nil, nil } func (m *mockChunkStore) SetChunkFilterer(f chunk.RequestChunkFilterer) { diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 1a4d6701b1b18..81ad03805dc68 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -34,7 +34,7 @@ func WriteResponseJSON(r *http.Request, v any, w http.ResponseWriter) error { case *logproto.LabelResponse: version := loghttp.GetVersion(r.RequestURI) if version == loghttp.VersionV1 { - return WriteLabelResponseJSON(result.GetValues(), w) + return WriteLabelResponseJSON(result.GetValues(), result.GetStructuredMetadata(), w) } return marshal_legacy.WriteLabelResponseJSON(*result, w) @@ -65,10 +65,11 @@ func WriteQueryResponseJSON(data parser.Value, warnings []string, statistics sta // WriteLabelResponseJSON marshals a logproto.LabelResponse to v1 loghttp JSON // and then writes it to the provided io.Writer. -func WriteLabelResponseJSON(data []string, w io.Writer) error { +func WriteLabelResponseJSON(data []string, smNames []string, w io.Writer) error { v1Response := loghttp.LabelResponse{ - Status: "success", - Data: data, + Status: "success", + Data: data, + StructuredMetadata: smNames, } s := jsoniter.ConfigFastest.BorrowStream(w) From 6d457c2ef84fb67a7807e9347d59c3932cf5101b Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Thu, 12 Dec 2024 15:46:11 +0100 Subject: [PATCH 11/21] some cleaning after hackathon --- pkg/blockbuilder/builder/tsdb.go | 4 - pkg/bloombuild/common/tsdb.go | 48 +++++++----- pkg/bloombuild/common/tsdb_test.go | 13 +++- .../planner/strategies/chunksize.go | 71 ++++++++++-------- .../planner/strategies/splitkeyspace_test.go | 4 +- pkg/indexgateway/gateway.go | 75 ++++++++++--------- pkg/storage/stores/composite_store_test.go | 4 +- pkg/storage/stores/index/index.go | 1 + pkg/storage/stores/series/index/index.go | 1 - .../series/series_index_gateway_store.go | 2 +- .../stores/series/series_index_store.go | 2 +- .../stores/shipper/indexshipper/tsdb/head.go | 1 - .../shipper/indexshipper/tsdb/head_manager.go | 1 - .../indexshipper/tsdb/head_manager_test.go | 64 ++++++++++++++-- .../shipper/indexshipper/tsdb/index/index.go | 5 -- .../shipper/indexshipper/tsdb/index_client.go | 1 - .../shipper/indexshipper/tsdb/manager.go | 1 - 17 files changed, 183 insertions(+), 115 deletions(-) diff --git a/pkg/blockbuilder/builder/tsdb.go b/pkg/blockbuilder/builder/tsdb.go index 672d7e1134f2a..e90bedb3815ad 100644 --- a/pkg/blockbuilder/builder/tsdb.go +++ b/pkg/blockbuilder/builder/tsdb.go @@ -60,10 +60,6 @@ func (m *TsdbCreator) Append(userID string, ls labels.Labels, fprint uint64, chk return nil } -func (t *TsdbCreator) seriesStats(userID string, fp uint64) []string { - return nil -} - type chunkInfo struct { chunkMetas index.ChunkMetas tsdbFormat int diff --git a/pkg/bloombuild/common/tsdb.go b/pkg/bloombuild/common/tsdb.go index d32ef435def2a..aaa4f3c675420 100644 --- a/pkg/bloombuild/common/tsdb.go +++ b/pkg/bloombuild/common/tsdb.go @@ -128,27 +128,35 @@ func NewTSDBSeriesIter(ctx context.Context, user string, f sharding.ForSeries, b // TODO(salvacorts): Create a pool series := make([]*v1.Series, 0, 100) - if err := f.ForSeries(ctx, user, bounds, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { - select { - case <-ctx.Done(): - return true - default: - res := &v1.Series{ - Fingerprint: fp, - Chunks: make(v1.ChunkRefs, 0, len(chks)), + if err := f.ForSeries( + ctx, + user, + bounds, + 0, math.MaxInt64, + func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { + select { + case <-ctx.Done(): + return true + default: + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), + } + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + From: model.Time(chk.MinTime), + Through: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + series = append(series, res) + return false } - for _, chk := range chks { - res.Chunks = append(res.Chunks, v1.ChunkRef{ - From: model.Time(chk.MinTime), - Through: model.Time(chk.MaxTime), - Checksum: chk.Checksum, - }) - } - - series = append(series, res) - return false - } - }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")); err != nil { + }, + nil, + labels.MustNewMatcher(labels.MatchEqual, "", ""), + ); err != nil { return nil, err } diff --git a/pkg/bloombuild/common/tsdb_test.go b/pkg/bloombuild/common/tsdb_test.go index 7cfabd52ad7b7..f2d28279c807a 100644 --- a/pkg/bloombuild/common/tsdb_test.go +++ b/pkg/bloombuild/common/tsdb_test.go @@ -16,7 +16,16 @@ import ( type forSeriesTestImpl []*v1.Series -func (f forSeriesTestImpl) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f forSeriesTestImpl) ForSeries( + _ context.Context, + _ string, + _ index.FingerprintFilter, + _ model.Time, + _ model.Time, + fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), + _ []string, + _ ...*labels.Matcher, +) error { for i := range f { unmapped := make([]index.ChunkMeta, 0, len(f[i].Chunks)) for _, c := range f[i].Chunks { @@ -27,7 +36,7 @@ func (f forSeriesTestImpl) ForSeries(ctx context.Context, userID string, fpFilte }) } - fn(nil, f[i].Fingerprint, unmapped) + fn(nil, f[i].Fingerprint, unmapped, nil) } return nil } diff --git a/pkg/bloombuild/planner/strategies/chunksize.go b/pkg/bloombuild/planner/strategies/chunksize.go index 907cee957a94b..99c30f82eb9f7 100644 --- a/pkg/bloombuild/planner/strategies/chunksize.go +++ b/pkg/bloombuild/planner/strategies/chunksize.go @@ -243,39 +243,46 @@ func (s *ChunkSizeStrategy) sizedSeriesIter( currentBatch = newSeriesBatch(idx.tsdbIdentifier) for _, gap := range idx.gaps { - if err := idx.tsdb.ForSeries(ctx, tenant, gap, 0, math.MaxInt64, func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { - select { - case <-ctx.Done(): - return true - default: - var seriesSize uint64 - for _, chk := range chks { - seriesSize += uint64(chk.KB * 1024) + if err := idx.tsdb.ForSeries( + ctx, + tenant, + gap, + 0, math.MaxInt64, + func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { + select { + case <-ctx.Done(): + return true + default: + var seriesSize uint64 + for _, chk := range chks { + seriesSize += uint64(chk.KB * 1024) + } + + // Cut a new batch IF the current batch is not empty (so we add at least one series to the batch) + // AND Adding this series to the batch would exceed the target task size. + if currentBatch.Len() > 0 && currentBatch.Size()+seriesSize > targetTaskSizeBytes { + batches = append(batches, currentBatch) + currentBatch = newSeriesBatch(idx.tsdbIdentifier) + } + + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), + } + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + From: model.Time(chk.MinTime), + Through: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + currentBatch.Append(res, seriesSize) + return false } - - // Cut a new batch IF the current batch is not empty (so we add at least one series to the batch) - // AND Adding this series to the batch would exceed the target task size. - if currentBatch.Len() > 0 && currentBatch.Size()+seriesSize > targetTaskSizeBytes { - batches = append(batches, currentBatch) - currentBatch = newSeriesBatch(idx.tsdbIdentifier) - } - - res := &v1.Series{ - Fingerprint: fp, - Chunks: make(v1.ChunkRefs, 0, len(chks)), - } - for _, chk := range chks { - res.Chunks = append(res.Chunks, v1.ChunkRef{ - From: model.Time(chk.MinTime), - Through: model.Time(chk.MaxTime), - Checksum: chk.Checksum, - }) - } - - currentBatch.Append(res, seriesSize) - return false - } - }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")); err != nil { + }, + nil, labels.MustNewMatcher(labels.MatchEqual, "", ""), + ); err != nil { return nil, 0, err } diff --git a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go index 5e2ebbc78264d..6040a51c386fe 100644 --- a/pkg/bloombuild/planner/strategies/splitkeyspace_test.go +++ b/pkg/bloombuild/planner/strategies/splitkeyspace_test.go @@ -333,7 +333,7 @@ func newFakeForSeries(series []*v1.Series) *fakeForSeries { } } -func (f fakeForSeries) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f fakeForSeries) ForSeries(_ context.Context, _ string, ff index.FingerprintFilter, _ model.Time, _ model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, _ ...*labels.Matcher) error { overlapping := make([]*v1.Series, 0, len(f.series)) for _, s := range f.series { if ff.Match(s.Fingerprint) { @@ -352,7 +352,7 @@ func (f fakeForSeries) ForSeries(ctx context.Context, userID string, fpFilter in }) } - if fn(labels.EmptyLabels(), s.Fingerprint, chunks) { + if fn(labels.EmptyLabels(), s.Fingerprint, chunks, nil) { break } } diff --git a/pkg/indexgateway/gateway.go b/pkg/indexgateway/gateway.go index 7a7a6618ffd6f..65c04be52ef92 100644 --- a/pkg/indexgateway/gateway.go +++ b/pkg/indexgateway/gateway.go @@ -607,44 +607,51 @@ func accumulateChunksToShards( var mtx sync.Mutex - if err := forSeries.ForSeries(ctx, user, v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, stats *tsdb_index.StreamStats) (stop bool) { - mtx.Lock() - defer mtx.Unlock() - - // check if this is a fingerprint we need - if _, ok := filteredM[fp]; !ok { - return false - } + if err := forSeries.ForSeries( + ctx, + user, + v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), + req.From, req.Through, + func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, stats *tsdb_index.StreamStats) (stop bool) { + mtx.Lock() + defer mtx.Unlock() + + // check if this is a fingerprint we need + if _, ok := filteredM[fp]; !ok { + return false + } - filteredChks := filteredM[fp] - var j int - - outer: - for i := range filteredChks { - for j < len(chks) { - switch filteredChks[i].Cmp(chks[j]) { - case iter.Less: - // this chunk is not in the queried index, continue checking other chunks - continue outer - case iter.Greater: - // next chunk in index but didn't pass filter; continue - j++ - continue - case iter.Eq: - // a match; set the sizing info - filteredChks[i].KB = chks[j].KB - filteredChks[i].Entries = chks[j].Entries - j++ - continue outer + filteredChks := filteredM[fp] + var j int + + outer: + for i := range filteredChks { + for j < len(chks) { + switch filteredChks[i].Cmp(chks[j]) { + case iter.Less: + // this chunk is not in the queried index, continue checking other chunks + continue outer + case iter.Greater: + // next chunk in index but didn't pass filter; continue + j++ + continue + case iter.Eq: + // a match; set the sizing info + filteredChks[i].KB = chks[j].KB + filteredChks[i].Entries = chks[j].Entries + j++ + continue outer + } } - } - // we've finished this index's chunks; no need to keep checking filtered chunks - break - } + // we've finished this index's chunks; no need to keep checking filtered chunks + break + } - return false - }, nil, p.Matchers...); err != nil { + return false + }, + nil, p.Matchers..., + ); err != nil { return nil, nil, err } diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index cb7722cf5c58e..623fc88419688 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -51,7 +51,7 @@ func (m mockStore) GetSeries(_ context.Context, _ string, _, _ model.Time, _ ... return nil, nil } -func (m mockStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { +func (m mockStore) LabelNamesForMetricName(_ context.Context, _ string, _ model.Time, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, []string, error) { return nil, nil, nil } @@ -216,7 +216,7 @@ func (m mockStoreLabel) LabelValuesForMetricName(_ context.Context, _ string, _, return m.values, nil } -func (m mockStoreLabel) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { +func (m mockStoreLabel) LabelNamesForMetricName(_ context.Context, _ string, _ model.Time, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, []string, error) { return m.values, nil, nil } diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 737cf2919cfc5..3e37a7eb3842c 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -25,6 +25,7 @@ type Filterable interface { type BaseReader interface { GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) + // LabelNamesForMetricName returns the label names for the given metric name and the available structured metadata fields. LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) } diff --git a/pkg/storage/stores/series/index/index.go b/pkg/storage/stores/series/index/index.go index bfc4e385ba2f8..5011c7e8f0815 100644 --- a/pkg/storage/stores/series/index/index.go +++ b/pkg/storage/stores/series/index/index.go @@ -16,7 +16,6 @@ type ReadClient interface { type WriteClient interface { NewWriteBatch() WriteBatch BatchWrite(context.Context, WriteBatch) error - // h11: UpdateSeriesStats(userID string, fp uint64, stats *StreamStats)?? } // Client is a client for the storage of the index (e.g. DynamoDB or Bigtable). diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index fa69ee50baafd..881fda50bbb97 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -37,7 +37,7 @@ type IndexGatewayClientStore struct { logger log.Logger } -func (c *IndexGatewayClientStore) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { +func (c *IndexGatewayClientStore) UpdateSeriesStats(_ string, _ uint64, _ *index.StreamStats) { level.Warn(c.logger).Log("msg", "UpdateSeriesStats called on index gateway client store, but it does not support it") } diff --git a/pkg/storage/stores/series/series_index_store.go b/pkg/storage/stores/series/series_index_store.go index 18cf564aa14b1..444137703fbaa 100644 --- a/pkg/storage/stores/series/series_index_store.go +++ b/pkg/storage/stores/series/series_index_store.go @@ -76,7 +76,7 @@ type IndexReaderWriter struct { writeDedupeCache cache.Cache } -func (c *IndexReaderWriter) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { +func (c *IndexReaderWriter) UpdateSeriesStats(_ string, _ uint64, _ *index.StreamStats) { level.Warn(util_log.Logger).Log("msg", "UpdateSeriesStats called on index gateway client store, but it does not support it") } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index c8e978b42125a..80e81cf9c874a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -311,7 +311,6 @@ func (s *stripeSeries) resetSeriesStats() { } type memSeries struct { - // (h11) potentially add stats? sync.RWMutex ref uint64 // The unique reference within a *Head ls labels.Labels diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 97929c2afc029..2e1bb48a3547a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -840,7 +840,6 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c stats *index.StreamStats ) - // h11: Pass stream stats fp, err := idx.Series(ps.At(), 0, math.MaxInt64, &ls, &chks, &stats) if err != nil { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go index dc57537ee982f..90a7826919cbb 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager_test.go @@ -110,7 +110,14 @@ func Test_TenantHeads_Append(t *testing.T) { } _ = h.Append("fake", ls, ls.Hash(), chks) - found, err := h.GetChunkRefs(context.Background(), "fake", 0, 100, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + found, err := h.GetChunkRefs( + context.Background(), + "fake", + 0, 100, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), + ) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs("fake", ls.Hash(), chks), found) @@ -158,7 +165,14 @@ func Test_TenantHeads_MultiRead(t *testing.T) { // ensure we're only returned the data from the correct tenant for _, tenant := range tenants { - found, err := h.GetChunkRefs(context.Background(), tenant.user, 0, 100, nil, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + found, err := h.GetChunkRefs( + context.Background(), + tenant.user, + 0, 100, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), + ) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(tenant.user, tenant.ls.Hash(), chks), found) } @@ -240,7 +254,14 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, recoverHead(mgr.name, mgr.dir, mgr.activeHeads, grp.wals, false)) for _, c := range cases { - refs, err := mgr.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) + refs, err := mgr.GetChunkRefs( + context.Background(), + c.User, + 0, math.MaxInt64, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), + ) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(c.User, c.Fingerprint, c.Chunks), refs) } @@ -289,7 +310,14 @@ func Test_HeadManager_QueryAfterRotate(t *testing.T) { mgr.tick(nextPeriod) // synthetic tick to rotate head for _, c := range cases { - refs, err := mgr.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) + refs, err := mgr.GetChunkRefs( + context.Background(), + c.User, + 0, math.MaxInt64, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), + ) require.Nil(t, err) require.Equal(t, chunkMetasToChunkRefs(c.User, c.Fingerprint, c.Chunks), refs) } @@ -359,7 +387,14 @@ func Test_HeadManager_Lifecycle(t *testing.T) { multiIndex := NewMultiIndex(IndexSlice{mgr, mgr.tsdbManager.(noopTSDBManager).tenantHeads}) for _, c := range cases { - refs, err := multiIndex.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) + refs, err := multiIndex.GetChunkRefs( + context.Background(), + c.User, + 0, math.MaxInt64, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), + ) require.Nil(t, err) lbls := labels.NewBuilder(c.Labels) @@ -388,7 +423,14 @@ func Test_HeadManager_Lifecycle(t *testing.T) { // Ensure old + new data is queryable for _, c := range append(cases, newCase) { - refs, err := multiIndex.GetChunkRefs(context.Background(), c.User, 0, math.MaxInt64, nil, nil, nil, labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+")) + refs, err := multiIndex.GetChunkRefs( + context.Background(), + c.User, + 0, math.MaxInt64, + nil, + nil, nil, + labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), + ) require.Nil(t, err) lbls := labels.NewBuilder(c.Labels) @@ -619,7 +661,15 @@ func BenchmarkTenantHeads(b *testing.B) { tenant := r % nTenants // nolint:ineffassign,staticcheck - res, _ = heads.GetChunkRefs(context.Background(), fmt.Sprint(tenant), 0, math.MaxInt64, nil, res, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + res, _ = heads.GetChunkRefs( + context.Background(), + fmt.Sprint(tenant), + 0, math.MaxInt64, + nil, + res, + nil, + labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), + ) }(r) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index df745db58dcc3..468de70080cdb 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -2619,11 +2619,6 @@ func overlap(from, through, chkFrom, chkThrough int64) bool { return from <= chkThrough && through > chkFrom } -// h11: add a mutex. -// Reset takes the muted to reset all the stats -// AddStructuredMetadata takes the mutes and updates the stats -// The index manager calls Reset when the index is written -// In the ingester push logic, we keep updating the index stats type StreamStats struct { mu sync.RWMutex StructuredMetadataFieldNames map[string]struct{} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index d600b186877ee..1b7f7648e2399 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -125,7 +125,6 @@ func (c *IndexClient) GetChunkRefs(ctx context.Context, userID string, from, thr return nil, err } - // h11: somehow check if the predicates have structured metadata labels var filterLabelNames []string filters := syntax.ExtractLabelFiltersBeforeParser(predicate.Plan().AST) for _, f := range filters { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index dd9ea5aebb284..8f93906b6899a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -199,7 +199,6 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe } // The builder now has the up-to-date series data, we can now reset the stats - // TODO(h11): We need to write the stats to the builder here head.ResetSeriesStats() return nil From 427deec509cada194e6c3143c15fa25eb2b1885a Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Thu, 12 Dec 2024 16:19:54 +0100 Subject: [PATCH 12/21] Encode/decode only for TSDB >= V4 --- pkg/storage/config/schema_config.go | 8 ++- .../shipper/indexshipper/tsdb/index/index.go | 66 +++++++++++-------- .../shipper/indexshipper/tsdb/querier_test.go | 2 +- .../shipper/indexshipper/tsdb/util_test.go | 2 +- 4 files changed, 45 insertions(+), 33 deletions(-) diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 2c9a6a4605e84..443fb4e2d93b0 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -141,7 +141,7 @@ type PeriodConfig struct { IndexType string `yaml:"store" doc:"description=store and object_store below affect which key is used. Which index to use. Either tsdb or boltdb-shipper. Following stores are deprecated: aws, aws-dynamo, gcp, gcp-columnkey, bigtable, bigtable-hashed, cassandra, grpc."` // type of object client to use. ObjectType string `yaml:"object_store" doc:"description=Which store to use for the chunks. Either aws (alias s3), azure, gcs, alibabacloud, bos, cos, swift, filesystem, or a named_store (refer to named_stores_config). Following stores are deprecated: aws-dynamo, gcp, gcp-columnkey, bigtable, bigtable-hashed, cassandra, grpc."` - Schema string `yaml:"schema" doc:"description=The schema version to use, current recommended schema is v13."` + Schema string `yaml:"schema" doc:"description=The schema version to use, current recommended schema is v14."` IndexTables IndexPeriodicTableConfig `yaml:"index" doc:"description=Configures how the index is updated and stored."` ChunkTables PeriodicTableConfig `yaml:"chunks" doc:"description=Configured how the chunks are updated and stored."` RowShards uint32 `yaml:"row_shards" doc:"default=16|description=How many shards will be created. Only used if schema is v10 or greater."` @@ -439,8 +439,10 @@ func (cfg *PeriodConfig) TSDBFormat() (int, error) { switch { case sver <= 12: return index.FormatV2, nil - default: // for v13 and above + case sver == 13: return index.FormatV3, nil + default: // for v14 and above + return index.FormatV4, nil } } @@ -469,7 +471,7 @@ func (cfg PeriodConfig) validate() error { } switch v { - case 10, 11, 12, 13: + case 10, 11, 12, 13, 14: if cfg.RowShards == 0 { return fmt.Errorf("must have row_shards > 0 (current: %d) for schema (%s)", cfg.RowShards, cfg.Schema) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 468de70080cdb..8b8050eb8b512 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -57,6 +57,9 @@ const ( // FormatV3 represents 3 version of index. It adds support for // paging through batches of chunks within a series FormatV3 = 3 + // FormatV4 represents 4 version of index. It adds support for + // structured metadata stats + FormatV4 = 4 IndexFilename = "index" @@ -468,25 +471,25 @@ func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model. w.buf2.PutUvarint32(valueIndex) } - // h11: write stream stats - var smFields map[string]struct{} - if len(stats) > 0 { - smFields = stats[0].StructuredMetadataFieldNames - } + if w.Version >= FormatV4 { + var smFields map[string]struct{} + if len(stats) > 0 { + smFields = stats[0].StructuredMetadataFieldNames + } - w.buf2.PutUvarint(len(smFields)) - for sm := range smFields { - var err error - cacheEntry, ok := w.symbolCache[sm] - nameIndex := cacheEntry.index - if !ok { - nameIndex, err = w.symbols.ReverseLookup(sm) - if err != nil { - return errors.Errorf("symbol entry for %q does not exist, %v", sm, err) + w.buf2.PutUvarint(len(smFields)) + for sm := range smFields { + var err error + cacheEntry, ok := w.symbolCache[sm] + nameIndex := cacheEntry.index + if !ok { + nameIndex, err = w.symbols.ReverseLookup(sm) + if err != nil { + return errors.Errorf("symbol entry for %q does not exist, %v", sm, err) + } } + w.buf2.PutUvarint32(nameIndex) } - // h11: Should we add this to w.labelNames? - w.buf2.PutUvarint32(nameIndex) } w.addChunks(chunks, &w.buf2, &w.buf1, ChunkPageSize) @@ -1305,7 +1308,7 @@ func newReader(b ByteSlice, c io.Closer) (*Reader, error) { } r.version = int(r.b.Range(4, 5)[0]) - if r.version != FormatV1 && r.version != FormatV2 && r.version != FormatV3 { + if r.version != FormatV1 && r.version != FormatV2 && r.version != FormatV3 && r.version != FormatV4 { return nil, errors.Errorf("unknown index file version %d", r.version) } @@ -1731,12 +1734,14 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, er offsetsMap[off] = struct{}{} } - smOffset, err := r.dec.SmNamesOffsetsFor(&d) - if err != nil { - return nil, nil, errors.Wrap(err, "get structured metadata offsets") - } - for _, off := range smOffset { - smOffsetsMap[off] = struct{}{} + if r.version >= FormatV4 { + smOffset, err := r.dec.SmNamesOffsetsFor(&d) + if err != nil { + return nil, nil, errors.Wrap(err, "get structured metadata offsets") + } + for _, off := range smOffset { + smOffsetsMap[off] = struct{}{} + } } } @@ -1752,6 +1757,7 @@ func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, []string, er sort.Strings(names) // Lookup the unique structured metadata symbols. + // Note: empty if TSDB version is less than V4. smNames := make([]string, 0, len(smOffsetsMap)) for o := range smOffsetsMap { name, err := r.lookupSymbol(o) @@ -2224,14 +2230,16 @@ func (dec *Decoder) prepSeries(version int, b []byte, lbls *labels.Labels, chks *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } - if err := dec.readSeriesStats(version, &d, stats); err != nil { - return nil, 0, errors.Wrap(err, "read series stats") + if version >= FormatV4 { + if err := dec.readSeriesStats(&d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") + } } return &d, fprint, nil } -func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **StreamStats) error { +func (dec *Decoder) readSeriesStats(d *encoding.Decbuf, stats **StreamStats) error { nSMFieldNames := d.Uvarint() fields := make(map[string]struct{}, nSMFieldNames) @@ -2296,8 +2304,10 @@ func (dec *Decoder) prepSeriesBy(version int, b []byte, lbls *labels.Labels, chk *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } - if err := dec.readSeriesStats(version, &d, stats); err != nil { - return nil, 0, errors.Wrap(err, "read series stats") + if version >= FormatV4 { + if err := dec.readSeriesStats(&d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") + } } return &d, fprint, nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go index 93c7d3fc14bd4..75986b23e5159 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go @@ -24,7 +24,7 @@ func mustParseLabels(s string) labels.Labels { func TestQueryIndex(t *testing.T) { dir := t.TempDir() - b := NewBuilder(index.FormatV3) + b := NewBuilder(index.FormatV4) cases := []struct { labels labels.Labels chunks []index.ChunkMeta diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go index 10957a3510b84..d4a9d75c82412 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go @@ -18,7 +18,7 @@ type LoadableSeries struct { } func BuildIndex(t testing.TB, dir string, cases []LoadableSeries) *TSDBFile { - b := NewBuilder(index.FormatV3) + b := NewBuilder(index.FormatV4) for _, s := range cases { b.AddSeries(s.Labels, model.Fingerprint(s.Labels.Hash()), s.Chunks) From f3d955ef27269cf2c88ab2413b3ebcc44e7aacb9 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Fri, 13 Dec 2024 15:51:12 +0100 Subject: [PATCH 13/21] fix tests --- pkg/indexgateway/gateway_test.go | 3 ++- pkg/querier/ingester_querier_test.go | 4 ++-- pkg/querier/querier_mock_test.go | 2 +- pkg/querier/queryrange/codec.go | 2 +- pkg/querier/queryrange/codec_test.go | 23 +++++++++++-------- pkg/querier/queryrange/labels_cache.go | 9 ++++---- .../indexshipper/tsdb/index/index_test.go | 6 ----- pkg/util/marshal/marshal_test.go | 8 +++++-- tools/tsdb/index-analyzer/analytics.go | 2 +- 9 files changed, 31 insertions(+), 28 deletions(-) diff --git a/pkg/indexgateway/gateway_test.go b/pkg/indexgateway/gateway_test.go index 9396e865da71b..de7d64577f56a 100644 --- a/pkg/indexgateway/gateway_test.go +++ b/pkg/indexgateway/gateway_test.go @@ -454,6 +454,7 @@ func TestAccumulateChunksToShards(t *testing.T) { _ labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, + _ *tsdb_index.StreamStats, ) (stop bool), _ ...*labels.Matcher) error { for _, s := range series { @@ -466,7 +467,7 @@ func TestAccumulateChunksToShards(t *testing.T) { }) } - if stop := fn(nil, s[0].ref.FingerprintModel(), chks); stop { + if stop := fn(nil, s[0].ref.FingerprintModel(), chks, nil); stop { return nil } } diff --git a/pkg/querier/ingester_querier_test.go b/pkg/querier/ingester_querier_test.go index 268191bd17a72..7f140d8f2c5a2 100644 --- a/pkg/querier/ingester_querier_test.go +++ b/pkg/querier/ingester_querier_test.go @@ -39,7 +39,7 @@ func TestIngesterQuerier_earlyExitOnQuorum(t *testing.T) { "label": { method: "Label", testFn: func(ingesterQuerier *IngesterQuerier) error { - _, err := ingesterQuerier.Label(context.Background(), nil) + _, _, err := ingesterQuerier.Label(context.Background(), nil) return err }, retVal: new(logproto.LabelResponse), @@ -250,7 +250,7 @@ func TestIngesterQuerierFetchesResponsesFromPartitionIngesters(t *testing.T) { "label": { method: "Label", testFn: func(ingesterQuerier *IngesterQuerier) error { - _, err := ingesterQuerier.Label(ctx, nil) + _, _, err := ingesterQuerier.Label(ctx, nil) return err }, retVal: new(logproto.LabelResponse), diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index e290ce516ac6d..78b05f93a74af 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -378,7 +378,7 @@ func (s *storeMock) LabelValuesForMetricName(ctx context.Context, userID string, return args.Get(0).([]string), args.Error(1) } -func (s *storeMock) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { +func (s *storeMock) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, m ...*labels.Matcher) ([]string, []string, error) { args := s.Called(ctx, userID, from, through, metricName, m) return args.Get(0).([]string), nil, args.Error(1) } diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 80b5ccd7f5bb1..665f31b79b242 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -1508,7 +1508,7 @@ func (Codec) MergeResponse(responses ...queryrangebase.Response) (queryrangebase uniqueNames := make(map[string]struct{}) names := []string{} uniqueSmNames := make(map[string]struct{}) - smNames := []string{} + var smNames []string // only unique name should be merged for _, res := range responses { diff --git a/pkg/querier/queryrange/codec_test.go b/pkg/querier/queryrange/codec_test.go index fbb10873f9865..a1431b7702dc0 100644 --- a/pkg/querier/queryrange/codec_test.go +++ b/pkg/querier/queryrange/codec_test.go @@ -1746,14 +1746,16 @@ func Test_codec_MergeResponse(t *testing.T) { "loki labels", []queryrangebase.Response{ &LokiLabelNamesResponse{ - Status: "success", - Version: 1, - Data: []string{"foo", "bar", "buzz"}, + Status: "success", + Version: 1, + Data: []string{"foo", "bar", "buzz"}, + StructuredMetadata: []string{"traceID"}, }, &LokiLabelNamesResponse{ - Status: "success", - Version: 1, - Data: []string{"foo", "bar", "buzz"}, + Status: "success", + Version: 1, + Data: []string{"foo", "bar", "buzz"}, + StructuredMetadata: []string{"email"}, }, &LokiLabelNamesResponse{ Status: "success", @@ -1762,10 +1764,11 @@ func Test_codec_MergeResponse(t *testing.T) { }, }, &LokiLabelNamesResponse{ - Statistics: stats.Result{Summary: stats.Summary{Splits: 3}}, - Status: "success", - Version: 1, - Data: []string{"foo", "bar", "buzz", "blip", "blop"}, + Statistics: stats.Result{Summary: stats.Summary{Splits: 3}}, + Status: "success", + Version: 1, + Data: []string{"foo", "bar", "buzz", "blip", "blop"}, + StructuredMetadata: []string{"traceID", "email"}, }, "", }, diff --git a/pkg/querier/queryrange/labels_cache.go b/pkg/querier/queryrange/labels_cache.go index 2b946760aa1fd..c63004576c242 100644 --- a/pkg/querier/queryrange/labels_cache.go +++ b/pkg/querier/queryrange/labels_cache.go @@ -75,10 +75,11 @@ func (p labelsExtractor) Extract(_, _ int64, res resultscache.Response, _, _ int func (p labelsExtractor) ResponseWithoutHeaders(resp queryrangebase.Response) queryrangebase.Response { labelsResp := resp.(*LokiLabelNamesResponse) return &LokiLabelNamesResponse{ - Status: labelsResp.Status, - Data: labelsResp.Data, - Version: labelsResp.Version, - Statistics: labelsResp.Statistics, + Status: labelsResp.Status, + Data: labelsResp.Data, + StructuredMetadata: labelsResp.StructuredMetadata, + Version: labelsResp.Version, + Statistics: labelsResp.Statistics, } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go index d0b7f528df2d7..123afb7a5c16f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go @@ -786,12 +786,6 @@ func TestDecoder_ChunkSamples(t *testing.T) { d.Uvarint() } - // Read structured metadata names stats - k = d.Uvarint() - for i := 0; i < k; i++ { - d.Uvarint() - } - require.Equal(t, len(tc.chunkMetas), d.Uvarint()) for i, cs := range ir.dec.chunksSample[postings.At()].chunks { require.Equal(t, tc.expectedChunkSamples[i].idx, cs.idx) diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index cbe61eafff4c8..8f37240234b05 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -456,8 +456,12 @@ var labelTests = []struct { "test", "value", }, + StructuredMetadata: []string{ + "traceID", + "email", + }, }, - `{"status": "success", "data": ["label1", "test", "value"]}`, + `{"status": "success", "data": ["label1", "test", "value"], "structured_metadata": ["traceID", "email"]}`, }, } @@ -622,7 +626,7 @@ func Test_WriteQueryResponseJSON(t *testing.T) { func Test_WriteLabelResponseJSON(t *testing.T) { for i, labelTest := range labelTests { var b bytes.Buffer - err := WriteLabelResponseJSON(labelTest.actual.GetValues(), &b) + err := WriteLabelResponseJSON(labelTest.actual.GetValues(), labelTest.actual.GetStructuredMetadata(), &b) require.NoError(t, err) require.JSONEqf(t, labelTest.expected, b.String(), "Label Test %d failed", i) diff --git a/tools/tsdb/index-analyzer/analytics.go b/tools/tsdb/index-analyzer/analytics.go index ac97779c920f8..43e7642d0260a 100644 --- a/tools/tsdb/index-analyzer/analytics.go +++ b/tools/tsdb/index-analyzer/analytics.go @@ -74,7 +74,7 @@ func analyze(indexShipper indexshipper.IndexShipper, tableName string, tenants [ "", nil, model.Earliest, model.Latest, - func(_ labels.Labels, _ model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { + func(_ labels.Labels, _ model.Fingerprint, chks []tsdb_index.ChunkMeta, _ *tsdb_index.StreamStats) (stop bool) { if len(chks) > maxChunksPerSeries { maxChunksPerSeries = len(chks) if len(chks) > 1000 { From b629aa10cb6722680c57193bfb2a35c19b1752a9 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Fri, 13 Dec 2024 16:30:38 +0100 Subject: [PATCH 14/21] Test encoding/decoding stream stats --- .../shipper/indexshipper/tsdb/index/index.go | 38 ++--- .../indexshipper/tsdb/index/index_test.go | 153 +++++++++++++++++- 2 files changed, 164 insertions(+), 27 deletions(-) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 8b8050eb8b512..83be2601bd125 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -409,7 +409,7 @@ func (w *Creator) writeMeta() error { // fingerprint differs from what labels.Hash() produces. For example, // multitenant TSDBs embed a tenant label, but the actual series has no such // label and so the derived fingerprint differs. -func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks []ChunkMeta, stats ...*StreamStats) error { +func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks []ChunkMeta, stats *StreamStats) error { if err := w.ensureStage(idxStageSeries); err != nil { return err } @@ -473,8 +473,8 @@ func (w *Creator) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model. if w.Version >= FormatV4 { var smFields map[string]struct{} - if len(stats) > 0 { - smFields = stats[0].StructuredMetadataFieldNames + if stats != nil { + smFields = stats.StructuredMetadataFieldNames } w.buf2.PutUvarint(len(smFields)) @@ -2230,16 +2230,24 @@ func (dec *Decoder) prepSeries(version int, b []byte, lbls *labels.Labels, chks *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } - if version >= FormatV4 { - if err := dec.readSeriesStats(&d, stats); err != nil { - return nil, 0, errors.Wrap(err, "read series stats") - } + if err := dec.readSeriesStats(version, &d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") } return &d, fprint, nil } -func (dec *Decoder) readSeriesStats(d *encoding.Decbuf, stats **StreamStats) error { +func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **StreamStats) error { + // Even if < V4, we want to return up empty stats if requested + if stats != nil { + if *stats == nil { + *stats = NewStreamStats() + } + } + if version < FormatV4 { + return nil + } + nSMFieldNames := d.Uvarint() fields := make(map[string]struct{}, nSMFieldNames) @@ -2253,13 +2261,7 @@ func (dec *Decoder) readSeriesStats(d *encoding.Decbuf, stats **StreamStats) err fields[ln] = struct{}{} } - - if stats != nil { - if *stats == nil { - *stats = NewStreamStats() - } - (*stats).StructuredMetadataFieldNames = fields - } + (*stats).StructuredMetadataFieldNames = fields return nil } @@ -2304,10 +2306,8 @@ func (dec *Decoder) prepSeriesBy(version int, b []byte, lbls *labels.Labels, chk *lbls = append(*lbls, labels.Label{Name: ln, Value: lv}) } - if version >= FormatV4 { - if err := dec.readSeriesStats(&d, stats); err != nil { - return nil, 0, errors.Wrap(err, "read series stats") - } + if err := dec.readSeriesStats(version, &d, stats); err != nil { + return nil, 0, errors.Wrap(err, "read series stats") } return &d, fprint, nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go index 123afb7a5c16f..04a7fdab7ca67 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index_test.go @@ -174,10 +174,10 @@ func TestIndexRW_Postings(t *testing.T) { // Postings lists are only written if a series with the respective // reference was added before. - require.NoError(t, iw.AddSeries(1, series[0], model.Fingerprint(series[0].Hash()), nil)) - require.NoError(t, iw.AddSeries(2, series[1], model.Fingerprint(series[1].Hash()), nil)) - require.NoError(t, iw.AddSeries(3, series[2], model.Fingerprint(series[2].Hash()), nil)) - require.NoError(t, iw.AddSeries(4, series[3], model.Fingerprint(series[3].Hash()), nil)) + require.NoError(t, iw.AddSeries(1, series[0], model.Fingerprint(series[0].Hash()), nil, nil)) + require.NoError(t, iw.AddSeries(2, series[1], model.Fingerprint(series[1].Hash()), nil, nil)) + require.NoError(t, iw.AddSeries(3, series[2], model.Fingerprint(series[2].Hash()), nil, nil)) + require.NoError(t, iw.AddSeries(4, series[3], model.Fingerprint(series[3].Hash()), nil, nil)) _, err = iw.Close(false) require.NoError(t, err) @@ -266,7 +266,7 @@ func TestPostingsMany(t *testing.T) { }) for i, s := range series { - require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s, model.Fingerprint(s.Hash()), nil)) + require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s, model.Fingerprint(s.Hash()), nil, nil)) } _, err = iw.Close(false) require.NoError(t, err) @@ -394,7 +394,7 @@ func TestPersistence_index_e2e(t *testing.T) { mi := newMockIndex() for i, s := range input { - err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks, nil) require.NoError(t, err) require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...)) @@ -740,7 +740,7 @@ func TestDecoder_ChunkSamples(t *testing.T) { } for i, l := range lbls { - err = iw.AddSeries(storage.SeriesRef(i), l, model.Fingerprint(l.Hash()), tc.chunkMetas) + err = iw.AddSeries(storage.SeriesRef(i), l, model.Fingerprint(l.Hash()), tc.chunkMetas, nil) require.NoError(t, err) } @@ -804,6 +804,143 @@ func TestDecoder_ChunkSamples(t *testing.T) { } } +func TestStreamStats(t *testing.T) { + dir := t.TempDir() + streams := []labels.Labels{ + {{Name: "fizz", Value: "buzz"}}, + {{Name: "ping", Value: "pong"}}, + } + smLabels := []string{"traceID", "email"} + + symbSeen := map[string]struct{}{} + var symbols []string + for _, lbs := range streams { + for _, l := range lbs { + if _, ok := symbSeen[l.Name]; !ok { + symbSeen[l.Name] = struct{}{} + symbols = append(symbols, l.Name) + } + if _, ok := symbSeen[l.Value]; !ok { + symbSeen[l.Value] = struct{}{} + symbols = append(symbols, l.Value) + } + } + } + for _, l := range smLabels { + if _, ok := symbSeen[l]; !ok { + symbSeen[l] = struct{}{} + symbols = append(symbols, l) + } + } + sort.Strings(symbols) + + now := model.Now() + chunkMetas := []ChunkMeta{ + { + MinTime: int64(now), + MaxTime: int64(now.Add(30 * time.Minute)), + }, + { + MinTime: int64(now.Add(40 * time.Minute)), + MaxTime: int64(now.Add(80 * time.Minute)), + }, + { + MinTime: int64(now.Add(90 * time.Minute)), + MaxTime: int64(now.Add(120 * time.Minute)), + }, + { + MinTime: int64(now.Add(130 * time.Minute)), + MaxTime: int64(now.Add(150 * time.Minute)), + }, + } + + for _, tc := range []struct { + name string + format int + writeStructuredMetadata bool + expectStructuredMetadata bool + }{ + { + name: "Old index format", + format: FormatV3, + writeStructuredMetadata: true, + expectStructuredMetadata: false, + }, + { + name: "with structured metadata", + format: FormatV4, + writeStructuredMetadata: true, + expectStructuredMetadata: true, + }, + { + name: "without structured metadata", + format: FormatV4, + writeStructuredMetadata: false, + expectStructuredMetadata: false, + }, + } { + t.Run(tc.name, func(t *testing.T) { + iw, err := NewFileWriterWithVersion(context.Background(), tc.format, filepath.Join(dir, tc.name)) + require.NoError(t, err) + + for _, s := range symbols { + require.NoError(t, iw.AddSymbol(s)) + } + + var stats *StreamStats + if tc.writeStructuredMetadata { + stats = &StreamStats{ + StructuredMetadataFieldNames: map[string]struct{}{ + "traceID": {}, + "email": {}, + }, + } + } + + for i, l := range streams { + err = iw.AddSeries(storage.SeriesRef(i), l, model.Fingerprint(l.Hash()), chunkMetas, stats) + require.NoError(t, err) + } + + _, err = iw.Close(false) + require.NoError(t, err) + + ir, err := NewFileReader(filepath.Join(dir, tc.name)) + require.NoError(t, err) + + postings, err := ir.Postings("fizz", nil, "buzz") + require.NoError(t, err) + + require.True(t, postings.Next()) + var outStats *StreamStats + var lset labels.Labels + var chks []ChunkMeta + + // there should be no chunk samples + require.Nil(t, ir.dec.chunksSample[postings.At()]) + + // read series so that chunk samples get built + _, err = ir.Series(postings.At(), 0, math.MaxInt64, &lset, &chks, &outStats) + require.NoError(t, err) + + require.Equal(t, chunkMetas, chks) + require.Equal(t, lset, streams[0]) + + if tc.expectStructuredMetadata { + require.Equal(t, len(outStats.StructuredMetadataFieldNames), len(stats.StructuredMetadataFieldNames)) + for k := range stats.StructuredMetadataFieldNames { + _, ok := outStats.StructuredMetadataFieldNames[k] + require.True(t, ok) + } + } else { + require.Empty(t, outStats.StructuredMetadataFieldNames) + } + + require.NoError(t, ir.Close()) + }) + } +} + func TestChunkSamples_getChunkSampleForQueryStarting(t *testing.T) { for name, tc := range map[string]struct { chunkSamples *chunkSamples @@ -997,7 +1134,7 @@ func BenchmarkInitReader_ReadOffsetTable(b *testing.B) { } for i, s := range input { - err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks, nil) require.NoError(b, err) } From 72cf18679e208e3f0f97d9b937e4cb8944fea3d6 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 11:54:15 +0100 Subject: [PATCH 15/21] test stats in flush loop --- pkg/indexgateway/gateway.go | 7 +-- pkg/ingester/flush.go | 1 - pkg/ingester/flush_test.go | 89 +++++++++++++++++++++++++++++++-- pkg/ingester/index/bitprefix.go | 6 +-- pkg/ingester/index/index.go | 8 +-- pkg/ingester/index/multi.go | 6 +-- pkg/ingester/instance.go | 9 ++-- pkg/ingester/stream.go | 2 - 8 files changed, 101 insertions(+), 27 deletions(-) diff --git a/pkg/indexgateway/gateway.go b/pkg/indexgateway/gateway.go index 65c04be52ef92..5c8a9f5517423 100644 --- a/pkg/indexgateway/gateway.go +++ b/pkg/indexgateway/gateway.go @@ -329,12 +329,13 @@ func (g *Gateway) LabelNamesForMetricName(ctx context.Context, req *logproto.Lab } matchers = matcherExpr.Mts } - names, _, err := g.indexQuerier.LabelNamesForMetricName(ctx, instanceID, req.From, req.Through, req.MetricName, matchers...) + names, smNames, err := g.indexQuerier.LabelNamesForMetricName(ctx, instanceID, req.From, req.Through, req.MetricName, matchers...) if err != nil { return nil, err } return &logproto.LabelResponse{ - Values: names, + Values: names, + StructuredMetadata: smNames, }, nil } @@ -612,7 +613,7 @@ func accumulateChunksToShards( user, v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, - func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, stats *tsdb_index.StreamStats) (stop bool) { + func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta, _ *tsdb_index.StreamStats) (stop bool) { mtx.Lock() defer mtx.Unlock() diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 63c10057e5f93..9b9ce0869ede8 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -448,7 +448,6 @@ func (i *Ingester) flushChunks(ctx context.Context, fp model.Fingerprint, labelP } firstTime, lastTime := util.RoundToMilliseconds(c.chunk.Bounds()) - // c.chunk already has memchunk that contains symbolizer and structured metadata (can add series stats) ch := chunk.NewChunk( userID, fp, metric, chunkenc.NewFacade(c.chunk, i.cfg.BlockSize, i.cfg.TargetChunkSize), diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index e64f858a5b50f..de0bff0e3cbee 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -318,12 +318,24 @@ func TestFlushMaxAge(t *testing.T) { now := time.Unix(0, 0) firstEntries := []logproto.Entry{ - {Timestamp: now.Add(time.Nanosecond), Line: "1"}, - {Timestamp: now.Add(time.Minute), Line: "2"}, + { + Timestamp: now.Add(time.Nanosecond), + Line: "1", + StructuredMetadata: logproto.FromLabelsToLabelAdapters(labels.FromStrings("traceID", "1234")), + }, + { + Timestamp: now.Add(time.Minute), + Line: "2", + StructuredMetadata: logproto.FromLabelsToLabelAdapters(labels.FromStrings("traceID", "5678", "user", "fake1")), + }, } secondEntries := []logproto.Entry{ - {Timestamp: now.Add(time.Second * 61), Line: "3"}, + { + Timestamp: now.Add(time.Second * 61), + Line: "3", + StructuredMetadata: logproto.FromLabelsToLabelAdapters(labels.FromStrings("user", "fake2")), + }, } req := &logproto.PushRequest{Streams: []logproto.Stream{ @@ -341,6 +353,19 @@ func TestFlushMaxAge(t *testing.T) { // ensure chunk is not flushed after flush period elapses store.checkData(t, map[string][]logproto.Stream{}) + // Structured metadata stats shouldn't be on the store + store.checkStats(t, map[string]map[uint64]*index.StreamStats{}) + // but they should be available on the ingester + start, end := now.Add(-time.Hour), now.Add(time.Hour) + lbs, err := ing.Label(ctx, &logproto.LabelRequest{ + Start: &end, + End: &start, + Query: `{app="l"}`, + }) + require.NoError(t, err) + require.ElementsMatch(t, lbs.Values, []string{"app"}) + require.ElementsMatch(t, lbs.StructuredMetadata, []string{"traceID", "user"}) + req2 := &logproto.PushRequest{Streams: []logproto.Stream{ {Labels: model.LabelSet{"app": "l"}.String(), Entries: secondEntries}, }} @@ -357,6 +382,27 @@ func TestFlushMaxAge(t *testing.T) { }, }) + // Structured metadata stats should now be on the store + store.checkStats(t, map[string]map[uint64]*index.StreamStats{ + userID: { + labels.FromStrings("app", "l").Hash(): { + StructuredMetadataFieldNames: map[string]struct{}{ + "traceID": {}, + "user": {}, + }, + }, + }, + }) + // and they should be on the ingester anymore + lbs, err = ing.Label(ctx, &logproto.LabelRequest{ + Start: &end, + End: &start, + Query: `{app="l"}`, + }) + require.NoError(t, err) + require.Empty(t, lbs.Values) + require.Empty(t, lbs.StructuredMetadata) + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), ing)) } @@ -379,7 +425,9 @@ type testStore struct { mtx sync.Mutex // Chunks keyed by userID. chunks map[string][]chunk.Chunk - onPut func(ctx context.Context, chunks []chunk.Chunk) error + // Stats keyed by userID and stream + stats map[string]map[uint64]*index.StreamStats + onPut func(ctx context.Context, chunks []chunk.Chunk) error } // Note: the ingester New() function creates it's own WAL first which we then override if specified. @@ -388,6 +436,7 @@ type testStore struct { func newTestStore(t require.TestingT, cfg Config, walOverride WAL) (*testStore, *Ingester) { store := &testStore{ chunks: map[string][]chunk.Chunk{}, + stats: map[string]map[uint64]*index.StreamStats{}, } readRingMock := mockReadRingWithOneActiveIngester() @@ -463,7 +512,19 @@ func (s *testStore) Put(ctx context.Context, chunks []chunk.Chunk) error { return nil } -func (s *testStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index.StreamStats) error { +func (s *testStore) UpdateSeriesStats(_ context.Context, _, _ model.Time, userID string, fp uint64, stats *index.StreamStats) error { + s.mtx.Lock() + defer s.mtx.Unlock() + + if _, ok := s.stats[userID]; !ok { + s.stats[userID] = map[uint64]*index.StreamStats{} + } + + if _, ok := s.stats[userID][fp]; !ok { + s.stats[userID][fp] = index.NewStreamStats() + } + s.stats[userID][fp].Merge(stats) + return nil } @@ -587,6 +648,24 @@ func (s *testStore) getChunksForUser(userID string) []chunk.Chunk { return s.chunks[userID] } +func (s *testStore) checkStats(t *testing.T, expected map[string]map[uint64]*index.StreamStats) { + for userID, expectedStreams := range expected { + userStreams, ok := s.stats[userID] + require.True(t, ok) + + for fp, expectedStats := range expectedStreams { + stat, ok := userStreams[fp] + require.True(t, ok) + + require.Len(t, stat.StructuredMetadataFieldNames, len(expectedStats.StructuredMetadataFieldNames)) + for name := range expectedStats.StructuredMetadataFieldNames { + _, ok := stat.StructuredMetadataFieldNames[name] + require.True(t, ok) + } + } + } +} + func buildStreamsFromChunk(t *testing.T, lbs string, chk chunkenc.Chunk) logproto.Stream { it, err := chk.Iterator(context.TODO(), time.Unix(0, 0), time.Unix(1000, 0), logproto.FORWARD, log.NewNoopPipeline().ForStream(labels.Labels{})) require.NoError(t, err) diff --git a/pkg/ingester/index/bitprefix.go b/pkg/ingester/index/bitprefix.go index 631f0febfb027..38df9381e49d0 100644 --- a/pkg/ingester/index/bitprefix.go +++ b/pkg/ingester/index/bitprefix.go @@ -169,9 +169,9 @@ func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *logq } // LabelNames returns all label names. -func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, []string, error) { +func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, error) { if err := ii.validateShard(shard); err != nil { - return nil, nil, err + return nil, err } var extractor func(unlockIndex) []string @@ -206,7 +206,7 @@ func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, []st results = append(results, shardResult) } - return mergeStringSlices(results), nil, nil + return mergeStringSlices(results), nil } // LabelValues returns the values for the given label. diff --git a/pkg/ingester/index/index.go b/pkg/ingester/index/index.go index 70ca09c109e64..67246b081c544 100644 --- a/pkg/ingester/index/index.go +++ b/pkg/ingester/index/index.go @@ -30,7 +30,7 @@ var ErrInvalidShardQuery = errors.New("incompatible index shard query") type Interface interface { Add(labels []logproto.LabelAdapter, fp model.Fingerprint) labels.Labels Lookup(matchers []*labels.Matcher, shard *logql.Shard) ([]model.Fingerprint, error) - LabelNames(shard *logql.Shard) ([]string, []string, error) + LabelNames(shard *logql.Shard) ([]string, error) LabelValues(name string, shard *logql.Shard) ([]string, error) Delete(labels labels.Labels, fp model.Fingerprint) } @@ -183,10 +183,10 @@ func (ii *InvertedIndex) Lookup(matchers []*labels.Matcher, s *logql.Shard) ([]m } // LabelNames returns all label names. -func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, []string, error) { +func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, error) { shard, err := ii.validateShard(s) if err != nil { - return nil, nil, err + return nil, err } shards := ii.getShards(shard) results := make([][]string, 0, len(shards)) @@ -195,7 +195,7 @@ func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, []string, error) results = append(results, shardResult) } - return mergeStringSlices(results), nil, nil + return mergeStringSlices(results), nil } // LabelValues returns the values for the given label. diff --git a/pkg/ingester/index/multi.go b/pkg/ingester/index/multi.go index 321a7b1b2cc76..293be18016b16 100644 --- a/pkg/ingester/index/multi.go +++ b/pkg/ingester/index/multi.go @@ -85,7 +85,7 @@ func (m *Multi) Lookup(t time.Time, matchers []*labels.Matcher, shard *logql.Sha return m.indexFor(t).Lookup(matchers, shard) } -func (m *Multi) LabelNames(t time.Time, shard *logql.Shard) ([]string, []string, error) { +func (m *Multi) LabelNames(t time.Time, shard *logql.Shard) ([]string, error) { return m.indexFor(t).LabelNames(shard) } @@ -116,8 +116,8 @@ func (noopInvertedIndex) Lookup(_ []*labels.Matcher, _ *logql.Shard) ([]model.Fi return nil, nil } -func (noopInvertedIndex) LabelNames(shard *logql.Shard) ([]string, []string, error) { - return nil, nil, nil +func (noopInvertedIndex) LabelNames(_ *logql.Shard) ([]string, error) { + return nil, nil } func (noopInvertedIndex) LabelValues(_ string, _ *logql.Shard) ([]string, error) { diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index dac6b33366be3..55ca01d68be34 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -581,17 +581,14 @@ func (i *instance) label(ctx context.Context, req *logproto.LabelRequest, matche Values: labels, }, nil } - names, smNames, err := i.index.LabelNames(*req.Start, nil) + names, err := i.index.LabelNames(*req.Start, nil) if err != nil { return nil, err } labels = make([]string, len(names)) copy(labels, names) - smLabels := make([]string, len(smNames)) - copy(smLabels, smNames) return &logproto.LabelResponse{ - Values: labels, - StructuredMetadata: smLabels, + Values: labels, }, nil } @@ -631,7 +628,7 @@ type UniqueValues map[string]struct{} func (i *instance) LabelsWithValues(ctx context.Context, startTime time.Time, matchers ...*labels.Matcher) (map[string]UniqueValues, error) { labelMap := make(map[string]UniqueValues) if len(matchers) == 0 { - labelsFromIndex, _, err := i.index.LabelNames(startTime, nil) + labelsFromIndex, err := i.index.LabelNames(startTime, nil) if err != nil { return nil, err } diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 2bafd9c53c316..2f86703b79065 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -362,8 +362,6 @@ func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry, usa s.handleLoggingOfDuplicateEntry(entries[i]) } - // h11: index fliushing and chunks flushing happen independently - // we need to have the stats in sync: when the index stats are flushed, we need to reset the stats s.openStreamStats.AddStructuredMetadata(entries[i].StructuredMetadata) s.entryCt++ From 1b19558b1fb3b71038c35ff6f3fb3e834b31a41a Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 11:55:41 +0100 Subject: [PATCH 16/21] fixup --- pkg/ingester/flush_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index de0bff0e3cbee..5fc5dd93500d8 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -393,7 +393,7 @@ func TestFlushMaxAge(t *testing.T) { }, }, }) - // and they should be on the ingester anymore + // but not in the ingester anymore lbs, err = ing.Label(ctx, &logproto.LabelRequest{ Start: &end, End: &start, From 452da0025d028852a0e1a024c8939320637ceea7 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 12:16:23 +0100 Subject: [PATCH 17/21] Fix test --- .../stores/shipper/indexshipper/tsdb/index/index.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index 83be2601bd125..e14d2d18e8fb2 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -2248,6 +2248,11 @@ func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **Str return nil } + // If stats are not requested (stats are nil), we can just skip through the stats + if stats == nil { + return dec.skipSeriesStats(d) + } + nSMFieldNames := d.Uvarint() fields := make(map[string]struct{}, nSMFieldNames) @@ -2266,6 +2271,14 @@ func (dec *Decoder) readSeriesStats(version int, d *encoding.Decbuf, stats **Str return nil } +func (dec *Decoder) skipSeriesStats(d *encoding.Decbuf) error { + nSMFieldNames := d.Uvarint() + for i := 0; i < nSMFieldNames; i++ { + _ = d.Uvarint() + } + return d.Err() +} + // prepSeriesBy returns series labels and chunks for a series and only returning selected `by` label names. // If `by` is empty, it returns all labels for the series. func (dec *Decoder) prepSeriesBy(version int, b []byte, lbls *labels.Labels, chks *[]ChunkMeta, stats **StreamStats, by map[string]struct{}) (*encoding.Decbuf, uint64, error) { From 1cf2f6d9346e6b8042bf54218033da65f68f0807 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 13:00:04 +0100 Subject: [PATCH 18/21] docs --- docs/sources/shared/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 9cf3b2a18ed0f..5af468c210805 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -4256,7 +4256,7 @@ The `period_config` block configures what index schemas should be used for from # gcp-columnkey, bigtable, bigtable-hashed, cassandra, grpc. [object_store: | default = ""] -# The schema version to use, current recommended schema is v13. +# The schema version to use, current recommended schema is v14. [schema: | default = ""] # Configures how the index is updated and stored. From 0e6c2f7b92803074ef84e71a1c93b59320ee8a8d Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 13:03:26 +0100 Subject: [PATCH 19/21] lint --- pkg/ingester/flush.go | 2 +- pkg/ingester/stream.go | 2 +- pkg/querier/querier_mock_test.go | 5 ++--- pkg/storage/stores/series/series_index_gateway_store.go | 2 +- pkg/storage/stores/series/series_index_store.go | 2 +- pkg/storage/stores/series_store_write.go | 2 +- pkg/storage/stores/series_store_write_test.go | 2 +- .../stores/shipper/indexshipper/tsdb/head_manager.go | 4 ++-- pkg/storage/stores/shipper/indexshipper/tsdb/index.go | 4 ++-- .../stores/shipper/indexshipper/tsdb/index_client.go | 9 ++++----- .../shipper/indexshipper/tsdb/index_shipper_querier.go | 2 +- .../stores/shipper/indexshipper/tsdb/lazy_index.go | 2 +- .../stores/shipper/indexshipper/tsdb/multi_file_index.go | 2 +- .../stores/shipper/indexshipper/tsdb/multitenant.go | 2 +- .../shipper/indexshipper/tsdb/sharding/for_series.go | 2 +- pkg/storage/util_test.go | 2 +- 16 files changed, 22 insertions(+), 24 deletions(-) diff --git a/pkg/ingester/flush.go b/pkg/ingester/flush.go index 9b9ce0869ede8..934f75ff921c2 100644 --- a/pkg/ingester/flush.go +++ b/pkg/ingester/flush.go @@ -15,7 +15,6 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/tenant" "github.com/grafana/dskit/user" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -24,6 +23,7 @@ import ( "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/storage/chunk" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/util" util_log "github.com/grafana/loki/v3/pkg/util/log" ) diff --git a/pkg/ingester/stream.go b/pkg/ingester/stream.go index 2f86703b79065..e1a568c7646b2 100644 --- a/pkg/ingester/stream.go +++ b/pkg/ingester/stream.go @@ -11,7 +11,6 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -25,6 +24,7 @@ import ( "github.com/grafana/loki/v3/pkg/logql/log" "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/runtime" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/flagext" util_log "github.com/grafana/loki/v3/pkg/util/log" diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 78b05f93a74af..54d303ce2a321 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "math" "time" @@ -25,18 +24,18 @@ import ( "google.golang.org/grpc/health/grpc_health_v1" grpc_metadata "google.golang.org/grpc/metadata" - logql_log "github.com/grafana/loki/v3/pkg/logql/log" - "github.com/grafana/loki/v3/pkg/distributor/clientpool" "github.com/grafana/loki/v3/pkg/ingester/client" "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql" + logql_log "github.com/grafana/loki/v3/pkg/logql/log" "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/validation" diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index 881fda50bbb97..76912192d67a9 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -6,7 +6,6 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -15,6 +14,7 @@ import ( statscontext "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" ) diff --git a/pkg/storage/stores/series/series_index_store.go b/pkg/storage/stores/series/series_index_store.go index 444137703fbaa..be868e7ed9948 100644 --- a/pkg/storage/stores/series/series_index_store.go +++ b/pkg/storage/stores/series/series_index_store.go @@ -8,7 +8,6 @@ import ( "sync" "github.com/go-kit/log/level" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" @@ -28,6 +27,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/stores" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" series_index "github.com/grafana/loki/v3/pkg/storage/stores/series/index" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/constants" diff --git a/pkg/storage/stores/series_store_write.go b/pkg/storage/stores/series_store_write.go index b120f16dace7f..54c14e44d1703 100644 --- a/pkg/storage/stores/series_store_write.go +++ b/pkg/storage/stores/series_store_write.go @@ -4,7 +4,6 @@ import ( "context" "github.com/go-kit/log/level" - index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -14,6 +13,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores/index" + index2 "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/util/constants" "github.com/grafana/loki/v3/pkg/util/spanlogger" ) diff --git a/pkg/storage/stores/series_store_write_test.go b/pkg/storage/stores/series_store_write_test.go index a8785c9753daa..5f7969deb43fa 100644 --- a/pkg/storage/stores/series_store_write_test.go +++ b/pkg/storage/stores/series_store_write_test.go @@ -4,7 +4,6 @@ import ( "context" "testing" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -15,6 +14,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) type mockCache struct { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 2e1bb48a3547a..8c5032ad8b727 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -759,7 +759,7 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } -func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, _ []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil, nil @@ -812,7 +812,7 @@ func (t *tenantHeads) Volume(ctx context.Context, userID string, from, through m return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index 04a22068a6a10..d5f715a9c5f95 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -71,7 +71,7 @@ type NoopIndex struct{} func (NoopIndex) Close() error { return nil } func (NoopIndex) Bounds() (_, through model.Time) { return } -func (NoopIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, filterLabelNames []string, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, filterLabelNames []string, _ []ChunkRef, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]ChunkRef, error) { return nil, nil } @@ -96,6 +96,6 @@ func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAc return nil } -func (NoopIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (NoopIndex) ForSeries(_ context.Context, _ string, _ index.FingerprintFilter, _ model.Time, _ model.Time, _ func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, _ ...*labels.Matcher) error { return nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index 1b7f7648e2399..c6b91d5395ad5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -8,24 +8,23 @@ import ( "time" "github.com/go-kit/log/level" - "github.com/grafana/loki/v3/pkg/logql" - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" - util_log "github.com/grafana/loki/v3/pkg/util/log" - "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/querier/astmapper" + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" "github.com/grafana/loki/v3/pkg/storage/chunk" "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/v3/pkg/util" + util_log "github.com/grafana/loki/v3/pkg/util/log" ) // implements stores.Index diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index c37f4867719df..46793b635b1a1 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -134,7 +134,7 @@ func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, t return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta, *tsdbindex.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (i *indexShipperQuerier) ForSeries(ctx context.Context, userID string, fpFilter tsdbindex.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []tsdbindex.ChunkMeta, *tsdbindex.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { idx, err := i.indices(ctx, from, through, userID) if err != nil { return err diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index adfd060ff6e5b..7b2e443cda7ad 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -81,7 +81,7 @@ func (f LazyIndex) Volume(ctx context.Context, userID string, from, through mode return i.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f LazyIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { i, err := f() if err != nil { return err diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index c93a561ccacca..438018ebd37a0 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -384,7 +384,7 @@ func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through mo }) } -func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (i MultiIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, matchers...) }) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index f6dec53a76203..f30cba447384f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -98,6 +98,6 @@ func (m *MultiTenantIndex) Volume(ctx context.Context, userID string, from, thro return m.idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (m *MultiTenantIndex) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { return m.idx.ForSeries(ctx, userID, fpFilter, from, through, fn, nil, withTenantLabelMatcher(userID, matchers)...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go index 11230c1dabd28..cebe21a99b961 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go @@ -38,6 +38,6 @@ type ForSeriesFunc func( matchers ...*labels.Matcher, ) error -func (f ForSeriesFunc) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), filterLabelNames []string, matchers ...*labels.Matcher) error { +func (f ForSeriesFunc) ForSeries(ctx context.Context, userID string, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta, *index.StreamStats) (stop bool), _ []string, matchers ...*labels.Matcher) error { return f(ctx, userID, fpFilter, from, through, fn, matchers...) } diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 3d5fe8255a2bc..5b0496023f400 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -2,7 +2,6 @@ package storage import ( "context" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "sort" "testing" "time" @@ -27,6 +26,7 @@ import ( "github.com/grafana/loki/v3/pkg/storage/config" "github.com/grafana/loki/v3/pkg/storage/stores" index_stats "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" loki_util "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/constants" From 85dc0f4e2fe3949af70872658d2b06231b15cdb8 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 16:08:29 +0100 Subject: [PATCH 20/21] Tests to tsdb pkg --- pkg/blockbuilder/builder/tsdb.go | 1 + .../shipper/indexshipper/tsdb/builder.go | 6 +-- .../shipper/indexshipper/tsdb/builder_test.go | 11 ++++- .../indexshipper/tsdb/compactor_test.go | 20 ++++++++- .../stores/shipper/indexshipper/tsdb/head.go | 22 ++++++--- .../shipper/indexshipper/tsdb/head_manager.go | 5 +-- .../shipper/indexshipper/tsdb/head_read.go | 45 ++++++++++--------- .../shipper/indexshipper/tsdb/index_client.go | 1 - .../shipper/indexshipper/tsdb/manager.go | 3 +- .../shipper/indexshipper/tsdb/querier_test.go | 28 +++++++++--- .../tsdb/single_file_index_test.go | 31 +++++++++++-- .../shipper/indexshipper/tsdb/util_test.go | 3 +- tools/tsdb/migrate-versions/main_test.go | 2 +- tools/tsdb/tsdb-map/main.go | 2 +- 14 files changed, 130 insertions(+), 50 deletions(-) diff --git a/pkg/blockbuilder/builder/tsdb.go b/pkg/blockbuilder/builder/tsdb.go index e90bedb3815ad..9bded199379e1 100644 --- a/pkg/blockbuilder/builder/tsdb.go +++ b/pkg/blockbuilder/builder/tsdb.go @@ -121,6 +121,7 @@ func (m *TsdbCreator) create(ctx context.Context, nodeName string, tableRanges [ // so queries route to the chunks which actually exist. model.Fingerprint(fp), matchingChks, + nil, ) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go index a368ce98fa9a6..af9455990bce9 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go @@ -43,7 +43,7 @@ func NewBuilder(version int) *Builder { } // TODO(h11): stats should be variadic but I added it to avoid changinf it in many places -func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats ...*index.StreamStats) { +func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) { id := ls.String() s, ok := b.streams[id] if !ok { @@ -61,8 +61,8 @@ func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index // But I'm not sure if then we should only reset the stream stats when we rotate the index. s.chunks = append(s.chunks, chks...) - if len(stats) > 0 { - s.stats.Merge(stats[0]) + if stats != nil { + s.stats.Merge(stats) } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder_test.go index 9ccf972151a15..b8c91cbc3ae90 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder_test.go @@ -7,8 +7,10 @@ import ( "testing" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) @@ -25,10 +27,17 @@ func Test_Build(t *testing.T) { chunks: buildChunkMetas(1, 5), } + stats := index.NewStreamStats() + stats.AddStructuredMetadata(logproto.FromLabelsToLabelAdapters(labels.FromStrings( + "traceID", "123", + "spanID", "456", + ))) + builder.AddSeries( lbls1, stream.fp, stream.chunks, + stats, ) return context.Background(), builder, tmpDir @@ -87,7 +96,7 @@ func Test_Build(t *testing.T) { symbolsList = append(symbolsList, symbols.At()) } - require.Equal(t, symbolsList, []string{"a", "b", "bar", "foo"}) + require.Equal(t, []string{"a", "b", "bar", "foo", "spanID", "traceID"}, symbolsList) }) t.Run("write index with correct version", func(t *testing.T) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index aae56643c7770..e85e13d3e2195 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -124,10 +124,17 @@ func setupMultiTenantIndex(t *testing.T, indexFormat int, userStreams map[string lb.Set(TenantLabel, userID) withTenant := lb.Labels() + stats := index.NewStreamStats() + stats.AddStructuredMetadata(logproto.FromLabelsToLabelAdapters(labels.FromStrings( + "traceID", "123", + "spanID", "456", + ))) + b.AddSeries( withTenant, stream.fp, stream.chunks, + stats, ) } } @@ -161,6 +168,7 @@ func setupPerTenantIndex(t *testing.T, indexFormat int, streams []stream, destDi stream.labels, stream.fp, stream.chunks, + stream.stats, ) } @@ -186,10 +194,18 @@ func buildStream(lbls labels.Labels, chunks index.ChunkMetas, userLabel string) if userLabel != "" { lbls = labels.NewBuilder(lbls.Copy()).Set("user_id", userLabel).Labels() } + + stats := index.NewStreamStats() + stats.AddStructuredMetadata(logproto.FromLabelsToLabelAdapters(labels.FromStrings( + "traceID", "123", + "spanID", "456", + ))) + return stream{ labels: lbls, fp: model.Fingerprint(lbls.Hash()), chunks: chunks, + stats: stats, } } @@ -893,10 +909,10 @@ func setupCompactedIndex(t *testing.T) *testContext { require.NoError(t, err) builder := NewBuilder(indexFormat) stream := buildStream(lbls1, buildChunkMetas(shiftTableStart(0), shiftTableStart(10)), "") - builder.AddSeries(stream.labels, stream.fp, stream.chunks) + builder.AddSeries(stream.labels, stream.fp, stream.chunks, stream.stats) stream = buildStream(lbls2, buildChunkMetas(shiftTableStart(0), shiftTableStart(20)), "") - builder.AddSeries(stream.labels, stream.fp, stream.chunks) + builder.AddSeries(stream.labels, stream.fp, stream.chunks, stream.stats) builder.FinalizeChunks() diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go index 80e81cf9c874a..b6469ac0296ba 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head.go @@ -201,7 +201,9 @@ func newSeriesHashmap() *seriesHashmap { func (m *seriesHashmap) get(hash uint64, ls labels.Labels) *memSeries { for _, s := range m.m[hash] { - if labels.Equal(s.ls, ls) { + // We might pass nil labels to get the series by hash only. + // Passing the labels is an extra check to ensure we have the right series. + if ls == nil || labels.Equal(s.ls, ls) { return s } } @@ -260,6 +262,13 @@ func (s *stripeSeries) getByID(id uint64) *memSeries { return x.m[id] } +func (s *stripeSeries) getByFP(fp uint64) *memSeries { + x := s.hashes[fp&uint64(s.shards-1)] + x.RLock() + defer x.RUnlock() + return x.get(fp, nil) +} + // Append adds chunks to the correct series and returns whether a new series was added func (s *stripeSeries) Append( ls labels.Labels, @@ -293,7 +302,7 @@ func (s *stripeSeries) Append( } func (s *stripeSeries) updateSeriesStats(fp uint64, stats *index.StreamStats) { - series := s.getByID(fp) + series := s.getByFP(fp) series.Lock() defer series.Unlock() @@ -316,13 +325,14 @@ type memSeries struct { ls labels.Labels fp uint64 chks index.ChunkMetas - stats index.StreamStats + stats *index.StreamStats } func newMemSeries(ref uint64, ls labels.Labels, fp uint64) *memSeries { return &memSeries{ - ref: ref, - ls: ls, - fp: fp, + ref: ref, + ls: ls, + fp: fp, + stats: index.NewStreamStats(), } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 8c5032ad8b727..b1c5021148fce 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -697,7 +697,6 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, fprint uint64, chk } func (t *tenantHeads) updateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { - // (h11) : don't create head.extract to a different function to just get head := t.getOrCreateTenantHead(userID) head.updateSeriesStats(fp, stats) } @@ -821,7 +820,7 @@ func (t *tenantHeads) ForSeries(ctx context.Context, userID string, fpFilter ind } // helper only used in building TSDBs -func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, head *Head) error) error { +func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, stats *index.StreamStats, head *Head) error) error { for i, shard := range t.tenants { t.locks[i].RLock() defer t.locks[i].RUnlock() @@ -846,7 +845,7 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, c return errors.Wrapf(err, "iterating postings for tenant: %s", tenant) } - if err := fn(tenant, ls, fp, chks, head); err != nil { + if err := fn(tenant, ls, fp, chks, stats, head); err != nil { return err } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index 415bb83d2c6ae..42659ddf63677 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -94,26 +94,29 @@ func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, []s labelNames := h.head.postings.LabelNames() sort.Strings(labelNames) - SMFieldNames := make(map[string]struct{}) - for _, s := range h.head.series.series { - s.RLock() - for _, ms := range s.m { - ms.RLock() - for ln := range ms.stats.StructuredMetadataFieldNames { - SMFieldNames[ln] = struct{}{} - } - ms.RUnlock() - } - s.RUnlock() - } - - structuredMetadataFieldNames := make([]string, 0, len(SMFieldNames)) - for name := range SMFieldNames { - structuredMetadataFieldNames = append(structuredMetadataFieldNames, name) - } - sort.Strings(structuredMetadataFieldNames) - - return labelNames, structuredMetadataFieldNames, nil + // On the file index we don't return the SM fields unless we pass some matchers. + // For consistency, we won't return them here. + // Here's the code that would return them: + //SMFieldNames := make(map[string]struct{}) + //for _, s := range h.head.series.series { + // s.RLock() + // for _, ms := range s.m { + // ms.RLock() + // for ln := range ms.stats.StructuredMetadataFieldNames { + // SMFieldNames[ln] = struct{}{} + // } + // ms.RUnlock() + // } + // s.RUnlock() + //} + // + //structuredMetadataFieldNames := make([]string, 0, len(SMFieldNames)) + //for name := range SMFieldNames { + // structuredMetadataFieldNames = append(structuredMetadataFieldNames, name) + //} + //sort.Strings(structuredMetadataFieldNames) + + return labelNames, nil, nil } return labelNamesWithMatchers(h, matchers...) @@ -152,7 +155,7 @@ func (h *headIndexReader) Series(ref storage.SeriesRef, from int64, through int6 *lbls = append((*lbls)[:0], s.ls...) if stats != nil { - *stats = &s.stats + *stats = s.stats } queryBounds := newBounds(model.Time(from), model.Time(through)) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index c6b91d5395ad5..432808ccf0688 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -182,7 +182,6 @@ func (c *IndexClient) LabelValuesForMetricName(ctx context.Context, userID strin } // tsdb no longer uses the __metric_name__="logs" hack, so we can ignore metric names! -// h11: Stopped implementing the interface, needs fix func (c *IndexClient) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, _ string, matchers ...*labels.Matcher) ([]string, []string, error) { return c.idx.LabelNames(ctx, userID, from, through, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go index 8f93906b6899a..353c9d2ed23f7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/manager.go @@ -160,7 +160,7 @@ type chunkInfo struct { func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshipper.IndexShipper, tableRanges []config.TableRange) (err error) { periods := make(map[string]*Builder) - if err := heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, head *Head) error { + if err := heads.forAll(func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas, stats *index.StreamStats, head *Head) error { // chunks may overlap index period bounds, in which case they're written to multiple pds := make(map[string]chunkInfo) @@ -195,6 +195,7 @@ func (m *tsdbManager) buildFromHead(heads *tenantHeads, indexShipper indexshippe // so queries route to the chunks which actually exist. model.Fingerprint(fp), matchingChks, + stats, ) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go index 75986b23e5159..99876559b93b2 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier_test.go @@ -10,6 +10,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" + "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) @@ -28,6 +29,7 @@ func TestQueryIndex(t *testing.T) { cases := []struct { labels labels.Labels chunks []index.ChunkMeta + stats *index.StreamStats }{ { labels: mustParseLabels(`{foo="bar"}`), @@ -87,8 +89,15 @@ func TestQueryIndex(t *testing.T) { }, }, } + + inStats := index.NewStreamStats() + inStats.AddStructuredMetadata(logproto.FromLabelsToLabelAdapters(labels.FromStrings( + "traceID", "123", + "spanID", "456", + ))) + for _, s := range cases { - b.AddSeries(s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) + b.AddSeries(s.labels, model.Fingerprint(s.labels.Hash()), s.chunks, inStats) } dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) Identifier { @@ -109,20 +118,29 @@ func TestQueryIndex(t *testing.T) { require.Nil(t, err) var ( - chks []index.ChunkMeta - ls labels.Labels + chks []index.ChunkMeta + ls labels.Labels + stats *index.StreamStats ) require.True(t, p.Next()) - _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, nil) + _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, &stats) require.Nil(t, err) require.Equal(t, cases[0].labels.String(), ls.String()) require.Equal(t, cases[0].chunks, chks) + require.Len(t, stats.StructuredMetadataFieldNames, len(inStats.StructuredMetadataFieldNames)) + for f := range inStats.StructuredMetadataFieldNames { + require.Contains(t, stats.StructuredMetadataFieldNames, f) + } require.True(t, p.Next()) - _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, nil) + _, err = reader.Series(p.At(), 0, math.MaxInt64, &ls, &chks, &stats) require.Nil(t, err) require.Equal(t, cases[1].labels.String(), ls.String()) require.Equal(t, cases[1].chunks, chks) + require.Len(t, stats.StructuredMetadataFieldNames, len(inStats.StructuredMetadataFieldNames)) + for f := range inStats.StructuredMetadataFieldNames { + require.Contains(t, stats.StructuredMetadataFieldNames, f) + } require.False(t, p.Next()) mint, maxt := reader.Bounds() diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go index d269cafbf0441..22d33c19a8094 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index_test.go @@ -41,6 +41,13 @@ func TestSingleIdx(t *testing.T) { Checksum: 2, }, }, + Stats: &index.StreamStats{ + StructuredMetadataFieldNames: map[string]struct{}{ + "traceID": {}, + "spanID": {}, + "email": {}, + }, + }, }, { Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), @@ -51,6 +58,13 @@ func TestSingleIdx(t *testing.T) { Checksum: 3, }, }, + Stats: &index.StreamStats{ + StructuredMetadataFieldNames: map[string]struct{}{ + "traceID": {}, + "spanID": {}, + "userID": {}, + }, + }, }, { Labels: mustParseLabels(`{foo="bard", bazz="bozz", bonk="borb"}`), @@ -61,6 +75,13 @@ func TestSingleIdx(t *testing.T) { Checksum: 4, }, }, + Stats: &index.StreamStats{ + StructuredMetadataFieldNames: map[string]struct{}{ + "traceID": {}, + "spanID": {}, + "userAgent": {}, + }, + }, }, } @@ -80,6 +101,7 @@ func TestSingleIdx(t *testing.T) { head := NewHead("fake", NewMetrics(nil), log.NewNopLogger()) for _, x := range cases { _, _ = head.Append(x.Labels, x.Labels.Hash(), x.Chunks) + head.updateSeriesStats(x.Labels.Hash(), x.Stats) } reader := head.Index() return NewTSDBIndex(reader) @@ -176,18 +198,19 @@ func TestSingleIdx(t *testing.T) { 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) + ls, sm, err := idx.LabelNames(context.Background(), "fake", 9, 10) require.Nil(t, err) sort.Strings(ls) require.Equal(t, []string{"bazz", "bonk", "foo"}, ls) + require.Empty(t, sm) }) 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")) + ls, sm, err := idx.LabelNames(context.Background(), "fake", 9, 10, labels.MustNewMatcher(labels.MatchRegexp, "bazz", "buzz|bozz")) require.Nil(t, err) - sort.Strings(ls) - require.Equal(t, []string{"bazz", "foo"}, ls) + require.ElementsMatch(t, []string{"bazz", "bonk", "foo"}, ls) + require.ElementsMatch(t, []string{"spanID", "traceID", "userAgent", "userID"}, sm) }) t.Run("LabelValues", func(t *testing.T) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go index d4a9d75c82412..9f700f410dfee 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/util_test.go @@ -15,13 +15,14 @@ import ( type LoadableSeries struct { Labels labels.Labels Chunks index.ChunkMetas + Stats *index.StreamStats } func BuildIndex(t testing.TB, dir string, cases []LoadableSeries) *TSDBFile { b := NewBuilder(index.FormatV4) for _, s := range cases { - b.AddSeries(s.Labels, model.Fingerprint(s.Labels.Hash()), s.Chunks) + b.AddSeries(s.Labels, model.Fingerprint(s.Labels.Hash()), s.Chunks, s.Stats) } dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) Identifier { diff --git a/tools/tsdb/migrate-versions/main_test.go b/tools/tsdb/migrate-versions/main_test.go index 7b211864292f8..fa79c572ac15d 100644 --- a/tools/tsdb/migrate-versions/main_test.go +++ b/tools/tsdb/migrate-versions/main_test.go @@ -76,7 +76,7 @@ func TestMigrateTables(t *testing.T) { KB: 1, Entries: 1, }, - }) + }, nil) id, err := b.Build(context.Background(), tempDir, func(from, through model.Time, checksum uint32) tsdb.Identifier { id := tsdb.SingleTenantTSDBIdentifier{ diff --git a/tools/tsdb/tsdb-map/main.go b/tools/tsdb/tsdb-map/main.go index 9f35b53fe48c6..ae3059307c1ec 100644 --- a/tools/tsdb/tsdb-map/main.go +++ b/tools/tsdb/tsdb-map/main.go @@ -85,7 +85,7 @@ func main() { MaxTime: int64(entry.Through), KB: ((3 << 20) / 4) / 1024, // guess: 0.75mb, 1/2 of the max size, rounded to KB Entries: 10000, // guess: 10k entries - }}) + }}, nil) return false, nil }) }); err != nil { From b839014c9804ff302a4d11ba77b6ef2f7436c6d8 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 16 Dec 2024 16:48:21 +0100 Subject: [PATCH 21/21] Test compactor and lint fixes --- pkg/storage/store.go | 2 +- .../series/series_index_gateway_store.go | 2 +- pkg/storage/stores/series_store_write_test.go | 4 +--- .../shipper/indexshipper/tsdb/builder.go | 1 - .../indexshipper/tsdb/compactor_test.go | 19 ++++++++++--------- .../stores/shipper/indexshipper/tsdb/index.go | 4 ++-- .../indexshipper/tsdb/single_file_index.go | 2 +- .../stores/shipper/indexshipper/tsdb/store.go | 7 ++----- pkg/storage/util_test.go | 2 +- 9 files changed, 19 insertions(+), 24 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 657acd1171746..eb8e743d4d5cc 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -596,7 +596,7 @@ func filterChunksByTime(from, through model.Time, chunks []chunk.Chunk) []chunk. type failingChunkWriter struct{} -func (f failingChunkWriter) UpdateSeriesStats(ctx context.Context, from, through model.Time, userID string, fp uint64, stats *index2.StreamStats) error { +func (f failingChunkWriter) UpdateSeriesStats(_ context.Context, _, _ model.Time, _ string, _ uint64, _ *index2.StreamStats) error { return nil } diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index 76912192d67a9..c20a679cff0bb 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -89,7 +89,7 @@ func (c *IndexGatewayClientStore) GetSeries(ctx context.Context, _ string, from, } // LabelNamesForMetricName retrieves all label names for a metric name. -func (c *IndexGatewayClientStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { +func (c *IndexGatewayClientStore) LabelNamesForMetricName(ctx context.Context, _ string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { resp, err := c.client.LabelNamesForMetricName(ctx, &logproto.LabelNamesForMetricNameRequest{ MetricName: metricName, From: from, diff --git a/pkg/storage/stores/series_store_write_test.go b/pkg/storage/stores/series_store_write_test.go index 5f7969deb43fa..ce1cb1359bf81 100644 --- a/pkg/storage/stores/series_store_write_test.go +++ b/pkg/storage/stores/series_store_write_test.go @@ -48,9 +48,7 @@ type mockIndexWriter struct { called int } -func (m *mockIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *index.StreamStats) { - return -} +func (m *mockIndexWriter) UpdateSeriesStats(_ string, _ uint64, _ *index.StreamStats) {} func (m *mockIndexWriter) IndexChunk(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { m.called++ diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go index af9455990bce9..b2f7b44ade610 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/builder.go @@ -42,7 +42,6 @@ func NewBuilder(version int) *Builder { } } -// TODO(h11): stats should be variadic but I added it to avoid changinf it in many places func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) { id := ls.String() s, ok := b.streams[id] diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index e85e13d3e2195..edc6b1d0d7bba 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -124,17 +124,11 @@ func setupMultiTenantIndex(t *testing.T, indexFormat int, userStreams map[string lb.Set(TenantLabel, userID) withTenant := lb.Labels() - stats := index.NewStreamStats() - stats.AddStructuredMetadata(logproto.FromLabelsToLabelAdapters(labels.FromStrings( - "traceID", "123", - "spanID", "456", - ))) - b.AddSeries( withTenant, stream.fp, stream.chunks, - stats, + stream.stats, ) } } @@ -253,7 +247,7 @@ func TestCompactor_Compact(t *testing.T) { periodConfig := config.PeriodConfig{ IndexTables: config.IndexPeriodicTableConfig{ PeriodicTableConfig: config.PeriodicTableConfig{Period: config.ObjectStorageIndexRequiredPeriod}}, - Schema: "v12", + Schema: "v14", } indexBkts := IndexBuckets(now, now, []config.TableRange{periodConfig.GetIndexTableNumberRange(config.DayTime{Time: now})}) @@ -627,6 +621,13 @@ func TestCompactor_Compact(t *testing.T) { actualChunks = map[string]index.ChunkMetas{} err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) (stop bool) { actualChunks[lbls.String()] = chks + + require.NotNil(t, stats) + require.Len(t, stats.StructuredMetadataFieldNames, 2) + for _, k := range []string{"traceID", "spanID"} { + require.Contains(t, stats.StructuredMetadataFieldNames, k) + } + return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) require.NoError(t, err) @@ -840,7 +841,7 @@ func TestCompactedIndex(t *testing.T) { require.NoError(t, err) foundChunks := map[string]index.ChunkMetas{} - err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta, stats *index.StreamStats) (stop bool) { + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta, _ *index.StreamStats) (stop bool) { foundChunks[lbls.String()] = append(index.ChunkMetas{}, chks...) return false }, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index d5f715a9c5f95..b2368204bd2d4 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -71,7 +71,7 @@ type NoopIndex struct{} func (NoopIndex) Close() error { return nil } func (NoopIndex) Bounds() (_, through model.Time) { return } -func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, filterLabelNames []string, _ []ChunkRef, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]ChunkRef, error) { +func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, _ []string, _ []ChunkRef, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]ChunkRef, error) { return nil, nil } @@ -79,7 +79,7 @@ func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, filt func (NoopIndex) Series(_ context.Context, _ string, _, _ model.Time, _ []Series, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]Series, error) { return nil, nil } -func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { +func (NoopIndex) LabelNames(_ context.Context, _ string, _, _ model.Time, _ ...*labels.Matcher) ([]string, []string, error) { return nil, nil, nil } func (NoopIndex) LabelValues(_ context.Context, _ string, _, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, error) { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 27d777dc62d27..0a39a6cb297dc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -292,7 +292,7 @@ func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Ti return res, nil } -func (i *TSDBIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { +func (i *TSDBIndex) LabelNames(_ context.Context, _ string, _, _ model.Time, matchers ...*labels.Matcher) ([]string, []string, error) { if len(matchers) == 0 { return i.reader.LabelNames() } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go index d65e4ce60d7b5..20acdef026838 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/store.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/store.go @@ -186,15 +186,12 @@ func (f failingIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbin return fmt.Errorf("index writer is not initialized due to tsdb store being initialized in read-only mode") } -func (f failingIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) { - return +func (f failingIndexWriter) UpdateSeriesStats(_ string, _ uint64, _ *tsdbindex.StreamStats) { } type noopIndexWriter struct{} -func (f noopIndexWriter) UpdateSeriesStats(userID string, fp uint64, stats *tsdbindex.StreamStats) { - return -} +func (f noopIndexWriter) UpdateSeriesStats(_ string, _ uint64, _ *tsdbindex.StreamStats) {} func (f noopIndexWriter) Append(_ string, _ labels.Labels, _ uint64, _ tsdbindex.ChunkMetas) error { return nil diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 5b0496023f400..8ecbcc1453ea5 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -225,7 +225,7 @@ func (m *mockChunkStore) LabelValuesForMetricName(_ context.Context, _ string, _ return nil, nil } -func (m *mockChunkStore) LabelNamesForMetricName(ctx context.Context, userID string, from model.Time, through model.Time, metricName string, matchers ...*labels.Matcher) ([]string, []string, error) { +func (m *mockChunkStore) LabelNamesForMetricName(_ context.Context, _ string, _ model.Time, _ model.Time, _ string, _ ...*labels.Matcher) ([]string, []string, error) { return nil, nil, nil }