diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 36603f1be1d3f..d5411f8b1c9d0 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -2695,18 +2695,18 @@ ring: # CLI flag: -bloom-compactor.compaction-interval [compaction_interval: | default = 10m] -# How many index periods (days) to wait before building bloom filters for a -# table. This can be used to lower cost by not re-writing data to object storage -# too frequently since recent data changes more often. -# CLI flag: -bloom-compactor.min-table-compaction-period -[min_table_compaction_period: | default = 1] - -# The maximum number of index periods (days) to build bloom filters for a table. -# This can be used to lower cost by not trying to compact older data which -# doesn't change. This can be optimized by aligning it with the maximum -# `reject_old_samples_max_age` setting of any tenant. -# CLI flag: -bloom-compactor.max-table-compaction-period -[max_table_compaction_period: | default = 7] +# Newest day-table offset (from today, inclusive) to compact. Increase to lower +# cost by not re-writing data to object storage too frequently since recent data +# changes more often at the cost of not having blooms available as quickly. +# CLI flag: -bloom-compactor.min-table-offset +[min_table_offset: | default = 1] + +# Oldest day-table offset (from today, inclusive) to compact. This can be used +# to lower cost by not trying to compact older data which doesn't change. This +# can be optimized by aligning it with the maximum `reject_old_samples_max_age` +# setting of any tenant. +# CLI flag: -bloom-compactor.max-table-offset +[max_table_offset: | default = 2] # Number of workers to run in parallel for compaction. # CLI flag: -bloom-compactor.worker-parallelism @@ -2871,11 +2871,18 @@ The `limits_config` block configures global and per-tenant limits in Loki. # CLI flag: -querier.tsdb-max-query-parallelism [tsdb_max_query_parallelism: | default = 128] -# Maximum number of bytes assigned to a single sharded query. Also expressible -# in human readable forms (1GB, etc). +# Target maximum number of bytes assigned to a single sharded query. Also +# expressible in human readable forms (1GB, etc). Note: This is a _target_ and +# not an absolute limit. The actual limit can be higher, but the query planner +# will try to build shards up to this limit. # CLI flag: -querier.tsdb-max-bytes-per-shard [tsdb_max_bytes_per_shard: | default = 600MB] +# sharding strategy to use in query planning. Suggested to use bounded once all +# nodes can recognize it. +# CLI flag: -limits.tsdb-sharding-strategy +[tsdb_sharding_strategy: | default = "power_of_two"] + # Cardinality limit for index queries. # CLI flag: -store.cardinality-limit [cardinality_limit: | default = 100000] diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index ddfe552cb2ad8..25994c08b6a35 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -55,8 +55,7 @@ type Compactor struct { sharding util_ring.TenantSharding - metrics *Metrics - btMetrics *v1.Metrics + metrics *Metrics } func New( @@ -67,7 +66,7 @@ func New( fetcherProvider stores.ChunkFetcherProvider, sharding util_ring.TenantSharding, limits Limits, - store bloomshipper.Store, + store bloomshipper.StoreWithMetrics, logger log.Logger, r prometheus.Registerer, ) (*Compactor, error) { @@ -78,6 +77,7 @@ func New( sharding: sharding, limits: limits, bloomStore: store, + metrics: NewMetrics(r, store.BloomMetrics()), } tsdbStore, err := NewTSDBStores(schemaCfg, storeCfg, clientMetrics, logger) @@ -86,10 +86,6 @@ func New( } c.tsdbStore = tsdbStore - // initialize metrics - c.btMetrics = v1.NewMetrics(prometheus.WrapRegistererWithPrefix("loki_bloom_tokenizer_", r)) - c.metrics = NewMetrics(r, c.btMetrics) - chunkLoader := NewStoreChunkLoader( fetcherProvider, c.metrics, @@ -258,12 +254,12 @@ func (c *Compactor) runOne(ctx context.Context) error { func (c *Compactor) tables(ts time.Time) *dayRangeIterator { // adjust the minimum by one to make it inclusive, which is more intuitive // for a configuration variable - adjustedMin := min(c.cfg.MinTableCompactionPeriod - 1) - minCompactionPeriod := time.Duration(adjustedMin) * config.ObjectStorageIndexRequiredPeriod - maxCompactionPeriod := time.Duration(c.cfg.MaxTableCompactionPeriod) * config.ObjectStorageIndexRequiredPeriod + adjustedMin := min(c.cfg.MinTableOffset - 1) + minCompactionDelta := time.Duration(adjustedMin) * config.ObjectStorageIndexRequiredPeriod + maxCompactionDelta := time.Duration(c.cfg.MaxTableOffset) * config.ObjectStorageIndexRequiredPeriod - from := ts.Add(-maxCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) - through := ts.Add(-minCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) + from := ts.Add(-maxCompactionDelta).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) + through := ts.Add(-minCompactionDelta).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) fromDay := config.NewDayTime(model.TimeFromUnixNano(from)) throughDay := config.NewDayTime(model.TimeFromUnixNano(through)) diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go index a80399503f4e7..72ff9c5cc2f19 100644 --- a/pkg/bloomcompactor/config.go +++ b/pkg/bloomcompactor/config.go @@ -22,14 +22,14 @@ type Config struct { // section and the ingester configuration by default). Ring ring.RingConfig `yaml:"ring,omitempty" doc:"description=Defines the ring to be used by the bloom-compactor servers. In case this isn't configured, this block supports inheriting configuration from the common ring section."` // Enabled configures whether bloom-compactors should be used to compact index values into bloomfilters - Enabled bool `yaml:"enabled"` - CompactionInterval time.Duration `yaml:"compaction_interval"` - MinTableCompactionPeriod int `yaml:"min_table_compaction_period"` - MaxTableCompactionPeriod int `yaml:"max_table_compaction_period"` - WorkerParallelism int `yaml:"worker_parallelism"` - RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"` - RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"` - CompactionRetries int `yaml:"compaction_retries"` + Enabled bool `yaml:"enabled"` + CompactionInterval time.Duration `yaml:"compaction_interval"` + MinTableOffset int `yaml:"min_table_offset"` + MaxTableOffset int `yaml:"max_table_offset"` + WorkerParallelism int `yaml:"worker_parallelism"` + RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"` + RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"` + CompactionRetries int `yaml:"compaction_retries"` MaxCompactionParallelism int `yaml:"max_compaction_parallelism"` } @@ -40,15 +40,14 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.CompactionInterval, "bloom-compactor.compaction-interval", 10*time.Minute, "Interval at which to re-run the compaction operation.") f.IntVar(&cfg.WorkerParallelism, "bloom-compactor.worker-parallelism", 1, "Number of workers to run in parallel for compaction.") // TODO(owen-d): This is a confusing name. Rename it to `min_table_offset` - f.IntVar(&cfg.MinTableCompactionPeriod, "bloom-compactor.min-table-compaction-period", 1, "How many index periods (days) to wait before building bloom filters for a table. This can be used to lower cost by not re-writing data to object storage too frequently since recent data changes more often.") + f.IntVar(&cfg.MinTableOffset, "bloom-compactor.min-table-offset", 1, "Newest day-table offset (from today, inclusive) to compact. Increase to lower cost by not re-writing data to object storage too frequently since recent data changes more often at the cost of not having blooms available as quickly.") // TODO(owen-d): ideally we'd set this per tenant based on their `reject_old_samples_max_age` setting, // but due to how we need to discover tenants, we can't do that yet. Tenant+Period discovery is done by // iterating the table periods in object storage and looking for tenants within that period. // In order to have this done dynamically, we'd need to account for tenant specific overrides, which are also // dynamically reloaded. // I'm doing it the simple way for now. - // TODO(owen-d): This is a confusing name. Rename it to `max_table_offset` - f.IntVar(&cfg.MaxTableCompactionPeriod, "bloom-compactor.max-table-compaction-period", 7, "The maximum number of index periods (days) to build bloom filters for a table. This can be used to lower cost by not trying to compact older data which doesn't change. This can be optimized by aligning it with the maximum `reject_old_samples_max_age` setting of any tenant.") + f.IntVar(&cfg.MaxTableOffset, "bloom-compactor.max-table-offset", 2, "Oldest day-table offset (from today, inclusive) to compact. This can be used to lower cost by not trying to compact older data which doesn't change. This can be optimized by aligning it with the maximum `reject_old_samples_max_age` setting of any tenant.") f.DurationVar(&cfg.RetryMinBackoff, "bloom-compactor.compaction-retries-min-backoff", 10*time.Second, "Minimum backoff time between retries.") f.DurationVar(&cfg.RetryMaxBackoff, "bloom-compactor.compaction-retries-max-backoff", time.Minute, "Maximum backoff time between retries.") f.IntVar(&cfg.CompactionRetries, "bloom-compactor.compaction-retries", 3, "Number of retries to perform when compaction fails.") @@ -67,8 +66,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { } func (cfg *Config) Validate() error { - if cfg.MinTableCompactionPeriod > cfg.MaxTableCompactionPeriod { - return fmt.Errorf("min_compaction_age must be less than or equal to max_compaction_age") + if cfg.MinTableOffset > cfg.MaxTableOffset { + return fmt.Errorf("min-table-offset (%d) must be less than or equal to max-table-offset (%d)", cfg.MinTableOffset, cfg.MaxTableOffset) } if cfg.Ring.ReplicationFactor != ringReplicationFactor { return errors.New("Replication factor must not be changed as it will not take effect") diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index 1f2e58dabcceb..2f67d8f90dcdd 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -217,7 +217,7 @@ func (b *LazyBlockBuilderIterator) Next() bool { return false } - b.curr = v1.NewBlock(reader) + b.curr = v1.NewBlock(reader, b.metrics.bloomMetrics) return true } diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go index 29f579a8e777d..fe1b2a09b5a07 100644 --- a/pkg/bloomcompactor/spec_test.go +++ b/pkg/bloomcompactor/spec_test.go @@ -50,7 +50,7 @@ func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fro _, err = builder.BuildFrom(itr) require.Nil(t, err) - res = append(res, v1.NewBlock(reader)) + res = append(res, v1.NewBlock(reader, v1.NewMetrics(nil))) ref := genBlockRef(data[minIdx].Series.Fingerprint, data[maxIdx-1].Series.Fingerprint) t.Log("create block", ref) refs = append(refs, ref) @@ -74,7 +74,7 @@ func dummyBloomGen(t *testing.T, opts v1.BlockOptions, store v1.Iterator[*v1.Ser for i, b := range blocks { bqs = append(bqs, &bloomshipper.CloseableBlockQuerier{ BlockRef: refs[i], - BlockQuerier: v1.NewBlockQuerier(b), + BlockQuerier: v1.NewBlockQuerier(b, false), }) } @@ -152,7 +152,7 @@ func TestSimpleBloomGenerator(t *testing.T) { expectedRefs := v1.PointerSlice(data) outputRefs := make([]*v1.SeriesWithBloom, 0, len(data)) for _, block := range outputBlocks { - bq := block.Querier() + bq := v1.NewBlockQuerier(block, false) for bq.Next() { outputRefs = append(outputRefs, bq.At()) } diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go index ddfd78c2974b4..db2adf9fdc74f 100644 --- a/pkg/bloomcompactor/tsdb.go +++ b/pkg/bloomcompactor/tsdb.go @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/storage" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" ) const ( @@ -121,34 +122,22 @@ func (b *BloomTSDBStore) LoadTSDB( } }() - return NewTSDBSeriesIter(ctx, idx, bounds) + return NewTSDBSeriesIter(ctx, tenant, idx, bounds) } -// TSDBStore is an interface for interacting with the TSDB, -// modeled off a relevant subset of the `tsdb.TSDBIndex` struct -type forSeries interface { - ForSeries( - ctx context.Context, - fpFilter index.FingerprintFilter, - from model.Time, - through model.Time, - fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), - matchers ...*labels.Matcher, - ) error -} - -func NewTSDBSeriesIter(ctx context.Context, f forSeries, bounds v1.FingerprintBounds) (v1.Iterator[*v1.Series], error) { +func NewTSDBSeriesIter(ctx context.Context, user string, f sharding.ForSeries, bounds v1.FingerprintBounds) (v1.Iterator[*v1.Series], error) { // 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) { + func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { select { case <-ctx.Done(): - return + return true default: res := &v1.Series{ Fingerprint: fp, @@ -163,6 +152,7 @@ func NewTSDBSeriesIter(ctx context.Context, f forSeries, bounds v1.FingerprintBo } series = append(series, res) + return false } }, labels.MustNewMatcher(labels.MatchEqual, "", ""), diff --git a/pkg/bloomcompactor/tsdb_test.go b/pkg/bloomcompactor/tsdb_test.go index 91ad1719375ac..30fc668a5a927 100644 --- a/pkg/bloomcompactor/tsdb_test.go +++ b/pkg/bloomcompactor/tsdb_test.go @@ -17,10 +17,11 @@ 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), + fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) bool, _ ...*labels.Matcher, ) error { for i := range f { @@ -61,7 +62,7 @@ func TestTSDBSeriesIter(t *testing.T) { }, } srcItr := v1.NewSliceIter(input) - itr, err := NewTSDBSeriesIter(context.Background(), forSeriesTestImpl(input), v1.NewBounds(0, math.MaxUint64)) + itr, err := NewTSDBSeriesIter(context.Background(), "", forSeriesTestImpl(input), v1.NewBounds(0, math.MaxUint64)) require.NoError(t, err) v1.EqualIterators[*v1.Series]( @@ -78,7 +79,7 @@ func TestTSDBSeriesIter_Expiry(t *testing.T) { t.Run("expires on creation", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) cancel() - itr, err := NewTSDBSeriesIter(ctx, forSeriesTestImpl{ + itr, err := NewTSDBSeriesIter(ctx, "", forSeriesTestImpl{ {}, // a single entry }, v1.NewBounds(0, math.MaxUint64)) require.Error(t, err) @@ -87,7 +88,7 @@ func TestTSDBSeriesIter_Expiry(t *testing.T) { t.Run("expires during consumption", func(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) - itr, err := NewTSDBSeriesIter(ctx, forSeriesTestImpl{ + itr, err := NewTSDBSeriesIter(ctx, "", forSeriesTestImpl{ {}, {}, }, v1.NewBounds(0, math.MaxUint64)) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index f76d6a55d2a09..804ed7490dcf1 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -51,7 +51,6 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/services" "github.com/grafana/dskit/tenant" - "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "go.uber.org/atomic" @@ -63,6 +62,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/constants" + "github.com/grafana/loki/pkg/util/spanlogger" ) var errGatewayUnhealthy = errors.New("bloom-gateway is unhealthy in the ring") @@ -196,15 +196,17 @@ func (g *Gateway) stopping(_ error) error { // FilterChunkRefs implements BloomGatewayServer func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunkRefRequest) (*logproto.FilterChunkRefResponse, error) { - sp, ctx := opentracing.StartSpanFromContext(ctx, "bloomgateway.FilterChunkRefs") - defer sp.Finish() - tenantID, err := tenant.TenantID(ctx) if err != nil { return nil, err } - logger := log.With(g.logger, "tenant", tenantID) + sp, ctx := spanlogger.NewWithLogger( + ctx, + log.With(g.logger, "tenant", tenantID), + "bloomgateway.FilterChunkRefs", + ) + defer sp.Finish() // start time == end time --> empty response if req.From.Equal(req.Through) { @@ -237,7 +239,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk }, nil } - sp.LogKV( + sp.Log( "filters", len(filters), "days", len(seriesByDay), "series_requested", len(req.Refs), @@ -254,7 +256,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk // TODO(owen-d): include capacity in constructor? task.responses = responsesPool.Get(len(seriesForDay.series)) - level.Debug(g.logger).Log( + level.Debug(sp).Log( "msg", "created task for day", "task", task.ID, "day", seriesForDay.day, @@ -276,7 +278,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk for _, task := range tasks { task := task task.enqueueTime = time.Now() - level.Info(logger).Log("msg", "enqueue task", "task", task.ID, "table", task.table, "series", len(task.series)) + level.Info(sp).Log("msg", "enqueue task", "task", task.ID, "table", task.table, "series", len(task.series)) // TODO(owen-d): gracefully handle full queues if err := g.queue.Enqueue(tenantID, nil, task, func() { @@ -289,7 +291,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk go g.consumeTask(ctx, task, tasksCh) } - sp.LogKV("enqueue_duration", time.Since(queueStart).String()) + sp.Log("enqueue_duration", time.Since(queueStart).String()) remaining := len(tasks) @@ -305,7 +307,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk case <-ctx.Done(): return nil, errors.Wrap(ctx.Err(), "request failed") case task := <-tasksCh: - level.Info(logger).Log("msg", "task done", "task", task.ID, "err", task.Err()) + level.Info(sp).Log("msg", "task done", "task", task.ID, "err", task.Err()) if task.Err() != nil { return nil, errors.Wrap(task.Err(), "request failed") } @@ -314,7 +316,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk } } - sp.LogKV("msg", "received all responses") + sp.Log("msg", "received all responses") filtered := filterChunkRefs(req, responses) @@ -333,7 +335,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk g.metrics.requestedChunks.Observe(float64(preFilterChunks)) g.metrics.filteredChunks.Observe(float64(preFilterChunks - postFilterChunks)) - level.Info(logger).Log( + level.Info(sp).Log( "msg", "return filtered chunk refs", "requested_series", preFilterSeries, "filtered_series", preFilterSeries-postFilterSeries, diff --git a/pkg/bloomgateway/client.go b/pkg/bloomgateway/client.go index 6497848923ab1..05eae0360952c 100644 --- a/pkg/bloomgateway/client.go +++ b/pkg/bloomgateway/client.go @@ -35,6 +35,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" "github.com/grafana/loki/pkg/util/constants" + util_log "github.com/grafana/loki/pkg/util/log" ) var ( @@ -270,9 +271,6 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t "from", from.Time(), "through", through.Time(), "num_refs", len(rs.groups), - "refs", JoinFunc(rs.groups, ",", func(e *logproto.GroupedChunkRefs) string { - return model.Fingerprint(e.Fingerprint).String() - }), "plan", plan.String(), "plan_hash", plan.Hash(), ) @@ -357,6 +355,16 @@ func replicationSetsWithBounds(subRing ring.ReadRing, instances []ring.InstanceD return nil, errors.Wrap(err, "bloom gateway get ring") } + if len(tr) == 0 { + level.Warn(util_log.Logger).Log( + "subroutine", "replicationSetsWithBounds", + "msg", "instance has no token ranges - should not be possible", + "instance", inst.Id, + "n_instances", len(instances), + ) + continue + } + // NB(owen-d): this will send requests to the wrong nodes if RF>1 since it only checks the // first token when assigning replicasets rs, err := subRing.Get(tr[0], BlocksOwnerRead, bufDescs, bufHosts, bufZones) diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 9fc6aca57dc11..3fb9b49b66ae0 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -90,7 +90,17 @@ func (p *processor) processBlocks(ctx context.Context, data []blockWithTasks) er } start := time.Now() - bqs, err := p.store.FetchBlocks(ctx, refs, bloomshipper.WithFetchAsync(true), bloomshipper.WithIgnoreNotFound(true)) + bqs, err := p.store.FetchBlocks( + ctx, + refs, + bloomshipper.WithFetchAsync(true), + bloomshipper.WithIgnoreNotFound(true), + // NB(owen-d): we relinquish bloom pages to a pool + // after iteration for performance (alloc reduction). + // This is safe to do here because we do not capture + // the underlying bloom []byte outside of iteration + bloomshipper.WithPool(true), + ) level.Debug(p.logger).Log("msg", "fetch blocks", "count", len(bqs), "duration", time.Since(start), "err", err) if err != nil { diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 0eb94f68c7dbf..48d0f99f9004d 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -334,7 +334,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, // } // } querier := &bloomshipper.CloseableBlockQuerier{ - BlockQuerier: v1.NewBlockQuerier(block), + BlockQuerier: v1.NewBlockQuerier(block, false), BlockRef: blockRef, } queriers = append(queriers, querier) diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index e3060f873b875..86127a3177f91 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -35,6 +35,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/fetcher" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/constants" "github.com/grafana/loki/pkg/validation" ) @@ -364,6 +365,14 @@ func (s *testStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ ch return nil, nil, nil } +func (s *testStore) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) (*logproto.ShardsResponse, error) { + return nil, nil +} + +func (s *testStore) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + func (s *testStore) GetSchemaConfigs() []config.PeriodConfig { return defaultPeriodConfigs } diff --git a/pkg/ingester/index/bitprefix.go b/pkg/ingester/index/bitprefix.go index 8235c2821d6ca..524bd56a69858 100644 --- a/pkg/ingester/index/bitprefix.go +++ b/pkg/ingester/index/bitprefix.go @@ -8,7 +8,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) @@ -48,7 +48,7 @@ func NewBitPrefixWithShards(totalShards uint32) (*BitPrefixInvertedIndex, error) }, nil } -func (ii *BitPrefixInvertedIndex) getShards(shard *astmapper.ShardAnnotation) ([]*indexShard, bool) { +func (ii *BitPrefixInvertedIndex) getShards(shard *logql.Shard) ([]*indexShard, bool) { if shard == nil { return ii.shards, false } @@ -63,13 +63,18 @@ func (ii *BitPrefixInvertedIndex) getShards(shard *astmapper.ShardAnnotation) ([ // Conversely, if the requested shard is 1_of_2, but the index has a factor of 4, // we can _exactly_ match ob1 => (ob10, ob11) and know all fingerprints in those // resulting shards have the requested ob1 prefix (don't need to filter). - var filter bool - if shard.Of > len(ii.shards) { - filter = true + // NB(owen-d): this only applies when using the old power-of-two shards, + // which are superseded by the new bounded sharding strategy. + filter := true + + switch shard.Variant() { + case logql.PowerOfTwoVersion: + if shard.PowerOfTwo.Of <= len(ii.shards) { + filter = false + } } - requestedShard := shard.TSDB() - minFp, maxFp := requestedShard.GetFromThrough() + minFp, maxFp := shard.GetFromThrough() // Determine how many bits we need to take from // the requested shard's min/max fingerprint values @@ -102,12 +107,17 @@ func (ii *BitPrefixInvertedIndex) shardForFP(fp model.Fingerprint) int { return int(fp >> (64 - localShard.RequiredBits())) } -func (ii *BitPrefixInvertedIndex) validateShard(shard *astmapper.ShardAnnotation) error { +func (ii *BitPrefixInvertedIndex) validateShard(shard *logql.Shard) error { if shard == nil { return nil } - return shard.TSDB().Validate() + switch shard.Variant() { + case logql.PowerOfTwoVersion: + return shard.PowerOfTwo.TSDB().Validate() + } + return nil + } // Add a fingerprint under the specified labels. @@ -119,7 +129,7 @@ func (ii *BitPrefixInvertedIndex) Add(labels []logproto.LabelAdapter, fp model.F } // Lookup all fingerprints for the provided matchers. -func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *astmapper.ShardAnnotation) ([]model.Fingerprint, error) { +func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *logql.Shard) ([]model.Fingerprint, error) { if err := ii.validateShard(shard); err != nil { return nil, err } @@ -143,7 +153,7 @@ func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *astm // Because bit prefix order is also ascending order, // the merged fingerprints from ascending shards are also in order. if filter { - minFP, maxFP := shard.TSDB().GetFromThrough() + minFP, maxFP := shard.GetFromThrough() minIdx := sort.Search(len(result), func(i int) bool { return result[i] >= minFP }) @@ -159,7 +169,7 @@ func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *astm } // LabelNames returns all label names. -func (ii *BitPrefixInvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ([]string, error) { +func (ii *BitPrefixInvertedIndex) LabelNames(shard *logql.Shard) ([]string, error) { if err := ii.validateShard(shard); err != nil { return nil, err } @@ -171,7 +181,6 @@ func (ii *BitPrefixInvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ( // Therefore it's more performant to request shard factors lower or equal to the // inverted index factor if filter { - s := shard.TSDB() extractor = func(x unlockIndex) (results []string) { @@ -179,7 +188,7 @@ func (ii *BitPrefixInvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ( for name, entry := range x { for _, valEntry := range entry.fps { for _, fp := range valEntry.fps { - if s.Match(fp) { + if shard.Match(fp) { results = append(results, name) continue outer } @@ -201,7 +210,7 @@ func (ii *BitPrefixInvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ( } // LabelValues returns the values for the given label. -func (ii *BitPrefixInvertedIndex) LabelValues(name string, shard *astmapper.ShardAnnotation) ([]string, error) { +func (ii *BitPrefixInvertedIndex) LabelValues(name string, shard *logql.Shard) ([]string, error) { if err := ii.validateShard(shard); err != nil { return nil, err } @@ -209,7 +218,6 @@ func (ii *BitPrefixInvertedIndex) LabelValues(name string, shard *astmapper.Shar var extractor func(indexEntry) []string shards, filter := ii.getShards(shard) if filter { - s := shard.TSDB() extractor = func(x indexEntry) []string { results := make([]string, 0, len(x.fps)) @@ -217,7 +225,7 @@ func (ii *BitPrefixInvertedIndex) LabelValues(name string, shard *astmapper.Shar outer: for val, valEntry := range x.fps { for _, fp := range valEntry.fps { - if s.Match(fp) { + if shard.Match(fp) { results = append(results, val) continue outer } diff --git a/pkg/ingester/index/bitprefix_test.go b/pkg/ingester/index/bitprefix_test.go index d4afb9f635727..4c67ac4c5e32c 100644 --- a/pkg/ingester/index/bitprefix_test.go +++ b/pkg/ingester/index/bitprefix_test.go @@ -2,6 +2,7 @@ package index import ( "fmt" + "math" "sort" "testing" @@ -10,6 +11,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) @@ -18,24 +20,24 @@ func Test_BitPrefixGetShards(t *testing.T) { for _, tt := range []struct { total uint32 filter bool - shard *astmapper.ShardAnnotation + shard *logql.Shard expected []uint32 }{ // equal factors - {16, false, &astmapper.ShardAnnotation{Shard: 0, Of: 16}, []uint32{0}}, - {16, false, &astmapper.ShardAnnotation{Shard: 4, Of: 16}, []uint32{4}}, - {16, false, &astmapper.ShardAnnotation{Shard: 15, Of: 16}, []uint32{15}}, + {16, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 0, Of: 16}).Ptr(), []uint32{0}}, + {16, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 4, Of: 16}).Ptr(), []uint32{4}}, + {16, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 15, Of: 16}).Ptr(), []uint32{15}}, // idx factor a larger factor of 2 - {32, false, &astmapper.ShardAnnotation{Shard: 0, Of: 16}, []uint32{0, 1}}, - {32, false, &astmapper.ShardAnnotation{Shard: 4, Of: 16}, []uint32{8, 9}}, - {32, false, &astmapper.ShardAnnotation{Shard: 15, Of: 16}, []uint32{30, 31}}, - {64, false, &astmapper.ShardAnnotation{Shard: 15, Of: 16}, []uint32{60, 61, 62, 63}}, + {32, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 0, Of: 16}).Ptr(), []uint32{0, 1}}, + {32, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 4, Of: 16}).Ptr(), []uint32{8, 9}}, + {32, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 15, Of: 16}).Ptr(), []uint32{30, 31}}, + {64, false, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 15, Of: 16}).Ptr(), []uint32{60, 61, 62, 63}}, // // idx factor a smaller factor of 2 - {8, true, &astmapper.ShardAnnotation{Shard: 0, Of: 16}, []uint32{0}}, - {8, true, &astmapper.ShardAnnotation{Shard: 4, Of: 16}, []uint32{2}}, - {8, true, &astmapper.ShardAnnotation{Shard: 15, Of: 16}, []uint32{7}}, + {8, true, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 0, Of: 16}).Ptr(), []uint32{0}}, + {8, true, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 4, Of: 16}).Ptr(), []uint32{2}}, + {8, true, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 15, Of: 16}).Ptr(), []uint32{7}}, } { tt := tt t.Run(tt.shard.String()+fmt.Sprintf("_total_%d", tt.total), func(t *testing.T) { @@ -52,11 +54,105 @@ func Test_BitPrefixGetShards(t *testing.T) { } } +func Test_BitPrefixGetShards_Bounded(t *testing.T) { + for _, tt := range []struct { + total uint32 + shard *logql.Shard + expected []uint32 + }{ + { + 4, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0b01 << 62, + Max: 0b10 << 62, + }, + }, + ).Ptr(), + []uint32{1, 2}, + }, + { + 4, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0b10 << 62, + Max: 0b11 << 62, + }, + }, + ).Ptr(), + []uint32{2, 3}, + }, + { + 8, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0b00 << 62, + Max: 0b101 << 61, + }, + }, + ).Ptr(), + []uint32{0, 1, 2, 3, 4, 5}, + }, + { + 8, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0b00 << 62, + Max: 0b110 << 61, + }, + }, + ).Ptr(), + []uint32{0, 1, 2, 3, 4, 5, 6}, + }, + { + 8, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0b00 << 62, + Max: 0b111 << 61, + }, + }, + ).Ptr(), + []uint32{0, 1, 2, 3, 4, 5, 6, 7}, + }, + { + 8, + logql.NewBoundedShard( + logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 0, + Max: math.MaxUint64, + }, + }, + ).Ptr(), + []uint32{0, 1, 2, 3, 4, 5, 6, 7}, + }, + } { + t.Run(tt.shard.String()+fmt.Sprintf("_total_%d", tt.total), func(t *testing.T) { + ii, err := NewBitPrefixWithShards(tt.total) + require.Nil(t, err) + res, filter := ii.getShards(tt.shard) + require.True(t, filter) // always need to filter bounded shards + resInt := []uint32{} + for _, r := range res { + resInt = append(resInt, r.shard) + } + require.Equal(t, tt.expected, resInt) + }) + } + +} + func Test_BitPrefixValidateShards(t *testing.T) { ii, err := NewBitPrefixWithShards(32) require.Nil(t, err) - require.NoError(t, ii.validateShard(&astmapper.ShardAnnotation{Shard: 1, Of: 16})) - require.Error(t, ii.validateShard(&astmapper.ShardAnnotation{Shard: 1, Of: 15})) + require.NoError(t, ii.validateShard(logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 1, Of: 16}).Ptr())) + require.Error(t, ii.validateShard(logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 1, Of: 15}).Ptr())) } func Test_BitPrefixCreation(t *testing.T) { @@ -116,10 +212,10 @@ func Test_BitPrefix_hash_mapping(t *testing.T) { []*labels.Matcher{{Type: labels.MatchEqual, Name: "compose_project", Value: "loki-tsdb-storage-s3"}}, - &astmapper.ShardAnnotation{ + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: int(expShard), Of: requestedFactor, - }, + }).Ptr(), ) require.NoError(t, err) require.Len(t, res, 1) @@ -147,7 +243,7 @@ func Test_BitPrefixNoMatcherLookup(t *testing.T) { require.Nil(t, err) expShard := uint32(fp >> (64 - index.NewShard(0, 16).RequiredBits())) ii.Add(logproto.FromLabelsToLabelAdapters(lbs), fp) - ids, err = ii.Lookup(nil, &astmapper.ShardAnnotation{Shard: int(expShard), Of: 16}) + ids, err = ii.Lookup(nil, logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: int(expShard), Of: 16}).Ptr()) require.Nil(t, err) require.Equal(t, fp, ids[0]) } @@ -171,10 +267,10 @@ func Test_BitPrefixConsistentMapping(t *testing.T) { shardMax := 8 for i := 0; i < shardMax; i++ { - shard := &astmapper.ShardAnnotation{ + shard := logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: i, Of: shardMax, - } + }).Ptr() aIDs, err := a.Lookup([]*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), diff --git a/pkg/ingester/index/index.go b/pkg/ingester/index/index.go index aff7352c8d59c..064c0ddc45ba7 100644 --- a/pkg/ingester/index/index.go +++ b/pkg/ingester/index/index.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/storage/stores/series" ) @@ -28,9 +29,9 @@ 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 *astmapper.ShardAnnotation) ([]model.Fingerprint, error) - LabelNames(shard *astmapper.ShardAnnotation) ([]string, error) - LabelValues(name string, shard *astmapper.ShardAnnotation) ([]string, error) + Lookup(matchers []*labels.Matcher, shard *logql.Shard) ([]model.Fingerprint, error) + LabelNames(shard *logql.Shard) ([]string, error) + LabelValues(name string, shard *logql.Shard) ([]string, error) Delete(labels labels.Labels, fp model.Fingerprint) } @@ -71,14 +72,20 @@ func (ii *InvertedIndex) getShards(shard *astmapper.ShardAnnotation) []*indexSha return result } -func (ii *InvertedIndex) validateShard(shard *astmapper.ShardAnnotation) error { +func (ii *InvertedIndex) validateShard(shard *logql.Shard) (*astmapper.ShardAnnotation, error) { if shard == nil { - return nil + return nil, nil + } + + s := shard.PowerOfTwo + if s == nil { + return nil, errors.New("inverted index only supports shard annotations with `PowerOfTwo`") } - if int(ii.totalShards)%shard.Of != 0 || uint32(shard.Of) > ii.totalShards { - return fmt.Errorf("%w index_shard:%d query_shard:%v", ErrInvalidShardQuery, ii.totalShards, shard) + + if int(ii.totalShards)%s.Of != 0 || uint32(s.Of) > ii.totalShards { + return nil, fmt.Errorf("%w index_shard:%d query_shard:%v", ErrInvalidShardQuery, ii.totalShards, s) } - return nil + return s, nil } // Add a fingerprint under the specified labels. @@ -150,8 +157,9 @@ func labelsString(b *bytes.Buffer, ls labels.Labels) { } // Lookup all fingerprints for the provided matchers. -func (ii *InvertedIndex) Lookup(matchers []*labels.Matcher, shard *astmapper.ShardAnnotation) ([]model.Fingerprint, error) { - if err := ii.validateShard(shard); err != nil { +func (ii *InvertedIndex) Lookup(matchers []*labels.Matcher, s *logql.Shard) ([]model.Fingerprint, error) { + shard, err := ii.validateShard(s) + if err != nil { return nil, err } @@ -175,8 +183,9 @@ func (ii *InvertedIndex) Lookup(matchers []*labels.Matcher, shard *astmapper.Sha } // LabelNames returns all label names. -func (ii *InvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ([]string, error) { - if err := ii.validateShard(shard); err != nil { +func (ii *InvertedIndex) LabelNames(s *logql.Shard) ([]string, error) { + shard, err := ii.validateShard(s) + if err != nil { return nil, err } shards := ii.getShards(shard) @@ -190,8 +199,9 @@ func (ii *InvertedIndex) LabelNames(shard *astmapper.ShardAnnotation) ([]string, } // LabelValues returns the values for the given label. -func (ii *InvertedIndex) LabelValues(name string, shard *astmapper.ShardAnnotation) ([]string, error) { - if err := ii.validateShard(shard); err != nil { +func (ii *InvertedIndex) LabelValues(name string, s *logql.Shard) ([]string, error) { + shard, err := ii.validateShard(s) + if err != nil { return nil, err } shards := ii.getShards(shard) diff --git a/pkg/ingester/index/index_test.go b/pkg/ingester/index/index_test.go index bc6aaeebf344b..06625a357970d 100644 --- a/pkg/ingester/index/index_test.go +++ b/pkg/ingester/index/index_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/util" ) @@ -46,7 +47,10 @@ func Test_GetShards(t *testing.T) { func Test_ValidateShards(t *testing.T) { ii := NewWithShards(32) - require.NoError(t, ii.validateShard(&astmapper.ShardAnnotation{Shard: 1, Of: 16})) + _, err := ii.validateShard( + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 1, Of: 16}).Ptr(), + ) + require.NoError(t, err) } var ( @@ -108,7 +112,8 @@ func Test_hash_mapping(t *testing.T) { ii := NewWithShards(shard) ii.Add(logproto.FromLabelsToLabelAdapters(lbs), 1) - res, err := ii.Lookup([]*labels.Matcher{{Type: labels.MatchEqual, Name: "compose_project", Value: "loki-tsdb-storage-s3"}}, &astmapper.ShardAnnotation{Shard: int(labelsSeriesIDHash(lbs) % 16), Of: 16}) + x := logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: int(labelsSeriesIDHash(lbs) % 16), Of: 16}) + res, err := ii.Lookup([]*labels.Matcher{{Type: labels.MatchEqual, Name: "compose_project", Value: "loki-tsdb-storage-s3"}}, &x) require.NoError(t, err) require.Len(t, res, 1) require.Equal(t, model.Fingerprint(1), res[0]) @@ -131,7 +136,8 @@ func Test_NoMatcherLookup(t *testing.T) { // with shard param ii = NewWithShards(16) ii.Add(logproto.FromLabelsToLabelAdapters(lbs), 1) - ids, err = ii.Lookup(nil, &astmapper.ShardAnnotation{Shard: int(labelsSeriesIDHash(lbs) % 16), Of: 16}) + x := logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: int(labelsSeriesIDHash(lbs) % 16), Of: 16}) + ids, err = ii.Lookup(nil, &x) require.Nil(t, err) require.Equal(t, model.Fingerprint(1), ids[0]) } @@ -151,10 +157,10 @@ func Test_ConsistentMapping(t *testing.T) { shardMax := 8 for i := 0; i < shardMax; i++ { - shard := &astmapper.ShardAnnotation{ + shard := logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: i, Of: shardMax, - } + }).Ptr() aIDs, err := a.Lookup([]*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), diff --git a/pkg/ingester/index/multi.go b/pkg/ingester/index/multi.go index db4a7b642f6fe..0bfa57806ad0a 100644 --- a/pkg/ingester/index/multi.go +++ b/pkg/ingester/index/multi.go @@ -8,7 +8,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/storage/config" ) @@ -80,15 +80,15 @@ func (m *Multi) Delete(labels labels.Labels, fp model.Fingerprint) { } -func (m *Multi) Lookup(t time.Time, matchers []*labels.Matcher, shard *astmapper.ShardAnnotation) ([]model.Fingerprint, error) { +func (m *Multi) Lookup(t time.Time, matchers []*labels.Matcher, shard *logql.Shard) ([]model.Fingerprint, error) { return m.indexFor(t).Lookup(matchers, shard) } -func (m *Multi) LabelNames(t time.Time, shard *astmapper.ShardAnnotation) ([]string, error) { +func (m *Multi) LabelNames(t time.Time, shard *logql.Shard) ([]string, error) { return m.indexFor(t).LabelNames(shard) } -func (m *Multi) LabelValues(t time.Time, name string, shard *astmapper.ShardAnnotation) ([]string, error) { +func (m *Multi) LabelValues(t time.Time, name string, shard *logql.Shard) ([]string, error) { return m.indexFor(t).LabelValues(name, shard) } @@ -111,14 +111,14 @@ func (noopInvertedIndex) Add(_ []logproto.LabelAdapter, _ model.Fingerprint) lab func (noopInvertedIndex) Delete(_ labels.Labels, _ model.Fingerprint) {} -func (noopInvertedIndex) Lookup(_ []*labels.Matcher, _ *astmapper.ShardAnnotation) ([]model.Fingerprint, error) { +func (noopInvertedIndex) Lookup(_ []*labels.Matcher, _ *logql.Shard) ([]model.Fingerprint, error) { return nil, nil } -func (noopInvertedIndex) LabelNames(_ *astmapper.ShardAnnotation) ([]string, error) { +func (noopInvertedIndex) LabelNames(_ *logql.Shard) ([]string, error) { return nil, nil } -func (noopInvertedIndex) LabelValues(_ string, _ *astmapper.ShardAnnotation) ([]string, error) { +func (noopInvertedIndex) LabelValues(_ string, _ *logql.Shard) ([]string, error) { return nil, nil } diff --git a/pkg/ingester/index/multi_test.go b/pkg/ingester/index/multi_test.go index dc4568dab63b2..50d5db945edb8 100644 --- a/pkg/ingester/index/multi_test.go +++ b/pkg/ingester/index/multi_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" @@ -129,7 +130,9 @@ func TestMultiIndex(t *testing.T) { []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "foo", "foo"), }, - &astmapper.ShardAnnotation{Shard: int(expShard), Of: int(factor)}, + logql.NewPowerOfTwoShard( + astmapper.ShardAnnotation{Shard: int(expShard), Of: int(factor)}, + ).Ptr(), ) require.Nil(t, err) @@ -144,7 +147,7 @@ func TestMultiIndex(t *testing.T) { []*labels.Matcher{ labels.MustNewMatcher(labels.MatchEqual, "foo", "foo"), }, - &astmapper.ShardAnnotation{Shard: int(expShard), Of: int(factor)}, + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: int(expShard), Of: int(factor)}).Ptr(), ) require.Nil(t, err) diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 1f62821e1cc8b..82a124f5116a4 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -43,6 +43,7 @@ import ( "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/constants" "github.com/grafana/loki/pkg/validation" ) @@ -478,6 +479,14 @@ func (s *mockStore) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*la }, nil } +func (s *mockStore) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) (*logproto.ShardsResponse, error) { + return nil, nil +} + +func (s *mockStore) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + func (s *mockStore) Volume(_ context.Context, _ string, _, _ model.Time, limit int32, _ []string, _ string, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) { return &logproto.VolumeResponse{ Volumes: []logproto.Volume{ diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 64678da85a540..e0e9d5e4ca6b4 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -38,7 +38,6 @@ import ( "github.com/grafana/loki/pkg/logql/log" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel/stats" - "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/runtime" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/config" @@ -504,17 +503,10 @@ func (i *instance) QuerySample(ctx context.Context, req logql.SelectSampleParams stats := stats.FromContext(ctx) var iters []iter.SampleIterator - var shard *astmapper.ShardAnnotation - shards, err := logql.ParseShards(req.Shards) + shard, err := parseShardFromRequest(req.Shards) if err != nil { return nil, err } - if len(shards) > 1 { - return nil, errors.New("only one shard per ingester query is supported") - } - if len(shards) == 1 { - shard = &shards[0] - } selector, err := expr.Selector() if err != nil { return nil, err @@ -823,11 +815,11 @@ func (i *instance) forMatchingStreams( // and is used to select the correct inverted index ts time.Time, matchers []*labels.Matcher, - shards *astmapper.ShardAnnotation, + shard *logql.Shard, fn func(*stream) error, ) error { filters, matchers := util.SplitFiltersAndMatchers(matchers) - ids, err := i.index.Lookup(ts, matchers, shards) + ids, err := i.index.Lookup(ts, matchers, shard) if err != nil { return err } @@ -934,9 +926,9 @@ func (i *instance) openTailersCount() uint32 { return uint32(len(i.tailers)) } -func parseShardFromRequest(reqShards []string) (*astmapper.ShardAnnotation, error) { - var shard *astmapper.ShardAnnotation - shards, err := logql.ParseShards(reqShards) +func parseShardFromRequest(reqShards []string) (*logql.Shard, error) { + var shard *logql.Shard + shards, _, err := logql.ParseShards(reqShards) if err != nil { return nil, err } diff --git a/pkg/loghttp/params.go b/pkg/loghttp/params.go index df97a5c2e37fe..654c52e7725df 100644 --- a/pkg/loghttp/params.go +++ b/pkg/loghttp/params.go @@ -8,6 +8,7 @@ import ( "strings" "time" + "github.com/c2h5oh/datasize" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -200,3 +201,21 @@ func parseRegexQuery(httpRequest *http.Request) (string, error) { } return query, nil } + +func parseBytes(r *http.Request, field string, optional bool) (val datasize.ByteSize, err error) { + s := r.Form.Get(field) + + if s == "" { + if !optional { + return 0, fmt.Errorf("missing %s", field) + } + return val, nil + } + + if err := val.UnmarshalText([]byte(s)); err != nil { + return 0, errors.Wrapf(err, "invalid %s: %s", field, s) + } + + return val, nil + +} diff --git a/pkg/loghttp/query.go b/pkg/loghttp/query.go index 854ccd5ae7116..75f75c60ccc03 100644 --- a/pkg/loghttp/query.go +++ b/pkg/loghttp/query.go @@ -8,6 +8,7 @@ import ( "time" "unsafe" + "github.com/c2h5oh/datasize" "github.com/grafana/jsonparser" json "github.com/json-iterator/go" "github.com/prometheus/common/model" @@ -503,6 +504,17 @@ func ParseIndexStatsQuery(r *http.Request) (*RangeQuery, error) { return ParseRangeQuery(r) } +func ParseIndexShardsQuery(r *http.Request) (*RangeQuery, datasize.ByteSize, error) { + // TODO(owen-d): use a specific type/validation instead + // of using range query parameters (superset) + parsed, err := ParseRangeQuery(r) + if err != nil { + return nil, 0, err + } + targetBytes, err := parseBytes(r, "targetBytesPerShard", true) + return parsed, targetBytes, err +} + func NewVolumeRangeQueryWithDefaults(matchers string) *logproto.VolumeRequest { start, end, _ := determineBounds(time.Now(), "", "", "") step := (time.Duration(defaultQueryRangeStep(start, end)) * time.Second).Milliseconds() diff --git a/pkg/logproto/compat.go b/pkg/logproto/compat.go index 0e65a90da02fa..25b5269e1ae6c 100644 --- a/pkg/logproto/compat.go +++ b/pkg/logproto/compat.go @@ -11,6 +11,7 @@ import ( "time" "unsafe" + "github.com/c2h5oh/datasize" "github.com/cespare/xxhash/v2" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -354,6 +355,9 @@ func (m *FilterChunkRefRequest) GetStep() int64 { return 0 } +// TODO(owen-d): why does this return the hash of all the refs instead of the query? +// The latter should be significantly cheaper, more helpful (readable), and just as correct +// at being a unique identifier for the request. // GetQuery returns the query of the request. // The query is the hash for the input chunks refs and the filter expressions. func (m *FilterChunkRefRequest) GetQuery() string { @@ -402,3 +406,42 @@ func (m *FilterChunkRefRequest) WithStartEndForCache(start, end time.Time) resul return &clone } + +func (m *ShardsRequest) GetCachingOptions() (res definitions.CachingOptions) { return } + +func (m *ShardsRequest) GetStart() time.Time { + return time.Unix(0, m.From.UnixNano()) +} + +func (m *ShardsRequest) GetEnd() time.Time { + return time.Unix(0, m.Through.UnixNano()) +} + +func (m *ShardsRequest) GetStep() int64 { return 0 } + +func (m *ShardsRequest) WithStartEnd(start, end time.Time) definitions.Request { + clone := *m + clone.From = model.TimeFromUnixNano(start.UnixNano()) + clone.Through = model.TimeFromUnixNano(end.UnixNano()) + return &clone +} + +func (m *ShardsRequest) WithQuery(query string) definitions.Request { + clone := *m + clone.Query = query + return &clone +} + +func (m *ShardsRequest) WithStartEndForCache(start, end time.Time) resultscache.Request { + return m.WithStartEnd(start, end).(resultscache.Request) +} + +func (m *ShardsRequest) LogToSpan(sp opentracing.Span) { + fields := []otlog.Field{ + otlog.String("from", timestamp.Time(int64(m.From)).String()), + otlog.String("through", timestamp.Time(int64(m.Through)).String()), + otlog.String("query", m.GetQuery()), + otlog.String("target_bytes_per_shard", datasize.ByteSize(m.TargetBytesPerShard).HumanReadable()), + } + sp.LogFields(fields...) +} diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index 9a5f3f8e61af6..5fa5048e1a9d1 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -133,3 +133,10 @@ func (m *IndexStatsResponse) LoggingKeyValues() []interface{} { "entries", m.Entries, } } + +func (m *Shard) SpaceFor(stats *IndexStatsResponse, targetShardBytes uint64) bool { + curDelta := max(m.Stats.Bytes, targetShardBytes) - min(m.Stats.Bytes, targetShardBytes) + updated := m.Stats.Bytes + stats.Bytes + newDelta := max(updated, targetShardBytes) - min(updated, targetShardBytes) + return newDelta <= curDelta +} diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go new file mode 100644 index 0000000000000..d1c96c76bbed3 --- /dev/null +++ b/pkg/logproto/extensions_test.go @@ -0,0 +1,42 @@ +package logproto + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestShard_SpaceFor(t *testing.T) { + target := uint64(100) + shard := Shard{ + Stats: &IndexStatsResponse{ + Bytes: 50, + }, + } + + for _, tc := range []struct { + desc string + bytes uint64 + exp bool + }{ + { + desc: "full shard", + bytes: 50, + exp: true, + }, + { + desc: "overflow equal to underflow accepts", + bytes: 100, + exp: true, + }, + { + desc: "overflow", + bytes: 101, + exp: false, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, shard.SpaceFor(&IndexStatsResponse{Bytes: tc.bytes}, target), tc.exp) + }) + } +} diff --git a/pkg/logproto/indexgateway.pb.go b/pkg/logproto/indexgateway.pb.go index 86b2665e86b17..1229caebbb604 100644 --- a/pkg/logproto/indexgateway.pb.go +++ b/pkg/logproto/indexgateway.pb.go @@ -6,11 +6,18 @@ package logproto import ( context "context" fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" + stats "github.com/grafana/loki/pkg/logqlmodel/stats" + github_com_prometheus_common_model "github.com/prometheus/common/model" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" + io "io" math "math" + math_bits "math/bits" + reflect "reflect" + strings "strings" ) // Reference imports to suppress errors if they are not otherwise used. @@ -24,33 +31,453 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type ShardsRequest struct { + From github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=from,proto3,customtype=github.com/prometheus/common/model.Time" json:"from"` + Through github_com_prometheus_common_model.Time `protobuf:"varint,2,opt,name=through,proto3,customtype=github.com/prometheus/common/model.Time" json:"through"` + Query string `protobuf:"bytes,3,opt,name=query,proto3" json:"query"` + TargetBytesPerShard uint64 `protobuf:"varint,4,opt,name=target_bytes_per_shard,json=targetBytesPerShard,proto3" json:"targetBytesPerShard"` +} + +func (m *ShardsRequest) Reset() { *m = ShardsRequest{} } +func (*ShardsRequest) ProtoMessage() {} +func (*ShardsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_d27585148d0a52c8, []int{0} +} +func (m *ShardsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ShardsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ShardsRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ShardsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShardsRequest.Merge(m, src) +} +func (m *ShardsRequest) XXX_Size() int { + return m.Size() +} +func (m *ShardsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ShardsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ShardsRequest proto.InternalMessageInfo + +func (m *ShardsRequest) GetQuery() string { + if m != nil { + return m.Query + } + return "" +} + +func (m *ShardsRequest) GetTargetBytesPerShard() uint64 { + if m != nil { + return m.TargetBytesPerShard + } + return 0 +} + +type ShardsResponse struct { + Shards []Shard `protobuf:"bytes,1,rep,name=shards,proto3" json:"shards"` + Statistics stats.Result `protobuf:"bytes,2,opt,name=statistics,proto3" json:"statistics"` +} + +func (m *ShardsResponse) Reset() { *m = ShardsResponse{} } +func (*ShardsResponse) ProtoMessage() {} +func (*ShardsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_d27585148d0a52c8, []int{1} +} +func (m *ShardsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ShardsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ShardsResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ShardsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShardsResponse.Merge(m, src) +} +func (m *ShardsResponse) XXX_Size() int { + return m.Size() +} +func (m *ShardsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ShardsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ShardsResponse proto.InternalMessageInfo + +func (m *ShardsResponse) GetShards() []Shard { + if m != nil { + return m.Shards + } + return nil +} + +func (m *ShardsResponse) GetStatistics() stats.Result { + if m != nil { + return m.Statistics + } + return stats.Result{} +} + +type Shard struct { + Bounds FPBounds `protobuf:"bytes,1,opt,name=bounds,proto3" json:"bounds"` + Stats *IndexStatsResponse `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats"` +} + +func (m *Shard) Reset() { *m = Shard{} } +func (*Shard) ProtoMessage() {} +func (*Shard) Descriptor() ([]byte, []int) { + return fileDescriptor_d27585148d0a52c8, []int{2} +} +func (m *Shard) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Shard) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Shard.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Shard) XXX_Merge(src proto.Message) { + xxx_messageInfo_Shard.Merge(m, src) +} +func (m *Shard) XXX_Size() int { + return m.Size() +} +func (m *Shard) XXX_DiscardUnknown() { + xxx_messageInfo_Shard.DiscardUnknown(m) +} + +var xxx_messageInfo_Shard proto.InternalMessageInfo + +func (m *Shard) GetBounds() FPBounds { + if m != nil { + return m.Bounds + } + return FPBounds{} +} + +func (m *Shard) GetStats() *IndexStatsResponse { + if m != nil { + return m.Stats + } + return nil +} + +// FPBounds is identical to the definition in `pkg/storage/bloom/v1/bounds.FingerprintBounds` +// which ensures we can cast between them without allocations. +type FPBounds struct { + Min github_com_prometheus_common_model.Fingerprint `protobuf:"varint,1,opt,name=min,proto3,casttype=github.com/prometheus/common/model.Fingerprint" json:"min"` + Max github_com_prometheus_common_model.Fingerprint `protobuf:"varint,2,opt,name=max,proto3,casttype=github.com/prometheus/common/model.Fingerprint" json:"max"` +} + +func (m *FPBounds) Reset() { *m = FPBounds{} } +func (*FPBounds) ProtoMessage() {} +func (*FPBounds) Descriptor() ([]byte, []int) { + return fileDescriptor_d27585148d0a52c8, []int{3} +} +func (m *FPBounds) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FPBounds) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_FPBounds.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *FPBounds) XXX_Merge(src proto.Message) { + xxx_messageInfo_FPBounds.Merge(m, src) +} +func (m *FPBounds) XXX_Size() int { + return m.Size() +} +func (m *FPBounds) XXX_DiscardUnknown() { + xxx_messageInfo_FPBounds.DiscardUnknown(m) +} + +var xxx_messageInfo_FPBounds proto.InternalMessageInfo + +func (m *FPBounds) GetMin() github_com_prometheus_common_model.Fingerprint { + if m != nil { + return m.Min + } + return 0 +} + +func (m *FPBounds) GetMax() github_com_prometheus_common_model.Fingerprint { + if m != nil { + return m.Max + } + return 0 +} + +func init() { + proto.RegisterType((*ShardsRequest)(nil), "indexgatewaypb.ShardsRequest") + proto.RegisterType((*ShardsResponse)(nil), "indexgatewaypb.ShardsResponse") + proto.RegisterType((*Shard)(nil), "indexgatewaypb.Shard") + proto.RegisterType((*FPBounds)(nil), "indexgatewaypb.FPBounds") +} + func init() { proto.RegisterFile("pkg/logproto/indexgateway.proto", fileDescriptor_d27585148d0a52c8) } var fileDescriptor_d27585148d0a52c8 = []byte{ - // 361 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xb1, 0x4e, 0xc2, 0x40, - 0x18, 0xc7, 0xef, 0x12, 0x63, 0xf4, 0x34, 0x0e, 0xb7, 0x40, 0x40, 0xcf, 0xc4, 0x38, 0xe8, 0x42, - 0x8d, 0xbe, 0x80, 0xd1, 0x84, 0x86, 0x04, 0x4d, 0xc4, 0x84, 0x81, 0xc1, 0x78, 0xc5, 0x8f, 0xd2, - 0x50, 0x7a, 0xb5, 0xbd, 0x46, 0xd9, 0x7c, 0x04, 0x1f, 0xc3, 0x87, 0xf0, 0x01, 0x1c, 0x19, 0x19, - 0xe5, 0x58, 0x1c, 0x79, 0x04, 0xc3, 0x35, 0x85, 0x03, 0x4b, 0xe2, 0x04, 0xfd, 0xfd, 0x7f, 0xdf, - 0xff, 0x4b, 0xef, 0x4a, 0x0e, 0xc3, 0x9e, 0x6b, 0xf9, 0xc2, 0x0d, 0x23, 0x21, 0x85, 0xe5, 0x05, - 0x4f, 0xf0, 0xea, 0x72, 0x09, 0x2f, 0x7c, 0x50, 0xd1, 0x88, 0xee, 0x99, 0x2c, 0x74, 0x4a, 0xe5, - 0xa5, 0x81, 0xec, 0x4f, 0x2a, 0x9f, 0x7f, 0x6e, 0x90, 0xdd, 0xda, 0xcc, 0xb7, 0x53, 0x9f, 0xd6, - 0x08, 0xb9, 0x4b, 0x20, 0x1a, 0x68, 0x48, 0xcb, 0x95, 0xb9, 0xbf, 0xa0, 0x0d, 0x78, 0x4e, 0x20, - 0x96, 0xa5, 0xfd, 0xfc, 0x30, 0x0e, 0x45, 0x10, 0xc3, 0x19, 0xa6, 0x75, 0xb2, 0x63, 0x83, 0xbc, - 0xee, 0x26, 0x41, 0xaf, 0x01, 0x1d, 0x6a, 0xe8, 0x06, 0xce, 0xca, 0x0e, 0xd6, 0xa4, 0x69, 0xdb, - 0x11, 0xa2, 0x55, 0xb2, 0x6d, 0x83, 0xbc, 0x87, 0xc8, 0x83, 0x98, 0x96, 0x96, 0xec, 0x14, 0x66, - 0x4d, 0xe5, 0xdc, 0x6c, 0xde, 0xf3, 0x40, 0x0a, 0x75, 0xee, 0x80, 0x7f, 0xcb, 0xfb, 0x10, 0x57, - 0x45, 0x74, 0x03, 0x32, 0xf2, 0xda, 0xb3, 0x27, 0x7a, 0xb2, 0x98, 0x5c, 0xa3, 0x64, 0x3b, 0x0a, - 0x2b, 0xa6, 0xd1, 0xff, 0x48, 0x8a, 0x1a, 0x35, 0xb9, 0x9f, 0xac, 0x2e, 0x38, 0x5d, 0x19, 0xcb, - 0x71, 0xfe, 0xb1, 0xc1, 0x26, 0x5b, 0xb3, 0x17, 0x93, 0x5c, 0xc6, 0xe6, 0x05, 0xe9, 0xe3, 0xd7, - 0x34, 0xe7, 0x82, 0xcc, 0x70, 0x5e, 0x74, 0xa9, 0x8f, 0xb4, 0x29, 0xfc, 0xa4, 0x0f, 0xd4, 0x58, - 0x98, 0x92, 0xac, 0xa5, 0xf8, 0x37, 0xc8, 0x1a, 0xae, 0x5a, 0xc3, 0x31, 0x43, 0xa3, 0x31, 0x43, - 0xd3, 0x31, 0xc3, 0x6f, 0x8a, 0xe1, 0x0f, 0xc5, 0xf0, 0x97, 0x62, 0x78, 0xa8, 0x18, 0xfe, 0x56, - 0x0c, 0xff, 0x28, 0x86, 0xa6, 0x8a, 0xe1, 0xf7, 0x09, 0x43, 0xc3, 0x09, 0x43, 0xa3, 0x09, 0x43, - 0xad, 0x63, 0xd7, 0x93, 0xdd, 0xc4, 0xa9, 0xb4, 0x45, 0xdf, 0x72, 0x23, 0xde, 0xe1, 0x01, 0xb7, - 0x7c, 0xd1, 0xf3, 0x2c, 0xf3, 0x4b, 0x75, 0x36, 0xf5, 0xcf, 0xc5, 0x6f, 0x00, 0x00, 0x00, 0xff, - 0xff, 0x7a, 0x1a, 0x28, 0xb4, 0xf1, 0x02, 0x00, 0x00, + // 734 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xbf, 0x4f, 0xdb, 0x4c, + 0x18, 0xf6, 0x91, 0x84, 0x0f, 0x8e, 0x0f, 0x86, 0xfb, 0x7e, 0x60, 0x05, 0xb0, 0xa3, 0xa8, 0x52, + 0xd3, 0xc5, 0xae, 0xe8, 0x50, 0xb5, 0x12, 0x12, 0x75, 0xa5, 0x44, 0x88, 0xb4, 0xa2, 0x06, 0x31, + 0x30, 0x94, 0x3a, 0xe1, 0x70, 0x2c, 0x6c, 0x5f, 0xb8, 0x3b, 0xab, 0x61, 0xeb, 0x58, 0x75, 0xaa, + 0xfa, 0x1f, 0x54, 0xea, 0xd0, 0x3f, 0x85, 0x91, 0x11, 0x75, 0xb0, 0x4a, 0x58, 0xaa, 0x4c, 0xcc, + 0x9d, 0x2a, 0xdf, 0xd9, 0x89, 0x09, 0x41, 0xa2, 0x5d, 0x72, 0xe7, 0xe7, 0x7d, 0xde, 0xe7, 0xfd, + 0x79, 0x0a, 0xd4, 0xbb, 0x47, 0xae, 0xe9, 0x13, 0xb7, 0x4b, 0x09, 0x27, 0xa6, 0x17, 0x1e, 0xe0, + 0x9e, 0xeb, 0x70, 0xfc, 0xd6, 0x39, 0x31, 0x04, 0x84, 0x16, 0xf2, 0x58, 0xb7, 0x55, 0xfe, 0xd7, + 0x25, 0x2e, 0x91, 0xec, 0xe4, 0x26, 0x59, 0xe5, 0xa5, 0x6b, 0x32, 0xd9, 0x25, 0x35, 0x56, 0x52, + 0xe3, 0xb1, 0x1f, 0x90, 0x03, 0xec, 0x9b, 0x8c, 0x3b, 0x9c, 0xc9, 0x5f, 0xc9, 0xa8, 0x7e, 0x9e, + 0x82, 0xf3, 0xdb, 0x1d, 0x87, 0x1e, 0x30, 0x1b, 0x1f, 0x47, 0x98, 0x71, 0xb4, 0x09, 0x8b, 0x87, + 0x94, 0x04, 0x2a, 0xa8, 0x80, 0x5a, 0xc1, 0x7a, 0x7c, 0x1a, 0xeb, 0xca, 0xb7, 0x58, 0xbf, 0xef, + 0x7a, 0xbc, 0x13, 0xb5, 0x8c, 0x36, 0x09, 0xcc, 0x2e, 0x25, 0x01, 0xe6, 0x1d, 0x1c, 0x31, 0xb3, + 0x4d, 0x82, 0x80, 0x84, 0xa6, 0x50, 0x37, 0x76, 0xbc, 0x00, 0x0f, 0x62, 0x5d, 0xb8, 0xdb, 0xe2, + 0x17, 0xed, 0xc0, 0xbf, 0x78, 0x87, 0x92, 0xc8, 0xed, 0xa8, 0x53, 0x42, 0xef, 0xe9, 0xef, 0xeb, + 0x65, 0x0a, 0x76, 0x76, 0x41, 0x3a, 0x2c, 0x1d, 0x47, 0x98, 0x9e, 0xa8, 0x85, 0x0a, 0xa8, 0xcd, + 0x5a, 0xb3, 0x83, 0x58, 0x97, 0x80, 0x2d, 0x0f, 0xd4, 0x84, 0xff, 0x73, 0x87, 0xba, 0x98, 0xef, + 0xb7, 0x4e, 0x38, 0x66, 0xfb, 0x5d, 0x4c, 0xf7, 0x59, 0x52, 0xa5, 0x5a, 0xac, 0x80, 0x5a, 0xd1, + 0x5a, 0x1c, 0xc4, 0xfa, 0x3f, 0x92, 0x61, 0x25, 0x84, 0x2d, 0x4c, 0x45, 0x13, 0xec, 0x49, 0x60, + 0xf5, 0x13, 0x80, 0x0b, 0x59, 0x8f, 0x58, 0x97, 0x84, 0x0c, 0xa3, 0x35, 0x38, 0x2d, 0xf4, 0x98, + 0x0a, 0x2a, 0x85, 0xda, 0xdc, 0xea, 0x7f, 0xc6, 0xf5, 0x61, 0x19, 0x82, 0x6f, 0x2d, 0x24, 0xd5, + 0x0e, 0x62, 0x3d, 0x25, 0xdb, 0xe9, 0x89, 0x9e, 0x41, 0x98, 0x0c, 0xc1, 0x63, 0xdc, 0x6b, 0x33, + 0xd1, 0x99, 0xb9, 0xd5, 0x79, 0x43, 0xce, 0xc5, 0xc6, 0x2c, 0xf2, 0xb9, 0x85, 0x52, 0xd7, 0x1c, + 0xd1, 0xce, 0xdd, 0xab, 0xef, 0x01, 0x2c, 0x89, 0x20, 0x68, 0x1d, 0x4e, 0xb7, 0x48, 0x14, 0x8a, + 0x5c, 0x12, 0x21, 0x75, 0x3c, 0x97, 0xfa, 0x96, 0x25, 0xec, 0xa3, 0x74, 0x24, 0xdf, 0x4e, 0x4f, + 0xb4, 0x06, 0x4b, 0x22, 0x76, 0x9a, 0xc9, 0xb2, 0x31, 0x5c, 0xa3, 0x8d, 0x44, 0x69, 0x3b, 0xb1, + 0x65, 0xa5, 0xcb, 0x6e, 0x0b, 0xba, 0x2d, 0x8f, 0xea, 0x17, 0x00, 0x67, 0xb2, 0x18, 0x68, 0x13, + 0x16, 0x02, 0x2f, 0x14, 0xa9, 0x14, 0xad, 0x27, 0x83, 0x58, 0x4f, 0x3e, 0x7f, 0xc6, 0xba, 0x71, + 0x87, 0x81, 0xd7, 0xbd, 0xd0, 0xc5, 0xb4, 0x4b, 0xbd, 0x90, 0xdb, 0x89, 0x9b, 0x10, 0x73, 0x7a, + 0x22, 0xad, 0x4c, 0xcc, 0xe9, 0xfd, 0x91, 0x98, 0xd3, 0x5b, 0xfd, 0x50, 0x82, 0x7f, 0x8b, 0x7a, + 0x1a, 0xb2, 0x33, 0x68, 0x03, 0xc2, 0x57, 0xc9, 0xba, 0x08, 0x10, 0x2d, 0x8d, 0xaa, 0x1e, 0xa1, + 0xe9, 0xa3, 0x28, 0x2f, 0x4f, 0x36, 0xca, 0x96, 0x3c, 0x04, 0xa8, 0x09, 0xe7, 0x1a, 0x98, 0x3f, + 0xef, 0x44, 0xe1, 0x91, 0x8d, 0x0f, 0x51, 0x8e, 0x9e, 0x83, 0x33, 0xb1, 0x95, 0x5b, 0xac, 0x52, + 0xad, 0xaa, 0xa0, 0x3a, 0x9c, 0x6d, 0x60, 0xbe, 0x8d, 0xa9, 0x87, 0x19, 0x2a, 0x5f, 0x63, 0x4b, + 0x30, 0x53, 0x5a, 0x9a, 0x68, 0x1b, 0xea, 0xbc, 0x86, 0x8b, 0x4d, 0xa7, 0x85, 0xfd, 0x97, 0x4e, + 0x80, 0x59, 0x9d, 0xd0, 0x17, 0x98, 0x53, 0xaf, 0x9d, 0x7c, 0xa1, 0xda, 0xc8, 0xf3, 0x16, 0x4a, + 0x16, 0x63, 0x71, 0x8c, 0x99, 0xd3, 0x7f, 0x03, 0x55, 0x01, 0xed, 0x3a, 0x7e, 0x34, 0x1e, 0xe0, + 0xc1, 0x98, 0xdb, 0x04, 0xce, 0x1d, 0x22, 0x34, 0xe0, 0x4c, 0x52, 0x58, 0xb2, 0x66, 0xf9, 0x01, + 0xe5, 0xd7, 0xf2, 0xc6, 0x80, 0x6e, 0xee, 0x6c, 0x55, 0x41, 0xeb, 0xa2, 0xa5, 0xbb, 0xc4, 0x8f, + 0x02, 0x8c, 0x72, 0x01, 0x25, 0x92, 0xa9, 0xa8, 0x37, 0x0d, 0x43, 0x85, 0xa6, 0x1c, 0x8a, 0x7c, + 0xc0, 0x2b, 0x13, 0xdf, 0xfb, 0x30, 0x1b, 0xed, 0x36, 0x73, 0xb6, 0x30, 0xd6, 0xde, 0xd9, 0x85, + 0xa6, 0x9c, 0x5f, 0x68, 0xca, 0xd5, 0x85, 0x06, 0xde, 0xf5, 0x35, 0xf0, 0xb5, 0xaf, 0x81, 0xd3, + 0xbe, 0x06, 0xce, 0xfa, 0x1a, 0xf8, 0xde, 0xd7, 0xc0, 0x8f, 0xbe, 0xa6, 0x5c, 0xf5, 0x35, 0xf0, + 0xf1, 0x52, 0x53, 0xce, 0x2e, 0x35, 0xe5, 0xfc, 0x52, 0x53, 0xf6, 0xee, 0xe5, 0xf6, 0xde, 0xa5, + 0xce, 0xa1, 0x13, 0x3a, 0xa6, 0x4f, 0x8e, 0x3c, 0x33, 0xff, 0x27, 0xd0, 0x9a, 0x16, 0xc7, 0xa3, + 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xf7, 0xbd, 0xfc, 0xaf, 0x62, 0x06, 0x00, 0x00, +} + +func (this *ShardsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ShardsRequest) + if !ok { + that2, ok := that.(ShardsRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.From.Equal(that1.From) { + return false + } + if !this.Through.Equal(that1.Through) { + return false + } + if this.Query != that1.Query { + return false + } + if this.TargetBytesPerShard != that1.TargetBytesPerShard { + return false + } + return true +} +func (this *ShardsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ShardsResponse) + if !ok { + that2, ok := that.(ShardsResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Shards) != len(that1.Shards) { + return false + } + for i := range this.Shards { + if !this.Shards[i].Equal(&that1.Shards[i]) { + return false + } + } + if !this.Statistics.Equal(&that1.Statistics) { + return false + } + return true +} +func (this *Shard) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Shard) + if !ok { + that2, ok := that.(Shard) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Bounds.Equal(&that1.Bounds) { + return false + } + if !this.Stats.Equal(that1.Stats) { + return false + } + return true +} +func (this *FPBounds) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*FPBounds) + if !ok { + that2, ok := that.(FPBounds) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Min != that1.Min { + return false + } + if this.Max != that1.Max { + return false + } + return true +} +func (this *ShardsRequest) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&logproto.ShardsRequest{") + s = append(s, "From: "+fmt.Sprintf("%#v", this.From)+",\n") + s = append(s, "Through: "+fmt.Sprintf("%#v", this.Through)+",\n") + s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") + s = append(s, "TargetBytesPerShard: "+fmt.Sprintf("%#v", this.TargetBytesPerShard)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *ShardsResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&logproto.ShardsResponse{") + if this.Shards != nil { + vs := make([]*Shard, len(this.Shards)) + for i := range vs { + vs[i] = &this.Shards[i] + } + s = append(s, "Shards: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "Statistics: "+strings.Replace(this.Statistics.GoString(), `&`, ``, 1)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Shard) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&logproto.Shard{") + s = append(s, "Bounds: "+strings.Replace(this.Bounds.GoString(), `&`, ``, 1)+",\n") + if this.Stats != nil { + s = append(s, "Stats: "+fmt.Sprintf("%#v", this.Stats)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *FPBounds) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&logproto.FPBounds{") + s = append(s, "Min: "+fmt.Sprintf("%#v", this.Min)+",\n") + s = append(s, "Max: "+fmt.Sprintf("%#v", this.Max)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func valueToGoStringIndexgateway(v interface{}, typ string) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) } // Reference imports to suppress errors if they are not otherwise used. @@ -79,6 +506,9 @@ type IndexGatewayClient interface { // Note: this MUST be the same as the variant defined in // logproto.proto on the Querier service. GetVolume(ctx context.Context, in *VolumeRequest, opts ...grpc.CallOption) (*VolumeResponse, error) + // GetShards is an optimized implemented shard-planning implementation + // on the index gateway and not on the ingester. + GetShards(ctx context.Context, in *ShardsRequest, opts ...grpc.CallOption) (IndexGateway_GetShardsClient, error) } type indexGatewayClient struct { @@ -175,6 +605,38 @@ func (c *indexGatewayClient) GetVolume(ctx context.Context, in *VolumeRequest, o return out, nil } +func (c *indexGatewayClient) GetShards(ctx context.Context, in *ShardsRequest, opts ...grpc.CallOption) (IndexGateway_GetShardsClient, error) { + stream, err := c.cc.NewStream(ctx, &_IndexGateway_serviceDesc.Streams[1], "/indexgatewaypb.IndexGateway/GetShards", opts...) + if err != nil { + return nil, err + } + x := &indexGatewayGetShardsClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type IndexGateway_GetShardsClient interface { + Recv() (*ShardsResponse, error) + grpc.ClientStream +} + +type indexGatewayGetShardsClient struct { + grpc.ClientStream +} + +func (x *indexGatewayGetShardsClient) Recv() (*ShardsResponse, error) { + m := new(ShardsResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // IndexGatewayServer is the server API for IndexGateway service. type IndexGatewayServer interface { /// QueryIndex reads the indexes required for given query & sends back the batch of rows @@ -191,6 +653,9 @@ type IndexGatewayServer interface { // Note: this MUST be the same as the variant defined in // logproto.proto on the Querier service. GetVolume(context.Context, *VolumeRequest) (*VolumeResponse, error) + // GetShards is an optimized implemented shard-planning implementation + // on the index gateway and not on the ingester. + GetShards(*ShardsRequest, IndexGateway_GetShardsServer) error } // UnimplementedIndexGatewayServer can be embedded to have forward compatible implementations. @@ -218,6 +683,9 @@ func (*UnimplementedIndexGatewayServer) GetStats(ctx context.Context, req *Index func (*UnimplementedIndexGatewayServer) GetVolume(ctx context.Context, req *VolumeRequest) (*VolumeResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetVolume not implemented") } +func (*UnimplementedIndexGatewayServer) GetShards(req *ShardsRequest, srv IndexGateway_GetShardsServer) error { + return status.Errorf(codes.Unimplemented, "method GetShards not implemented") +} func RegisterIndexGatewayServer(s *grpc.Server, srv IndexGatewayServer) { s.RegisterService(&_IndexGateway_serviceDesc, srv) @@ -352,6 +820,27 @@ func _IndexGateway_GetVolume_Handler(srv interface{}, ctx context.Context, dec f return interceptor(ctx, in, info, handler) } +func _IndexGateway_GetShards_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(ShardsRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(IndexGatewayServer).GetShards(m, &indexGatewayGetShardsServer{stream}) +} + +type IndexGateway_GetShardsServer interface { + Send(*ShardsResponse) error + grpc.ServerStream +} + +type indexGatewayGetShardsServer struct { + grpc.ServerStream +} + +func (x *indexGatewayGetShardsServer) Send(m *ShardsResponse) error { + return x.ServerStream.SendMsg(m) +} + var _IndexGateway_serviceDesc = grpc.ServiceDesc{ ServiceName: "indexgatewaypb.IndexGateway", HandlerType: (*IndexGatewayServer)(nil), @@ -387,6 +876,912 @@ var _IndexGateway_serviceDesc = grpc.ServiceDesc{ Handler: _IndexGateway_QueryIndex_Handler, ServerStreams: true, }, + { + StreamName: "GetShards", + Handler: _IndexGateway_GetShards_Handler, + ServerStreams: true, + }, }, Metadata: "pkg/logproto/indexgateway.proto", } + +func (m *ShardsRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ShardsRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShardsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.TargetBytesPerShard != 0 { + i = encodeVarintIndexgateway(dAtA, i, uint64(m.TargetBytesPerShard)) + i-- + dAtA[i] = 0x20 + } + if len(m.Query) > 0 { + i -= len(m.Query) + copy(dAtA[i:], m.Query) + i = encodeVarintIndexgateway(dAtA, i, uint64(len(m.Query))) + i-- + dAtA[i] = 0x1a + } + if m.Through != 0 { + i = encodeVarintIndexgateway(dAtA, i, uint64(m.Through)) + i-- + dAtA[i] = 0x10 + } + if m.From != 0 { + i = encodeVarintIndexgateway(dAtA, i, uint64(m.From)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ShardsResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ShardsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShardsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintIndexgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + if len(m.Shards) > 0 { + for iNdEx := len(m.Shards) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Shards[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintIndexgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Shard) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Shard) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Shard) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Stats != nil { + { + size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintIndexgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + { + size, err := m.Bounds.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintIndexgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + +func (m *FPBounds) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *FPBounds) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *FPBounds) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Max != 0 { + i = encodeVarintIndexgateway(dAtA, i, uint64(m.Max)) + i-- + dAtA[i] = 0x10 + } + if m.Min != 0 { + i = encodeVarintIndexgateway(dAtA, i, uint64(m.Min)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintIndexgateway(dAtA []byte, offset int, v uint64) int { + offset -= sovIndexgateway(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *ShardsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.From != 0 { + n += 1 + sovIndexgateway(uint64(m.From)) + } + if m.Through != 0 { + n += 1 + sovIndexgateway(uint64(m.Through)) + } + l = len(m.Query) + if l > 0 { + n += 1 + l + sovIndexgateway(uint64(l)) + } + if m.TargetBytesPerShard != 0 { + n += 1 + sovIndexgateway(uint64(m.TargetBytesPerShard)) + } + return n +} + +func (m *ShardsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Shards) > 0 { + for _, e := range m.Shards { + l = e.Size() + n += 1 + l + sovIndexgateway(uint64(l)) + } + } + l = m.Statistics.Size() + n += 1 + l + sovIndexgateway(uint64(l)) + return n +} + +func (m *Shard) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.Bounds.Size() + n += 1 + l + sovIndexgateway(uint64(l)) + if m.Stats != nil { + l = m.Stats.Size() + n += 1 + l + sovIndexgateway(uint64(l)) + } + return n +} + +func (m *FPBounds) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Min != 0 { + n += 1 + sovIndexgateway(uint64(m.Min)) + } + if m.Max != 0 { + n += 1 + sovIndexgateway(uint64(m.Max)) + } + return n +} + +func sovIndexgateway(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozIndexgateway(x uint64) (n int) { + return sovIndexgateway(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *ShardsRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ShardsRequest{`, + `From:` + fmt.Sprintf("%v", this.From) + `,`, + `Through:` + fmt.Sprintf("%v", this.Through) + `,`, + `Query:` + fmt.Sprintf("%v", this.Query) + `,`, + `TargetBytesPerShard:` + fmt.Sprintf("%v", this.TargetBytesPerShard) + `,`, + `}`, + }, "") + return s +} +func (this *ShardsResponse) String() string { + if this == nil { + return "nil" + } + repeatedStringForShards := "[]Shard{" + for _, f := range this.Shards { + repeatedStringForShards += strings.Replace(strings.Replace(f.String(), "Shard", "Shard", 1), `&`, ``, 1) + "," + } + repeatedStringForShards += "}" + s := strings.Join([]string{`&ShardsResponse{`, + `Shards:` + repeatedStringForShards + `,`, + `Statistics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statistics), "Result", "stats.Result", 1), `&`, ``, 1) + `,`, + `}`, + }, "") + return s +} +func (this *Shard) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Shard{`, + `Bounds:` + strings.Replace(strings.Replace(this.Bounds.String(), "FPBounds", "FPBounds", 1), `&`, ``, 1) + `,`, + `Stats:` + strings.Replace(fmt.Sprintf("%v", this.Stats), "IndexStatsResponse", "IndexStatsResponse", 1) + `,`, + `}`, + }, "") + return s +} +func (this *FPBounds) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&FPBounds{`, + `Min:` + fmt.Sprintf("%v", this.Min) + `,`, + `Max:` + fmt.Sprintf("%v", this.Max) + `,`, + `}`, + }, "") + return s +} +func valueToStringIndexgateway(v interface{}) string { + rv := reflect.ValueOf(v) + if rv.IsNil() { + return "nil" + } + pv := reflect.Indirect(rv).Interface() + return fmt.Sprintf("*%v", pv) +} +func (m *ShardsRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ShardsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) + } + m.From = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.From |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Through", wireType) + } + m.Through = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Through |= github_com_prometheus_common_model.Time(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + 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 ErrInvalidLengthIndexgateway + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthIndexgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Query = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetBytesPerShard", wireType) + } + m.TargetBytesPerShard = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TargetBytesPerShard |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipIndexgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ShardsResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ShardsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthIndexgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthIndexgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, Shard{}) + if err := m.Shards[len(m.Shards)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Statistics", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthIndexgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthIndexgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Statistics.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipIndexgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Shard) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Shard: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Shard: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Bounds", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthIndexgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthIndexgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Bounds.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthIndexgateway + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthIndexgateway + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Stats == nil { + m.Stats = &IndexStatsResponse{} + } + if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipIndexgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FPBounds) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: FPBounds: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FPBounds: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Min", wireType) + } + m.Min = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Min |= github_com_prometheus_common_model.Fingerprint(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Max", wireType) + } + m.Max = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Max |= github_com_prometheus_common_model.Fingerprint(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipIndexgateway(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthIndexgateway + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipIndexgateway(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthIndexgateway + } + iNdEx += length + if iNdEx < 0 { + return 0, ErrInvalidLengthIndexgateway + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowIndexgateway + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipIndexgateway(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + if iNdEx < 0 { + return 0, ErrInvalidLengthIndexgateway + } + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthIndexgateway = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowIndexgateway = fmt.Errorf("proto: integer overflow") +) diff --git a/pkg/logproto/indexgateway.proto b/pkg/logproto/indexgateway.proto index af34e03a279cb..33cfbec317d57 100644 --- a/pkg/logproto/indexgateway.proto +++ b/pkg/logproto/indexgateway.proto @@ -2,7 +2,9 @@ syntax = "proto3"; package indexgatewaypb; +import "gogoproto/gogo.proto"; import "pkg/logproto/logproto.proto"; +import "pkg/logqlmodel/stats/stats.proto"; option go_package = "github.com/grafana/loki/pkg/logproto"; @@ -25,4 +27,55 @@ service IndexGateway { // Note: this MUST be the same as the variant defined in // logproto.proto on the Querier service. rpc GetVolume(logproto.VolumeRequest) returns (logproto.VolumeResponse) {} + + // GetShards is an optimized implemented shard-planning implementation + // on the index gateway and not on the ingester. + rpc GetShards(ShardsRequest) returns (stream ShardsResponse); +} + +message ShardsRequest { + int64 from = 1 [ + (gogoproto.customtype) = "github.com/prometheus/common/model.Time", + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "from" + ]; + int64 through = 2 [ + (gogoproto.customtype) = "github.com/prometheus/common/model.Time", + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "through" + ]; + string query = 3 [(gogoproto.jsontag) = "query"]; + uint64 target_bytes_per_shard = 4 [(gogoproto.jsontag) = "targetBytesPerShard"]; +} + +message ShardsResponse { + repeated Shard shards = 1 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "shards" + ]; + stats.Result statistics = 2 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "statistics" + ]; +} + +message Shard { + FPBounds bounds = 1 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "bounds" + ]; + logproto.IndexStatsResponse stats = 2 [(gogoproto.jsontag) = "stats"]; +} + +// FPBounds is identical to the definition in `pkg/storage/bloom/v1/bounds.FingerprintBounds` +// which ensures we can cast between them without allocations. +message FPBounds { + uint64 min = 1 [ + (gogoproto.casttype) = "github.com/prometheus/common/model.Fingerprint", + (gogoproto.jsontag) = "min" + ]; + uint64 max = 2 [ + (gogoproto.casttype) = "github.com/prometheus/common/model.Fingerprint", + (gogoproto.jsontag) = "max" + ]; } diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index d50ae7d1e5db4..7302df0cfb212 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -386,6 +386,7 @@ func (m *SampleQueryRequest) GetDeletes() []*Delete { return nil } +// TODO(owen-d): fix. This will break rollouts as soon as the internal repr is changed. type Plan struct { Raw []byte `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` } @@ -1740,6 +1741,7 @@ func (m *LabelNamesForMetricNameRequest) GetMetricName() string { return "" } +// TODO(owen-d): fix. This will break rollouts as soon as the internal repr is changed. type LineFilter struct { Raw []byte `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` } diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index bf175168cfd93..c50246a1b57b5 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -83,6 +83,7 @@ message SampleQueryRequest { Plan plan = 6 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; } +// TODO(owen-d): fix. This will break rollouts as soon as the internal repr is changed. message Plan { bytes raw = 1; } @@ -297,6 +298,7 @@ message LabelNamesForMetricNameRequest { ]; } +// TODO(owen-d): fix. This will break rollouts as soon as the internal repr is changed. message LineFilter { bytes raw = 1; } diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 11f94132a4bf6..5dea1144d9a18 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -16,7 +16,6 @@ import ( "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/logqlmodel/metadata" "github.com/grafana/loki/pkg/logqlmodel/stats" - "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/util" util_log "github.com/grafana/loki/pkg/util/log" ) @@ -75,7 +74,7 @@ func (ng *DownstreamEngine) Query(ctx context.Context, p Params) Query { // DownstreamSampleExpr is a SampleExpr which signals downstream computation type DownstreamSampleExpr struct { - shard *astmapper.ShardAnnotation + shard *Shard syntax.SampleExpr } @@ -108,7 +107,7 @@ func (d DownstreamSampleExpr) Pretty(level int) string { // DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation type DownstreamLogSelectorExpr struct { - shard *astmapper.ShardAnnotation + shard *Shard syntax.LogSelectorExpr } @@ -302,50 +301,6 @@ func (e *QuantileSketchMergeExpr) Walk(f syntax.WalkFn) { } } -type Shards []astmapper.ShardAnnotation - -func (xs Shards) Encode() (encoded []string) { - for _, shard := range xs { - encoded = append(encoded, shard.String()) - } - - return encoded -} - -// ParseShards parses a list of string encoded shards -func ParseShards(strs []string) (Shards, error) { - if len(strs) == 0 { - return nil, nil - } - shards := make([]astmapper.ShardAnnotation, 0, len(strs)) - - for _, str := range strs { - shard, err := astmapper.ParseShard(str) - if err != nil { - return nil, err - } - shards = append(shards, shard) - } - return shards, nil -} - -func ParseShardCount(strs []string) int { - if len(strs) == 0 { - return 0 - } - - for _, str := range strs { - shard, err := astmapper.ParseShard(str) - if err != nil { - continue - } - - return shard.Of - } - - return 0 -} - type Downstreamable interface { Downstreamer(context.Context) Downstreamer } @@ -435,7 +390,7 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( case DownstreamSampleExpr: // downstream to a querier - var shards []astmapper.ShardAnnotation + var shards Shards if e.shard != nil { shards = append(shards, *e.shard) } @@ -443,7 +398,7 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( results, err := ev.Downstream(ctx, []DownstreamQuery{{ Params: ParamsWithShardsOverride{ Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: e.SampleExpr}, - ShardsOverride: Shards(shards).Encode(), + ShardsOverride: shards.Encode(), }, }}, acc) if err != nil { diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index b3b8c6f37e48a..68afe83cceead 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -96,7 +96,8 @@ func TestMappingEquivalence(t *testing.T) { qry := regular.Query(params) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{}) + strategy := NewPowerOfTwoStrategy(ConstantShards(shards)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{}) // TODO (callum) refactor this test so that we won't need to set every // possible sharding config option to true when we have multiple in the future if tc.approximate { @@ -166,7 +167,8 @@ func TestMappingEquivalenceSketches(t *testing.T) { qry := regular.Query(params) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(shards)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) _, _, mapped, err := mapper.Parse(params.GetExpression()) require.NoError(t, err) @@ -200,7 +202,8 @@ func TestMappingEquivalenceSketches(t *testing.T) { qry := regular.Query(params) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(shards)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) _, _, mapped, err := mapper.Parse(params.GetExpression()) require.NoError(t, err) @@ -265,7 +268,8 @@ func TestShardCounter(t *testing.T) { require.NoError(t, err) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(shards)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) noop, _, mapped, err := mapper.Parse(params.GetExpression()) require.NoError(t, err) @@ -620,10 +624,10 @@ func TestFormat_ShardedExpr(t *testing.T) { name: "ConcatSampleExpr", in: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 3, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -636,10 +640,10 @@ func TestFormat_ShardedExpr(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 3, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -652,10 +656,10 @@ func TestFormat_ShardedExpr(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 3, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -701,7 +705,8 @@ func TestPrettierWithoutShards(t *testing.T) { q := `((quantile_over_time(0.5,{foo="bar"} | json | unwrap bytes[1d]) by (cluster) > 42) and (count by (cluster)(max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace)) > 10))` e := syntax.MustParseExpr(q) - mapper := NewShardMapper(ConstantShards(4), nilShardMetrics, []string{}) + strategy := NewPowerOfTwoStrategy(ConstantShards(4)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{}) _, _, mapped, err := mapper.Parse(e) require.NoError(t, err) got := syntax.Prettify(mapped) @@ -738,42 +743,3 @@ and 10` assert.Equal(t, expected, got) } - -func TestParseShardCount(t *testing.T) { - for _, st := range []struct { - name string - shards []string - expected int - }{ - { - name: "empty shards", - shards: []string{}, - expected: 0, - }, - { - name: "single shard", - shards: []string{"0_of_3"}, - expected: 3, - }, - { - name: "single shard with error", - shards: []string{"0_of_"}, - expected: 0, - }, - { - name: "multiple shards", - shards: []string{"0_of_3", "0_of_4"}, - expected: 3, - }, - { - name: "multiple shards with errors", - shards: []string{"_of_3", "0_of_4"}, - expected: 4, - }, - } { - t.Run(st.name, func(t *testing.T) { - require.Equal(t, st.expected, ParseShardCount(st.shards)) - }) - - } -} diff --git a/pkg/logql/explain_test.go b/pkg/logql/explain_test.go index 307aa10cfa98d..84364b633c228 100644 --- a/pkg/logql/explain_test.go +++ b/pkg/logql/explain_test.go @@ -28,7 +28,8 @@ func TestExplain(t *testing.T) { defaultEv := NewDefaultEvaluator(querier, 30*time.Second) downEv := &DownstreamEvaluator{Downstreamer: MockDownstreamer{regular}, defaultEvaluator: defaultEv} - mapper := NewShardMapper(ConstantShards(4), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(4)) + mapper := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) _, _, expr, err := mapper.Parse(syntax.MustParseExpr(query)) require.NoError(t, err) diff --git a/pkg/logql/mapper_metrics.go b/pkg/logql/mapper_metrics.go index 3588231700a63..4ec8cb8454c78 100644 --- a/pkg/logql/mapper_metrics.go +++ b/pkg/logql/mapper_metrics.go @@ -42,9 +42,10 @@ func newMapperMetrics(registerer prometheus.Registerer, mapper string) *MapperMe ConstLabels: prometheus.Labels{"mapper": mapper}, }, []string{"type"}), DownstreamFactor: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{ - Namespace: constants.Loki, - Name: "query_frontend_shard_factor", - Help: "Number of downstream queries per request", + Namespace: constants.Loki, + Name: "query_frontend_shard_factor", + Help: "Number of downstream queries per request", + // 1 -> 65k shards Buckets: prometheus.ExponentialBuckets(1, 4, 8), ConstLabels: prometheus.Labels{"mapper": mapper}, }), diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 1399ac15ef9f5..bd5bfec77db5e 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -2,10 +2,12 @@ package logql import ( "context" + "fmt" "strconv" "strings" "time" + "github.com/c2h5oh/datasize" "github.com/dustin/go-humanize" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -32,6 +34,7 @@ const ( QueryTypeLabels = "labels" QueryTypeSeries = "series" QueryTypeStats = "stats" + QueryTypeShards = "shards" QueryTypeVolume = "volume" latencyTypeSlow = "slow" @@ -128,6 +131,11 @@ func RecordRangeAndInstantQueryMetrics( logValues := make([]interface{}, 0, 50) + var bloomRatio float64 // what % are filtered + if stats.Index.TotalChunks > 0 { + bloomRatio = float64(stats.Index.TotalChunks-stats.Index.PostFilterChunks) / float64(stats.Index.TotalChunks) + } + logValues = append(logValues, []interface{}{ "latency", latencyType, // this can be used to filter log lines. "query", query, @@ -191,6 +199,9 @@ func RecordRangeAndInstantQueryMetrics( "ingester_post_filter_lines", stats.Ingester.Store.Chunk.GetPostFilterLines(), // Time spent being blocked on congestion control. "congestion_control_latency", stats.CongestionControlLatency(), + "index_total_chunks", stats.Index.TotalChunks, + "index_post_bloom_filter_chunks", stats.Index.PostFilterChunks, + "index_bloom_filter_ratio", fmt.Sprintf("%.2f", bloomRatio), }...) logValues = append(logValues, tagsToKeyValues(queryTags)...) @@ -370,6 +381,58 @@ func RecordStatsQueryMetrics(ctx context.Context, log log.Logger, start, end tim execLatency.WithLabelValues(status, queryType, "").Observe(stats.Summary.ExecTime) } +func RecordShardsQueryMetrics( + ctx context.Context, + log log.Logger, + start, + end time.Time, + query string, + targetBytesPerShard uint64, + status string, + shards int, + stats logql_stats.Result, +) { + var ( + logger = fixLogger(ctx, log) + latencyType = latencyTypeFast + queryType = QueryTypeShards + ) + + // Tag throughput metric by latency type based on a threshold. + // Latency below the threshold is fast, above is slow. + if stats.Summary.ExecTime > slowQueryThresholdSecond { + latencyType = latencyTypeSlow + } + + var bloomRatio float64 // what % are filtered + if stats.Index.TotalChunks > 0 { + bloomRatio = float64(stats.Index.TotalChunks-stats.Index.PostFilterChunks) / float64(stats.Index.TotalChunks) + } + logValues := make([]interface{}, 0, 15) + logValues = append(logValues, + "latency", latencyType, + "query_type", queryType, + "start", start.Format(time.RFC3339Nano), + "end", end.Format(time.RFC3339Nano), + "start_delta", time.Since(start), + "end_delta", time.Since(end), + "length", end.Sub(start), + "duration", time.Duration(int64(stats.Summary.ExecTime*float64(time.Second))), + "status", status, + "query", query, + "query_hash", util.HashedQuery(query), + "target_bytes_per_shard", datasize.ByteSize(targetBytesPerShard).HumanReadable(), + "shards", shards, + "index_total_chunks", stats.Index.TotalChunks, + "index_post_bloom_filter_chunks", stats.Index.PostFilterChunks, + "index_bloom_filter_ratio", fmt.Sprintf("%.2f", bloomRatio), + ) + + level.Info(logger).Log(logValues...) + + execLatency.WithLabelValues(status, queryType, "").Observe(stats.Summary.ExecTime) +} + func RecordVolumeQueryMetrics(ctx context.Context, log log.Logger, start, end time.Time, query string, limit uint32, step time.Duration, status string, stats logql_stats.Result) { var ( logger = fixLogger(ctx, log) diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index a8fb04c826a6a..a1c17c86da036 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -7,35 +7,22 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/grafana/loki/pkg/util/math" - "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/querier/astmapper" - "github.com/grafana/loki/pkg/storage/stores/index/stats" util_log "github.com/grafana/loki/pkg/util/log" ) -type ShardResolver interface { - Shards(expr syntax.Expr) (int, uint64, error) - GetStats(e syntax.Expr) (stats.Stats, error) -} - -type ConstantShards int - -func (s ConstantShards) Shards(_ syntax.Expr) (int, uint64, error) { return int(s), 0, nil } -func (s ConstantShards) GetStats(_ syntax.Expr) (stats.Stats, error) { return stats.Stats{}, nil } - const ( ShardQuantileOverTime = "quantile_over_time" ) type ShardMapper struct { - shards ShardResolver + shards ShardingStrategy metrics *MapperMetrics quantileOverTimeSharding bool } -func NewShardMapper(resolver ShardResolver, metrics *MapperMetrics, shardAggregation []string) ShardMapper { +func NewShardMapper(strategy ShardingStrategy, metrics *MapperMetrics, shardAggregation []string) ShardMapper { quantileOverTimeSharding := false for _, a := range shardAggregation { if a == ShardQuantileOverTime { @@ -43,7 +30,7 @@ func NewShardMapper(resolver ShardResolver, metrics *MapperMetrics, shardAggrega } } return ShardMapper{ - shards: resolver, + shards: strategy, metrics: metrics, quantileOverTimeSharding: quantileOverTimeSharding, } @@ -148,71 +135,70 @@ func (m ShardMapper) mapBinOpExpr(e *syntax.BinOpExpr, r *downstreamRecorder, to e.RHS = rhsSampleExpr // We take the maximum bytes per shard of both sides of the operation - bytesPerShard := uint64(math.Max(int(lhsBytesPerShard), int(rhsBytesPerShard))) + bytesPerShard := uint64(max(int(lhsBytesPerShard), int(rhsBytesPerShard))) return e, bytesPerShard, nil } func (m ShardMapper) mapLogSelectorExpr(expr syntax.LogSelectorExpr, r *downstreamRecorder) (syntax.LogSelectorExpr, uint64, error) { var head *ConcatLogSelectorExpr - shards, bytesPerShard, err := m.shards.Shards(expr) + shards, maxBytesPerShard, err := m.shards.Shards(expr) if err != nil { return nil, 0, err } - if shards == 0 { + if len(shards) == 0 { return &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: nil, LogSelectorExpr: expr, }, - }, bytesPerShard, nil + }, maxBytesPerShard, nil } - for i := shards - 1; i >= 0; i-- { + + for i := len(shards) - 1; i >= 0; i-- { head = &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ - Shard: i, - Of: shards, - }, + shard: &shards[i], LogSelectorExpr: expr, }, next: head, } } - r.Add(shards, StreamsKey) - return head, bytesPerShard, nil + r.Add(len(shards), StreamsKey) + return head, maxBytesPerShard, nil } func (m ShardMapper) mapSampleExpr(expr syntax.SampleExpr, r *downstreamRecorder) (syntax.SampleExpr, uint64, error) { var head *ConcatSampleExpr - shards, bytesPerShard, err := m.shards.Shards(expr) + shards, maxBytesPerShard, err := m.shards.Shards(expr) + if err != nil { return nil, 0, err } - if shards == 0 { + + if len(shards) == 0 { return &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ shard: nil, SampleExpr: expr, }, - }, bytesPerShard, nil + }, maxBytesPerShard, nil } - for shard := shards - 1; shard >= 0; shard-- { + + for i := len(shards) - 1; i >= 0; i-- { head = &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ - Shard: shard, - Of: shards, - }, + shard: &shards[i], SampleExpr: expr, }, next: head, } } - r.Add(shards, MetricsKey) - return head, bytesPerShard, nil + r.Add(len(shards), MetricsKey) + + return head, maxBytesPerShard, nil } // turn a vector aggr into a wrapped+sharded variant, @@ -273,7 +259,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr } // We take the maximum bytes per shard of both sides of the operation - bytesPerShard := uint64(math.Max(int(lhsBytesPerShard), int(rhsBytesPerShard))) + bytesPerShard := uint64(max(int(lhsBytesPerShard), int(rhsBytesPerShard))) return &syntax.BinOpExpr{ SampleExpr: lhs, @@ -307,7 +293,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *syntax.VectorAggregationExpr "msg", "unexpected operation which appears shardable, ignoring", "operation", expr.Operation, ) - exprStats, err := m.shards.GetStats(expr) + exprStats, err := m.shards.Resolver().GetStats(expr) if err != nil { return nil, 0, err } @@ -366,7 +352,7 @@ var rangeMergeMap = map[string]string{ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, r *downstreamRecorder, topLevel bool) (syntax.SampleExpr, uint64, error) { if !expr.Shardable(topLevel) { - return noOp(expr, m.shards) + return noOp(expr, m.shards.Resolver()) } switch expr.Operation { @@ -442,7 +428,7 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, } // We take the maximum bytes per shard of both sides of the operation - bytesPerShard := uint64(math.Max(int(lhsBytesPerShard), int(rhsBytesPerShard))) + bytesPerShard := uint64(max(int(lhsBytesPerShard), int(rhsBytesPerShard))) return &syntax.BinOpExpr{ SampleExpr: lhs, @@ -452,7 +438,7 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, case syntax.OpRangeTypeQuantile: if !m.quantileOverTimeSharding { - return noOp(expr, m.shards) + return noOp(expr, m.shards.Resolver()) } potentialConflict := syntax.ReducesLabels(expr) @@ -460,12 +446,15 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, return m.mapSampleExpr(expr, r) } - shards, bytesPerShard, err := m.shards.Shards(expr) + // TODO(owen-d): integrate bounded sharding with quantile over time + // I'm not doing this now because it uses a separate code path and may not handle + // bounded shards in the same way + shards, bytesPerShard, err := m.shards.Resolver().Shards(expr) if err != nil { return nil, 0, err } if shards == 0 { - return noOp(expr, m.shards) + return noOp(expr, m.shards.Resolver()) } // quantile_over_time() by (foo) -> @@ -475,11 +464,12 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, downstreams := make([]DownstreamSampleExpr, 0, shards) expr.Operation = syntax.OpRangeTypeQuantileSketch for shard := shards - 1; shard >= 0; shard-- { + s := NewPowerOfTwoShard(astmapper.ShardAnnotation{ + Shard: shard, + Of: shards, + }) downstreams = append(downstreams, DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ - Shard: shard, - Of: shards, - }, + shard: &s, SampleExpr: expr, }) } @@ -493,7 +483,7 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, default: // don't shard if there's not an appropriate optimization - return noOp(expr, m.shards) + return noOp(expr, m.shards.Resolver()) } } diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index e7fbb9d5204c3..472bc51806041 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -20,20 +20,20 @@ func TestShardedStringer(t *testing.T) { { in: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, }, next: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -51,7 +51,9 @@ func TestShardedStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) + + strategy := NewPowerOfTwoStrategy(ConstantShards(2)) + m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) for _, tc := range []struct { in syntax.SampleExpr @@ -69,10 +71,10 @@ func TestMapSampleExpr(t *testing.T) { }, out: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -85,10 +87,10 @@ func TestMapSampleExpr(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -113,7 +115,8 @@ func TestMapSampleExpr(t *testing.T) { } func TestMappingStrings(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(2)) + m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) for _, tc := range []struct { in string out string @@ -452,7 +455,8 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) { }, } { t.Run(tc.in, func(t *testing.T) { - shardedMapper := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) + + shardedMapper := NewShardMapper(NewPowerOfTwoStrategy(ConstantShards(2)), nilShardMetrics, []string{ShardQuantileOverTime}) ast, err := syntax.ParseExpr(tc.in) require.Nil(t, err) @@ -462,7 +466,7 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) { require.Equal(t, removeWhiteSpace(tc.out), removeWhiteSpace(sharded.String())) - unshardedMapper := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{}) + unshardedMapper := NewShardMapper(NewPowerOfTwoStrategy(ConstantShards(2)), nilShardMetrics, []string{}) ast, err = syntax.ParseExpr(tc.in) require.Nil(t, err) @@ -476,7 +480,8 @@ func TestMappingStrings_NoProbabilisticSharding(t *testing.T) { } func TestMapping(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{}) + strategy := NewPowerOfTwoStrategy(ConstantShards(2)) + m := NewShardMapper(strategy, nilShardMetrics, []string{}) for _, tc := range []struct { in string @@ -487,20 +492,20 @@ func TestMapping(t *testing.T) { in: `{foo="bar"}`, expr: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, }, next: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -513,10 +518,10 @@ func TestMapping(t *testing.T) { in: `{foo="bar"} |= "error"`, expr: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.PipelineExpr{ Left: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, @@ -534,10 +539,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatLogSelectorExpr{ DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), LogSelectorExpr: &syntax.PipelineExpr{ Left: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, @@ -561,10 +566,10 @@ func TestMapping(t *testing.T) { in: `rate({foo="bar"}[5m])`, expr: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -577,10 +582,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -599,10 +604,10 @@ func TestMapping(t *testing.T) { in: `count_over_time({foo="bar"}[5m])`, expr: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeCount, Left: &syntax.LogRange{ @@ -615,10 +620,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeCount, Left: &syntax.LogRange{ @@ -640,10 +645,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -660,10 +665,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -691,10 +696,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeTopK, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -707,10 +712,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -733,10 +738,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -753,10 +758,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -785,10 +790,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -805,10 +810,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -832,10 +837,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -852,10 +857,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -892,10 +897,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -914,10 +919,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -954,10 +959,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -974,10 +979,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -1009,10 +1014,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1029,10 +1034,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1071,10 +1076,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1093,10 +1098,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1123,10 +1128,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1143,10 +1148,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1185,10 +1190,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1207,10 +1212,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1236,10 +1241,10 @@ func TestMapping(t *testing.T) { Grouping: &syntax.Grouping{}, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1256,10 +1261,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1291,10 +1296,10 @@ func TestMapping(t *testing.T) { Operation: syntax.OpTypeSum, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1316,10 +1321,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1350,10 +1355,10 @@ func TestMapping(t *testing.T) { }, Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1372,10 +1377,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1461,10 +1466,10 @@ func TestMapping(t *testing.T) { RHS: &syntax.VectorAggregationExpr{ Left: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 0, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Left: &syntax.RangeAggregationExpr{ Left: &syntax.LogRange{ @@ -1484,10 +1489,10 @@ func TestMapping(t *testing.T) { }, next: &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ + shard: NewPowerOfTwoShard(astmapper.ShardAnnotation{ Shard: 1, Of: 2, - }, + }).Ptr(), SampleExpr: &syntax.VectorAggregationExpr{ Left: &syntax.RangeAggregationExpr{ Left: &syntax.LogRange{ @@ -1661,7 +1666,8 @@ func TestStringTrimming(t *testing.T) { }, } { t.Run(tc.expr.String(), func(t *testing.T) { - m := NewShardMapper(ConstantShards(tc.shards), nilShardMetrics, []string{ShardQuantileOverTime}) + strategy := NewPowerOfTwoStrategy(ConstantShards(tc.shards)) + m := NewShardMapper(strategy, nilShardMetrics, []string{ShardQuantileOverTime}) _, _, mappedExpr, err := m.Parse(tc.expr) require.Nil(t, err) require.Equal(t, removeWhiteSpace(tc.expected), removeWhiteSpace(mappedExpr.String())) @@ -1680,7 +1686,7 @@ func TestShardTopk(t *testing.T) { sum_over_time({job="foo"} | json | unwrap bytes(bytes)[1m]) ) )` - m := NewShardMapper(ConstantShards(5), nilShardMetrics, []string{ShardQuantileOverTime}) + m := NewShardMapper(NewPowerOfTwoStrategy(ConstantShards(5)), nilShardMetrics, []string{ShardQuantileOverTime}) _, _, mappedExpr, err := m.Parse(syntax.MustParseExpr(expr)) require.NoError(t, err) diff --git a/pkg/logql/shards.go b/pkg/logql/shards.go new file mode 100644 index 0000000000000..7ca7f67cb367d --- /dev/null +++ b/pkg/logql/shards.go @@ -0,0 +1,248 @@ +package logql + +import ( + "encoding/json" + + "github.com/grafana/dskit/multierror" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/astmapper" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" +) + +type Shards []Shard + +type ShardVersion uint8 + +const ( + PowerOfTwoVersion ShardVersion = iota + BoundedVersion +) + +func (v ShardVersion) Strategy(resolver ShardResolver, defaultTargetShardBytes uint64) ShardingStrategy { + switch v { + case BoundedVersion: + return NewDynamicBoundsStrategy(resolver, defaultTargetShardBytes) + default: + // TODO(owen-d): refactor, ugly, etc, but the power of two strategy already populated + // the default target shard bytes through it's resolver + return NewPowerOfTwoStrategy(resolver) + } +} + +func (v ShardVersion) String() string { + switch v { + case PowerOfTwoVersion: + return "power_of_two" + case BoundedVersion: + return "bounded" + default: + return "unknown" + } +} + +var validStrategies = map[string]ShardVersion{ + PowerOfTwoVersion.String(): PowerOfTwoVersion, + BoundedVersion.String(): BoundedVersion, +} + +func ParseShardVersion(s string) (ShardVersion, error) { + v, ok := validStrategies[s] + if !ok { + return PowerOfTwoVersion, errors.Errorf("invalid shard version %s", s) + } + return v, nil +} + +type ShardResolver interface { + Shards(expr syntax.Expr) (int, uint64, error) + ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ([]logproto.Shard, error) + GetStats(e syntax.Expr) (stats.Stats, error) +} + +type ConstantShards int + +func (s ConstantShards) Shards(_ syntax.Expr) (int, uint64, error) { return int(s), 0, nil } +func (s ConstantShards) ShardingRanges(_ syntax.Expr, _ uint64) ([]logproto.Shard, error) { + return sharding.LinearShards(int(s), 0), nil +} +func (s ConstantShards) GetStats(_ syntax.Expr) (stats.Stats, error) { return stats.Stats{}, nil } + +type ShardingStrategy interface { + Shards(expr syntax.Expr) (shards Shards, maxBytesPerShard uint64, err error) + Resolver() ShardResolver +} + +type DynamicBoundsStrategy struct { + resolver ShardResolver + targetBytesPerShard uint64 +} + +func (s DynamicBoundsStrategy) Shards(expr syntax.Expr) (Shards, uint64, error) { + shards, err := s.resolver.ShardingRanges(expr, s.targetBytesPerShard) + if err != nil { + return nil, 0, err + } + + var maxBytes uint64 + res := make(Shards, 0, len(shards)) + for _, shard := range shards { + if shard.Stats != nil { + maxBytes = max(maxBytes, shard.Stats.Bytes) + } + res = append(res, NewBoundedShard(shard)) + } + + return res, maxBytes, nil +} + +func (s DynamicBoundsStrategy) Resolver() ShardResolver { + return s.resolver +} + +func NewDynamicBoundsStrategy(resolver ShardResolver, targetBytesPerShard uint64) DynamicBoundsStrategy { + return DynamicBoundsStrategy{resolver: resolver, targetBytesPerShard: targetBytesPerShard} +} + +type PowerOfTwoStrategy struct { + resolver ShardResolver +} + +func NewPowerOfTwoStrategy(resolver ShardResolver) PowerOfTwoStrategy { + return PowerOfTwoStrategy{resolver: resolver} +} + +func (s PowerOfTwoStrategy) Resolver() ShardResolver { + return s.resolver +} + +func (s PowerOfTwoStrategy) Shards(expr syntax.Expr) (Shards, uint64, error) { + factor, bytesPerShard, err := s.resolver.Shards(expr) + if err != nil { + return nil, 0, err + } + + if factor == 0 { + return nil, bytesPerShard, nil + } + + res := make(Shards, 0, factor) + for i := 0; i < factor; i++ { + res = append(res, NewPowerOfTwoShard(astmapper.ShardAnnotation{Of: factor, Shard: i})) + } + return res, bytesPerShard, nil +} + +// Shard represents a shard annotation +// It holds either a power of two shard (legacy) or a bounded shard +type Shard struct { + PowerOfTwo *astmapper.ShardAnnotation + Bounded *logproto.Shard +} + +func (s *Shard) Variant() ShardVersion { + if s.Bounded != nil { + return BoundedVersion + } + + return PowerOfTwoVersion +} + +// implement FingerprintFilter +func (s *Shard) Match(fp model.Fingerprint) bool { + if s.Bounded != nil { + return v1.BoundsFromProto(s.Bounded.Bounds).Match(fp) + } + + return s.PowerOfTwo.Match(fp) +} + +func (s *Shard) GetFromThrough() (model.Fingerprint, model.Fingerprint) { + if s.Bounded != nil { + return v1.BoundsFromProto(s.Bounded.Bounds).GetFromThrough() + } + + return s.PowerOfTwo.TSDB().GetFromThrough() +} + +// convenience method for unaddressability concerns using constructors in literals (tests) +func (s Shard) Ptr() *Shard { + return &s +} + +func NewBoundedShard(shard logproto.Shard) Shard { + return Shard{Bounded: &shard} +} + +func NewPowerOfTwoShard(shard astmapper.ShardAnnotation) Shard { + return Shard{PowerOfTwo: &shard} +} + +func (s Shard) String() string { + if s.Bounded != nil { + b, err := json.Marshal(s.Bounded) + if err != nil { + panic(err) + } + return string(b) + } + + return s.PowerOfTwo.String() +} + +func (xs Shards) Encode() (encoded []string) { + for _, shard := range xs { + encoded = append(encoded, shard.String()) + } + + return encoded +} + +// ParseShards parses a list of string encoded shards +func ParseShards(strs []string) (Shards, ShardVersion, error) { + if len(strs) == 0 { + return nil, PowerOfTwoVersion, nil + } + shards := make(Shards, 0, len(strs)) + + var prevVersion ShardVersion + for i, str := range strs { + shard, version, err := ParseShard(str) + if err != nil { + return nil, PowerOfTwoVersion, err + } + + if i == 0 { + prevVersion = version + } else if prevVersion != version { + return nil, PowerOfTwoVersion, errors.New("shards must be of the same version") + } + shards = append(shards, shard) + } + return shards, prevVersion, nil +} + +func ParseShard(s string) (Shard, ShardVersion, error) { + + var bounded logproto.Shard + v2Err := json.Unmarshal([]byte(s), &bounded) + if v2Err == nil { + return Shard{Bounded: &bounded}, BoundedVersion, nil + } + + old, v1Err := astmapper.ParseShard(s) + if v1Err == nil { + return Shard{PowerOfTwo: &old}, PowerOfTwoVersion, nil + } + + err := errors.Wrap( + multierror.New(v1Err, v2Err).Err(), + "failed to parse shard", + ) + return Shard{}, PowerOfTwoVersion, err +} diff --git a/pkg/logql/shards_test.go b/pkg/logql/shards_test.go new file mode 100644 index 0000000000000..fd0adb35f881f --- /dev/null +++ b/pkg/logql/shards_test.go @@ -0,0 +1,188 @@ +package logql + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/querier/astmapper" +) + +func TestShardString(t *testing.T) { + for _, rc := range []struct { + shard Shard + exp string + }{ + { + shard: Shard{ + PowerOfTwo: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + }, + exp: "1_of_2", + }, + { + shard: Shard{ + Bounded: &logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }, + }, + exp: `{"bounds":{"min":1,"max":2},"stats":null}`, + }, + { + shard: Shard{ + Bounded: &logproto.Shard{ + Stats: &logproto.IndexStatsResponse{ + Bytes: 1, + }, + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }, + }, + exp: `{"bounds":{"min":1,"max":2},"stats":{"streams":0,"chunks":0,"bytes":1,"entries":0}}`, + }, + { + // when more than one are present, + // return the newest successful version (v2) + shard: Shard{ + Bounded: &logproto.Shard{ + Stats: &logproto.IndexStatsResponse{ + Bytes: 1, + }, + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }, + PowerOfTwo: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + }, + exp: `{"bounds":{"min":1,"max":2},"stats":{"streams":0,"chunks":0,"bytes":1,"entries":0}}`, + }, + } { + t.Run(fmt.Sprintf("%+v", rc.shard), func(t *testing.T) { + require.Equal(t, rc.exp, rc.shard.String()) + }) + } +} + +func TestParseShard(t *testing.T) { + for _, rc := range []struct { + str string + version ShardVersion + exp Shard + }{ + { + str: "1_of_2", + version: PowerOfTwoVersion, + exp: Shard{ + PowerOfTwo: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + }, + }, + { + str: `{"bounds":{"min":1,"max":2},"stats":null}`, + version: BoundedVersion, + exp: Shard{ + Bounded: &logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }, + }, + }, + { + str: `{"bounds":{"min":1,"max":2},"stats":{"streams":0,"chunks":0,"bytes":1,"entries":0}}`, + version: BoundedVersion, + exp: Shard{ + Bounded: &logproto.Shard{ + Stats: &logproto.IndexStatsResponse{ + Bytes: 1, + }, + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }, + }, + }, + } { + t.Run(rc.str, func(t *testing.T) { + shard, version, err := ParseShard(rc.str) + require.NoError(t, err) + require.Equal(t, rc.version, version) + require.Equal(t, rc.exp, shard) + }) + } +} + +func TestParseShards(t *testing.T) { + for _, rc := range []struct { + strs []string + version ShardVersion + exp Shards + err bool + }{ + { + strs: []string{"1_of_2", "1_of_2"}, + version: PowerOfTwoVersion, + exp: Shards{ + NewPowerOfTwoShard(astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }), + NewPowerOfTwoShard(astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }), + }, + }, + { + strs: []string{`{"bounds":{"min":1,"max":2},"stats":null}`, `{"bounds":{"min":1,"max":2},"stats":null}`}, + version: BoundedVersion, + exp: Shards{ + NewBoundedShard(logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }), + NewBoundedShard(logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: 1, + Max: 2, + }, + }), + }, + }, + { + strs: []string{`{"bounds":{"min":1,"max":2},"stats":null}`, "1_of_2"}, + version: PowerOfTwoVersion, + err: true, + }, + } { + t.Run(fmt.Sprintf("%+v", rc.strs), func(t *testing.T) { + shards, version, err := ParseShards(rc.strs) + if rc.err { + require.Error(t, err) + return + } + require.NoError(t, err) + require.Equal(t, rc.version, version) + require.Equal(t, rc.exp, shards) + }) + } +} diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 72b8429e11bf9..7f41e45be60d2 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -34,6 +34,20 @@ type MockQuerier struct { streams []logproto.Stream } +func (q MockQuerier) extractOldShard(xs []string) (*astmapper.ShardAnnotation, error) { + parsed, version, err := ParseShards(xs) + if err != nil { + return nil, err + } + + if version != PowerOfTwoVersion { + return nil, fmt.Errorf("unsupported shard version: %d", version) + } + + return parsed[0].PowerOfTwo, nil + +} + func (q MockQuerier) SelectLogs(_ context.Context, req SelectLogParams) (iter.EntryIterator, error) { expr, err := req.LogSelector() if err != nil { @@ -48,11 +62,10 @@ func (q MockQuerier) SelectLogs(_ context.Context, req SelectLogParams) (iter.En var shard *astmapper.ShardAnnotation if len(req.Shards) > 0 { - shards, err := ParseShards(req.Shards) + shard, err = q.extractOldShard(req.Shards) if err != nil { return nil, err } - shard = &shards[0] } var matched []logproto.Stream @@ -174,11 +187,10 @@ func (q MockQuerier) SelectSamples(_ context.Context, req SelectSampleParams) (i var shard *astmapper.ShardAnnotation if len(req.Shards) > 0 { - shards, err := ParseShards(req.Shards) + shard, err = q.extractOldShard(req.Shards) if err != nil { return nil, err } - shard = &shards[0] } var matched []logproto.Stream diff --git a/pkg/logqlmodel/stats/context.go b/pkg/logqlmodel/stats/context.go index 31344b01c585d..7f2b9cfaf40bb 100644 --- a/pkg/logqlmodel/stats/context.go +++ b/pkg/logqlmodel/stats/context.go @@ -43,6 +43,7 @@ type Context struct { querier Querier ingester Ingester caches Caches + index Index // store is the store statistics collected across the query path store Store @@ -115,6 +116,11 @@ func (c *Context) Caches() Caches { } } +// Index returns the index statistics accumulated so far. +func (c *Context) Index() Index { + return c.index +} + // Reset clears the statistics. func (c *Context) Reset() { c.mtx.Lock() @@ -125,6 +131,7 @@ func (c *Context) Reset() { c.ingester.Reset() c.result.Reset() c.caches.Reset() + c.index.Reset() } // Result calculates the summary based on store and ingester data. @@ -137,6 +144,7 @@ func (c *Context) Result(execTime time.Duration, queueTime time.Duration, totalE }, Ingester: c.ingester, Caches: c.caches, + Index: c.index, }) r.ComputeSummary(execTime, queueTime, totalEntriesReturned) @@ -162,6 +170,15 @@ func JoinIngesters(ctx context.Context, inc Ingester) { stats.ingester.Merge(inc) } +// JoinIndex joins the index statistics in a concurrency-safe manner. +func JoinIndex(ctx context.Context, index Index) { + stats := FromContext(ctx) + stats.mtx.Lock() + defer stats.mtx.Unlock() + + stats.index.Merge(index) +} + // ComputeSummary compute the summary of the statistics. func (r *Result) ComputeSummary(execTime time.Duration, queueTime time.Duration, totalEntriesReturned int) { r.Summary.TotalBytesProcessed = r.Querier.Store.Chunk.DecompressedBytes + r.Querier.Store.Chunk.HeadChunkBytes + @@ -226,6 +243,11 @@ func (i *Ingester) Merge(m Ingester) { i.TotalReached += m.TotalReached } +func (i *Index) Merge(m Index) { + i.TotalChunks += m.TotalChunks + i.PostFilterChunks += m.PostFilterChunks +} + func (c *Caches) Merge(m Caches) { c.Chunk.Merge(m.Chunk) c.Index.Merge(m.Index) @@ -267,6 +289,7 @@ func (r *Result) Merge(m Result) { r.Ingester.Merge(m.Ingester) r.Caches.Merge(m.Caches) r.Summary.Merge(m.Summary) + r.Index.Merge(m.Index) r.ComputeSummary(ConvertSecondsToNanoseconds(r.Summary.ExecTime+m.Summary.ExecTime), ConvertSecondsToNanoseconds(r.Summary.QueueTime+m.Summary.QueueTime), int(r.Summary.TotalEntriesReturned)) } diff --git a/pkg/logqlmodel/stats/stats.pb.go b/pkg/logqlmodel/stats/stats.pb.go index f25143272599b..e26bedf55d33b 100644 --- a/pkg/logqlmodel/stats/stats.pb.go +++ b/pkg/logqlmodel/stats/stats.pb.go @@ -32,6 +32,7 @@ type Result struct { Querier Querier `protobuf:"bytes,2,opt,name=querier,proto3" json:"querier"` Ingester Ingester `protobuf:"bytes,3,opt,name=ingester,proto3" json:"ingester"` Caches Caches `protobuf:"bytes,4,opt,name=caches,proto3" json:"cache"` + Index Index `protobuf:"bytes,5,opt,name=index,proto3" json:"index"` } func (m *Result) Reset() { *m = Result{} } @@ -94,6 +95,13 @@ func (m *Result) GetCaches() Caches { return Caches{} } +func (m *Result) GetIndex() Index { + if m != nil { + return m.Index + } + return Index{} +} + type Caches struct { Chunk Cache `protobuf:"bytes,1,opt,name=chunk,proto3" json:"chunk"` Index Cache `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` @@ -342,6 +350,62 @@ func (m *Summary) GetTotalStructuredMetadataBytesProcessed() int64 { return 0 } +// Statistics from Index queries +// TODO(owen-d): include bytes. +// Needs some index methods added to return _sized_ chunk refs to know +type Index struct { + // Total chunks + TotalChunks int64 `protobuf:"varint,1,opt,name=totalChunks,proto3" json:"totalChunks"` + // Post-filtered chunks + PostFilterChunks int64 `protobuf:"varint,2,opt,name=postFilterChunks,proto3" json:"postFilterChunks"` +} + +func (m *Index) Reset() { *m = Index{} } +func (*Index) ProtoMessage() {} +func (*Index) Descriptor() ([]byte, []int) { + return fileDescriptor_6cdfe5d2aea33ebb, []int{3} +} +func (m *Index) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Index) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Index.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Index) XXX_Merge(src proto.Message) { + xxx_messageInfo_Index.Merge(m, src) +} +func (m *Index) XXX_Size() int { + return m.Size() +} +func (m *Index) XXX_DiscardUnknown() { + xxx_messageInfo_Index.DiscardUnknown(m) +} + +var xxx_messageInfo_Index proto.InternalMessageInfo + +func (m *Index) GetTotalChunks() int64 { + if m != nil { + return m.TotalChunks + } + return 0 +} + +func (m *Index) GetPostFilterChunks() int64 { + if m != nil { + return m.PostFilterChunks + } + return 0 +} + type Querier struct { Store Store `protobuf:"bytes,1,opt,name=store,proto3" json:"store"` } @@ -349,7 +413,7 @@ type Querier struct { func (m *Querier) Reset() { *m = Querier{} } func (*Querier) ProtoMessage() {} func (*Querier) Descriptor() ([]byte, []int) { - return fileDescriptor_6cdfe5d2aea33ebb, []int{3} + return fileDescriptor_6cdfe5d2aea33ebb, []int{4} } func (m *Querier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +464,7 @@ type Ingester struct { func (m *Ingester) Reset() { *m = Ingester{} } func (*Ingester) ProtoMessage() {} func (*Ingester) Descriptor() ([]byte, []int) { - return fileDescriptor_6cdfe5d2aea33ebb, []int{4} + return fileDescriptor_6cdfe5d2aea33ebb, []int{5} } func (m *Ingester) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -483,7 +547,7 @@ type Store struct { func (m *Store) Reset() { *m = Store{} } func (*Store) ProtoMessage() {} func (*Store) Descriptor() ([]byte, []int) { - return fileDescriptor_6cdfe5d2aea33ebb, []int{5} + return fileDescriptor_6cdfe5d2aea33ebb, []int{6} } func (m *Store) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -585,7 +649,7 @@ type Chunk struct { func (m *Chunk) Reset() { *m = Chunk{} } func (*Chunk) ProtoMessage() {} func (*Chunk) Descriptor() ([]byte, []int) { - return fileDescriptor_6cdfe5d2aea33ebb, []int{6} + return fileDescriptor_6cdfe5d2aea33ebb, []int{7} } func (m *Chunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -691,7 +755,7 @@ type Cache struct { func (m *Cache) Reset() { *m = Cache{} } func (*Cache) ProtoMessage() {} func (*Cache) Descriptor() ([]byte, []int) { - return fileDescriptor_6cdfe5d2aea33ebb, []int{7} + return fileDescriptor_6cdfe5d2aea33ebb, []int{8} } func (m *Cache) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -780,6 +844,7 @@ func init() { proto.RegisterType((*Result)(nil), "stats.Result") proto.RegisterType((*Caches)(nil), "stats.Caches") proto.RegisterType((*Summary)(nil), "stats.Summary") + proto.RegisterType((*Index)(nil), "stats.Index") proto.RegisterType((*Querier)(nil), "stats.Querier") proto.RegisterType((*Ingester)(nil), "stats.Ingester") proto.RegisterType((*Store)(nil), "stats.Store") @@ -790,87 +855,90 @@ func init() { func init() { proto.RegisterFile("pkg/logqlmodel/stats/stats.proto", fileDescriptor_6cdfe5d2aea33ebb) } var fileDescriptor_6cdfe5d2aea33ebb = []byte{ - // 1274 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0x4b, 0x6f, 0xe3, 0x54, - 0x14, 0x8e, 0x27, 0x75, 0xd2, 0xde, 0x3e, 0xe7, 0xb6, 0xc3, 0x64, 0x18, 0x64, 0x97, 0xc0, 0x88, - 0x22, 0x50, 0x23, 0x1e, 0x12, 0x02, 0x31, 0x12, 0x72, 0x87, 0x4a, 0x95, 0x5a, 0x51, 0x4e, 0x40, - 0x42, 0xb0, 0x72, 0xec, 0xdb, 0xc4, 0xaa, 0x63, 0xa7, 0xf6, 0x75, 0x99, 0xae, 0xe0, 0x27, 0xb0, - 0x62, 0xc3, 0x1f, 0x60, 0xc3, 0x8a, 0x15, 0x6b, 0x36, 0xb3, 0xec, 0x72, 0x56, 0x16, 0x4d, 0x37, - 0xc8, 0xab, 0x91, 0xf8, 0x03, 0xe8, 0x3e, 0xe2, 0x57, 0xec, 0x99, 0x6c, 0xea, 0x7b, 0xbe, 0xf3, - 0x7d, 0xe7, 0x3e, 0x72, 0xce, 0xb9, 0xb7, 0x68, 0x77, 0x72, 0x3e, 0xec, 0xb9, 0xfe, 0xf0, 0xc2, - 0x1d, 0xfb, 0x36, 0x71, 0x7b, 0x21, 0x35, 0x69, 0x28, 0xfe, 0xee, 0x4f, 0x02, 0x9f, 0xfa, 0x58, - 0xe5, 0xc6, 0xeb, 0x3b, 0x43, 0x7f, 0xe8, 0x73, 0xa4, 0xc7, 0x46, 0xc2, 0xd9, 0xfd, 0x4f, 0x41, - 0x2d, 0x20, 0x61, 0xe4, 0x52, 0xfc, 0x29, 0x6a, 0x87, 0xd1, 0x78, 0x6c, 0x06, 0x57, 0x1d, 0x65, - 0x57, 0xd9, 0x5b, 0xfd, 0x70, 0x63, 0x5f, 0x84, 0xe9, 0x0b, 0xd4, 0xd8, 0x7c, 0x16, 0xeb, 0x8d, - 0x24, 0xd6, 0x67, 0x34, 0x98, 0x0d, 0x98, 0xf4, 0x22, 0x22, 0x81, 0x43, 0x82, 0xce, 0x9d, 0x82, - 0xf4, 0x6b, 0x81, 0x66, 0x52, 0x49, 0x83, 0xd9, 0x00, 0x3f, 0x46, 0xcb, 0x8e, 0x37, 0x24, 0x21, - 0x25, 0x41, 0xa7, 0xc9, 0xb5, 0x9b, 0x52, 0x7b, 0x24, 0x61, 0x63, 0x4b, 0x8a, 0x53, 0x22, 0xa4, - 0x23, 0xfc, 0x31, 0x6a, 0x59, 0xa6, 0x35, 0x22, 0x61, 0x67, 0x89, 0x8b, 0xd7, 0xa5, 0xf8, 0x80, - 0x83, 0xc6, 0xba, 0x94, 0xaa, 0x9c, 0x04, 0x92, 0xdb, 0xfd, 0x6d, 0x09, 0xb5, 0x04, 0x03, 0x7f, - 0x80, 0x54, 0x6b, 0x14, 0x79, 0xe7, 0x72, 0xcf, 0x6b, 0x79, 0x7d, 0x4e, 0xce, 0x28, 0x20, 0x3e, - 0x4c, 0xe2, 0x78, 0x36, 0x79, 0x2a, 0xf7, 0x5a, 0x23, 0xe1, 0x14, 0x10, 0x1f, 0xb6, 0xcc, 0x80, - 0x9f, 0xb2, 0xdc, 0x63, 0x51, 0xb3, 0x21, 0x35, 0x92, 0x03, 0xf2, 0x8b, 0x0f, 0xd0, 0x2a, 0xa7, - 0x89, 0x1f, 0x48, 0xee, 0xb0, 0x28, 0xdd, 0x96, 0xd2, 0x3c, 0x11, 0xf2, 0x06, 0x3e, 0x44, 0x6b, - 0x97, 0xbe, 0x1b, 0x8d, 0x89, 0x8c, 0xa2, 0x56, 0x44, 0xd9, 0x91, 0x51, 0x0a, 0x4c, 0x28, 0x58, - 0x2c, 0x4e, 0xc8, 0x7e, 0xb2, 0xd9, 0x6a, 0x5a, 0x2f, 0x8b, 0x93, 0x67, 0x42, 0xc1, 0x62, 0x9b, - 0x72, 0xcd, 0x01, 0x71, 0x65, 0x98, 0xf6, 0xcb, 0x36, 0x95, 0x23, 0x42, 0xde, 0xc0, 0x3f, 0xa0, - 0x6d, 0xc7, 0x0b, 0xa9, 0xe9, 0xd1, 0x13, 0x42, 0x03, 0xc7, 0x92, 0xc1, 0x96, 0x2b, 0x82, 0x3d, - 0x94, 0xc1, 0xaa, 0x04, 0x50, 0x05, 0x76, 0xff, 0x6a, 0xa1, 0xb6, 0xcc, 0x79, 0xfc, 0x2d, 0xba, - 0x3f, 0xb8, 0xa2, 0x24, 0x3c, 0x0d, 0x7c, 0x8b, 0x84, 0x21, 0xb1, 0x4f, 0x49, 0xd0, 0x27, 0x96, - 0xef, 0xd9, 0x3c, 0x61, 0x9a, 0xc6, 0xc3, 0x24, 0xd6, 0xeb, 0x28, 0x50, 0xe7, 0x60, 0x61, 0x5d, - 0xc7, 0xab, 0x0c, 0x7b, 0x27, 0x0b, 0x5b, 0x43, 0x81, 0x3a, 0x07, 0x3e, 0x42, 0xdb, 0xd4, 0xa7, - 0xa6, 0x6b, 0x14, 0xa6, 0xe5, 0x39, 0xd7, 0x34, 0xee, 0xb3, 0x43, 0xa8, 0x70, 0x43, 0x15, 0x98, - 0x86, 0x3a, 0x2e, 0x4c, 0xc5, 0x73, 0x30, 0x1f, 0xaa, 0xe8, 0x86, 0x2a, 0x10, 0xef, 0xa1, 0x65, - 0xf2, 0x94, 0x58, 0xdf, 0x38, 0x63, 0xc2, 0xb3, 0x4f, 0x31, 0xd6, 0x58, 0x35, 0xcf, 0x30, 0x48, - 0x47, 0xf8, 0x3d, 0xb4, 0x72, 0x11, 0x91, 0x88, 0x70, 0x6a, 0x8b, 0x53, 0xd7, 0x93, 0x58, 0xcf, - 0x40, 0xc8, 0x86, 0x78, 0x1f, 0xa1, 0x30, 0x1a, 0x88, 0x3e, 0x12, 0xf2, 0x3c, 0x6a, 0x1a, 0x1b, - 0x49, 0xac, 0xe7, 0x50, 0xc8, 0x8d, 0xf1, 0x31, 0xda, 0xe1, 0xab, 0xfb, 0xd2, 0xa3, 0x22, 0x1d, - 0x69, 0x14, 0x78, 0xc4, 0xe6, 0x49, 0xd3, 0x34, 0x3a, 0x49, 0xac, 0x57, 0xfa, 0xa1, 0x12, 0xc5, - 0x5d, 0xd4, 0x0a, 0x27, 0xae, 0x43, 0xc3, 0xce, 0x0a, 0xd7, 0x23, 0x56, 0xbf, 0x02, 0x01, 0xf9, - 0xe5, 0x9c, 0x91, 0x19, 0xd8, 0x61, 0x07, 0xe5, 0x38, 0x1c, 0x01, 0xf9, 0x4d, 0x57, 0x75, 0xea, - 0x87, 0xf4, 0xd0, 0x71, 0x29, 0x09, 0xf8, 0xe9, 0x75, 0x56, 0x4b, 0xab, 0x2a, 0xf9, 0xa1, 0x12, - 0xc5, 0x3f, 0xa1, 0x47, 0x1c, 0xef, 0xd3, 0x20, 0xb2, 0x68, 0x14, 0x10, 0xfb, 0x84, 0x50, 0xd3, - 0x36, 0xa9, 0x59, 0x4a, 0x89, 0x35, 0x1e, 0xfe, 0xdd, 0x24, 0xd6, 0x17, 0x13, 0xc0, 0x62, 0xb4, - 0xee, 0xe7, 0xa8, 0x2d, 0x7b, 0x3e, 0x6b, 0x93, 0x21, 0xf5, 0x03, 0x52, 0xea, 0xac, 0x7d, 0x86, - 0x65, 0x6d, 0x92, 0x53, 0x40, 0x7c, 0xba, 0x7f, 0xdc, 0x41, 0xcb, 0x47, 0x59, 0x6b, 0x5f, 0xe3, - 0x73, 0x02, 0x61, 0x75, 0x2c, 0xea, 0x4d, 0x35, 0xb6, 0x58, 0x7b, 0xc9, 0xe3, 0x50, 0xb0, 0xf0, - 0x21, 0xc2, 0xdc, 0x3e, 0x60, 0xad, 0x3a, 0x3c, 0x31, 0x29, 0xd7, 0x8a, 0xa2, 0x7a, 0x2d, 0x89, - 0xf5, 0x0a, 0x2f, 0x54, 0x60, 0xe9, 0xec, 0x06, 0xb7, 0x43, 0x59, 0x43, 0xd9, 0xec, 0x12, 0x87, - 0x82, 0x85, 0x3f, 0x43, 0x1b, 0x59, 0x05, 0xf4, 0x89, 0x47, 0x65, 0xc1, 0xe0, 0x24, 0xd6, 0x4b, - 0x1e, 0x28, 0xd9, 0xd9, 0x79, 0xa9, 0x0b, 0x9f, 0xd7, 0xaf, 0x4b, 0x48, 0xe5, 0xfe, 0x74, 0x62, - 0xb1, 0x09, 0x20, 0x67, 0xb2, 0x3d, 0x65, 0x13, 0xa7, 0x1e, 0x28, 0xd9, 0xf8, 0x2b, 0x74, 0x2f, - 0x87, 0x3c, 0xf1, 0x7f, 0xf4, 0x5c, 0xdf, 0xb4, 0xd3, 0x53, 0x7b, 0x90, 0xc4, 0x7a, 0x35, 0x01, - 0xaa, 0x61, 0xf6, 0x1b, 0x58, 0x05, 0x8c, 0xd7, 0x73, 0x33, 0xfb, 0x0d, 0xe6, 0xbd, 0x50, 0x81, - 0x61, 0x0b, 0x3d, 0x60, 0xc5, 0x7b, 0x05, 0xe4, 0x8c, 0x04, 0xc4, 0xb3, 0x88, 0x9d, 0xe5, 0x5f, - 0x67, 0x7d, 0x57, 0xd9, 0x5b, 0x36, 0x1e, 0x25, 0xb1, 0xfe, 0x66, 0x2d, 0x69, 0x96, 0xa4, 0x50, - 0x1f, 0x27, 0x7b, 0x00, 0x94, 0xae, 0x57, 0x86, 0xd5, 0x3c, 0x00, 0x66, 0xfb, 0x03, 0x72, 0x16, - 0x1e, 0x12, 0x6a, 0x8d, 0xd2, 0xd6, 0x96, 0xdf, 0x5f, 0xc1, 0x0b, 0x15, 0x18, 0xfe, 0x0e, 0x75, - 0x2c, 0x9f, 0xa7, 0xbb, 0xe3, 0x7b, 0x07, 0xbe, 0x47, 0x03, 0xdf, 0x3d, 0x36, 0x29, 0xf1, 0xac, - 0x2b, 0xde, 0xfd, 0x9a, 0xc6, 0x1b, 0x49, 0xac, 0xd7, 0x72, 0xa0, 0xd6, 0xd3, 0xfd, 0x53, 0x45, - 0x2a, 0xdf, 0x01, 0x4b, 0x8c, 0x11, 0x31, 0x6d, 0xb1, 0x1d, 0x56, 0xab, 0xf9, 0x8c, 0x2c, 0x7a, - 0xa0, 0x64, 0x17, 0xb4, 0xa2, 0x2b, 0xa9, 0x15, 0x5a, 0xd1, 0x8f, 0x4a, 0x36, 0x3e, 0x40, 0x77, - 0x6d, 0x62, 0xf9, 0xe3, 0x49, 0xc0, 0x1b, 0x83, 0x98, 0x5a, 0x6c, 0xea, 0x5e, 0x12, 0xeb, 0xf3, - 0x4e, 0x98, 0x87, 0xca, 0x41, 0xc4, 0x1a, 0xda, 0xd5, 0x41, 0xc4, 0x32, 0xe6, 0x21, 0xfc, 0x18, - 0x6d, 0x96, 0xd7, 0x21, 0x5a, 0xfe, 0x76, 0x12, 0xeb, 0x65, 0x17, 0x94, 0x01, 0x26, 0xe7, 0x59, - 0xfe, 0x24, 0x9a, 0xb8, 0x8e, 0x65, 0x32, 0xf9, 0x4a, 0x26, 0x2f, 0xb9, 0xa0, 0x0c, 0x30, 0xf9, - 0xa4, 0xd4, 0xda, 0x51, 0x26, 0x2f, 0xb9, 0xa0, 0x0c, 0xe0, 0x09, 0xda, 0x4d, 0x0f, 0xb6, 0xa6, - 0xf9, 0xca, 0xab, 0xe2, 0xed, 0x24, 0xd6, 0x5f, 0xc9, 0x85, 0x57, 0x32, 0xf0, 0x15, 0x7a, 0x2b, - 0x7f, 0x86, 0x75, 0x93, 0x8a, 0x0b, 0xe4, 0x9d, 0x24, 0xd6, 0x17, 0xa1, 0xc3, 0x22, 0xa4, 0xee, - 0xdf, 0x4d, 0xa4, 0xf2, 0x47, 0x1b, 0xeb, 0xbe, 0x44, 0x5c, 0xb8, 0x87, 0x7e, 0xe4, 0x15, 0x7a, - 0x7f, 0x1e, 0x87, 0x82, 0x85, 0xbf, 0x40, 0x5b, 0x64, 0x76, 0x4d, 0x5f, 0x44, 0xec, 0x16, 0x11, - 0x3d, 0x4c, 0x35, 0x76, 0x92, 0x58, 0x9f, 0xf3, 0xc1, 0x1c, 0x82, 0x3f, 0x41, 0xeb, 0x12, 0xe3, - 0x6d, 0x55, 0x3c, 0x9d, 0x54, 0xe3, 0x6e, 0x12, 0xeb, 0x45, 0x07, 0x14, 0x4d, 0x26, 0xe4, 0x6f, - 0x3d, 0x20, 0x16, 0x71, 0x2e, 0xd3, 0x87, 0x12, 0x17, 0x16, 0x1c, 0x50, 0x34, 0xd9, 0x93, 0x87, - 0x03, 0xfc, 0xb2, 0x10, 0xe5, 0xc5, 0x9f, 0x3c, 0x29, 0x08, 0xd9, 0x90, 0xbd, 0xa4, 0x02, 0xb1, - 0x56, 0x51, 0x4b, 0xaa, 0x78, 0x49, 0xcd, 0x30, 0x48, 0x47, 0xec, 0x00, 0xed, 0x7c, 0xf3, 0x6d, - 0x67, 0xd7, 0x57, 0x1e, 0x87, 0x82, 0xc5, 0xea, 0x8d, 0x37, 0xca, 0x63, 0xe2, 0x0d, 0xe9, 0xa8, - 0x4f, 0x82, 0xcb, 0xf4, 0x7d, 0xc4, 0xeb, 0x6d, 0xce, 0x09, 0xf3, 0x90, 0x31, 0xb8, 0xbe, 0xd1, - 0x1a, 0xcf, 0x6f, 0xb4, 0xc6, 0x8b, 0x1b, 0x4d, 0xf9, 0x79, 0xaa, 0x29, 0xbf, 0x4f, 0x35, 0xe5, - 0xd9, 0x54, 0x53, 0xae, 0xa7, 0x9a, 0xf2, 0xcf, 0x54, 0x53, 0xfe, 0x9d, 0x6a, 0x8d, 0x17, 0x53, - 0x4d, 0xf9, 0xe5, 0x56, 0x6b, 0x5c, 0xdf, 0x6a, 0x8d, 0xe7, 0xb7, 0x5a, 0xe3, 0xfb, 0xf7, 0x87, - 0x0e, 0x1d, 0x45, 0x83, 0x7d, 0xcb, 0x1f, 0xf7, 0x86, 0x81, 0x79, 0x66, 0x7a, 0x66, 0xcf, 0xf5, - 0xcf, 0x9d, 0x5e, 0xd5, 0xff, 0xb7, 0x83, 0x16, 0xff, 0xef, 0xf5, 0xa3, 0xff, 0x03, 0x00, 0x00, - 0xff, 0xff, 0x39, 0xf0, 0xb8, 0xbf, 0xfe, 0x0e, 0x00, 0x00, + // 1325 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0xcf, 0x8f, 0xdb, 0x44, + 0x14, 0x4e, 0x36, 0x75, 0xb2, 0x9d, 0xcd, 0xee, 0xb6, 0xb3, 0x5b, 0x9a, 0x52, 0x64, 0x2f, 0x81, + 0x8a, 0x22, 0xd0, 0x46, 0x05, 0x24, 0x04, 0xa2, 0x12, 0x72, 0xca, 0x4a, 0x95, 0xb6, 0xa2, 0xbc, + 0x80, 0x84, 0xe0, 0xe4, 0xd8, 0xb3, 0x89, 0x55, 0xc7, 0xce, 0xda, 0xe3, 0xd2, 0x95, 0x90, 0xe0, + 0x4f, 0xe0, 0xc4, 0x85, 0x3b, 0xe2, 0xc2, 0x89, 0x13, 0x67, 0x2e, 0x3d, 0xf6, 0xd8, 0x93, 0x45, + 0xd3, 0x0b, 0xf2, 0xa9, 0x7f, 0x02, 0x9a, 0x37, 0x13, 0xff, 0x8a, 0xd3, 0xee, 0x25, 0x9e, 0xf7, + 0xbd, 0xef, 0x7b, 0xf3, 0xf3, 0xbd, 0x99, 0x90, 0x83, 0xf9, 0x83, 0xc9, 0xc0, 0x0b, 0x26, 0xa7, + 0xde, 0x2c, 0x70, 0x98, 0x37, 0x88, 0xb8, 0xc5, 0x23, 0xf9, 0x7b, 0x38, 0x0f, 0x03, 0x1e, 0x50, + 0x0d, 0x8d, 0xd7, 0xf7, 0x27, 0xc1, 0x24, 0x40, 0x64, 0x20, 0x5a, 0xd2, 0xd9, 0xff, 0x7d, 0x83, + 0xb4, 0x81, 0x45, 0xb1, 0xc7, 0xe9, 0x27, 0xa4, 0x13, 0xc5, 0xb3, 0x99, 0x15, 0x9e, 0xf5, 0x9a, + 0x07, 0xcd, 0x9b, 0x5b, 0x1f, 0xec, 0x1c, 0xca, 0x30, 0x23, 0x89, 0x9a, 0xbb, 0x8f, 0x13, 0xa3, + 0x91, 0x26, 0xc6, 0x92, 0x06, 0xcb, 0x86, 0x90, 0x9e, 0xc6, 0x2c, 0x74, 0x59, 0xd8, 0xdb, 0x28, + 0x49, 0xbf, 0x92, 0x68, 0x2e, 0x55, 0x34, 0x58, 0x36, 0xe8, 0x6d, 0xb2, 0xe9, 0xfa, 0x13, 0x16, + 0x71, 0x16, 0xf6, 0x5a, 0xa8, 0xdd, 0x55, 0xda, 0xbb, 0x0a, 0x36, 0x2f, 0x29, 0x71, 0x46, 0x84, + 0xac, 0x45, 0x3f, 0x22, 0x6d, 0xdb, 0xb2, 0xa7, 0x2c, 0xea, 0x5d, 0x40, 0xf1, 0xb6, 0x12, 0x0f, + 0x11, 0x34, 0xb7, 0x95, 0x54, 0x43, 0x12, 0x28, 0x2e, 0xbd, 0x45, 0x34, 0xd7, 0x77, 0xd8, 0xa3, + 0x9e, 0x86, 0xa2, 0x6e, 0xd6, 0xa3, 0xc3, 0x1e, 0xe5, 0x1a, 0xa4, 0x80, 0xfc, 0xf4, 0x7f, 0xbb, + 0x40, 0xda, 0xc3, 0x4c, 0x6d, 0x4f, 0x63, 0xff, 0x81, 0x5a, 0xa6, 0x6e, 0xb1, 0xcb, 0x42, 0x8f, + 0x82, 0x02, 0xf2, 0x93, 0x77, 0xb8, 0xf1, 0x32, 0x49, 0xb1, 0x43, 0x31, 0xb3, 0x10, 0x37, 0x46, + 0x2d, 0x4b, 0x59, 0xb3, 0xa3, 0x34, 0x8a, 0x03, 0xea, 0x4b, 0x87, 0x64, 0x0b, 0x69, 0x72, 0x4f, + 0xd5, 0xa2, 0x94, 0xa5, 0x7b, 0x4a, 0x5a, 0x24, 0x42, 0xd1, 0xa0, 0x47, 0xa4, 0xfb, 0x30, 0xf0, + 0xe2, 0x19, 0x53, 0x51, 0xb4, 0x9a, 0x28, 0xfb, 0x2a, 0x4a, 0x89, 0x09, 0x25, 0x4b, 0xc4, 0x89, + 0xc4, 0x2e, 0x2f, 0x47, 0xd3, 0x7e, 0x59, 0x9c, 0x22, 0x13, 0x4a, 0x96, 0x98, 0x94, 0x67, 0x8d, + 0x99, 0xa7, 0xc2, 0x74, 0x5e, 0x36, 0xa9, 0x02, 0x11, 0x8a, 0x06, 0xfd, 0x9e, 0xec, 0xb9, 0x7e, + 0xc4, 0x2d, 0x9f, 0xdf, 0x63, 0x3c, 0x74, 0x6d, 0x15, 0x6c, 0xb3, 0x26, 0xd8, 0x75, 0x15, 0xac, + 0x4e, 0x00, 0x75, 0x60, 0xff, 0xef, 0x36, 0xe9, 0xa8, 0x34, 0xa1, 0xdf, 0x90, 0xab, 0xe3, 0x33, + 0xce, 0xa2, 0xfb, 0x61, 0x60, 0xb3, 0x28, 0x62, 0xce, 0x7d, 0x16, 0x8e, 0x98, 0x1d, 0xf8, 0x0e, + 0x1e, 0x98, 0x96, 0x79, 0x3d, 0x4d, 0x8c, 0x75, 0x14, 0x58, 0xe7, 0x10, 0x61, 0x3d, 0xd7, 0xaf, + 0x0d, 0xbb, 0x91, 0x87, 0x5d, 0x43, 0x81, 0x75, 0x0e, 0x7a, 0x97, 0xec, 0xf1, 0x80, 0x5b, 0x9e, + 0x59, 0xea, 0x16, 0xcf, 0x5c, 0xcb, 0xbc, 0x2a, 0x16, 0xa1, 0xc6, 0x0d, 0x75, 0x60, 0x16, 0xea, + 0xb8, 0xd4, 0x15, 0x9e, 0xc1, 0x62, 0xa8, 0xb2, 0x1b, 0xea, 0x40, 0x7a, 0x93, 0x6c, 0xb2, 0x47, + 0xcc, 0xfe, 0xda, 0x9d, 0x31, 0x3c, 0x7d, 0x4d, 0xb3, 0x2b, 0x0a, 0xc0, 0x12, 0x83, 0xac, 0x45, + 0xdf, 0x23, 0x17, 0x4f, 0x63, 0x16, 0x33, 0xa4, 0xb6, 0x91, 0xba, 0x9d, 0x26, 0x46, 0x0e, 0x42, + 0xde, 0xa4, 0x87, 0x84, 0x44, 0xf1, 0x58, 0x96, 0x9e, 0x08, 0xcf, 0x51, 0xcb, 0xdc, 0x49, 0x13, + 0xa3, 0x80, 0x42, 0xa1, 0x4d, 0x8f, 0xc9, 0x3e, 0x8e, 0xee, 0x0b, 0x9f, 0xcb, 0xe3, 0xc8, 0xe3, + 0xd0, 0x67, 0x0e, 0x1e, 0x9a, 0x96, 0xd9, 0x4b, 0x13, 0xa3, 0xd6, 0x0f, 0xb5, 0x28, 0xed, 0x93, + 0x76, 0x34, 0xf7, 0x5c, 0x1e, 0xf5, 0x2e, 0xa2, 0x9e, 0x88, 0xfc, 0x95, 0x08, 0xa8, 0x2f, 0x72, + 0xa6, 0x56, 0xe8, 0x44, 0x3d, 0x52, 0xe0, 0x20, 0x02, 0xea, 0x9b, 0x8d, 0xea, 0x7e, 0x10, 0xf1, + 0x23, 0xd7, 0xe3, 0x2c, 0xc4, 0xd5, 0xeb, 0x6d, 0x55, 0x46, 0x55, 0xf1, 0x43, 0x2d, 0x4a, 0x7f, + 0x22, 0x37, 0x10, 0x1f, 0xf1, 0x30, 0xb6, 0x79, 0x1c, 0x32, 0xe7, 0x1e, 0xe3, 0x96, 0x63, 0x71, + 0xab, 0x72, 0x24, 0xba, 0x18, 0xfe, 0xdd, 0x34, 0x31, 0xce, 0x27, 0x80, 0xf3, 0xd1, 0xfa, 0x3f, + 0x12, 0x0d, 0x0b, 0x2f, 0xbd, 0x45, 0xb6, 0x50, 0x31, 0x14, 0x25, 0x33, 0x52, 0xc9, 0xb2, 0x2b, + 0x92, 0xba, 0x00, 0x43, 0xd1, 0xa0, 0x9f, 0x93, 0x4b, 0xf3, 0x6c, 0x3e, 0x4a, 0x27, 0xb3, 0x61, + 0x3f, 0x4d, 0x8c, 0x15, 0x1f, 0xac, 0x20, 0xfd, 0xcf, 0x48, 0x47, 0x5d, 0x52, 0xa2, 0x48, 0x47, + 0x3c, 0x08, 0x59, 0xa5, 0xae, 0x8f, 0x04, 0x96, 0x17, 0x69, 0xa4, 0x80, 0xfc, 0xf4, 0xff, 0xdc, + 0x20, 0x9b, 0x77, 0xf3, 0xbb, 0xa8, 0x8b, 0x63, 0x03, 0x26, 0xaa, 0x88, 0xcc, 0x76, 0xcd, 0xbc, + 0x24, 0x8a, 0x5b, 0x11, 0x87, 0x92, 0x45, 0x8f, 0x08, 0x2d, 0xcc, 0xe8, 0x9e, 0xc5, 0x51, 0x2b, + 0x27, 0xf1, 0x5a, 0x9a, 0x18, 0x35, 0x5e, 0xa8, 0xc1, 0xb2, 0xde, 0x4d, 0xb4, 0x23, 0x95, 0xc1, + 0x79, 0xef, 0x0a, 0x87, 0x92, 0x45, 0x3f, 0x25, 0x3b, 0x79, 0xfe, 0x8d, 0x98, 0xcf, 0x55, 0xba, + 0xd2, 0x34, 0x31, 0x2a, 0x1e, 0xa8, 0xd8, 0xf9, 0x7a, 0x69, 0xe7, 0x5e, 0xaf, 0x5f, 0x2f, 0x10, + 0x0d, 0xfd, 0x59, 0xc7, 0x6a, 0x63, 0xd8, 0x89, 0xda, 0xef, 0xbc, 0xe3, 0xcc, 0x03, 0x15, 0x9b, + 0x7e, 0x49, 0xae, 0x14, 0x90, 0x3b, 0xc1, 0x0f, 0xbe, 0x17, 0x58, 0x4e, 0xb6, 0x6a, 0xd7, 0xd2, + 0xc4, 0xa8, 0x27, 0x40, 0x3d, 0x2c, 0xf6, 0xc0, 0x2e, 0x61, 0x58, 0x4d, 0x5a, 0xf9, 0x1e, 0xac, + 0x7a, 0xa1, 0x06, 0xa3, 0x36, 0xb9, 0x26, 0x4a, 0xc7, 0x19, 0xb0, 0x13, 0x16, 0x32, 0xdf, 0x66, + 0x4e, 0x7e, 0xfa, 0x7b, 0xdb, 0x07, 0xcd, 0x9b, 0x9b, 0xe6, 0x8d, 0x34, 0x31, 0xde, 0x5c, 0x4b, + 0x5a, 0xa6, 0x08, 0xac, 0x8f, 0x93, 0x3f, 0x3f, 0x2a, 0x97, 0xbb, 0xc0, 0xd6, 0x3c, 0x3f, 0x96, + 0xf3, 0x03, 0x76, 0x12, 0x1d, 0x31, 0x6e, 0x4f, 0xb3, 0xc2, 0x5a, 0x9c, 0x5f, 0xc9, 0x0b, 0x35, + 0x18, 0xfd, 0x96, 0xf4, 0xec, 0x00, 0x8f, 0xbb, 0x1b, 0xf8, 0xc3, 0xc0, 0xe7, 0x61, 0xe0, 0x1d, + 0x5b, 0x9c, 0xf9, 0xf6, 0x19, 0xd6, 0xde, 0x96, 0xf9, 0x46, 0x9a, 0x18, 0x6b, 0x39, 0xb0, 0xd6, + 0xd3, 0xff, 0x4b, 0x23, 0x1a, 0xce, 0x40, 0x1c, 0x8c, 0x29, 0xb3, 0x1c, 0x39, 0x1d, 0x51, 0x29, + 0x8a, 0x27, 0xb2, 0xec, 0x81, 0x8a, 0x5d, 0xd2, 0xca, 0x9a, 0xa8, 0xd5, 0x68, 0x65, 0x35, 0xac, + 0xd8, 0x74, 0x48, 0x2e, 0x3b, 0xcc, 0x0e, 0x66, 0xf3, 0x10, 0xcb, 0x92, 0xec, 0x5a, 0x4e, 0xea, + 0x4a, 0x9a, 0x18, 0xab, 0x4e, 0x58, 0x85, 0xaa, 0x41, 0xe4, 0x18, 0x3a, 0xf5, 0x41, 0xe4, 0x30, + 0x56, 0x21, 0x7a, 0x9b, 0xec, 0x56, 0xc7, 0x21, 0x2f, 0x9c, 0xbd, 0x34, 0x31, 0xaa, 0x2e, 0xa8, + 0x02, 0x42, 0x8e, 0xa7, 0xfc, 0x4e, 0x3c, 0xf7, 0x5c, 0xdb, 0x12, 0xf2, 0x8b, 0xb9, 0xbc, 0xe2, + 0x82, 0x2a, 0x20, 0xe4, 0xf3, 0xca, 0xc5, 0x42, 0x72, 0x79, 0xc5, 0x05, 0x55, 0x80, 0xce, 0xc9, + 0x41, 0xb6, 0xb0, 0x6b, 0x4a, 0xbf, 0xba, 0xa8, 0xde, 0x4e, 0x13, 0xe3, 0x95, 0x5c, 0x78, 0x25, + 0x83, 0x9e, 0x91, 0xb7, 0x8a, 0x6b, 0xb8, 0xae, 0x53, 0x79, 0x7d, 0xbd, 0x93, 0x26, 0xc6, 0x79, + 0xe8, 0x70, 0x1e, 0x52, 0xff, 0x9f, 0x16, 0xd1, 0xf0, 0xc9, 0x28, 0xaa, 0x2f, 0x93, 0xd7, 0xfd, + 0x51, 0x10, 0xfb, 0xa5, 0xda, 0x5f, 0xc4, 0xa1, 0x64, 0x89, 0xeb, 0x8b, 0x2d, 0x1f, 0x09, 0xa7, + 0xb1, 0xb8, 0x45, 0x64, 0x0d, 0xd3, 0xe4, 0xf5, 0x55, 0xf5, 0xc1, 0x0a, 0x42, 0x3f, 0x26, 0xdb, + 0x0a, 0xc3, 0xb2, 0x2a, 0x1f, 0x6e, 0x9a, 0x79, 0x39, 0x4d, 0x8c, 0xb2, 0x03, 0xca, 0xa6, 0x10, + 0xe2, 0x4b, 0x13, 0x98, 0xcd, 0xdc, 0x87, 0xd9, 0x33, 0x0d, 0x85, 0x25, 0x07, 0x94, 0x4d, 0xf1, + 0xe0, 0x42, 0x00, 0x2f, 0x0b, 0x99, 0x5e, 0xf8, 0xe0, 0xca, 0x40, 0xc8, 0x9b, 0xe2, 0x1d, 0x17, + 0xca, 0xb1, 0xca, 0x5c, 0xd2, 0xe4, 0x3b, 0x6e, 0x89, 0x41, 0xd6, 0x12, 0x0b, 0xe8, 0x14, 0x8b, + 0x6f, 0x27, 0xbf, 0xbe, 0x8a, 0x38, 0x94, 0x2c, 0x91, 0x6f, 0x58, 0x28, 0x8f, 0x99, 0x3f, 0xe1, + 0xd3, 0x11, 0x0b, 0x1f, 0x66, 0xaf, 0x33, 0xcc, 0xb7, 0x15, 0x27, 0xac, 0x42, 0xe6, 0xf8, 0xc9, + 0x33, 0xbd, 0xf1, 0xf4, 0x99, 0xde, 0x78, 0xf1, 0x4c, 0x6f, 0xfe, 0xbc, 0xd0, 0x9b, 0x7f, 0x2c, + 0xf4, 0xe6, 0xe3, 0x85, 0xde, 0x7c, 0xb2, 0xd0, 0x9b, 0xff, 0x2e, 0xf4, 0xe6, 0x7f, 0x0b, 0xbd, + 0xf1, 0x62, 0xa1, 0x37, 0x7f, 0x79, 0xae, 0x37, 0x9e, 0x3c, 0xd7, 0x1b, 0x4f, 0x9f, 0xeb, 0x8d, + 0xef, 0xde, 0x9f, 0xb8, 0x7c, 0x1a, 0x8f, 0x0f, 0xed, 0x60, 0x36, 0x98, 0x84, 0xd6, 0x89, 0xe5, + 0x5b, 0x03, 0x2f, 0x78, 0xe0, 0x0e, 0xea, 0xfe, 0x90, 0x8f, 0xdb, 0xf8, 0x77, 0xfb, 0xc3, 0xff, + 0x03, 0x00, 0x00, 0xff, 0xff, 0x22, 0x3f, 0x1a, 0x51, 0xaf, 0x0f, 0x00, 0x00, } func (this *Result) Equal(that interface{}) bool { @@ -904,6 +972,9 @@ func (this *Result) Equal(that interface{}) bool { if !this.Caches.Equal(&that1.Caches) { return false } + if !this.Index.Equal(&that1.Index) { + return false + } return true } func (this *Caches) Equal(that interface{}) bool { @@ -1008,6 +1079,33 @@ func (this *Summary) Equal(that interface{}) bool { } return true } +func (this *Index) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Index) + if !ok { + that2, ok := that.(Index) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.TotalChunks != that1.TotalChunks { + return false + } + if this.PostFilterChunks != that1.PostFilterChunks { + return false + } + return true +} func (this *Querier) Equal(that interface{}) bool { if that == nil { return this == nil @@ -1207,12 +1305,13 @@ func (this *Result) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 8) + s := make([]string, 0, 9) s = append(s, "&stats.Result{") s = append(s, "Summary: "+strings.Replace(this.Summary.GoString(), `&`, ``, 1)+",\n") s = append(s, "Querier: "+strings.Replace(this.Querier.GoString(), `&`, ``, 1)+",\n") s = append(s, "Ingester: "+strings.Replace(this.Ingester.GoString(), `&`, ``, 1)+",\n") s = append(s, "Caches: "+strings.Replace(this.Caches.GoString(), `&`, ``, 1)+",\n") + s = append(s, "Index: "+strings.Replace(this.Index.GoString(), `&`, ``, 1)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -1254,6 +1353,17 @@ func (this *Summary) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *Index) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&stats.Index{") + s = append(s, "TotalChunks: "+fmt.Sprintf("%#v", this.TotalChunks)+",\n") + s = append(s, "PostFilterChunks: "+fmt.Sprintf("%#v", this.PostFilterChunks)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *Querier) GoString() string { if this == nil { return "nil" @@ -1357,6 +1467,16 @@ func (m *Result) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.Index.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStats(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a { size, err := m.Caches.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1588,6 +1708,39 @@ func (m *Summary) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *Index) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Index) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Index) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.PostFilterChunks != 0 { + i = encodeVarintStats(dAtA, i, uint64(m.PostFilterChunks)) + i-- + dAtA[i] = 0x10 + } + if m.TotalChunks != 0 { + i = encodeVarintStats(dAtA, i, uint64(m.TotalChunks)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *Querier) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1898,6 +2051,8 @@ func (m *Result) Size() (n int) { n += 1 + l + sovStats(uint64(l)) l = m.Caches.Size() n += 1 + l + sovStats(uint64(l)) + l = m.Index.Size() + n += 1 + l + sovStats(uint64(l)) return n } @@ -1971,6 +2126,21 @@ func (m *Summary) Size() (n int) { return n } +func (m *Index) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TotalChunks != 0 { + n += 1 + sovStats(uint64(m.TotalChunks)) + } + if m.PostFilterChunks != 0 { + n += 1 + sovStats(uint64(m.PostFilterChunks)) + } + return n +} + func (m *Querier) Size() (n int) { if m == nil { return 0 @@ -2118,6 +2288,7 @@ func (this *Result) String() string { `Querier:` + strings.Replace(strings.Replace(this.Querier.String(), "Querier", "Querier", 1), `&`, ``, 1) + `,`, `Ingester:` + strings.Replace(strings.Replace(this.Ingester.String(), "Ingester", "Ingester", 1), `&`, ``, 1) + `,`, `Caches:` + strings.Replace(strings.Replace(this.Caches.String(), "Caches", "Caches", 1), `&`, ``, 1) + `,`, + `Index:` + strings.Replace(strings.Replace(this.Index.String(), "Index", "Index", 1), `&`, ``, 1) + `,`, `}`, }, "") return s @@ -2160,6 +2331,17 @@ func (this *Summary) String() string { }, "") return s } +func (this *Index) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Index{`, + `TotalChunks:` + fmt.Sprintf("%v", this.TotalChunks) + `,`, + `PostFilterChunks:` + fmt.Sprintf("%v", this.PostFilterChunks) + `,`, + `}`, + }, "") + return s +} func (this *Querier) String() string { if this == nil { return "nil" @@ -2404,6 +2586,39 @@ func (m *Result) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStats + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStats + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Index.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStats(dAtA[iNdEx:]) @@ -3010,6 +3225,97 @@ func (m *Summary) Unmarshal(dAtA []byte) error { } return nil } +func (m *Index) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Index: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Index: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalChunks", wireType) + } + m.TotalChunks = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TotalChunks |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PostFilterChunks", wireType) + } + m.PostFilterChunks = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PostFilterChunks |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipStats(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStats + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthStats + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Querier) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/logqlmodel/stats/stats.proto b/pkg/logqlmodel/stats/stats.proto index 20ad6077392b6..84219f11510be 100644 --- a/pkg/logqlmodel/stats/stats.proto +++ b/pkg/logqlmodel/stats/stats.proto @@ -26,6 +26,10 @@ message Result { (gogoproto.nullable) = false, (gogoproto.jsontag) = "cache" ]; + Index index = 5 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "index" + ]; } message Caches { @@ -96,6 +100,16 @@ message Summary { int64 totalStructuredMetadataBytesProcessed = 12 [(gogoproto.jsontag) = "totalStructuredMetadataBytesProcessed"]; } +// Statistics from Index queries +// TODO(owen-d): include bytes. +// Needs some index methods added to return _sized_ chunk refs to know +message Index { + // Total chunks + int64 totalChunks = 1 [(gogoproto.jsontag) = "totalChunks"]; + // Post-filtered chunks + int64 postFilterChunks = 2 [(gogoproto.jsontag) = "postFilterChunks"]; +} + message Querier { Store store = 1 [ (gogoproto.nullable) = false, diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index eb513910f1707..5eab58e357c53 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -312,7 +312,7 @@ type Loki struct { querierAPI *querier.QuerierAPI ingesterQuerier *querier.IngesterQuerier Store storage.Store - BloomStore bloomshipper.Store + BloomStore bloomshipper.StoreWithMetrics tableManager *index.TableManager frontend Frontend ruler *base_ruler.Ruler diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index d3a9a4842adc7..1c9a8c189a5b3 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -414,6 +414,7 @@ func (t *Loki) initQuerier() (services.Service, error) { t.querierAPI = querier.NewQuerierAPI(t.Cfg.Querier, t.Querier, t.Overrides, logger) indexStatsHTTPMiddleware := querier.WrapQuerySpanAndTimeout("query.IndexStats", t.Overrides) + indexShardsHTTPMiddleware := querier.WrapQuerySpanAndTimeout("query.IndexShards", t.Overrides) volumeHTTPMiddleware := querier.WrapQuerySpanAndTimeout("query.VolumeInstant", t.Overrides) volumeRangeHTTPMiddleware := querier.WrapQuerySpanAndTimeout("query.VolumeRange", t.Overrides) seriesHTTPMiddleware := querier.WrapQuerySpanAndTimeout("query.Series", t.Overrides) @@ -465,6 +466,7 @@ func (t *Loki) initQuerier() (services.Service, error) { if querierWorkerServiceConfig.QuerierRunningStandalone() { labelsHTTPMiddleware = middleware.Merge(httpMiddleware, labelsHTTPMiddleware) indexStatsHTTPMiddleware = middleware.Merge(httpMiddleware, indexStatsHTTPMiddleware) + indexShardsHTTPMiddleware = middleware.Merge(httpMiddleware, indexShardsHTTPMiddleware) volumeHTTPMiddleware = middleware.Merge(httpMiddleware, volumeHTTPMiddleware) volumeRangeHTTPMiddleware = middleware.Merge(httpMiddleware, volumeRangeHTTPMiddleware) seriesHTTPMiddleware = middleware.Merge(httpMiddleware, seriesHTTPMiddleware) @@ -495,6 +497,7 @@ func (t *Loki) initQuerier() (services.Service, error) { router.Path("/loki/api/v1/series").Methods("GET", "POST").Handler(seriesHTTPMiddleware.Wrap(httpHandler)) router.Path("/loki/api/v1/index/stats").Methods("GET", "POST").Handler(indexStatsHTTPMiddleware.Wrap(httpHandler)) + router.Path("/loki/api/v1/index/shards").Methods("GET", "POST").Handler(indexShardsHTTPMiddleware.Wrap(httpHandler)) router.Path("/loki/api/v1/index/volume").Methods("GET", "POST").Handler(volumeHTTPMiddleware.Wrap(httpHandler)) router.Path("/loki/api/v1/index/volume_range").Methods("GET", "POST").Handler(volumeRangeHTTPMiddleware.Wrap(httpHandler)) @@ -1038,6 +1041,7 @@ func (t *Loki) initQueryFrontend() (_ services.Service, err error) { t.Server.HTTP.Path("/loki/api/v1/label/{name}/values").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/series").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/index/stats").Methods("GET", "POST").Handler(frontendHandler) + t.Server.HTTP.Path("/loki/api/v1/index/shards").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/index/volume").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/loki/api/v1/index/volume_range").Methods("GET", "POST").Handler(frontendHandler) t.Server.HTTP.Path("/api/prom/query").Methods("GET", "POST").Handler(frontendHandler) diff --git a/pkg/querier/handler.go b/pkg/querier/handler.go index 47a4c15e07511..3e57d61396945 100644 --- a/pkg/querier/handler.go +++ b/pkg/querier/handler.go @@ -93,6 +93,18 @@ func (h *Handler) Do(ctx context.Context, req queryrangebase.Request) (queryrang return nil, err } return &queryrange.IndexStatsResponse{Response: result}, nil + case *logproto.ShardsRequest: + request := loghttp.NewRangeQueryWithDefaults() + request.Start = concrete.From.Time() + request.End = concrete.Through.Time() + request.Query = concrete.GetQuery() + request.UpdateStep() + result, err := h.api.IndexShardsHandler(ctx, request, concrete.TargetBytesPerShard) + if err != nil { + return nil, err + } + return &queryrange.ShardsResponse{Response: result}, nil + case *logproto.VolumeRequest: result, err := h.api.VolumeHandler(ctx, concrete) if err != nil { diff --git a/pkg/querier/http.go b/pkg/querier/http.go index a508bf9f7286b..664e3b2ba4c9e 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -309,6 +309,39 @@ func (q *QuerierAPI) IndexStatsHandler(ctx context.Context, req *loghttp.RangeQu return resp, err } +func (q *QuerierAPI) IndexShardsHandler(ctx context.Context, req *loghttp.RangeQuery, targetBytesPerShard uint64) (*logproto.ShardsResponse, error) { + timer := prometheus.NewTimer(logql.QueryTime.WithLabelValues(logql.QueryTypeShards)) + defer timer.ObserveDuration() + + start := time.Now() + statsCtx, ctx := stats.NewContext(ctx) + + resp, err := q.querier.IndexShards(ctx, req, targetBytesPerShard) + queueTime, _ := ctx.Value(httpreq.QueryQueueTimeHTTPHeader).(time.Duration) + + resLength := 0 + if resp != nil { + resLength = len(resp.Shards) + stats.JoinResults(ctx, resp.Statistics) + } + + statResult := statsCtx.Result(time.Since(start), queueTime, resLength) + + log := spanlogger.FromContext(ctx) + statResult.Log(level.Debug(log)) + + status := 200 + if err != nil { + status, _ = serverutil.ClientHTTPStatusAndError(err) + } + + logql.RecordShardsQueryMetrics( + ctx, log, req.Start, req.End, req.Query, targetBytesPerShard, strconv.Itoa(status), resLength, statResult, + ) + + return resp, err +} + // TODO(trevorwhitney): add test for the handler split // VolumeHandler queries the index label volumes related to the passed matchers and given time range. diff --git a/pkg/querier/multi_tenant_querier.go b/pkg/querier/multi_tenant_querier.go index 2849830141167..6338b51e978a6 100644 --- a/pkg/querier/multi_tenant_querier.go +++ b/pkg/querier/multi_tenant_querier.go @@ -199,6 +199,44 @@ func (q *MultiTenantQuerier) IndexStats(ctx context.Context, req *loghttp.RangeQ return &merged, nil } +func (q *MultiTenantQuerier) IndexShards( + ctx context.Context, + req *loghttp.RangeQuery, + targetBytesPerShard uint64, +) (*logproto.ShardsResponse, error) { + tenantIDs, err := tenant.TenantIDs(ctx) + if err != nil { + return nil, err + } + + if len(tenantIDs) == 1 { + return q.Querier.IndexShards(ctx, req, targetBytesPerShard) + } + + responses := make([]*logproto.ShardsResponse, len(tenantIDs)) + for i, id := range tenantIDs { + singleContext := user.InjectOrgID(ctx, id) + resp, err := q.Querier.IndexShards(singleContext, req, targetBytesPerShard) + if err != nil { + return nil, err + } + + responses[i] = resp + } + + // TODO(owen-d): better merging + var highestIdx int + var highestVal int + for i, resp := range responses { + if len(resp.Shards) > highestVal { + highestIdx = i + highestVal = len(resp.Shards) + } + } + + return responses[highestIdx], nil +} + func (q *MultiTenantQuerier) Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index a91293c977968..d368900b21e68 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -11,6 +11,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores/index" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/indexgateway" "github.com/go-kit/log/level" "github.com/grafana/dskit/httpgrpc" @@ -92,6 +93,7 @@ type Querier interface { Series(ctx context.Context, req *logproto.SeriesRequest) (*logproto.SeriesResponse, error) Tail(ctx context.Context, req *logproto.TailRequest, categorizedLabels bool) (*Tailer, error) IndexStats(ctx context.Context, req *loghttp.RangeQuery) (*stats.Stats, error) + IndexShards(ctx context.Context, req *loghttp.RangeQuery, targetBytesPerShard uint64) (*logproto.ShardsResponse, error) Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) } @@ -772,6 +774,45 @@ func (q *SingleTenantQuerier) IndexStats(ctx context.Context, req *loghttp.Range ) } +func (q *SingleTenantQuerier) IndexShards( + ctx context.Context, + req *loghttp.RangeQuery, + targetBytesPerShard uint64, +) (*logproto.ShardsResponse, error) { + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + start, end, err := validateQueryTimeRangeLimits(ctx, userID, q.limits, req.Start, req.End) + if err != nil { + return nil, err + } + + // Enforce the query timeout while querying backends + queryTimeout := q.limits.QueryTimeout(ctx, userID) + ctx, cancel := context.WithDeadline(ctx, time.Now().Add(queryTimeout)) + defer cancel() + + p, err := indexgateway.ExtractShardRequestMatchersAndAST(req.Query) + if err != nil { + return nil, err + } + + shards, err := q.store.GetShards( + ctx, + userID, + model.TimeFromUnixNano(start.UnixNano()), + model.TimeFromUnixNano(end.UnixNano()), + targetBytesPerShard, + p, + ) + if err != nil { + return nil, err + } + return shards, nil +} + func (q *SingleTenantQuerier) Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) { sp, ctx := opentracing.StartSpanFromContext(ctx, "Querier.Volume") defer sp.Finish() diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 268e05528f781..ed34a91bcaf7b 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -31,6 +31,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/fetcher" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/validation" ) @@ -371,6 +372,14 @@ func (s *storeMock) Stats(_ context.Context, _ string, _, _ model.Time, _ ...*la return nil, nil } +func (s *storeMock) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) (*logproto.ShardsResponse, error) { + return nil, nil +} + +func (s *storeMock) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + func (s *storeMock) Volume(ctx context.Context, userID string, from, through model.Time, _ int32, targetLabels []string, _ string, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) { args := s.Called(ctx, userID, from, through, targetLabels, matchers) return args.Get(0).(*logproto.VolumeResponse), args.Error(1) @@ -547,6 +556,18 @@ func (q *querierMock) IndexStats(_ context.Context, _ *loghttp.RangeQuery) (*sta return nil, nil } +func (q *querierMock) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) ([]logproto.Shard, error) { + return nil, nil +} + +func (q *querierMock) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + +func (q *querierMock) IndexShards(_ context.Context, _ *loghttp.RangeQuery, _ uint64) (*logproto.ShardsResponse, error) { + return nil, errors.New("unimplemented") +} + func (q *querierMock) Volume(ctx context.Context, req *logproto.VolumeRequest) (*logproto.VolumeResponse, error) { args := q.MethodCalled("Volume", ctx, req) diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 44de02408b4df..76d726c240fa8 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -346,6 +346,18 @@ func (Codec) DecodeRequest(_ context.Context, r *http.Request, _ []string) (quer Through: through, Matchers: req.Query, }, err + case IndexShardsOp: + req, targetBytes, err := loghttp.ParseIndexShardsQuery(r) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + from, through := util.RoundToMilliseconds(req.Start, req.End) + return &logproto.ShardsRequest{ + From: from, + Through: through, + Query: req.Query, + TargetBytesPerShard: targetBytes.Bytes(), + }, err case VolumeOp: req, err := loghttp.ParseVolumeInstantQuery(r) if err != nil { @@ -521,6 +533,19 @@ func (Codec) DecodeHTTPGrpcRequest(ctx context.Context, r *httpgrpc.HTTPRequest) Through: through, Matchers: req.Query, }, ctx, err + case IndexShardsOp: + req, targetBytes, err := loghttp.ParseIndexShardsQuery(httpReq) + if err != nil { + return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + from, through := util.RoundToMilliseconds(req.Start, req.End) + return &logproto.ShardsRequest{ + From: from, + Through: through, + Query: req.Query, + TargetBytesPerShard: targetBytes.Bytes(), + }, ctx, nil + case VolumeOp: req, err := loghttp.ParseVolumeInstantQuery(httpReq) if err != nil { @@ -789,6 +814,25 @@ func (c Codec) EncodeRequest(ctx context.Context, r queryrangebase.Request) (*ht Header: header, } return req.WithContext(ctx), nil + case *logproto.ShardsRequest: + params := url.Values{ + "start": []string{fmt.Sprintf("%d", request.From.Time().UnixNano())}, + "end": []string{fmt.Sprintf("%d", request.Through.Time().UnixNano())}, + "query": []string{request.GetQuery()}, + "targetBytesPerShard": []string{fmt.Sprintf("%d", request.TargetBytesPerShard)}, + } + u := &url.URL{ + Path: "/loki/api/v1/index/shards", + RawQuery: params.Encode(), + } + req := &http.Request{ + Method: "GET", + RequestURI: u.String(), // This is what the httpgrpc code looks at. + URL: u, + Body: http.NoBody, + Header: header, + } + return req.WithContext(ctx), nil default: return nil, httpgrpc.Errorf(http.StatusInternalServerError, fmt.Sprintf("invalid request format, got (%T)", r)) } @@ -897,6 +941,15 @@ func decodeResponseJSONFrom(buf []byte, req queryrangebase.Request, headers http Response: &resp, Headers: httpResponseHeadersToPromResponseHeaders(headers), }, nil + case *logproto.ShardsRequest: + var resp logproto.ShardsResponse + if err := json.Unmarshal(buf, &resp); err != nil { + return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) + } + return &ShardsResponse{ + Response: &resp, + Headers: httpResponseHeadersToPromResponseHeaders(headers), + }, nil case *logproto.VolumeRequest: var resp logproto.VolumeResponse if err := json.Unmarshal(buf, &resp); err != nil { @@ -1013,6 +1066,8 @@ func decodeResponseProtobuf(r *http.Response, req queryrangebase.Request) (query return resp.GetLabels().WithHeaders(headers), nil case *logproto.IndexStatsRequest: return resp.GetStats().WithHeaders(headers), nil + case *logproto.ShardsRequest: + return resp.GetShardsResponse().WithHeaders(headers), nil default: switch concrete := resp.Response.(type) { case *QueryResponse_Prom: @@ -1110,6 +1165,10 @@ func encodeResponseJSONTo(version loghttp.Version, res queryrangebase.Response, if err := marshal.WriteIndexStatsResponseJSON(response.Response, w); err != nil { return err } + case *ShardsResponse: + if err := marshal.WriteIndexShardsResponseJSON(response.Response, w); err != nil { + return err + } case *VolumeResponse: if err := marshal.WriteVolumeResponseJSON(response.Response, w); err != nil { return err diff --git a/pkg/querier/queryrange/codec_test.go b/pkg/querier/queryrange/codec_test.go index fa6fa9e036711..b615d73b09c01 100644 --- a/pkg/querier/queryrange/codec_test.go +++ b/pkg/querier/queryrange/codec_test.go @@ -1593,6 +1593,10 @@ var ( "queryReferencedStructuredMetadata": true } }, + "index": { + "postFilterChunks": 0, + "totalChunks": 0 + }, "cache": { "chunk": { "entriesFound": 0, diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index cadfceeee20e3..8a305176b6870 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -22,6 +22,7 @@ import ( "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/querier/astmapper" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" ) @@ -290,7 +291,7 @@ func TestInstanceFor(t *testing.T) { Params: logql.ParamsWithShardsOverride{ Params: newParams(), ShardsOverride: logql.Shards{ - {Shard: 0, Of: 2}, + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 0, Of: 2}), }.Encode(), }, }, @@ -298,7 +299,7 @@ func TestInstanceFor(t *testing.T) { Params: logql.ParamsWithShardsOverride{ Params: newParams(), ShardsOverride: logql.Shards{ - {Shard: 1, Of: 2}, + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 1, Of: 2}), }.Encode(), }, }, @@ -363,8 +364,10 @@ func TestInstanceDownstream(t *testing.T) { queries := []logql.DownstreamQuery{ { Params: logql.ParamsWithShardsOverride{ - Params: logql.ParamsWithExpressionOverride{Params: params, ExpressionOverride: expr}, - ShardsOverride: logql.Shards{{Shard: 0, Of: 2}}.Encode(), + Params: logql.ParamsWithExpressionOverride{Params: params, ExpressionOverride: expr}, + ShardsOverride: logql.Shards{ + logql.NewPowerOfTwoShard(astmapper.ShardAnnotation{Shard: 0, Of: 2}), + }.Encode(), }, }, } diff --git a/pkg/querier/queryrange/extensions.go b/pkg/querier/queryrange/extensions.go index 75d4ce2cb4edd..46a8ebdc17084 100644 --- a/pkg/querier/queryrange/extensions.go +++ b/pkg/querier/queryrange/extensions.go @@ -220,3 +220,19 @@ func (m *QuantileSketchResponse) WithHeaders(h []queryrangebase.PrometheusRespon m.Headers = h return m } + +func (m *ShardsResponse) GetHeaders() []*queryrangebase.PrometheusResponseHeader { + if m != nil { + return convertPrometheusResponseHeadersToPointers(m.Headers) + } + return nil +} + +func (m *ShardsResponse) SetHeader(name, value string) { + m.Headers = setHeader(m.Headers, name, value) +} + +func (m *ShardsResponse) WithHeaders(h []queryrangebase.PrometheusResponseHeader) queryrangebase.Response { + m.Headers = h + return m +} diff --git a/pkg/querier/queryrange/limits/definitions.go b/pkg/querier/queryrange/limits/definitions.go index 9e1232b750797..f2e55728a594e 100644 --- a/pkg/querier/queryrange/limits/definitions.go +++ b/pkg/querier/queryrange/limits/definitions.go @@ -27,6 +27,7 @@ type Limits interface { TSDBMaxQueryParallelism(context.Context, string) int // TSDBMaxBytesPerShard returns the limit to the number of bytes a single shard TSDBMaxBytesPerShard(string) int + TSDBShardingStrategy(userID string) string RequiredLabels(context.Context, string) []string RequiredNumberLabels(context.Context, string) int diff --git a/pkg/querier/queryrange/marshal.go b/pkg/querier/queryrange/marshal.go index 4480b06adcfac..473b3714464e2 100644 --- a/pkg/querier/queryrange/marshal.go +++ b/pkg/querier/queryrange/marshal.go @@ -200,6 +200,8 @@ func QueryResponseUnwrap(res *QueryResponse) (queryrangebase.Response, error) { return concrete.Labels, nil case *QueryResponse_Stats: return concrete.Stats, nil + case *QueryResponse_ShardsResponse: + return concrete.ShardsResponse, nil case *QueryResponse_Prom: return concrete.Prom, nil case *QueryResponse_Streams: @@ -243,6 +245,8 @@ func QueryResponseWrap(res queryrangebase.Response) (*QueryResponse, error) { p.Response = &QueryResponse_TopkSketches{response} case *QuantileSketchResponse: p.Response = &QueryResponse_QuantileSketches{response} + case *ShardsResponse: + p.Response = &QueryResponse_ShardsResponse{response} default: return nil, fmt.Errorf("invalid response format, got (%T)", res) } @@ -311,6 +315,8 @@ func (Codec) QueryRequestUnwrap(ctx context.Context, req *QueryRequest) (queryra return concrete.Instant, ctx, nil case *QueryRequest_Stats: return concrete.Stats, ctx, nil + case *QueryRequest_ShardsRequest: + return concrete.ShardsRequest, ctx, nil case *QueryRequest_Volume: return concrete.Volume, ctx, nil case *QueryRequest_Streams: @@ -330,7 +336,7 @@ func (Codec) QueryRequestUnwrap(ctx context.Context, req *QueryRequest) (queryra LabelRequest: *concrete.Labels, }, ctx, nil default: - return nil, ctx, fmt.Errorf("unsupported request type, got (%T)", req.Request) + return nil, ctx, fmt.Errorf("unsupported request type while unwrapping, got (%T)", req.Request) } } @@ -353,8 +359,10 @@ func (Codec) QueryRequestWrap(ctx context.Context, r queryrangebase.Request) (*Q result.Request = &QueryRequest_Instant{Instant: req} case *LokiRequest: result.Request = &QueryRequest_Streams{Streams: req} + case *logproto.ShardsRequest: + result.Request = &QueryRequest_ShardsRequest{ShardsRequest: req} default: - return nil, fmt.Errorf("unsupported request type, got (%T)", r) + return nil, fmt.Errorf("unsupported request type while wrapping, got (%T)", r) } // Add query tags diff --git a/pkg/querier/queryrange/prometheus_test.go b/pkg/querier/queryrange/prometheus_test.go index 80e4f5367afb0..624c8b5194eec 100644 --- a/pkg/querier/queryrange/prometheus_test.go +++ b/pkg/querier/queryrange/prometheus_test.go @@ -13,6 +13,10 @@ import ( ) var emptyStats = `"stats": { + "index": { + "postFilterChunks": 0, + "totalChunks": 0 + }, "ingester" : { "store": { "chunksDownloadTime": 0, diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index cbc541a044044..38477b8b83912 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -831,6 +831,43 @@ func (m *QuantileSketchResponse) XXX_DiscardUnknown() { var xxx_messageInfo_QuantileSketchResponse proto.InternalMessageInfo +type ShardsResponse struct { + Response *github_com_grafana_loki_pkg_logproto.ShardsResponse `protobuf:"bytes,1,opt,name=response,proto3,customtype=github.com/grafana/loki/pkg/logproto.ShardsResponse" json:"response,omitempty"` + Headers []github_com_grafana_loki_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader `protobuf:"bytes,2,rep,name=Headers,proto3,customtype=github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" json:"-"` +} + +func (m *ShardsResponse) Reset() { *m = ShardsResponse{} } +func (*ShardsResponse) ProtoMessage() {} +func (*ShardsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_51b9d53b40d11902, []int{13} +} +func (m *ShardsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ShardsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ShardsResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ShardsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShardsResponse.Merge(m, src) +} +func (m *ShardsResponse) XXX_Size() int { + return m.Size() +} +func (m *ShardsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ShardsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ShardsResponse proto.InternalMessageInfo + type QueryResponse struct { Status *rpc.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` // Types that are valid to be assigned to Response: @@ -842,13 +879,14 @@ type QueryResponse struct { // *QueryResponse_Volume // *QueryResponse_TopkSketches // *QueryResponse_QuantileSketches + // *QueryResponse_ShardsResponse Response isQueryResponse_Response `protobuf_oneof:"response"` } func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{13} + return fileDescriptor_51b9d53b40d11902, []int{14} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,6 +946,9 @@ type QueryResponse_TopkSketches struct { type QueryResponse_QuantileSketches struct { QuantileSketches *QuantileSketchResponse `protobuf:"bytes,9,opt,name=quantileSketches,proto3,oneof"` } +type QueryResponse_ShardsResponse struct { + ShardsResponse *ShardsResponse `protobuf:"bytes,10,opt,name=shardsResponse,proto3,oneof"` +} func (*QueryResponse_Series) isQueryResponse_Response() {} func (*QueryResponse_Labels) isQueryResponse_Response() {} @@ -917,6 +958,7 @@ func (*QueryResponse_Streams) isQueryResponse_Response() {} func (*QueryResponse_Volume) isQueryResponse_Response() {} func (*QueryResponse_TopkSketches) isQueryResponse_Response() {} func (*QueryResponse_QuantileSketches) isQueryResponse_Response() {} +func (*QueryResponse_ShardsResponse) isQueryResponse_Response() {} func (m *QueryResponse) GetResponse() isQueryResponse_Response { if m != nil { @@ -988,6 +1030,13 @@ func (m *QueryResponse) GetQuantileSketches() *QuantileSketchResponse { return nil } +func (m *QueryResponse) GetShardsResponse() *ShardsResponse { + if x, ok := m.GetResponse().(*QueryResponse_ShardsResponse); ok { + return x.ShardsResponse + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*QueryResponse) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -999,6 +1048,7 @@ func (*QueryResponse) XXX_OneofWrappers() []interface{} { (*QueryResponse_Volume)(nil), (*QueryResponse_TopkSketches)(nil), (*QueryResponse_QuantileSketches)(nil), + (*QueryResponse_ShardsResponse)(nil), } } @@ -1010,6 +1060,7 @@ type QueryRequest struct { // *QueryRequest_Instant // *QueryRequest_Streams // *QueryRequest_Volume + // *QueryRequest_ShardsRequest Request isQueryRequest_Request `protobuf_oneof:"request"` Metadata map[string]string `protobuf:"bytes,7,rep,name=metadata,proto3" json:"metadata" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } @@ -1017,7 +1068,7 @@ type QueryRequest struct { func (m *QueryRequest) Reset() { *m = QueryRequest{} } func (*QueryRequest) ProtoMessage() {} func (*QueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{14} + return fileDescriptor_51b9d53b40d11902, []int{15} } func (m *QueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1071,13 +1122,17 @@ type QueryRequest_Streams struct { type QueryRequest_Volume struct { Volume *logproto.VolumeRequest `protobuf:"bytes,6,opt,name=volume,proto3,oneof"` } +type QueryRequest_ShardsRequest struct { + ShardsRequest *logproto.ShardsRequest `protobuf:"bytes,8,opt,name=shardsRequest,proto3,oneof"` +} -func (*QueryRequest_Series) isQueryRequest_Request() {} -func (*QueryRequest_Labels) isQueryRequest_Request() {} -func (*QueryRequest_Stats) isQueryRequest_Request() {} -func (*QueryRequest_Instant) isQueryRequest_Request() {} -func (*QueryRequest_Streams) isQueryRequest_Request() {} -func (*QueryRequest_Volume) isQueryRequest_Request() {} +func (*QueryRequest_Series) isQueryRequest_Request() {} +func (*QueryRequest_Labels) isQueryRequest_Request() {} +func (*QueryRequest_Stats) isQueryRequest_Request() {} +func (*QueryRequest_Instant) isQueryRequest_Request() {} +func (*QueryRequest_Streams) isQueryRequest_Request() {} +func (*QueryRequest_Volume) isQueryRequest_Request() {} +func (*QueryRequest_ShardsRequest) isQueryRequest_Request() {} func (m *QueryRequest) GetRequest() isQueryRequest_Request { if m != nil { @@ -1128,6 +1183,13 @@ func (m *QueryRequest) GetVolume() *logproto.VolumeRequest { return nil } +func (m *QueryRequest) GetShardsRequest() *logproto.ShardsRequest { + if x, ok := m.GetRequest().(*QueryRequest_ShardsRequest); ok { + return x.ShardsRequest + } + return nil +} + func (m *QueryRequest) GetMetadata() map[string]string { if m != nil { return m.Metadata @@ -1144,6 +1206,7 @@ func (*QueryRequest) XXX_OneofWrappers() []interface{} { (*QueryRequest_Instant)(nil), (*QueryRequest_Streams)(nil), (*QueryRequest_Volume)(nil), + (*QueryRequest_ShardsRequest)(nil), } } @@ -1161,6 +1224,7 @@ func init() { proto.RegisterType((*VolumeResponse)(nil), "queryrange.VolumeResponse") proto.RegisterType((*TopKSketchesResponse)(nil), "queryrange.TopKSketchesResponse") proto.RegisterType((*QuantileSketchResponse)(nil), "queryrange.QuantileSketchResponse") + proto.RegisterType((*ShardsResponse)(nil), "queryrange.ShardsResponse") proto.RegisterType((*QueryResponse)(nil), "queryrange.QueryResponse") proto.RegisterType((*QueryRequest)(nil), "queryrange.QueryRequest") proto.RegisterMapType((map[string]string)(nil), "queryrange.QueryRequest.MetadataEntry") @@ -1171,102 +1235,107 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1514 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x5b, 0x6f, 0x1b, 0x45, - 0x1b, 0xf6, 0xfa, 0x18, 0x4f, 0x0e, 0x5f, 0xbe, 0x49, 0x94, 0xee, 0x97, 0xf6, 0xdb, 0xb5, 0x2c, - 0xd1, 0x06, 0x04, 0x6b, 0x9a, 0x94, 0x1e, 0x01, 0xd1, 0xa5, 0xad, 0x52, 0xd1, 0xa2, 0x76, 0x13, - 0x71, 0x81, 0xb8, 0x99, 0xd8, 0x13, 0x7b, 0xf1, 0x9e, 0xb2, 0x33, 0x0e, 0xcd, 0x1d, 0x3f, 0x00, - 0xa4, 0xfe, 0x0a, 0x84, 0x44, 0x55, 0x89, 0x5b, 0x2e, 0xb9, 0xa1, 0x97, 0xbd, 0xac, 0x2c, 0xb1, - 0x50, 0x97, 0x0b, 0x94, 0xab, 0xfe, 0x04, 0x34, 0x87, 0x5d, 0xef, 0xda, 0x6e, 0xeb, 0x14, 0x21, - 0xb5, 0x12, 0x37, 0xf6, 0x1c, 0xde, 0x67, 0xf6, 0xdd, 0xe7, 0x79, 0xdf, 0x77, 0x66, 0x16, 0x9c, - 0x0a, 0xba, 0xed, 0xc6, 0x5e, 0x0f, 0x87, 0x36, 0x0e, 0xf9, 0xff, 0x41, 0x88, 0xbc, 0x36, 0x4e, - 0x35, 0x8d, 0x20, 0xf4, 0xa9, 0x0f, 0xc1, 0x70, 0x64, 0x75, 0xbd, 0x6d, 0xd3, 0x4e, 0x6f, 0xc7, - 0x68, 0xfa, 0x6e, 0xa3, 0xed, 0xb7, 0xfd, 0x46, 0xdb, 0xf7, 0xdb, 0x0e, 0x46, 0x81, 0x4d, 0x64, - 0xb3, 0x11, 0x06, 0xcd, 0x06, 0xa1, 0x88, 0xf6, 0x88, 0xc0, 0xaf, 0x2e, 0x33, 0x43, 0xde, 0xe4, - 0x10, 0x39, 0xaa, 0x4b, 0x73, 0xde, 0xdb, 0xe9, 0xed, 0x36, 0xa8, 0xed, 0x62, 0x42, 0x91, 0x1b, - 0x48, 0x83, 0xe3, 0xcc, 0x3f, 0xc7, 0x6f, 0x0b, 0x64, 0xdc, 0x90, 0x93, 0xff, 0xcb, 0x4c, 0x92, - 0x2e, 0xa6, 0xcd, 0x8e, 0x9c, 0xaa, 0xc9, 0xa9, 0x3d, 0xc7, 0xf5, 0x5b, 0xd8, 0xe1, 0xbe, 0x10, - 0xf1, 0x2b, 0x2d, 0x96, 0x98, 0x45, 0xd0, 0x23, 0x1d, 0xfe, 0x23, 0x07, 0x3f, 0x7e, 0x21, 0x1d, - 0x3b, 0x88, 0xe0, 0x46, 0x0b, 0xef, 0xda, 0x9e, 0x4d, 0x6d, 0xdf, 0x23, 0xe9, 0xb6, 0x5c, 0xe4, - 0xec, 0x74, 0x8b, 0x8c, 0x52, 0x5c, 0xbf, 0x5f, 0x00, 0xb3, 0x37, 0xfc, 0xae, 0x6d, 0xe1, 0xbd, - 0x1e, 0x26, 0x14, 0x2e, 0x83, 0x12, 0xb7, 0x51, 0x95, 0x9a, 0xb2, 0x56, 0xb5, 0x44, 0x87, 0x8d, - 0x3a, 0xb6, 0x6b, 0x53, 0x35, 0x5f, 0x53, 0xd6, 0xe6, 0x2d, 0xd1, 0x81, 0x10, 0x14, 0x09, 0xc5, - 0x81, 0x5a, 0xa8, 0x29, 0x6b, 0x05, 0x8b, 0xb7, 0xe1, 0x2a, 0x98, 0xb1, 0x3d, 0x8a, 0xc3, 0x7d, - 0xe4, 0xa8, 0x55, 0x3e, 0x9e, 0xf4, 0xe1, 0x87, 0xa0, 0x42, 0x28, 0x0a, 0xe9, 0x36, 0x51, 0x8b, - 0x35, 0x65, 0x6d, 0x76, 0x7d, 0xd5, 0x10, 0x52, 0x18, 0xb1, 0x14, 0xc6, 0x76, 0x2c, 0x85, 0x39, - 0xf3, 0x20, 0xd2, 0x73, 0x77, 0x7f, 0xd3, 0x15, 0x2b, 0x06, 0xc1, 0x8b, 0xa0, 0x84, 0xbd, 0xd6, - 0x36, 0x51, 0x4b, 0x47, 0x40, 0x0b, 0x08, 0x3c, 0x0d, 0xaa, 0x2d, 0x3b, 0xc4, 0x4d, 0xc6, 0x99, - 0x5a, 0xae, 0x29, 0x6b, 0x0b, 0xeb, 0x4b, 0x46, 0x22, 0xed, 0x95, 0x78, 0xca, 0x1a, 0x5a, 0xb1, - 0xd7, 0x0b, 0x10, 0xed, 0xa8, 0x15, 0xce, 0x04, 0x6f, 0xc3, 0x3a, 0x28, 0x93, 0x0e, 0x0a, 0x5b, - 0x44, 0x9d, 0xa9, 0x15, 0xd6, 0xaa, 0x26, 0x38, 0x8c, 0x74, 0x39, 0x62, 0xc9, 0x7f, 0xf8, 0x05, - 0x28, 0x06, 0x0e, 0xf2, 0x54, 0xc0, 0xbd, 0x5c, 0x34, 0x52, 0x9c, 0xdf, 0x72, 0x90, 0x67, 0x9e, - 0xed, 0x47, 0x7a, 0x26, 0x9a, 0x43, 0xb4, 0x8b, 0x3c, 0xd4, 0x70, 0xfc, 0xae, 0xdd, 0x48, 0xcb, - 0xc8, 0x56, 0x31, 0x6e, 0x33, 0x34, 0xc3, 0x59, 0x7c, 0xd5, 0xfa, 0x2f, 0x79, 0x00, 0x99, 0x60, - 0xd7, 0x3d, 0x42, 0x91, 0x47, 0x5f, 0x46, 0xb7, 0xf7, 0x41, 0x99, 0x85, 0xfc, 0x36, 0xe1, 0xca, - 0x4d, 0x4b, 0xa4, 0xc4, 0x64, 0x99, 0x2c, 0x1e, 0x89, 0xc9, 0xd2, 0x44, 0x26, 0xcb, 0x2f, 0x64, - 0xb2, 0xf2, 0x8f, 0x30, 0xa9, 0x82, 0x22, 0xeb, 0xc1, 0x45, 0x50, 0x08, 0xd1, 0x57, 0x9c, 0xb8, - 0x39, 0x8b, 0x35, 0xeb, 0x3f, 0x14, 0xc1, 0x9c, 0x48, 0x0a, 0x12, 0xf8, 0x1e, 0xc1, 0xcc, 0xd9, - 0x2d, 0x5e, 0x58, 0x04, 0xbd, 0xd2, 0x59, 0x3e, 0x62, 0xc9, 0x19, 0xf8, 0x11, 0x28, 0x5e, 0x41, - 0x14, 0x71, 0xaa, 0x67, 0xd7, 0x97, 0xd3, 0xce, 0xb2, 0xb5, 0xd8, 0x9c, 0xb9, 0xc2, 0xd8, 0x3c, - 0x8c, 0xf4, 0x85, 0x16, 0xa2, 0xe8, 0x6d, 0xdf, 0xb5, 0x29, 0x76, 0x03, 0x7a, 0x60, 0x71, 0x24, - 0x7c, 0x0f, 0x54, 0xaf, 0x86, 0xa1, 0x1f, 0x6e, 0x1f, 0x04, 0x98, 0x4b, 0x53, 0x35, 0x8f, 0x1d, - 0x46, 0xfa, 0x12, 0x8e, 0x07, 0x53, 0x88, 0xa1, 0x25, 0x7c, 0x13, 0x94, 0x78, 0x87, 0x8b, 0x51, - 0x35, 0x97, 0x0e, 0x23, 0xfd, 0x3f, 0x1c, 0x92, 0x32, 0x17, 0x16, 0x59, 0xed, 0x4a, 0x53, 0x69, - 0x97, 0x84, 0x50, 0x39, 0x1d, 0x42, 0x2a, 0xa8, 0xec, 0xe3, 0x90, 0xb0, 0x65, 0x2a, 0x7c, 0x3c, - 0xee, 0xc2, 0xcb, 0x00, 0x30, 0x62, 0x6c, 0x42, 0xed, 0x26, 0xcb, 0x12, 0x46, 0xc6, 0xbc, 0x21, - 0x8a, 0xa0, 0x85, 0x49, 0xcf, 0xa1, 0x26, 0x94, 0x2c, 0xa4, 0x0c, 0xad, 0x54, 0x1b, 0xde, 0x53, - 0x40, 0x65, 0x13, 0xa3, 0x16, 0x0e, 0x89, 0x5a, 0xad, 0x15, 0xd6, 0x66, 0xd7, 0xdf, 0x30, 0xd2, - 0x15, 0xef, 0x56, 0xe8, 0xbb, 0x98, 0x76, 0x70, 0x8f, 0xc4, 0x02, 0x09, 0x6b, 0xb3, 0xdb, 0x8f, - 0xf4, 0x9d, 0x69, 0xe2, 0x61, 0xaa, 0x2a, 0xfb, 0xcc, 0xe7, 0x1c, 0x46, 0xba, 0xf2, 0x8e, 0x15, - 0xbb, 0x58, 0xff, 0x55, 0x01, 0xff, 0x65, 0x0a, 0x6f, 0xb1, 0xb5, 0x49, 0x2a, 0x21, 0x5d, 0x44, - 0x9b, 0x1d, 0x55, 0x61, 0xe1, 0x6d, 0x89, 0x4e, 0xba, 0x04, 0xe6, 0xff, 0x56, 0x09, 0x2c, 0x1c, - 0xbd, 0x04, 0xc6, 0x59, 0x58, 0x9c, 0x98, 0x85, 0xa5, 0x67, 0x65, 0x61, 0xfd, 0x9b, 0x82, 0xa8, - 0x38, 0xf1, 0xfb, 0x1d, 0x21, 0x27, 0xae, 0x25, 0x39, 0x51, 0xe0, 0xde, 0x26, 0xa1, 0x26, 0xd6, - 0xba, 0xde, 0xc2, 0x1e, 0xb5, 0x77, 0x6d, 0x1c, 0xbe, 0x20, 0x33, 0x52, 0xe1, 0x56, 0xc8, 0x86, - 0x5b, 0x3a, 0x56, 0x8a, 0xaf, 0x7c, 0xac, 0x8c, 0x64, 0x47, 0xe9, 0x25, 0xb2, 0xa3, 0xfe, 0x34, - 0x0f, 0x56, 0x98, 0x1c, 0x37, 0xd0, 0x0e, 0x76, 0x3e, 0x45, 0xee, 0x11, 0x25, 0x39, 0x99, 0x92, - 0xa4, 0x6a, 0xc2, 0x7f, 0x29, 0x9f, 0x82, 0xf2, 0xef, 0x14, 0x30, 0x13, 0xd7, 0x70, 0x68, 0x00, - 0x20, 0x60, 0xbc, 0x4c, 0x0b, 0xa2, 0x17, 0x18, 0x38, 0x4c, 0x46, 0xad, 0x94, 0x05, 0xfc, 0x12, - 0x94, 0x45, 0x4f, 0x66, 0xc1, 0xb1, 0x54, 0x16, 0xd0, 0x10, 0x23, 0xf7, 0x72, 0x0b, 0x05, 0x14, - 0x87, 0xe6, 0x05, 0xe6, 0x45, 0x3f, 0xd2, 0x4f, 0x3d, 0x8f, 0x22, 0x7e, 0x6e, 0x14, 0x38, 0x26, - 0xae, 0x78, 0xa6, 0x25, 0x9f, 0x50, 0xff, 0x56, 0x01, 0x8b, 0xcc, 0x51, 0x46, 0x4d, 0x12, 0x15, - 0x57, 0xc0, 0x4c, 0x28, 0xdb, 0xdc, 0xdd, 0xd9, 0xf5, 0xba, 0x91, 0xa5, 0x75, 0x02, 0x95, 0x66, - 0xf1, 0x41, 0xa4, 0x2b, 0x56, 0x82, 0x84, 0x1b, 0x19, 0x1a, 0xf3, 0x93, 0x68, 0x64, 0x90, 0x5c, - 0x86, 0xb8, 0x9f, 0xf2, 0x00, 0x5e, 0xf7, 0x5a, 0xf8, 0x0e, 0x0b, 0xbe, 0x61, 0x9c, 0xf6, 0xc6, - 0x3c, 0x3a, 0x31, 0x24, 0x65, 0xdc, 0xde, 0xbc, 0xd4, 0x8f, 0xf4, 0x73, 0xcf, 0x63, 0xe5, 0x39, - 0xe0, 0xd4, 0x2b, 0xa4, 0x03, 0x37, 0xff, 0xea, 0xef, 0x2b, 0xf7, 0xf3, 0x60, 0xe1, 0x33, 0xdf, - 0xe9, 0xb9, 0x38, 0x21, 0xce, 0x1d, 0x23, 0x4e, 0x1d, 0x12, 0x97, 0xb5, 0x35, 0xcf, 0xf5, 0x23, - 0x7d, 0x63, 0x2a, 0xd2, 0xb2, 0xc0, 0xd7, 0x97, 0xb0, 0x7b, 0x79, 0xb0, 0xbc, 0xed, 0x07, 0x9f, - 0x6c, 0xf1, 0x4b, 0x59, 0xaa, 0x2e, 0xe2, 0x31, 0xda, 0x96, 0x87, 0xb4, 0x31, 0xc4, 0x4d, 0x44, - 0x43, 0xfb, 0x8e, 0xb9, 0xd1, 0x8f, 0xf4, 0xc6, 0x54, 0x94, 0x0d, 0x41, 0xaf, 0x2f, 0x5d, 0x3f, - 0xe7, 0xc1, 0xca, 0xed, 0x1e, 0xf2, 0xa8, 0xed, 0x60, 0x41, 0x59, 0x42, 0xd8, 0xc1, 0x18, 0x61, - 0xda, 0x90, 0xb0, 0x2c, 0x46, 0x52, 0xf7, 0x41, 0x3f, 0xd2, 0x2f, 0x4c, 0x45, 0xdd, 0x24, 0xf8, - 0xeb, 0x4b, 0xe2, 0x8f, 0x45, 0x30, 0xcf, 0x2f, 0x16, 0x09, 0x77, 0x6f, 0x01, 0xb9, 0xe5, 0x4a, - 0xe6, 0x60, 0x7c, 0x46, 0x0b, 0x83, 0xa6, 0xb1, 0x25, 0x37, 0x63, 0x61, 0x01, 0xcf, 0x83, 0x32, - 0xe1, 0x27, 0x21, 0x59, 0x50, 0xb5, 0xd1, 0x5b, 0x43, 0xf6, 0xcc, 0xb5, 0x99, 0xb3, 0xa4, 0x3d, - 0xbb, 0xc3, 0x39, 0xec, 0x00, 0x10, 0x9f, 0x04, 0xeb, 0xa3, 0xc8, 0xf1, 0xe3, 0x01, 0x43, 0x0b, - 0x0c, 0x3c, 0x0b, 0x4a, 0xbc, 0x72, 0xcb, 0x7b, 0x78, 0xe6, 0xb1, 0xe3, 0x25, 0x74, 0x33, 0x67, - 0x09, 0x73, 0xb8, 0x0e, 0x8a, 0x41, 0xe8, 0xbb, 0x72, 0x17, 0x3d, 0x31, 0xfa, 0xcc, 0xf4, 0xb6, - 0xb3, 0x99, 0xb3, 0xb8, 0x2d, 0x3c, 0xc3, 0x8e, 0xbc, 0x6c, 0xbf, 0x22, 0xfc, 0x0a, 0xc1, 0x4a, - 0xd6, 0x08, 0x2c, 0x05, 0x89, 0x4d, 0xe1, 0x19, 0x50, 0xde, 0xe7, 0x65, 0x49, 0x5e, 0xfe, 0x56, - 0xd3, 0xa0, 0x6c, 0xc1, 0x62, 0xef, 0x25, 0x6c, 0xe1, 0x35, 0x30, 0x47, 0xfd, 0xa0, 0x1b, 0x17, - 0x00, 0x79, 0xfd, 0xa8, 0xa5, 0xb1, 0x93, 0x0a, 0xc4, 0x66, 0xce, 0xca, 0xe0, 0xe0, 0x2d, 0xb0, - 0xb8, 0x97, 0x09, 0x53, 0x4c, 0xf8, 0xd7, 0x8c, 0x11, 0x9e, 0x27, 0x67, 0xcf, 0x66, 0xce, 0x1a, - 0x43, 0x9b, 0x60, 0x98, 0x51, 0xf5, 0x3f, 0x0a, 0x60, 0x4e, 0xc6, 0x8c, 0xb8, 0x2b, 0x9c, 0x4b, - 0xc2, 0x40, 0x84, 0xcc, 0xff, 0x9f, 0x15, 0x06, 0xdc, 0x3c, 0x15, 0x05, 0xef, 0x26, 0x51, 0x20, - 0xe2, 0x67, 0x65, 0x98, 0xa5, 0x5c, 0xff, 0x14, 0x42, 0x2a, 0xbf, 0x11, 0x2b, 0x2f, 0xc2, 0xe6, - 0xf8, 0xe4, 0x7d, 0x37, 0x46, 0x49, 0xd9, 0x2f, 0x82, 0x8a, 0x2d, 0x3e, 0x37, 0x4c, 0x0a, 0x98, - 0xf1, 0xaf, 0x11, 0x4c, 0x48, 0x09, 0x80, 0x1b, 0x43, 0xf9, 0x45, 0xd4, 0x1c, 0x1b, 0x97, 0x3f, - 0x01, 0xc5, 0xea, 0x9f, 0x4e, 0xd4, 0x2f, 0x4b, 0xcc, 0xd8, 0x66, 0x95, 0xbc, 0x98, 0x94, 0x7e, - 0x13, 0xcc, 0xb8, 0x98, 0x22, 0x76, 0x96, 0x55, 0x2b, 0xbc, 0x6e, 0x9c, 0xcc, 0x4a, 0x35, 0xe4, - 0xdb, 0xb8, 0x29, 0x0d, 0xaf, 0x7a, 0x34, 0x3c, 0x90, 0xc7, 0x96, 0x04, 0xbd, 0x7a, 0x09, 0xcc, - 0x67, 0x0c, 0xe0, 0x22, 0x28, 0x74, 0x71, 0xfc, 0x65, 0x85, 0x35, 0xd9, 0xe5, 0x6e, 0x1f, 0x39, - 0x3d, 0xcc, 0x69, 0xaf, 0x5a, 0xa2, 0x73, 0x31, 0x7f, 0x5e, 0x31, 0xab, 0xa0, 0x12, 0x8a, 0xa7, - 0x98, 0xad, 0x87, 0x8f, 0xb5, 0xdc, 0xa3, 0xc7, 0x5a, 0xee, 0xe9, 0x63, 0x4d, 0xf9, 0x7a, 0xa0, - 0x29, 0xdf, 0x0f, 0x34, 0xe5, 0xc1, 0x40, 0x53, 0x1e, 0x0e, 0x34, 0xe5, 0xf7, 0x81, 0xa6, 0xfc, - 0x39, 0xd0, 0x72, 0x4f, 0x07, 0x9a, 0x72, 0xf7, 0x89, 0x96, 0x7b, 0xf8, 0x44, 0xcb, 0x3d, 0x7a, - 0xa2, 0xe5, 0x3e, 0x37, 0x8e, 0x56, 0xc2, 0x76, 0xca, 0x9c, 0x96, 0x8d, 0xbf, 0x02, 0x00, 0x00, - 0xff, 0xff, 0xe6, 0x4a, 0x9a, 0x06, 0x55, 0x15, 0x00, 0x00, + // 1586 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0xcb, 0x6f, 0x1b, 0x45, + 0x18, 0xf7, 0xfa, 0x19, 0x4f, 0x9a, 0x10, 0x26, 0x51, 0xba, 0xa4, 0xed, 0xae, 0x65, 0x89, 0x36, + 0x20, 0x58, 0xd3, 0xa4, 0xf4, 0x09, 0x88, 0x2e, 0x69, 0xe5, 0x8a, 0x16, 0xb5, 0x9b, 0x88, 0x03, + 0xe2, 0x32, 0x89, 0x27, 0xce, 0x92, 0x7d, 0x65, 0x67, 0x9c, 0x36, 0x37, 0xfe, 0x00, 0x90, 0xfa, + 0x57, 0x20, 0x24, 0xaa, 0x9e, 0x38, 0x71, 0xe4, 0x42, 0x8f, 0x3d, 0x56, 0x91, 0x30, 0x34, 0xbd, + 0xa0, 0x9c, 0x7a, 0xe3, 0x8a, 0xe6, 0xb1, 0xeb, 0x59, 0xdb, 0x69, 0x9d, 0x22, 0xa4, 0x46, 0xe2, + 0x62, 0xcf, 0xe3, 0xfb, 0xcd, 0xce, 0xfe, 0xbe, 0xdf, 0xf7, 0xed, 0x37, 0x03, 0xce, 0x44, 0x9b, + 0xed, 0xc6, 0x56, 0x07, 0xc7, 0x2e, 0x8e, 0xf9, 0xff, 0x4e, 0x8c, 0x82, 0x36, 0x56, 0x9a, 0x56, + 0x14, 0x87, 0x34, 0x84, 0xa0, 0x37, 0x32, 0xb7, 0xd0, 0x76, 0xe9, 0x46, 0x67, 0xd5, 0x5a, 0x0b, + 0xfd, 0x46, 0x3b, 0x6c, 0x87, 0x8d, 0x76, 0x18, 0xb6, 0x3d, 0x8c, 0x22, 0x97, 0xc8, 0x66, 0x23, + 0x8e, 0xd6, 0x1a, 0x84, 0x22, 0xda, 0x21, 0x02, 0x3f, 0x37, 0xc3, 0x0c, 0x79, 0x93, 0x43, 0xe4, + 0xa8, 0x29, 0xcd, 0x79, 0x6f, 0xb5, 0xb3, 0xde, 0xa0, 0xae, 0x8f, 0x09, 0x45, 0x7e, 0x94, 0x18, + 0xb0, 0xfd, 0x79, 0x61, 0x5b, 0x20, 0xdd, 0xa0, 0x85, 0xef, 0xb5, 0x11, 0xc5, 0x77, 0xd1, 0x8e, + 0x34, 0x38, 0x91, 0x31, 0x48, 0x1a, 0x72, 0xf2, 0xad, 0xcc, 0x24, 0xd9, 0xc4, 0x74, 0x6d, 0x43, + 0x4e, 0xd5, 0xe4, 0xd4, 0x96, 0xe7, 0x87, 0x2d, 0xec, 0xf1, 0xcd, 0x12, 0xf1, 0x2b, 0x2d, 0xa6, + 0x99, 0x45, 0xd4, 0x21, 0x1b, 0xfc, 0x47, 0x0e, 0x7e, 0xf6, 0x52, 0xbe, 0x56, 0x11, 0xc1, 0x8d, + 0x16, 0x5e, 0x77, 0x03, 0x97, 0xba, 0x61, 0x40, 0xd4, 0xb6, 0x5c, 0xe4, 0xfc, 0x68, 0x8b, 0xf4, + 0xfb, 0xa0, 0xfe, 0xb0, 0x00, 0xc6, 0x6f, 0x86, 0x9b, 0xae, 0x83, 0xb7, 0x3a, 0x98, 0x50, 0x38, + 0x03, 0x4a, 0xdc, 0x46, 0xd7, 0x6a, 0xda, 0x7c, 0xd5, 0x11, 0x1d, 0x36, 0xea, 0xb9, 0xbe, 0x4b, + 0xf5, 0x7c, 0x4d, 0x9b, 0x9f, 0x70, 0x44, 0x07, 0x42, 0x50, 0x24, 0x14, 0x47, 0x7a, 0xa1, 0xa6, + 0xcd, 0x17, 0x1c, 0xde, 0x86, 0x73, 0x60, 0xcc, 0x0d, 0x28, 0x8e, 0xb7, 0x91, 0xa7, 0x57, 0xf9, + 0x78, 0xda, 0x87, 0x9f, 0x80, 0x0a, 0xa1, 0x28, 0xa6, 0x2b, 0x44, 0x2f, 0xd6, 0xb4, 0xf9, 0xf1, + 0x85, 0x39, 0x4b, 0xf8, 0xca, 0x4a, 0x7c, 0x65, 0xad, 0x24, 0xbe, 0xb2, 0xc7, 0x1e, 0x75, 0xcd, + 0xdc, 0xfd, 0x3f, 0x4c, 0xcd, 0x49, 0x40, 0xf0, 0x32, 0x28, 0xe1, 0xa0, 0xb5, 0x42, 0xf4, 0xd2, + 0x21, 0xd0, 0x02, 0x02, 0xcf, 0x82, 0x6a, 0xcb, 0x8d, 0xf1, 0x1a, 0xe3, 0x4c, 0x2f, 0xd7, 0xb4, + 0xf9, 0xc9, 0x85, 0x69, 0x2b, 0x75, 0xed, 0x52, 0x32, 0xe5, 0xf4, 0xac, 0xd8, 0xeb, 0x45, 0x88, + 0x6e, 0xe8, 0x15, 0xce, 0x04, 0x6f, 0xc3, 0x3a, 0x28, 0x93, 0x0d, 0x14, 0xb7, 0x88, 0x3e, 0x56, + 0x2b, 0xcc, 0x57, 0x6d, 0xb0, 0xdf, 0x35, 0xe5, 0x88, 0x23, 0xff, 0xe1, 0xd7, 0xa0, 0x18, 0x79, + 0x28, 0xd0, 0x01, 0xdf, 0xe5, 0x94, 0xa5, 0x70, 0x7e, 0xdb, 0x43, 0x81, 0x7d, 0x7e, 0xb7, 0x6b, + 0x66, 0xe4, 0x1e, 0xa3, 0x75, 0x14, 0xa0, 0x86, 0x17, 0x6e, 0xba, 0x0d, 0xd5, 0x8d, 0x6c, 0x15, + 0xeb, 0x0e, 0x43, 0x33, 0x9c, 0xc3, 0x57, 0xad, 0xff, 0x96, 0x07, 0x90, 0x39, 0xec, 0x46, 0x40, + 0x28, 0x0a, 0xe8, 0xab, 0xf8, 0xed, 0x23, 0x50, 0x66, 0x31, 0xb1, 0x42, 0xb8, 0xe7, 0x46, 0x25, + 0x52, 0x62, 0xb2, 0x4c, 0x16, 0x0f, 0xc5, 0x64, 0x69, 0x28, 0x93, 0xe5, 0x97, 0x32, 0x59, 0xf9, + 0x4f, 0x98, 0xd4, 0x41, 0x91, 0xf5, 0xe0, 0x14, 0x28, 0xc4, 0xe8, 0x2e, 0x27, 0xee, 0x98, 0xc3, + 0x9a, 0xf5, 0x9f, 0x8a, 0xe0, 0x98, 0x08, 0x0a, 0x12, 0x85, 0x01, 0xc1, 0x6c, 0xb3, 0xcb, 0x3c, + 0xf3, 0x08, 0x7a, 0xe5, 0x66, 0xf9, 0x88, 0x23, 0x67, 0xe0, 0xa7, 0xa0, 0xb8, 0x84, 0x28, 0xe2, + 0x54, 0x8f, 0x2f, 0xcc, 0xa8, 0x9b, 0x65, 0x6b, 0xb1, 0x39, 0x7b, 0x96, 0xb1, 0xb9, 0xdf, 0x35, + 0x27, 0x5b, 0x88, 0xa2, 0xf7, 0x42, 0xdf, 0xa5, 0xd8, 0x8f, 0xe8, 0x8e, 0xc3, 0x91, 0xf0, 0x43, + 0x50, 0xbd, 0x16, 0xc7, 0x61, 0xbc, 0xb2, 0x13, 0x61, 0xee, 0x9a, 0xaa, 0x7d, 0x7c, 0xbf, 0x6b, + 0x4e, 0xe3, 0x64, 0x50, 0x41, 0xf4, 0x2c, 0xe1, 0x3b, 0xa0, 0xc4, 0x3b, 0xdc, 0x19, 0x55, 0x7b, + 0x7a, 0xbf, 0x6b, 0xbe, 0xc1, 0x21, 0x8a, 0xb9, 0xb0, 0xc8, 0xfa, 0xae, 0x34, 0x92, 0xef, 0x52, + 0x09, 0x95, 0x55, 0x09, 0xe9, 0xa0, 0xb2, 0x8d, 0x63, 0xc2, 0x96, 0xa9, 0xf0, 0xf1, 0xa4, 0x0b, + 0xaf, 0x02, 0xc0, 0x88, 0x71, 0x09, 0x75, 0xd7, 0x58, 0x94, 0x30, 0x32, 0x26, 0x2c, 0x91, 0x04, + 0x1d, 0x4c, 0x3a, 0x1e, 0xb5, 0xa1, 0x64, 0x41, 0x31, 0x74, 0x94, 0x36, 0x7c, 0xa0, 0x81, 0x4a, + 0x13, 0xa3, 0x16, 0x8e, 0x89, 0x5e, 0xad, 0x15, 0xe6, 0xc7, 0x17, 0xde, 0xb6, 0xd4, 0x8c, 0x77, + 0x3b, 0x0e, 0x7d, 0x4c, 0x37, 0x70, 0x87, 0x24, 0x0e, 0x12, 0xd6, 0xf6, 0xe6, 0x6e, 0xd7, 0x5c, + 0x1d, 0x45, 0x0f, 0x23, 0x65, 0xd9, 0x03, 0x9f, 0xb3, 0xdf, 0x35, 0xb5, 0xf7, 0x9d, 0x64, 0x8b, + 0xf5, 0xdf, 0x35, 0xf0, 0x26, 0xf3, 0xf0, 0x32, 0x5b, 0x9b, 0x28, 0x01, 0xe9, 0x23, 0xba, 0xb6, + 0xa1, 0x6b, 0x4c, 0xde, 0x8e, 0xe8, 0xa8, 0x29, 0x30, 0xff, 0xaf, 0x52, 0x60, 0xe1, 0xf0, 0x29, + 0x30, 0x89, 0xc2, 0xe2, 0xd0, 0x28, 0x2c, 0x1d, 0x14, 0x85, 0xf5, 0xef, 0x0a, 0x22, 0xe3, 0x24, + 0xef, 0x77, 0x88, 0x98, 0xb8, 0x9e, 0xc6, 0x44, 0x81, 0xef, 0x36, 0x95, 0x9a, 0x58, 0xeb, 0x46, + 0x0b, 0x07, 0xd4, 0x5d, 0x77, 0x71, 0xfc, 0x92, 0xc8, 0x50, 0xe4, 0x56, 0xc8, 0xca, 0x4d, 0xd5, + 0x4a, 0xf1, 0xb5, 0xd7, 0x4a, 0x5f, 0x74, 0x94, 0x5e, 0x21, 0x3a, 0xea, 0xcf, 0xf3, 0x60, 0x96, + 0xb9, 0xe3, 0x26, 0x5a, 0xc5, 0xde, 0x17, 0xc8, 0x3f, 0xa4, 0x4b, 0x4e, 0x2b, 0x2e, 0xa9, 0xda, + 0xf0, 0x7f, 0xca, 0x47, 0xa0, 0xfc, 0x07, 0x0d, 0x8c, 0x25, 0x39, 0x1c, 0x5a, 0x00, 0x08, 0x18, + 0x4f, 0xd3, 0x82, 0xe8, 0x49, 0x06, 0x8e, 0xd3, 0x51, 0x47, 0xb1, 0x80, 0xdf, 0x80, 0xb2, 0xe8, + 0xc9, 0x28, 0x38, 0xae, 0x44, 0x01, 0x8d, 0x31, 0xf2, 0xaf, 0xb6, 0x50, 0x44, 0x71, 0x6c, 0x5f, + 0x62, 0xbb, 0xd8, 0xed, 0x9a, 0x67, 0x5e, 0x44, 0x11, 0xaf, 0x1b, 0x05, 0x8e, 0x39, 0x57, 0x3c, + 0xd3, 0x91, 0x4f, 0xa8, 0x7f, 0xaf, 0x81, 0x29, 0xb6, 0x51, 0x46, 0x4d, 0xaa, 0x8a, 0x25, 0x30, + 0x16, 0xcb, 0x36, 0xdf, 0xee, 0xf8, 0x42, 0xdd, 0xca, 0xd2, 0x3a, 0x84, 0x4a, 0xbb, 0xf8, 0xa8, + 0x6b, 0x6a, 0x4e, 0x8a, 0x84, 0x8b, 0x19, 0x1a, 0xf3, 0xc3, 0x68, 0x64, 0x90, 0x5c, 0x86, 0xb8, + 0x5f, 0xf2, 0x00, 0xde, 0x60, 0x05, 0x36, 0x13, 0x5f, 0x4f, 0xa7, 0x9d, 0x81, 0x1d, 0x9d, 0xec, + 0x91, 0x32, 0x68, 0x6f, 0x5f, 0xd9, 0xed, 0x9a, 0x17, 0x5e, 0xc4, 0xca, 0x0b, 0xc0, 0xca, 0x2b, + 0xa8, 0xc2, 0xcd, 0xbf, 0xfe, 0xdf, 0x95, 0x87, 0x79, 0x30, 0xf9, 0x65, 0xe8, 0x75, 0x7c, 0x9c, + 0x12, 0xe7, 0x0f, 0x10, 0xa7, 0xf7, 0x88, 0xcb, 0xda, 0xda, 0x17, 0x76, 0xbb, 0xe6, 0xe2, 0x48, + 0xa4, 0x65, 0x81, 0x47, 0x97, 0xb0, 0x07, 0x79, 0x30, 0xb3, 0x12, 0x46, 0x9f, 0x2f, 0xf3, 0x43, + 0x99, 0x92, 0x17, 0xf1, 0x00, 0x6d, 0x33, 0x3d, 0xda, 0x18, 0xe2, 0x16, 0xa2, 0xb1, 0x7b, 0xcf, + 0x5e, 0xdc, 0xed, 0x9a, 0x8d, 0x91, 0x28, 0xeb, 0x81, 0x8e, 0x2e, 0x5d, 0xbf, 0xe6, 0xc1, 0xec, + 0x9d, 0x0e, 0x0a, 0xa8, 0xeb, 0x61, 0x41, 0x59, 0x4a, 0xd8, 0xce, 0x00, 0x61, 0x46, 0x8f, 0xb0, + 0x2c, 0x46, 0x52, 0xf7, 0xf1, 0x6e, 0xd7, 0xbc, 0x34, 0x12, 0x75, 0xc3, 0xe0, 0x47, 0x97, 0xc4, + 0x9f, 0xf3, 0x60, 0x72, 0x59, 0xd4, 0x4b, 0xc9, 0x1b, 0x90, 0x21, 0xe4, 0xa9, 0xb7, 0x0c, 0xd1, + 0xaa, 0x95, 0x45, 0x1c, 0x22, 0x54, 0xb3, 0xc0, 0xa3, 0x4b, 0xdb, 0xdf, 0x45, 0x30, 0xc1, 0xcf, + 0x63, 0x29, 0x6b, 0xef, 0x02, 0x59, 0xa9, 0x48, 0xce, 0x60, 0x52, 0xda, 0xc6, 0xd1, 0x9a, 0xb5, + 0x2c, 0x6b, 0x18, 0x61, 0x01, 0x2f, 0x82, 0x32, 0xe1, 0x05, 0xa4, 0xfc, 0x0e, 0x19, 0xfd, 0x87, + 0xad, 0x6c, 0xa9, 0xda, 0xcc, 0x39, 0xd2, 0x9e, 0x1d, 0x7d, 0x3d, 0x56, 0x37, 0x25, 0x05, 0x74, + 0xbd, 0x1f, 0x39, 0x58, 0x55, 0x31, 0xb4, 0xc0, 0xc0, 0xf3, 0xa0, 0xc4, 0x3f, 0x78, 0xf2, 0xfa, + 0x22, 0xf3, 0xd8, 0xc1, 0x2f, 0x4f, 0x33, 0xe7, 0x08, 0x73, 0xb8, 0x00, 0x8a, 0x51, 0x1c, 0xfa, + 0xb2, 0xf8, 0x38, 0xd9, 0xff, 0x4c, 0xf5, 0x6b, 0xdd, 0xcc, 0x39, 0xdc, 0x16, 0x9e, 0x63, 0x27, + 0x05, 0xf6, 0x99, 0x27, 0xfc, 0xe4, 0xc5, 0x32, 0x7d, 0x1f, 0x4c, 0x81, 0x24, 0xa6, 0xf0, 0x1c, + 0x28, 0x6f, 0xf3, 0x6c, 0x2e, 0xcf, 0xcc, 0x73, 0x2a, 0x28, 0x9b, 0xe7, 0xd9, 0x7b, 0x09, 0x5b, + 0x78, 0x1d, 0x1c, 0xa3, 0x61, 0xb4, 0x99, 0xe4, 0x4d, 0x79, 0x6a, 0xab, 0xa9, 0xd8, 0x61, 0x79, + 0xb5, 0x99, 0x73, 0x32, 0x38, 0x78, 0x1b, 0x4c, 0x6d, 0x65, 0xa2, 0x1b, 0x13, 0x7e, 0x09, 0xd4, + 0xc7, 0xf3, 0xf0, 0xa4, 0xd3, 0xcc, 0x39, 0x03, 0x68, 0xb8, 0x04, 0x26, 0x49, 0x46, 0xf2, 0xf2, + 0x56, 0x25, 0xf3, 0x5e, 0xd9, 0xa0, 0x68, 0xe6, 0x9c, 0x3e, 0x8c, 0x0d, 0x7a, 0x11, 0xc9, 0xcf, + 0xf6, 0x52, 0x79, 0xe2, 0xa0, 0x76, 0x21, 0x15, 0x93, 0x10, 0xde, 0xa9, 0x83, 0xc4, 0xc4, 0xcd, + 0x15, 0x2d, 0x7d, 0x90, 0x6a, 0x49, 0xa8, 0x70, 0xb6, 0x97, 0x22, 0xb9, 0x8a, 0x14, 0x84, 0xd4, + 0xcf, 0x62, 0xa2, 0x1f, 0x21, 0xbe, 0x13, 0xc3, 0x8b, 0x9e, 0x04, 0x25, 0xc5, 0x73, 0x19, 0x54, + 0x5c, 0x71, 0xd7, 0x33, 0x4c, 0x76, 0x83, 0x57, 0x41, 0x4c, 0x0e, 0x12, 0x00, 0x17, 0x7b, 0x22, + 0x12, 0xda, 0x3b, 0x3e, 0x28, 0xa2, 0x14, 0x94, 0x68, 0xe8, 0x6c, 0xaa, 0xa1, 0xb2, 0xc4, 0x0c, + 0x54, 0x0a, 0xe9, 0x8b, 0x49, 0x01, 0x5d, 0x03, 0x13, 0x09, 0xe5, 0x7c, 0x4a, 0x2a, 0xe8, 0xd4, + 0x41, 0x79, 0x2f, 0xc1, 0x67, 0x51, 0xb0, 0x09, 0xc6, 0x7c, 0x4c, 0x11, 0x3b, 0x8f, 0xe8, 0x15, + 0x9e, 0xc4, 0x4e, 0x67, 0x75, 0xd3, 0x73, 0x9b, 0x75, 0x4b, 0x1a, 0x5e, 0x0b, 0x68, 0xbc, 0x23, + 0x4b, 0xcf, 0x14, 0x3d, 0x77, 0x05, 0x4c, 0x64, 0x0c, 0xe0, 0x14, 0x28, 0x6c, 0xe2, 0xe4, 0x76, + 0x8c, 0x35, 0xd9, 0x01, 0x7d, 0x1b, 0x79, 0x1d, 0xcc, 0xbd, 0x57, 0x75, 0x44, 0xe7, 0x72, 0xfe, + 0xa2, 0x66, 0x57, 0x41, 0x25, 0x16, 0x4f, 0xb1, 0x5b, 0x8f, 0x9f, 0x1a, 0xb9, 0x27, 0x4f, 0x8d, + 0xdc, 0xf3, 0xa7, 0x86, 0xf6, 0xed, 0x9e, 0xa1, 0xfd, 0xb8, 0x67, 0x68, 0x8f, 0xf6, 0x0c, 0xed, + 0xf1, 0x9e, 0xa1, 0xfd, 0xb9, 0x67, 0x68, 0x7f, 0xed, 0x19, 0xb9, 0xe7, 0x7b, 0x86, 0x76, 0xff, + 0x99, 0x91, 0x7b, 0xfc, 0xcc, 0xc8, 0x3d, 0x79, 0x66, 0xe4, 0xbe, 0xb2, 0x0e, 0x97, 0x4f, 0x57, + 0xcb, 0x9c, 0xdd, 0xc5, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x20, 0xe2, 0xc2, 0x3a, 0x17, + 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -1798,6 +1867,42 @@ func (this *QuantileSketchResponse) Equal(that interface{}) bool { } return true } +func (this *ShardsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ShardsResponse) + if !ok { + that2, ok := that.(ShardsResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if that1.Response == nil { + if this.Response != nil { + return false + } + } else if !this.Response.Equal(*that1.Response) { + return false + } + if len(this.Headers) != len(that1.Headers) { + return false + } + for i := range this.Headers { + if !this.Headers[i].Equal(that1.Headers[i]) { + return false + } + } + return true +} func (this *QueryResponse) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2023,6 +2128,30 @@ func (this *QueryResponse_QuantileSketches) Equal(that interface{}) bool { } return true } +func (this *QueryResponse_ShardsResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryResponse_ShardsResponse) + if !ok { + that2, ok := that.(QueryResponse_ShardsResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.ShardsResponse.Equal(that1.ShardsResponse) { + return false + } + return true +} func (this *QueryRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2205,6 +2334,30 @@ func (this *QueryRequest_Volume) Equal(that interface{}) bool { } return true } +func (this *QueryRequest_ShardsRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*QueryRequest_ShardsRequest) + if !ok { + that2, ok := that.(QueryRequest_ShardsRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.ShardsRequest.Equal(that1.ShardsRequest) { + return false + } + return true +} func (this *LokiRequest) GoString() string { if this == nil { return "nil" @@ -2384,11 +2537,22 @@ func (this *QuantileSketchResponse) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *ShardsResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&queryrange.ShardsResponse{") + s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n") + s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *QueryResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 13) + s := make([]string, 0, 14) s = append(s, "&queryrange.QueryResponse{") if this.Status != nil { s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") @@ -2463,11 +2627,19 @@ func (this *QueryResponse_QuantileSketches) GoString() string { `QuantileSketches:` + fmt.Sprintf("%#v", this.QuantileSketches) + `}`}, ", ") return s } +func (this *QueryResponse_ShardsResponse) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&queryrange.QueryResponse_ShardsResponse{` + + `ShardsResponse:` + fmt.Sprintf("%#v", this.ShardsResponse) + `}`}, ", ") + return s +} func (this *QueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&queryrange.QueryRequest{") if this.Request != nil { s = append(s, "Request: "+fmt.Sprintf("%#v", this.Request)+",\n") @@ -2536,6 +2708,14 @@ func (this *QueryRequest_Volume) GoString() string { `Volume:` + fmt.Sprintf("%#v", this.Volume) + `}`}, ", ") return s } +func (this *QueryRequest_ShardsRequest) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&queryrange.QueryRequest_ShardsRequest{` + + `ShardsRequest:` + fmt.Sprintf("%#v", this.ShardsRequest) + `}`}, ", ") + return s +} func valueToGoStringQueryrange(v interface{}, typ string) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -3327,6 +3507,55 @@ func (m *QuantileSketchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) return len(dAtA) - i, nil } +func (m *ShardsResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ShardsResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ShardsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Headers) > 0 { + for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Headers[iNdEx].Size() + i -= size + if _, err := m.Headers[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if m.Response != nil { + { + size := m.Response.Size() + i -= size + if _, err := m.Response.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *QueryResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3531,6 +3760,26 @@ func (m *QueryResponse_QuantileSketches) MarshalToSizedBuffer(dAtA []byte) (int, } return len(dAtA) - i, nil } +func (m *QueryResponse_ShardsResponse) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *QueryResponse_ShardsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ShardsResponse != nil { + { + size, err := m.ShardsResponse.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x52 + } + return len(dAtA) - i, nil +} func (m *QueryRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3551,6 +3800,15 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Request != nil { + { + size := m.Request.Size() + i -= size + if _, err := m.Request.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } if len(m.Metadata) > 0 { for k := range m.Metadata { v := m.Metadata[k] @@ -3570,15 +3828,6 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x3a } } - if m.Request != nil { - { - size := m.Request.Size() - i -= size - if _, err := m.Request.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } - } return len(dAtA) - i, nil } @@ -3702,6 +3951,26 @@ func (m *QueryRequest_Volume) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *QueryRequest_ShardsRequest) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *QueryRequest_ShardsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ShardsRequest != nil { + { + size, err := m.ShardsRequest.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + return len(dAtA) - i, nil +} func encodeVarintQueryrange(dAtA []byte, offset int, v uint64) int { offset -= sovQueryrange(v) base := offset @@ -4043,6 +4312,25 @@ func (m *QuantileSketchResponse) Size() (n int) { return n } +func (m *ShardsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Response != nil { + l = m.Response.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + if len(m.Headers) > 0 { + for _, e := range m.Headers { + l = e.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + } + return n +} + func (m *QueryResponse) Size() (n int) { if m == nil { return 0 @@ -4155,6 +4443,18 @@ func (m *QueryResponse_QuantileSketches) Size() (n int) { } return n } +func (m *QueryResponse_ShardsResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ShardsResponse != nil { + l = m.ShardsResponse.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} func (m *QueryRequest) Size() (n int) { if m == nil { return 0 @@ -4247,6 +4547,18 @@ func (m *QueryRequest_Volume) Size() (n int) { } return n } +func (m *QueryRequest_ShardsRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ShardsRequest != nil { + l = m.ShardsRequest.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} func sovQueryrange(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 @@ -4430,6 +4742,17 @@ func (this *QuantileSketchResponse) String() string { }, "") return s } +func (this *ShardsResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&ShardsResponse{`, + `Response:` + fmt.Sprintf("%v", this.Response) + `,`, + `Headers:` + fmt.Sprintf("%v", this.Headers) + `,`, + `}`, + }, "") + return s +} func (this *QueryResponse) String() string { if this == nil { return "nil" @@ -4521,6 +4844,16 @@ func (this *QueryResponse_QuantileSketches) String() string { }, "") return s } +func (this *QueryResponse_ShardsResponse) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryResponse_ShardsResponse{`, + `ShardsResponse:` + strings.Replace(fmt.Sprintf("%v", this.ShardsResponse), "ShardsResponse", "ShardsResponse", 1) + `,`, + `}`, + }, "") + return s +} func (this *QueryRequest) String() string { if this == nil { return "nil" @@ -4602,6 +4935,16 @@ func (this *QueryRequest_Volume) String() string { }, "") return s } +func (this *QueryRequest_ShardsRequest) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&QueryRequest_ShardsRequest{`, + `ShardsRequest:` + strings.Replace(fmt.Sprintf("%v", this.ShardsRequest), "ShardsRequest", "logproto.ShardsRequest", 1) + `,`, + `}`, + }, "") + return s +} func valueToStringQueryrange(v interface{}) string { rv := reflect.ValueOf(v) if rv.IsNil() { @@ -6942,6 +7285,129 @@ func (m *QuantileSketchResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *ShardsResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ShardsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ShardsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Response == nil { + m.Response = &github_com_grafana_loki_pkg_logproto.ShardsResponse{} + } + if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Headers = append(m.Headers, github_com_grafana_loki_pkg_querier_queryrange_queryrangebase_definitions.PrometheusResponseHeader{}) + if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQueryrange(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *QueryResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -7287,6 +7753,41 @@ func (m *QueryResponse) Unmarshal(dAtA []byte) error { } m.Response = &QueryResponse_QuantileSketches{v} iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardsResponse", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ShardsResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Response = &QueryResponse_ShardsResponse{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) @@ -7677,6 +8178,41 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error { } m.Metadata[mapkey] = mapvalue iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShardsRequest", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &logproto.ShardsRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Request = &QueryRequest_ShardsRequest{v} + 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 f673464acfc0b..4d9fb84853757 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -5,6 +5,7 @@ package queryrange; import "github.com/gogo/googleapis/google/rpc/status.proto"; import "gogoproto/gogo.proto"; import "google/protobuf/timestamp.proto"; +import "pkg/logproto/indexgateway.proto"; import "pkg/logproto/logproto.proto"; import "pkg/logproto/sketch.proto"; import "pkg/logqlmodel/stats/stats.proto"; @@ -166,6 +167,14 @@ message QuantileSketchResponse { ]; } +message ShardsResponse { + indexgatewaypb.ShardsResponse response = 1 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/logproto.ShardsResponse"]; + repeated definitions.PrometheusResponseHeader Headers = 2 [ + (gogoproto.jsontag) = "-", + (gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions.PrometheusResponseHeader" + ]; +} + message QueryResponse { google.rpc.Status status = 1; oneof response { @@ -177,6 +186,7 @@ message QueryResponse { VolumeResponse volume = 7; TopKSketchesResponse topkSketches = 8; QuantileSketchResponse quantileSketches = 9; + ShardsResponse shardsResponse = 10; } } @@ -188,6 +198,7 @@ message QueryRequest { LokiInstantRequest instant = 4; LokiRequest streams = 5; logproto.VolumeRequest volume = 6; + indexgatewaypb.ShardsRequest shardsRequest = 8; } map metadata = 7 [(gogoproto.nullable) = false]; } diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index a6c32b1525862..8a11c546a7b64 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -189,13 +189,27 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que ast.maxShards, r, ast.statsHandler, + ast.next, ast.limits, ) if !ok { return ast.next.Do(ctx, r) } - mapper := logql.NewShardMapper(resolver, ast.metrics, ast.shardAggregation) + v := ast.limits.TSDBShardingStrategy(tenants[0]) + version, err := logql.ParseShardVersion(v) + if err != nil { + level.Warn(logger).Log( + "msg", "failed to parse shard version", + "fallback", version.String(), + "err", err.Error(), + "user", tenants[0], + "query", r.GetQuery(), + ) + } + strategy := version.Strategy(resolver, uint64(ast.limits.TSDBMaxBytesPerShard(tenants[0]))) + + mapper := logql.NewShardMapper(strategy, ast.metrics, ast.shardAggregation) noop, bytesPerShard, parsed, err := mapper.Parse(params.GetExpression()) if err != nil { @@ -232,9 +246,7 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que } // Merge index and volume stats result cache stats from shard resolver into the query stats. - res.Statistics.Caches.StatsResult.Merge(resolverStats.Caches().StatsResult) - res.Statistics.Caches.VolumeResult.Merge(resolverStats.Caches().VolumeResult) - + res.Statistics.Merge(resolverStats.Result(0, 0, 0)) value, err := marshal.NewResultValue(res.Data) if err != nil { return nil, err diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 5184ef62bb13c..28d71f8fa880a 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -397,6 +397,7 @@ const ( IndexStatsOp = "index_stats" VolumeOp = "volume" VolumeRangeOp = "volume_range" + IndexShardsOp = "index_shards" ) func getOperation(path string) string { @@ -415,6 +416,8 @@ func getOperation(path string) string { return VolumeOp case path == "/loki/api/v1/index/volume_range": return VolumeRangeOp + case path == "/loki/api/v1/index/shards": + return IndexShardsOp default: return "" } diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index 206822a50f6e8..f7ce311eee3f7 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -1380,6 +1380,9 @@ func (f fakeLimits) VolumeEnabled(_ string) bool { func (f fakeLimits) TSDBMaxBytesPerShard(_ string) int { return valid.DefaultTSDBMaxBytesPerShard } +func (f fakeLimits) TSDBShardingStrategy(string) string { + return logql.PowerOfTwoVersion.String() +} type ingesterQueryOpts struct { queryStoreOnly bool diff --git a/pkg/querier/queryrange/shard_resolver.go b/pkg/querier/queryrange/shard_resolver.go index 652637a724655..2808a4ae2eb21 100644 --- a/pkg/querier/queryrange/shard_resolver.go +++ b/pkg/querier/queryrange/shard_resolver.go @@ -3,14 +3,16 @@ package queryrange import ( "context" "fmt" - "math" + "net/http" strings "strings" "time" "github.com/dustin/go-humanize" + "github.com/efficientgo/core/errors" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/tenant" "github.com/opentracing/opentracing-go" "github.com/prometheus/common/model" @@ -18,13 +20,13 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" + logqlstats "github.com/grafana/loki/pkg/logqlmodel/stats" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/stats" - utilMath "github.com/grafana/loki/pkg/util/math" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/spanlogger" "github.com/grafana/loki/pkg/util/validation" - valid "github.com/grafana/loki/pkg/validation" ) func shardResolverForConf( @@ -35,14 +37,15 @@ func shardResolverForConf( maxParallelism int, maxShards int, r queryrangebase.Request, - handler queryrangebase.Handler, + statsHandler, next queryrangebase.Handler, limits Limits, ) (logql.ShardResolver, bool) { if conf.IndexType == config.TSDBType { return &dynamicShardResolver{ ctx: ctx, logger: logger, - handler: handler, + statsHandler: statsHandler, + next: next, limits: limits, from: model.Time(r.GetStart().UnixMilli()), through: model.Time(r.GetEnd().UnixMilli()), @@ -58,10 +61,13 @@ func shardResolverForConf( } type dynamicShardResolver struct { - ctx context.Context - handler queryrangebase.Handler - logger log.Logger - limits Limits + ctx context.Context + // TODO(owen-d): shouldn't have to fork handlers here -- one should just transparently handle the right logic + // depending on the underlying type? + statsHandler queryrangebase.Handler // index stats handler (hooked up to results cache, etc) + next queryrangebase.Handler // next handler in the chain (used for non-stats reqs) + logger log.Logger + limits Limits from, through model.Time maxParallelism int @@ -154,7 +160,7 @@ func (r *dynamicShardResolver) GetStats(e syntax.Expr) (stats.Stats, error) { grps = append(grps, syntax.MatcherRange{}) } - results, err := getStatsForMatchers(ctx, log, r.handler, r.from, r.through, grps, r.maxParallelism, r.defaultLookback) + results, err := getStatsForMatchers(ctx, log, r.statsHandler, r.from, r.through, grps, r.maxParallelism, r.defaultLookback) if err != nil { return stats.Stats{}, err } @@ -192,7 +198,7 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) { } maxBytesPerShard := validation.SmallestPositiveIntPerTenant(tenantIDs, r.limits.TSDBMaxBytesPerShard) - factor := guessShardFactor(combined, maxBytesPerShard, r.maxShards) + factor := sharding.GuessShardFactor(combined.Bytes, uint64(maxBytesPerShard), r.maxShards) var bytesPerShard = combined.Bytes if factor > 0 { @@ -211,34 +217,84 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) { return factor, bytesPerShard, nil } -// Since we shard by powers of two and we increase shard factor -// once each shard surpasses maxBytesPerShard, if the shard factor -// is at least two, the range of data per shard is (maxBytesPerShard/2, maxBytesPerShard] -// For instance, for a maxBytesPerShard of 500MB and a query touching 1000MB, we split into two shards of 500MB. -// If there are 1004MB, we split into four shards of 251MB. -func guessShardFactor(stats stats.Stats, maxBytesPerShard, maxShards int) int { - // If maxBytesPerShard is 0, we use the default value - // to avoid division by zero - if maxBytesPerShard < 1 { - maxBytesPerShard = valid.DefaultTSDBMaxBytesPerShard +func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ([]logproto.Shard, error) { + sp, ctx := opentracing.StartSpanFromContext(r.ctx, "dynamicShardResolver.ShardingRanges") + defer sp.Finish() + log := spanlogger.FromContext(ctx) + defer log.Finish() + + adjustedFrom := r.from + + // NB(owen-d): there should only ever be 1 matcher group passed + // to this call as we call it separately for different legs + // of binary ops, but I'm putting in the loop for completion + grps, err := syntax.MatcherGroups(expr) + if err != nil { + return nil, err } - minShards := float64(stats.Bytes) / float64(maxBytesPerShard) + for _, grp := range grps { + diff := grp.Interval + grp.Offset - // round up to nearest power of 2 - power := math.Ceil(math.Log2(minShards)) + // For instant queries, when start == end, + // we have a default lookback which we add here + if grp.Interval == 0 { + diff = diff + r.defaultLookback + } - // Since x^0 == 1 and we only support factors of 2 - // reset this edge case manually - factor := int(math.Pow(2, power)) - if maxShards > 0 { - factor = utilMath.Min(factor, maxShards) + // use the oldest adjustedFrom + if r.from.Add(-diff).Before(adjustedFrom) { + adjustedFrom = r.from.Add(-diff) + } } - // shortcut: no need to run any sharding logic when factor=1 - // as it's the same as no sharding - if factor == 1 { - factor = 0 + exprStr := expr.String() + // try to get shards for the given expression + // if it fails, fallback to linearshards based on stats + resp, err := r.next.Do(ctx, &logproto.ShardsRequest{ + From: adjustedFrom, + Through: r.through, + Query: expr.String(), + TargetBytesPerShard: targetBytesPerShard, + }) + + if err != nil { + // check unimplemented to fallback + // TODO(owen-d): fix if this isn't right + if resp, ok := httpgrpc.HTTPResponseFromError(err); ok && (resp.Code == http.StatusNotFound) { + n, bytesPerShard, err := r.Shards(expr) + if err != nil { + return nil, errors.Wrap(err, "falling back to building linear shards from stats") + } + level.Debug(log).Log( + "msg", "falling back to building linear shards from stats", + "bytes_per_shard", bytesPerShard, + "shards", n, + "query", exprStr, + ) + return sharding.LinearShards(n, uint64(n)*bytesPerShard), nil + } + + return nil, errors.Wrapf(err, "failed to get shards for expression, got %T: %+v", err, err) + + } + + casted, ok := resp.(*ShardsResponse) + if !ok { + return nil, fmt.Errorf("expected *ShardsResponse while querying index, got %T", resp) } - return factor + + // accumulate stats + logqlstats.JoinResults(ctx, casted.Response.Statistics) + + level.Debug(log).Log( + "msg", "retrieved sharding ranges", + "target_bytes_per_shard", targetBytesPerShard, + "shards", len(casted.Response.Shards), + "query", exprStr, + "total_chunks", casted.Response.Statistics.Index.TotalChunks, + "post_filter_chunks:", casted.Response.Statistics.Index.PostFilterChunks, + ) + + return casted.Response.Shards, err } diff --git a/pkg/querier/queryrange/split_by_interval.go b/pkg/querier/queryrange/split_by_interval.go index ef05aa969ec1e..2f693b438dbd0 100644 --- a/pkg/querier/queryrange/split_by_interval.go +++ b/pkg/querier/queryrange/split_by_interval.go @@ -223,7 +223,7 @@ func (h *splitByInterval) Do(ctx context.Context, r queryrangebase.Request) (que intervals[i], intervals[j] = intervals[j], intervals[i] } } - case *LokiSeriesRequest, *LabelRequest, *logproto.IndexStatsRequest, *logproto.VolumeRequest: + case *LokiSeriesRequest, *LabelRequest, *logproto.IndexStatsRequest, *logproto.VolumeRequest, *logproto.ShardsRequest: // Set this to 0 since this is not used in Series/Labels/Index Request. limit = 0 default: diff --git a/pkg/querier/queryrange/splitters.go b/pkg/querier/queryrange/splitters.go index eddcc10edf491..e9f2de5b41034 100644 --- a/pkg/querier/queryrange/splitters.go +++ b/pkg/querier/queryrange/splitters.go @@ -75,6 +75,15 @@ func (s *defaultSplitter) split(execTime time.Time, tenantIDs []string, req quer Matchers: r.GetMatchers(), }) } + case *logproto.ShardsRequest: + factory = func(start, end time.Time) { + reqs = append(reqs, &logproto.ShardsRequest{ + From: model.TimeFromUnix(start.Unix()), + Through: model.TimeFromUnix(end.Unix()), + Query: r.Query, + TargetBytesPerShard: r.TargetBytesPerShard, + }) + } case *logproto.VolumeRequest: factory = func(start, end time.Time) { reqs = append(reqs, &logproto.VolumeRequest{ diff --git a/pkg/querier/queryrange/stats.go b/pkg/querier/queryrange/stats.go index 71f93959c3b69..4cc9de2f3d2c9 100644 --- a/pkg/querier/queryrange/stats.go +++ b/pkg/querier/queryrange/stats.go @@ -35,6 +35,7 @@ const ( queryTypeLabel = "label" queryTypeStats = "stats" queryTypeVolume = "volume" + queryTypeShards = "shards" ) var ( @@ -160,6 +161,9 @@ func StatsCollectorMiddleware() queryrangebase.Middleware { responseStats = &stats.Result{} // TODO: support stats in proto totalEntries = 1 queryType = queryTypeStats + case *ShardsResponse: + responseStats = &r.Response.Statistics + queryType = queryTypeShards default: level.Warn(logger).Log("msg", fmt.Sprintf("cannot compute stats, unexpected type: %T", resp)) } diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index c9eef5fa33027..b1e534df731bd 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -14,6 +14,7 @@ type BlockMetadata struct { } type Block struct { + metrics *Metrics // covers series pages index BlockIndex // covers bloom pages @@ -26,9 +27,10 @@ type Block struct { initialized bool } -func NewBlock(reader BlockReader) *Block { +func NewBlock(reader BlockReader, metrics *Metrics) *Block { return &Block{ - reader: reader, + reader: reader, + metrics: metrics, } } @@ -87,19 +89,6 @@ func combineChecksums(index, blooms uint32) uint32 { return index ^ blooms } -// convenience method -func (b *Block) Querier() *BlockQuerier { - return NewBlockQuerier(b) -} - -func (b *Block) Series() *LazySeriesIter { - return NewLazySeriesIter(b) -} - -func (b *Block) Blooms() *LazyBloomIter { - return NewLazyBloomIter(b) -} - func (b *Block) Metadata() (BlockMetadata, error) { if err := b.LoadHeaders(); err != nil { return BlockMetadata{}, err @@ -123,11 +112,16 @@ type BlockQuerier struct { cur *SeriesWithBloom } -func NewBlockQuerier(b *Block) *BlockQuerier { +// NewBlockQuerier returns a new BlockQuerier for the given block. +// WARNING: If noCapture is true, the underlying byte slice of the bloom page +// will be returned to the pool for efficiency. This can only safely be used +// when the underlying bloom bytes don't escape the decoder, i.e. +// when loading blooms for querying (bloom-gw) but not for writing (bloom-compactor). +func NewBlockQuerier(b *Block, noCapture bool) *BlockQuerier { return &BlockQuerier{ block: b, series: NewLazySeriesIter(b), - blooms: NewLazyBloomIter(b), + blooms: NewLazyBloomIter(b, noCapture), } } diff --git a/pkg/storage/bloom/v1/bloom.go b/pkg/storage/bloom/v1/bloom.go index 661c60970ab8e..d740c170fcc92 100644 --- a/pkg/storage/bloom/v1/bloom.go +++ b/pkg/storage/bloom/v1/bloom.go @@ -17,7 +17,7 @@ import ( // gateways to OOM. // Figure out a decent maximum page size that we can process. // TODO(chaudum): Make max page size configurable -var maxPageSize = 32 << 20 // 32MB +var maxPageSize = 64 << 20 // 64MB var ErrPageTooLarge = errors.Errorf("bloom page too large: size limit is %.1fMiB", float64(maxPageSize)/float64(1<<20)) type Bloom struct { @@ -86,7 +86,7 @@ func LazyDecodeBloomPage(r io.Reader, pool chunkenc.ReaderPool, page BloomPageHe } defer pool.PutReader(decompressor) - b := make([]byte, page.DecompressedLen) + b := BlockPool.Get(page.DecompressedLen)[:page.DecompressedLen] if _, err = io.ReadFull(decompressor, b); err != nil { return nil, errors.Wrap(err, "decompressing bloom page") @@ -97,11 +97,13 @@ func LazyDecodeBloomPage(r io.Reader, pool chunkenc.ReaderPool, page BloomPageHe return decoder, nil } +// shortcut to skip allocations when we know the page is not compressed func LazyDecodeBloomPageNoCompression(r io.Reader, page BloomPageHeader) (*BloomPageDecoder, error) { + // data + checksum if page.Len != page.DecompressedLen+4 { return nil, errors.New("the Len and DecompressedLen of the page do not match") } - data := make([]byte, page.Len) + data := BlockPool.Get(page.Len)[:page.Len] _, err := io.ReadFull(r, data) if err != nil { @@ -152,6 +154,21 @@ type BloomPageDecoder struct { err error } +// Relinquish returns the underlying byte slice to the pool +// for efficiency. It's intended to be used as a +// perf optimization. +// This can only safely be used when the underlying bloom +// bytes don't escape the decoder: +// on reads in the bloom-gw but not in the bloom-compactor +func (d *BloomPageDecoder) Relinquish() { + data := d.data + d.data = nil + + if cap(data) > 0 { + BlockPool.Put(data) + } +} + func (d *BloomPageDecoder) Reset() { d.err = nil d.cur = nil @@ -259,8 +276,10 @@ func (b *BloomBlock) DecodeHeaders(r io.ReadSeeker) (uint32, error) { return checksum, nil } -func (b *BloomBlock) BloomPageDecoder(r io.ReadSeeker, pageIdx int) (*BloomPageDecoder, error) { +func (b *BloomBlock) BloomPageDecoder(r io.ReadSeeker, pageIdx int, metrics *Metrics) (res *BloomPageDecoder, err error) { if pageIdx < 0 || pageIdx >= len(b.pageHeaders) { + metrics.pagesSkipped.WithLabelValues(pageTypeBloom, skipReasonOOB).Inc() + metrics.bytesSkipped.WithLabelValues(pageTypeBloom, skipReasonOOB).Add(float64(b.pageHeaders[pageIdx].DecompressedLen)) return nil, fmt.Errorf("invalid page (%d) for bloom page decoding", pageIdx) } @@ -268,16 +287,30 @@ func (b *BloomBlock) BloomPageDecoder(r io.ReadSeeker, pageIdx int) (*BloomPageD // fmt.Printf("pageIdx=%d page=%+v size=%.2fMiB\n", pageIdx, page, float64(page.Len)/float64(1<<20)) if page.Len > maxPageSize { + metrics.pagesSkipped.WithLabelValues(pageTypeBloom, skipReasonTooLarge).Inc() + metrics.bytesSkipped.WithLabelValues(pageTypeBloom, skipReasonTooLarge).Add(float64(page.DecompressedLen)) return nil, ErrPageTooLarge } if _, err := r.Seek(int64(page.Offset), io.SeekStart); err != nil { + metrics.pagesSkipped.WithLabelValues(pageTypeBloom, skipReasonErr).Inc() + metrics.bytesSkipped.WithLabelValues(pageTypeBloom, skipReasonErr).Add(float64(page.DecompressedLen)) return nil, errors.Wrap(err, "seeking to bloom page") } if b.schema.encoding == chunkenc.EncNone { - return LazyDecodeBloomPageNoCompression(r, page) + res, err = LazyDecodeBloomPageNoCompression(r, page) + } else { + res, err = LazyDecodeBloomPage(r, b.schema.DecompressorPool(), page) + } + + if err != nil { + metrics.pagesSkipped.WithLabelValues(pageTypeBloom, skipReasonErr).Inc() + metrics.bytesSkipped.WithLabelValues(pageTypeBloom, skipReasonErr).Add(float64(page.DecompressedLen)) + return nil, errors.Wrap(err, "decoding bloom page") } - return LazyDecodeBloomPage(r, b.schema.DecompressorPool(), page) + metrics.pagesRead.WithLabelValues(pageTypeBloom).Inc() + metrics.bytesRead.WithLabelValues(pageTypeBloom).Add(float64(page.DecompressedLen)) + return res, nil } diff --git a/pkg/storage/bloom/v1/bloom_querier.go b/pkg/storage/bloom/v1/bloom_querier.go index d0dbdc1b3b550..535e2a379250b 100644 --- a/pkg/storage/bloom/v1/bloom_querier.go +++ b/pkg/storage/bloom/v1/bloom_querier.go @@ -7,6 +7,8 @@ type BloomQuerier interface { } type LazyBloomIter struct { + usePool bool + b *Block // state @@ -16,9 +18,15 @@ type LazyBloomIter struct { curPage *BloomPageDecoder } -func NewLazyBloomIter(b *Block) *LazyBloomIter { +// NewLazyBloomIter returns a new lazy bloom iterator. +// If pool is true, the underlying byte slice of the bloom page +// will be returned to the pool for efficiency. +// This can only safely be used when the underlying bloom +// bytes don't escape the decoder. +func NewLazyBloomIter(b *Block, pool bool) *LazyBloomIter { return &LazyBloomIter{ - b: b, + usePool: pool, + b: b, } } @@ -39,12 +47,18 @@ func (it *LazyBloomIter) Seek(offset BloomOffset) { // load the desired page if it.curPageIndex != offset.Page || it.curPage == nil { + // drop the current page if it exists and + // we're using the pool + if it.curPage != nil && it.usePool { + it.curPage.Relinquish() + } + r, err := it.b.reader.Blooms() if err != nil { it.err = errors.Wrap(err, "getting blooms reader") return } - decoder, err := it.b.blooms.BloomPageDecoder(r, offset.Page) + decoder, err := it.b.blooms.BloomPageDecoder(r, offset.Page, it.b.metrics) if err != nil { it.err = errors.Wrap(err, "loading bloom page") return @@ -83,6 +97,7 @@ func (it *LazyBloomIter) next() bool { it.curPage, err = it.b.blooms.BloomPageDecoder( r, it.curPageIndex, + it.b.metrics, ) if err != nil { it.err = err @@ -96,8 +111,14 @@ func (it *LazyBloomIter) next() bool { if it.curPage.Err() != nil { return false } + // we've exhausted the current page, progress to next it.curPageIndex++ + // drop the current page if it exists and + // we're using the pool + if it.usePool { + it.curPage.Relinquish() + } it.curPage = nil continue } diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index fd538ed161c09..542ba2c972790 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -3,12 +3,15 @@ package v1 import ( "fmt" "hash" + "math" "strings" + "unsafe" "github.com/pkg/errors" "github.com/prometheus/common/model" "golang.org/x/exp/slices" + "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/util/encoding" ) @@ -20,6 +23,30 @@ const ( After ) +var ( + // FullBounds is the bounds that covers the entire fingerprint space + FullBounds = NewBounds(0, model.Fingerprint(math.MaxUint64)) +) + +type FingerprintBounds struct { + Min, Max model.Fingerprint +} + +// Proto compat +// compiler check ensuring equal repr of underlying types +var _ FingerprintBounds = FingerprintBounds(logproto.FPBounds{}) + +func BoundsFromProto(pb logproto.FPBounds) FingerprintBounds { + return FingerprintBounds(pb) +} + +// Unsafe cast to avoid allocation. This _requires_ that the underlying types are the same +// which is checked by the compiler above +func MultiBoundsFromProto(pb []logproto.FPBounds) MultiFingerprintBounds { + //nolint:unconvert + return MultiFingerprintBounds(*(*MultiFingerprintBounds)(unsafe.Pointer(&pb))) +} + // ParseBoundsFromAddr parses a fingerprint bounds from a string func ParseBoundsFromAddr(s string) (FingerprintBounds, error) { parts := strings.Split(s, "-") @@ -40,10 +67,6 @@ func ParseBoundsFromParts(a, b string) (FingerprintBounds, error) { return NewBounds(minFingerprint, maxFingerprint), nil } -type FingerprintBounds struct { - Min, Max model.Fingerprint -} - func NewBounds(min, max model.Fingerprint) FingerprintBounds { return FingerprintBounds{Min: min, Max: max} } @@ -91,8 +114,16 @@ func (b FingerprintBounds) Match(fp model.Fingerprint) bool { return b.Cmp(fp) == Overlap } -// GetFromThrough implements TSDBs FingerprintFilter interface +// GetFromThrough implements TSDBs FingerprintFilter interface, +// NB(owen-d): adjusts to return `[from,through)` instead of `[from,through]` which the +// fingerprint bounds struct tracks. func (b FingerprintBounds) GetFromThrough() (model.Fingerprint, model.Fingerprint) { + from, through := b.Bounds() + return from, max(through+1, through) +} + +// Bounds returns the inclusive bounds [from,through] +func (b FingerprintBounds) Bounds() (model.Fingerprint, model.Fingerprint) { return b.Min, b.Max } diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go index 4dd01e60c1238..98fec7b0aafa0 100644 --- a/pkg/storage/bloom/v1/bounds_test.go +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -5,8 +5,29 @@ import ( "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" + + "github.com/grafana/loki/pkg/logproto" ) +func TestBoundsFromProto(t *testing.T) { + bounds := BoundsFromProto(logproto.FPBounds{ + Min: 10, + Max: 2000, + }) + assert.Equal(t, NewBounds(10, 2000), bounds) +} + +func TestMultiBoundsFromProto(t *testing.T) { + bounds := MultiBoundsFromProto([]logproto.FPBounds{ + {Min: 10, Max: 2000}, + {Min: 2001, Max: 4000}, + }) + assert.Equal(t, MultiFingerprintBounds{ + NewBounds(10, 2000), + NewBounds(2001, 4000), + }, bounds) +} + func Test_ParseFingerprint(t *testing.T) { t.Parallel() fp, err := model.ParseFingerprint("7d0") diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go index 8324475662c58..540d0a768ca7e 100644 --- a/pkg/storage/bloom/v1/builder_test.go +++ b/pkg/storage/bloom/v1/builder_test.go @@ -116,8 +116,8 @@ func TestBlockBuilder_RoundTrip(t *testing.T) { processedData = data[:lastProcessedIdx] } - block := NewBlock(tc.reader) - querier := NewBlockQuerier(block) + block := NewBlock(tc.reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, false) err = block.LoadHeaders() require.Nil(t, err) @@ -218,7 +218,7 @@ func TestMergeBuilder(t *testing.T) { itr := NewSliceIter[SeriesWithBloom](data[min:max]) _, err = builder.BuildFrom(itr) require.Nil(t, err) - blocks = append(blocks, NewPeekingIter[*SeriesWithBloom](NewBlockQuerier(NewBlock(reader)))) + blocks = append(blocks, NewPeekingIter[*SeriesWithBloom](NewBlockQuerier(NewBlock(reader, NewMetrics(nil)), false))) } // We're not testing the ability to extend a bloom in this test @@ -251,8 +251,8 @@ func TestMergeBuilder(t *testing.T) { _, _, err = mergeBuilder.Build(builder) require.Nil(t, err) - block := NewBlock(reader) - querier := NewBlockQuerier(block) + block := NewBlock(reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, false) EqualIterators[*SeriesWithBloom]( t, @@ -295,8 +295,8 @@ func TestBlockReset(t *testing.T) { itr := NewSliceIter[SeriesWithBloom](data) _, err = builder.BuildFrom(itr) require.Nil(t, err) - block := NewBlock(reader) - querier := NewBlockQuerier(block) + block := NewBlock(reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, false) rounds := make([][]model.Fingerprint, 2) @@ -361,8 +361,8 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { itr := NewSliceIter[SeriesWithBloom](xs[minIdx:maxIdx]) _, err = builder.BuildFrom(itr) require.Nil(t, err) - block := NewBlock(reader) - querier := NewBlockQuerier(block) + block := NewBlock(reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, false) // rather than use the block querier directly, collect it's data // so we can use it in a few places later @@ -423,7 +423,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { // ensure the new block contains one copy of all the data // by comparing it against an iterator over the source data - mergedBlockQuerier := NewBlockQuerier(NewBlock(reader)) + mergedBlockQuerier := NewBlockQuerier(NewBlock(reader, NewMetrics(nil)), false) sourceItr := NewSliceIter[*SeriesWithBloom](PointerSlice[SeriesWithBloom](xs)) EqualIterators[*SeriesWithBloom]( diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index b5a5c22984559..7ca7267b6ecc7 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -48,8 +48,8 @@ func TestFusedQuerier(t *testing.T) { _, err = builder.BuildFrom(itr) require.NoError(t, err) require.False(t, itr.Next()) - block := NewBlock(reader) - querier := NewBlockQuerier(block) + block := NewBlock(reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, true) n := 2 nReqs := numSeries / n @@ -142,8 +142,8 @@ func setupBlockForBenchmark(b *testing.B) (*BlockQuerier, [][]Request, []chan Ou itr := NewSliceIter[SeriesWithBloom](data) _, err = builder.BuildFrom(itr) require.Nil(b, err) - block := NewBlock(reader) - querier := NewBlockQuerier(block) + block := NewBlock(reader, NewMetrics(nil)) + querier := NewBlockQuerier(block, true) numRequestChains := 100 seriesPerRequest := 100 diff --git a/pkg/storage/bloom/v1/index.go b/pkg/storage/bloom/v1/index.go index 838ba891cb2f6..c69b4eb292be3 100644 --- a/pkg/storage/bloom/v1/index.go +++ b/pkg/storage/bloom/v1/index.go @@ -151,7 +151,16 @@ func (b *BlockIndex) DecodeHeaders(r io.ReadSeeker) (uint32, error) { } // decompress page and return an iterator over the bytes -func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHeaderWithOffset) (*SeriesPageDecoder, error) { +func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHeaderWithOffset, metrics *Metrics) (res *SeriesPageDecoder, err error) { + defer func() { + if err != nil { + metrics.pagesSkipped.WithLabelValues(pageTypeSeries, skipReasonErr).Inc() + metrics.bytesSkipped.WithLabelValues(pageTypeSeries).Add(float64(header.DecompressedLen)) + } else { + metrics.pagesRead.WithLabelValues(pageTypeSeries).Inc() + metrics.bytesRead.WithLabelValues(pageTypeSeries).Add(float64(header.DecompressedLen)) + } + }() if _, err := r.Seek(int64(header.Offset), io.SeekStart); err != nil { return nil, errors.Wrap(err, "seeking to series page") @@ -159,7 +168,7 @@ func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHead data := BlockPool.Get(header.Len)[:header.Len] defer BlockPool.Put(data) - _, err := io.ReadFull(r, data) + _, err = io.ReadFull(r, data) if err != nil { return nil, errors.Wrap(err, "reading series page") } @@ -180,7 +189,7 @@ func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHead return nil, errors.Wrap(err, "decompressing series page") } - res := &SeriesPageDecoder{ + res = &SeriesPageDecoder{ data: decompressed, header: header.SeriesHeader, } @@ -221,8 +230,8 @@ func aggregateHeaders(xs []SeriesHeader) SeriesHeader { return SeriesHeader{} } - fromFp, _ := xs[0].Bounds.GetFromThrough() - _, throughFP := xs[len(xs)-1].Bounds.GetFromThrough() + fromFp, _ := xs[0].Bounds.Bounds() + _, throughFP := xs[len(xs)-1].Bounds.Bounds() res := SeriesHeader{ Bounds: NewBounds(fromFp, throughFP), } diff --git a/pkg/storage/bloom/v1/index_querier.go b/pkg/storage/bloom/v1/index_querier.go index 142b6423185b6..8ba984d3df31c 100644 --- a/pkg/storage/bloom/v1/index_querier.go +++ b/pkg/storage/bloom/v1/index_querier.go @@ -74,6 +74,7 @@ func (it *LazySeriesIter) Seek(fp model.Fingerprint) error { it.curPage, err = it.b.index.NewSeriesPageDecoder( r, page, + it.b.metrics, ) if err != nil { return err @@ -107,6 +108,7 @@ func (it *LazySeriesIter) next() bool { it.curPage, err = it.b.index.NewSeriesPageDecoder( r, curHeader, + it.b.metrics, ) if err != nil { it.err = err diff --git a/pkg/storage/bloom/v1/metrics.go b/pkg/storage/bloom/v1/metrics.go index 138580a435054..c2204bf275505 100644 --- a/pkg/storage/bloom/v1/metrics.go +++ b/pkg/storage/bloom/v1/metrics.go @@ -3,6 +3,8 @@ package v1 import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/pkg/util/constants" ) type Metrics struct { @@ -15,6 +17,11 @@ type Metrics struct { blockSeriesIterated prometheus.Counter tokensTotal prometheus.Counter insertsTotal *prometheus.CounterVec + + pagesRead *prometheus.CounterVec + pagesSkipped *prometheus.CounterVec + bytesRead *prometheus.CounterVec + bytesSkipped *prometheus.CounterVec } const ( @@ -26,49 +33,86 @@ const ( collisionTypeFalse = "false" collisionTypeTrue = "true" collisionTypeCache = "cache" + + pageTypeBloom = "bloom" + pageTypeSeries = "series" + + skipReasonTooLarge = "too_large" + skipReasonErr = "err" + skipReasonOOB = "out_of_bounds" ) func NewMetrics(r prometheus.Registerer) *Metrics { return &Metrics{ sbfCreationTime: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Name: "bloom_creation_time_total", - Help: "Time spent creating scalable bloom filters", + Namespace: constants.Loki, + Name: "bloom_creation_time_total", + Help: "Time spent creating scalable bloom filters", }), bloomSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_size", - Help: "Size of the bloom filter in bytes", - Buckets: prometheus.ExponentialBucketsRange(128, 16777216, 8), + Namespace: constants.Loki, + Name: "bloom_size", + Help: "Size of the bloom filter in bytes", + Buckets: prometheus.ExponentialBucketsRange(1<<10, 512<<20, 10), }), hammingWeightRatio: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_hamming_weight_ratio", - Help: "Ratio of the hamming weight of the bloom filter to the number of bits in the bloom filter", - Buckets: prometheus.ExponentialBucketsRange(0.001, 1, 12), + Namespace: constants.Loki, + Name: "bloom_hamming_weight_ratio", + Help: "Ratio of the hamming weight of the bloom filter to the number of bits in the bloom filter", + Buckets: prometheus.ExponentialBucketsRange(0.001, 1, 12), }), estimatedCount: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_estimated_count", - Help: "Estimated number of elements in the bloom filter", - Buckets: prometheus.ExponentialBucketsRange(1, 33554432, 10), + Namespace: constants.Loki, + Name: "bloom_estimated_count", + Help: "Estimated number of elements in the bloom filter", + Buckets: prometheus.ExponentialBucketsRange(1, 33554432, 10), }), chunksIndexed: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Name: "bloom_chunks_indexed_total", - Help: "Number of chunks indexed in bloom filters, partitioned by type. Type can be iterated or copied, where iterated indicates the chunk data was fetched and ngrams for it's contents generated whereas copied indicates the chunk already existed in another source block and was copied to the new block", + Namespace: constants.Loki, + Name: "bloom_chunks_indexed_total", + Help: "Number of chunks indexed in bloom filters, partitioned by type. Type can be iterated or copied, where iterated indicates the chunk data was fetched and ngrams for it's contents generated whereas copied indicates the chunk already existed in another source block and was copied to the new block", }, []string{"type"}), chunksPerSeries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_chunks_per_series", - Help: "Number of chunks per series. Can be copied from an existing bloom or iterated", - Buckets: prometheus.ExponentialBucketsRange(1, 4096, 10), + Namespace: constants.Loki, + Name: "bloom_chunks_per_series", + Help: "Number of chunks per series. Can be copied from an existing bloom or iterated", + Buckets: prometheus.ExponentialBucketsRange(1, 4096, 10), }), blockSeriesIterated: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Name: "bloom_block_series_iterated_total", - Help: "Number of series iterated in existing blocks while generating new blocks", + Namespace: constants.Loki, + Name: "bloom_block_series_iterated_total", + Help: "Number of series iterated in existing blocks while generating new blocks", }), tokensTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Name: "bloom_tokens_total", - Help: "Number of tokens processed", + Namespace: constants.Loki, + Name: "bloom_tokens_total", + Help: "Number of tokens processed", }), insertsTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Name: "bloom_inserts_total", - Help: "Number of inserts into the bloom filter. collision type may be `false` (no collision), `cache` (found in token cache) or true (found in bloom filter). token_type may be either `raw` (the original ngram) or `chunk_prefixed` (the ngram with the chunk prefix)", + Namespace: constants.Loki, + Name: "bloom_inserts_total", + Help: "Number of inserts into the bloom filter. collision type may be `false` (no collision), `cache` (found in token cache) or true (found in bloom filter). token_type may be either `raw` (the original ngram) or `chunk_prefixed` (the ngram with the chunk prefix)", }, []string{"token_type", "collision"}), + + pagesRead: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "bloom_pages_read_total", + Help: "Number of bloom pages read", + }, []string{"type"}), + pagesSkipped: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "bloom_pages_skipped_total", + Help: "Number of bloom pages skipped during query iteration", + }, []string{"type", "reason"}), + bytesRead: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "bloom_bytes_read_total", + Help: "Number of bytes read from bloom pages", + }, []string{"type"}), + bytesSkipped: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: constants.Loki, + Name: "bloom_bytes_skipped_total", + Help: "Number of bytes skipped during query iteration", + }, []string{"type", "reason"}), } } diff --git a/pkg/storage/bloom/v1/test_util.go b/pkg/storage/bloom/v1/test_util.go index 3cd021de5f5ee..ed1dd5cc978c1 100644 --- a/pkg/storage/bloom/v1/test_util.go +++ b/pkg/storage/bloom/v1/test_util.go @@ -41,7 +41,7 @@ func MakeBlock(t testing.TB, nth int, fromFp, throughFp model.Fingerprint, fromT itr := NewSliceIter[SeriesWithBloom](data) _, err = builder.BuildFrom(itr) require.Nil(t, err) - block := NewBlock(reader) + block := NewBlock(reader, NewMetrics(nil)) return block, data, keys } diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index e6ad69a248fed..6b83e298a3957 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -32,10 +32,10 @@ var ( }, } - // 4KB -> 64MB + // 4KB -> 128MB BlockPool = BytePool{ pool: pool.New( - 4<<10, 64<<20, 4, + 4<<10, 128<<20, 4, func(size int) interface{} { return make([]byte, size) }), diff --git a/pkg/storage/chunk/predicate.go b/pkg/storage/chunk/predicate.go index 62a91c7a46437..e200f28fb55e8 100644 --- a/pkg/storage/chunk/predicate.go +++ b/pkg/storage/chunk/predicate.go @@ -6,6 +6,7 @@ import ( "github.com/grafana/loki/pkg/querier/plan" ) +// TODO(owen-d): rename. This is not a predicate and is confusing. type Predicate struct { Matchers []*labels.Matcher plan *plan.QueryPlan diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 706f630931cd2..b582f7e6c7156 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -364,9 +364,11 @@ func decodeReq(req logql.QueryParams) ([]*labels.Matcher, model.Time, model.Time return matchers, from, through, nil } +// TODO(owen-d): refactor this. Injecting shard labels via matchers is a big hack and we shouldn't continue +// doing it, _but_ it requires adding `fingerprintfilter` support to much of our storage interfaces func injectShardLabel(shards []string, matchers []*labels.Matcher) ([]*labels.Matcher, error) { if shards != nil { - parsed, err := logql.ParseShards(shards) + parsed, _, err := logql.ParseShards(shards) if err != nil { return nil, err } diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index d0fb516dcbff4..7df10aa076480 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -13,6 +13,8 @@ import ( "github.com/grafana/loki/pkg/storage/stores/index" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/grafana/loki/pkg/storage/stores/index/stats" + tsdb_index "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util" ) @@ -206,6 +208,83 @@ func (c CompositeStore) Volume(ctx context.Context, userID string, from, through return res, err } +func (c CompositeStore) GetShards( + ctx context.Context, + userID string, + from, through model.Time, + targetBytesPerShard uint64, + predicate chunk.Predicate, +) (*logproto.ShardsResponse, error) { + // TODO(owen-d): improve. Since shards aren't easily merge-able, + // we choose the store which returned the highest shard count. + // This is only used when a query crosses a schema boundary + var groups []*logproto.ShardsResponse + err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { + shards, err := store.GetShards(innerCtx, userID, from, through, targetBytesPerShard, predicate) + if err != nil { + return err + } + groups = append(groups, shards) + return nil + }) + + if err != nil { + return nil, err + } + + switch { + case len(groups) == 1: + return groups[0], nil + case len(groups) == 0: + return nil, nil + default: + sort.Slice(groups, func(i, j int) bool { + return len(groups[i].Shards) > len(groups[j].Shards) + }) + return groups[0], nil + } +} + +func (c CompositeStore) HasForSeries(from, through model.Time) (sharding.ForSeries, bool) { + var impls []sharding.ForSeries + _ = c.forStores(context.Background(), from, through, func(_ context.Context, from, through model.Time, store Store) error { + impl, ok := store.HasForSeries(from, through) + if ok { + impls = append(impls, impl) + } + return nil + }) + + if len(impls) == 0 { + return nil, false + } + + wrapped := sharding.ForSeriesFunc( + func( + ctx context.Context, + userID string, + fpFilter tsdb_index.FingerprintFilter, + from model.Time, + through model.Time, + fn func( + labels.Labels, + model.Fingerprint, + []tsdb_index.ChunkMeta, + ) (stop bool), + matchers ...*labels.Matcher, + ) error { + for _, impl := range impls { + if err := impl.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...); err != nil { + return err + } + } + return nil + }, + ) + + return wrapped, true +} + func (c CompositeStore) GetChunkFetcher(tm model.Time) *fetcher.Fetcher { // find the schema with the lowest start _after_ tm j := sort.Search(len(c.stores), func(j int) bool { diff --git a/pkg/storage/stores/composite_store_entry.go b/pkg/storage/stores/composite_store_entry.go index 7edbdab404fe6..200b701449a5d 100644 --- a/pkg/storage/stores/composite_store_entry.go +++ b/pkg/storage/stores/composite_store_entry.go @@ -19,6 +19,7 @@ import ( "github.com/grafana/loki/pkg/storage/errors" "github.com/grafana/loki/pkg/storage/stores/index" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" util_log "github.com/grafana/loki/pkg/util/log" "github.com/grafana/loki/pkg/util/spanlogger" "github.com/grafana/loki/pkg/util/validation" @@ -143,6 +144,25 @@ func (c *storeEntry) Volume(ctx context.Context, userID string, from, through mo return c.indexReader.Volume(ctx, userID, from, through, limit, targetLabels, aggregateBy, matchers...) } +func (c *storeEntry) GetShards( + ctx context.Context, + userID string, + from, through model.Time, + targetBytesPerShard uint64, + predicate chunk.Predicate, +) (*logproto.ShardsResponse, error) { + _, err := c.validateQueryTimeRange(ctx, userID, &from, &through) + if err != nil { + return nil, err + } + + return c.indexReader.GetShards(ctx, userID, from, through, targetBytesPerShard, predicate) +} + +func (c *storeEntry) HasForSeries(from, through model.Time) (sharding.ForSeries, bool) { + return c.indexReader.HasForSeries(from, through) +} + func (c *storeEntry) validateQueryTimeRange(ctx context.Context, userID string, from *model.Time, through *model.Time) (bool, error) { //nolint:ineffassign,staticcheck //Leaving ctx even though we don't currently use it, we want to make it available for when we might need it and hopefully will ensure us using the correct context at that time diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index b2878487f4606..d3fc6ba4dacfc 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -18,6 +18,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/fetcher" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" ) type mockStore int @@ -60,6 +61,14 @@ func (m mockStore) Volume(_ context.Context, _ string, _, _ model.Time, _ int32, return nil, nil } +func (m mockStore) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) (*logproto.ShardsResponse, error) { + return nil, nil +} + +func (m mockStore) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + func (m mockStore) Stop() {} func TestCompositeStore(t *testing.T) { diff --git a/pkg/storage/stores/index/index.go b/pkg/storage/stores/index/index.go index 41746e346f20b..db3ccfd2578e6 100644 --- a/pkg/storage/stores/index/index.go +++ b/pkg/storage/stores/index/index.go @@ -11,6 +11,8 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" loki_instrument "github.com/grafana/loki/pkg/util/instrument" ) @@ -28,6 +30,17 @@ type BaseReader interface { type StatsReader interface { Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) Volume(ctx context.Context, userID string, from, through model.Time, limit int32, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) (*logproto.VolumeResponse, error) + GetShards( + ctx context.Context, + userID string, + from, through model.Time, + targetBytesPerShard uint64, + predicate chunk.Predicate, + ) (*logproto.ShardsResponse, error) + + // If the underlying index supports it, this will return the ForSeries interface + // which is used in bloom-filter accelerated sharding calculation optimization. + HasForSeries(from, through model.Time) (sharding.ForSeries, bool) } type Reader interface { @@ -137,6 +150,24 @@ func (m MonitoredReaderWriter) Volume(ctx context.Context, userID string, from, return vol, nil } +func (m MonitoredReaderWriter) GetShards( + ctx context.Context, + userID string, + from, through model.Time, + targetBytesPerShard uint64, + predicate chunk.Predicate, +) (*logproto.ShardsResponse, error) { + var shards *logproto.ShardsResponse + if err := loki_instrument.TimeRequest(ctx, "shards", instrument.NewHistogramCollector(m.metrics.indexQueryLatency), instrument.ErrorCode, func(ctx context.Context) error { + var err error + shards, err = m.rw.GetShards(ctx, userID, from, through, targetBytesPerShard, predicate) + return err + }); err != nil { + return nil, err + } + return shards, nil +} + func (m MonitoredReaderWriter) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { m.rw.SetChunkFilterer(chunkFilter) } @@ -146,3 +177,29 @@ func (m MonitoredReaderWriter) IndexChunk(ctx context.Context, from, through mod return m.rw.IndexChunk(ctx, from, through, chk) }) } + +func (m MonitoredReaderWriter) HasForSeries(from, through model.Time) (sharding.ForSeries, bool) { + if impl, ok := m.rw.HasForSeries(from, through); ok { + wrapped := sharding.ForSeriesFunc( + func( + 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 { + 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 wrapped, true + } + return nil, false +} diff --git a/pkg/storage/stores/series/series_index_gateway_store.go b/pkg/storage/stores/series/series_index_gateway_store.go index 00059fe16c1a3..c3af8c0c8d3c6 100644 --- a/pkg/storage/stores/series/series_index_gateway_store.go +++ b/pkg/storage/stores/series/series_index_gateway_store.go @@ -13,15 +13,29 @@ import ( "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" ) +// NB(owen-d): mostly modeled off of the proto-generated `logproto.IndexGatewayClient`, +// but decoupled from explicit GRPC dependencies to work well with streaming grpc methods +type GatewayClient interface { + GetChunkRef(ctx context.Context, in *logproto.GetChunkRefRequest) (*logproto.GetChunkRefResponse, error) + GetSeries(ctx context.Context, in *logproto.GetSeriesRequest) (*logproto.GetSeriesResponse, error) + LabelNamesForMetricName(ctx context.Context, in *logproto.LabelNamesForMetricNameRequest) (*logproto.LabelResponse, error) + LabelValuesForMetricName(ctx context.Context, in *logproto.LabelValuesForMetricNameRequest) (*logproto.LabelResponse, error) + GetStats(ctx context.Context, in *logproto.IndexStatsRequest) (*logproto.IndexStatsResponse, error) + GetVolume(ctx context.Context, in *logproto.VolumeRequest) (*logproto.VolumeResponse, error) + + GetShards(ctx context.Context, in *logproto.ShardsRequest) (*logproto.ShardsResponse, error) +} + // IndexGatewayClientStore implements pkg/storage/stores/index.ReaderWriter type IndexGatewayClientStore struct { - client logproto.IndexGatewayClient + client GatewayClient logger log.Logger } -func NewIndexGatewayClientStore(client logproto.IndexGatewayClient, logger log.Logger) *IndexGatewayClientStore { +func NewIndexGatewayClientStore(client GatewayClient, logger log.Logger) *IndexGatewayClientStore { return &IndexGatewayClientStore{ client: client, logger: logger, @@ -111,6 +125,25 @@ func (c *IndexGatewayClientStore) Volume(ctx context.Context, _ string, from, th }) } +func (c *IndexGatewayClientStore) GetShards( + ctx context.Context, + _ string, + from, through model.Time, + targetBytesPerShard uint64, + predicate chunk.Predicate, +) (*logproto.ShardsResponse, error) { + resp, err := c.client.GetShards(ctx, &logproto.ShardsRequest{ + From: from, + Through: through, + Query: predicate.Plan().AST.String(), + TargetBytesPerShard: targetBytesPerShard, + }) + if err != nil { + return nil, err + } + return resp, nil +} + func (c *IndexGatewayClientStore) SetChunkFilterer(_ chunk.RequestChunkFilterer) { level.Warn(c.logger).Log("msg", "SetChunkFilterer called on index gateway client store, but it does not support it") } @@ -118,3 +151,10 @@ func (c *IndexGatewayClientStore) SetChunkFilterer(_ chunk.RequestChunkFilterer) func (c *IndexGatewayClientStore) IndexChunk(_ context.Context, _, _ model.Time, _ chunk.Chunk) error { return fmt.Errorf("index writes not supported on index gateway client") } + +// IndexGatewayClientStore does not implement tsdb.ForSeries; +// that is implemented by the index-gws themselves and will be +// called during the `GetShards() invocation` +func (c *IndexGatewayClientStore) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} diff --git a/pkg/storage/stores/series/series_index_gateway_store_test.go b/pkg/storage/stores/series/series_index_gateway_store_test.go index 125973eb49d42..8c65881d3c5b3 100644 --- a/pkg/storage/stores/series/series_index_gateway_store_test.go +++ b/pkg/storage/stores/series/series_index_gateway_store_test.go @@ -7,20 +7,15 @@ import ( "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" - "google.golang.org/grpc" "github.com/grafana/loki/pkg/logproto" ) type fakeClient struct { - logproto.IndexGatewayClient + GatewayClient } -func (fakeClient) GetChunkRef(_ context.Context, _ *logproto.GetChunkRefRequest, _ ...grpc.CallOption) (*logproto.GetChunkRefResponse, error) { - return &logproto.GetChunkRefResponse{}, nil -} - -func (fakeClient) GetSeries(_ context.Context, _ *logproto.GetSeriesRequest, _ ...grpc.CallOption) (*logproto.GetSeriesResponse, error) { +func (fakeClient) GetSeries(_ context.Context, _ *logproto.GetSeriesRequest) (*logproto.GetSeriesResponse, error) { return &logproto.GetSeriesResponse{}, nil } diff --git a/pkg/storage/stores/series/series_index_store.go b/pkg/storage/stores/series/series_index_store.go index 5d17963180556..50a036db3762c 100644 --- a/pkg/storage/stores/series/series_index_store.go +++ b/pkg/storage/stores/series/series_index_store.go @@ -3,6 +3,7 @@ package series import ( "context" "fmt" + "math" "sort" "sync" @@ -26,6 +27,7 @@ import ( "github.com/grafana/loki/pkg/storage/stores" "github.com/grafana/loki/pkg/storage/stores/index/stats" series_index "github.com/grafana/loki/pkg/storage/stores/series/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/constants" "github.com/grafana/loki/pkg/util/extract" @@ -758,3 +760,30 @@ func (c *IndexReaderWriter) Stats(_ context.Context, _ string, _, _ model.Time, func (c *IndexReaderWriter) Volume(_ context.Context, _ string, _, _ model.Time, _ int32, _ []string, _ string, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) { return nil, nil } + +// old index stores do not implement dynamic sharidng -- skip +func (c *IndexReaderWriter) GetShards( + _ context.Context, + _ string, + _, _ model.Time, + _ uint64, + _ chunk.Predicate, +) (*logproto.ShardsResponse, error) { + // should not be called for legacy indices at all, so just return a single shard covering everything + // could be improved by reading schema shards + return &logproto.ShardsResponse{ + Shards: []logproto.Shard{ + { + Bounds: logproto.FPBounds{ + Min: 0, + Max: math.MaxUint64, + }, + }, + }, + }, nil +} + +// old index stores do not implement tsdb.ForSeries -- skip +func (c *IndexReaderWriter) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go index b4118a55f61fb..ebd2d6a53ff8b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -95,8 +95,8 @@ type BlockDirectory struct { size int64 } -func (b BlockDirectory) Block() *v1.Block { - return v1.NewBlock(v1.NewDirectoryBlockReader(b.Path)) +func (b BlockDirectory) Block(metrics *v1.Metrics) *v1.Block { + return v1.NewBlock(v1.NewDirectoryBlockReader(b.Path), metrics) } func (b BlockDirectory) Size() int64 { @@ -120,9 +120,13 @@ func (b *BlockDirectory) resolveSize() error { // BlockQuerier returns a new block querier from the directory. // The passed function `close` is called when the the returned querier is closed. -func (b BlockDirectory) BlockQuerier(close func() error) *CloseableBlockQuerier { +func (b BlockDirectory) BlockQuerier( + usePool bool, + close func() error, + metrics *v1.Metrics, +) *CloseableBlockQuerier { return &CloseableBlockQuerier{ - BlockQuerier: v1.NewBlockQuerier(b.Block()), + BlockQuerier: v1.NewBlockQuerier(b.Block(metrics), usePool), BlockRef: b.BlockRef, close: close, } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index e5779a7294acf..f2e40534a0957 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -26,6 +26,10 @@ var downloadQueueCapacity = 10000 type options struct { ignoreNotFound bool // ignore 404s from object storage; default=true fetchAsync bool // dispatch downloading of block and return immediately; default=false + // return bloom blocks to pool after iteration; default=false + // NB(owen-d): this can only be safely used when blooms are not captured outside + // of iteration or it can introduce use-after-free bugs + usePool bool } func (o *options) apply(opts ...FetchOption) { @@ -48,6 +52,12 @@ func WithFetchAsync(v bool) FetchOption { } } +func WithPool(v bool) FetchOption { + return func(opts *options) { + opts.usePool = v + } +} + type fetcher interface { FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) FetchBlocks(ctx context.Context, refs []BlockRef, opts ...FetchOption) ([]*CloseableBlockQuerier, error) @@ -66,12 +76,21 @@ type Fetcher struct { q *downloadQueue[BlockRef, BlockDirectory] - cfg bloomStoreConfig - metrics *fetcherMetrics - logger log.Logger + cfg bloomStoreConfig + metrics *fetcherMetrics + bloomMetrics *v1.Metrics + logger log.Logger } -func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blocksCache Cache, reg prometheus.Registerer, logger log.Logger) (*Fetcher, error) { +func NewFetcher( + cfg bloomStoreConfig, + client Client, + metasCache cache.Cache, + blocksCache Cache, + reg prometheus.Registerer, + logger log.Logger, + bloomMetrics *v1.Metrics, +) (*Fetcher, error) { fetcher := &Fetcher{ cfg: cfg, client: client, @@ -79,6 +98,7 @@ func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blo blocksCache: blocksCache, localFSResolver: NewPrefixedResolver(cfg.workingDir, defaultKeyResolver{}), metrics: newFetcherMetrics(reg, constants.Loki, "bloom_store"), + bloomMetrics: bloomMetrics, logger: logger, } q, err := newDownloadQueue[BlockRef, BlockDirectory](downloadQueueCapacity, cfg.numWorkers, fetcher.processTask, logger) @@ -176,7 +196,7 @@ func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { // FetchBlocks implements fetcher func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef, opts ...FetchOption) ([]*CloseableBlockQuerier, error) { // apply fetch options - cfg := &options{ignoreNotFound: true, fetchAsync: false} + cfg := &options{ignoreNotFound: true, fetchAsync: false, usePool: false} cfg.apply(opts...) // first, resolve blocks from cache and enqueue missing blocks to download queue @@ -218,9 +238,13 @@ func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef, opts ...Fetc } found++ f.metrics.blocksFound.Inc() - results[i] = dir.BlockQuerier(func() error { - return f.blocksCache.Release(ctx, key) - }) + results[i] = dir.BlockQuerier( + cfg.usePool, + func() error { + return f.blocksCache.Release(ctx, key) + }, + f.bloomMetrics, + ) } // fetchAsync defines whether the function may return early or whether it @@ -248,9 +272,13 @@ func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef, opts ...Fetc case res := <-responses: found++ key := f.client.Block(refs[res.idx]).Addr() - results[res.idx] = res.item.BlockQuerier(func() error { - return f.blocksCache.Release(ctx, key) - }) + results[res.idx] = res.item.BlockQuerier( + cfg.usePool, + func() error { + return f.blocksCache.Release(ctx, key) + }, + f.bloomMetrics, + ) } } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index 78a681dac5014..e51d153098381 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -108,7 +108,7 @@ func TestMetasFetcher(t *testing.T) { c, err := NewBloomClient(cfg, oc, logger) require.NoError(t, err) - fetcher, err := NewFetcher(cfg, c, metasCache, nil, nil, logger) + fetcher, err := NewFetcher(cfg, c, metasCache, nil, nil, logger, v1.NewMetrics(nil)) require.NoError(t, err) // prepare metas cache @@ -286,7 +286,7 @@ func TestFetcher_LoadBlocksFromFS(t *testing.T) { c, err := NewBloomClient(cfg, oc, log.NewNopLogger()) require.NoError(t, err) - fetcher, err := NewFetcher(cfg, c, nil, nil, nil, log.NewNopLogger()) + fetcher, err := NewFetcher(cfg, c, nil, nil, nil, log.NewNopLogger(), v1.NewMetrics(nil)) require.NoError(t, err) found, missing, err := fetcher.loadBlocksFromFS(context.Background(), refs) @@ -314,7 +314,7 @@ func createBlockDir(t *testing.T, path string) { func TestFetcher_IsBlockDir(t *testing.T) { cfg := bloomStoreConfig{numWorkers: 1} - fetcher, _ := NewFetcher(cfg, nil, nil, nil, nil, log.NewNopLogger()) + fetcher, _ := NewFetcher(cfg, nil, nil, nil, nil, log.NewNopLogger(), v1.NewMetrics(nil)) t.Run("path does not exist", func(t *testing.T) { base := t.TempDir() diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 9f099d683cf4e..c8a162dbd66b9 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -35,6 +35,11 @@ type Store interface { Stop() } +type StoreWithMetrics interface { + Store + BloomMetrics() *v1.Metrics +} + type bloomStoreConfig struct { workingDir string numWorkers int @@ -123,8 +128,8 @@ func (b *bloomStoreEntry) FetchMetas(ctx context.Context, params MetaSearchParam } // FetchBlocks implements Store. -func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef, _ ...FetchOption) ([]*CloseableBlockQuerier, error) { - return b.fetcher.FetchBlocks(ctx, refs) +func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef, opts ...FetchOption) ([]*CloseableBlockQuerier, error) { + return b.fetcher.FetchBlocks(ctx, refs, opts...) } // Fetcher implements Store. @@ -144,12 +149,14 @@ func (b bloomStoreEntry) Stop() { } // Compiler check to ensure BloomStore implements the Store interface -var _ Store = &BloomStore{} +var _ StoreWithMetrics = &BloomStore{} type BloomStore struct { - stores []*bloomStoreEntry - storageConfig storage.Config - metrics *storeMetrics + stores []*bloomStoreEntry + storageConfig storage.Config + metrics *storeMetrics + bloomMetrics *v1.Metrics + logger log.Logger defaultKeyResolver // TODO(owen-d): impl schema aware resolvers } @@ -166,6 +173,7 @@ func NewBloomStore( store := &BloomStore{ storageConfig: storageConfig, metrics: newStoreMetrics(reg, constants.Loki, "bloom_store"), + bloomMetrics: v1.NewMetrics(reg), logger: logger, } @@ -204,7 +212,7 @@ func NewBloomStore( } regWithLabels := prometheus.WrapRegistererWith(prometheus.Labels{"store": periodicConfig.From.String()}, reg) - fetcher, err := NewFetcher(cfg, bloomClient, metasCache, blocksCache, regWithLabels, logger) + fetcher, err := NewFetcher(cfg, bloomClient, metasCache, blocksCache, regWithLabels, logger, store.bloomMetrics) if err != nil { return nil, errors.Wrapf(err, "creating fetcher for period %s", periodicConfig.From) } @@ -221,6 +229,10 @@ func NewBloomStore( return store, nil } +func (b *BloomStore) BloomMetrics() *v1.Metrics { + return b.bloomMetrics +} + // Impements KeyResolver func (b *BloomStore) Meta(ref MetaRef) (loc Location) { _ = b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { diff --git a/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go b/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go index 69553fc34b6c3..a2054e806b0cf 100644 --- a/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go +++ b/pkg/storage/stores/shipper/indexshipper/gatewayclient/gateway_client.go @@ -10,6 +10,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/gogo/status" "github.com/grafana/dskit/concurrency" "github.com/grafana/dskit/grpcclient" "github.com/grafana/dskit/instrument" @@ -23,11 +24,14 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "google.golang.org/grpc" + "google.golang.org/grpc/codes" "github.com/grafana/loki/pkg/distributor/clientpool" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/storage/stores/series/index" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/indexgateway" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/constants" "github.com/grafana/loki/pkg/util/discovery" util_math "github.com/grafana/loki/pkg/util/math" @@ -237,78 +241,184 @@ func (s *GatewayClient) QueryIndex(_ context.Context, _ *logproto.QueryIndexRequ panic("not implemented") } -func (s *GatewayClient) GetChunkRef(ctx context.Context, in *logproto.GetChunkRefRequest, opts ...grpc.CallOption) (*logproto.GetChunkRefResponse, error) { +func (s *GatewayClient) GetChunkRef(ctx context.Context, in *logproto.GetChunkRefRequest) (*logproto.GetChunkRefResponse, error) { var ( resp *logproto.GetChunkRefResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.GetChunkRef(ctx, in, opts...) + resp, err = client.GetChunkRef(ctx, in) return err }) return resp, err } -func (s *GatewayClient) GetSeries(ctx context.Context, in *logproto.GetSeriesRequest, opts ...grpc.CallOption) (*logproto.GetSeriesResponse, error) { +func (s *GatewayClient) GetSeries(ctx context.Context, in *logproto.GetSeriesRequest) (*logproto.GetSeriesResponse, error) { var ( resp *logproto.GetSeriesResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.GetSeries(ctx, in, opts...) + resp, err = client.GetSeries(ctx, in) return err }) return resp, err } -func (s *GatewayClient) LabelNamesForMetricName(ctx context.Context, in *logproto.LabelNamesForMetricNameRequest, opts ...grpc.CallOption) (*logproto.LabelResponse, error) { +func (s *GatewayClient) LabelNamesForMetricName(ctx context.Context, in *logproto.LabelNamesForMetricNameRequest) (*logproto.LabelResponse, error) { var ( resp *logproto.LabelResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.LabelNamesForMetricName(ctx, in, opts...) + resp, err = client.LabelNamesForMetricName(ctx, in) return err }) return resp, err } -func (s *GatewayClient) LabelValuesForMetricName(ctx context.Context, in *logproto.LabelValuesForMetricNameRequest, opts ...grpc.CallOption) (*logproto.LabelResponse, error) { +func (s *GatewayClient) LabelValuesForMetricName(ctx context.Context, in *logproto.LabelValuesForMetricNameRequest) (*logproto.LabelResponse, error) { var ( resp *logproto.LabelResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.LabelValuesForMetricName(ctx, in, opts...) + resp, err = client.LabelValuesForMetricName(ctx, in) return err }) return resp, err } -func (s *GatewayClient) GetStats(ctx context.Context, in *logproto.IndexStatsRequest, opts ...grpc.CallOption) (*logproto.IndexStatsResponse, error) { +func (s *GatewayClient) GetStats(ctx context.Context, in *logproto.IndexStatsRequest) (*logproto.IndexStatsResponse, error) { var ( resp *logproto.IndexStatsResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.GetStats(ctx, in, opts...) + resp, err = client.GetStats(ctx, in) return err }) return resp, err } -func (s *GatewayClient) GetVolume(ctx context.Context, in *logproto.VolumeRequest, opts ...grpc.CallOption) (*logproto.VolumeResponse, error) { +func (s *GatewayClient) GetVolume(ctx context.Context, in *logproto.VolumeRequest) (*logproto.VolumeResponse, error) { var ( resp *logproto.VolumeResponse err error ) err = s.poolDo(ctx, func(client logproto.IndexGatewayClient) error { - resp, err = client.GetVolume(ctx, in, opts...) + resp, err = client.GetVolume(ctx, in) return err }) return resp, err } +func (s *GatewayClient) GetShards( + ctx context.Context, + in *logproto.ShardsRequest, +) (res *logproto.ShardsResponse, err error) { + + // We try to get the shards from the index gateway, + // but if it's not implemented, we fall back to the stats. + // We limit the maximum number of errors to 2 to avoid + // cascading all requests to new node(s) when + // the idx-gw replicas start to update to a version + // which supports the new API. + var ( + maxErrs = 2 + errCt int + ) + + if err := s.poolDoWithStrategy( + ctx, + func(client logproto.IndexGatewayClient) error { + perReplicaResult := &logproto.ShardsResponse{} + streamer, err := client.GetShards(ctx, in) + if err != nil { + return errors.Wrap(err, "get shards") + } + + // TODO(owen-d): stream currently unused (buffered) because query planning doesn't expect a streamed response, + // but can be improved easily in the future by using a stream here. + for { + resp, err := streamer.Recv() + if err == io.EOF { + break + } + if err != nil { + return errors.WithStack(err) + } + perReplicaResult.Shards = append(perReplicaResult.Shards, resp.Shards...) + perReplicaResult.Statistics.Merge(resp.Statistics) + } + + // Since `poolDo` retries on error, we only want to set the response if we got a successful response. + // This avoids cases where we add duplicates to the response on retries. + res = perReplicaResult + + return nil + }, + func(err error) bool { + errCt++ + return errCt <= maxErrs + }, + ); err != nil { + if isUnimplementedCallError(err) { + return s.getShardsFromStatsFallback(ctx, in) + } + return nil, err + } + return res, nil +} + +func (s *GatewayClient) getShardsFromStatsFallback( + ctx context.Context, + in *logproto.ShardsRequest, +) (*logproto.ShardsResponse, error) { + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, errors.Wrap(err, "index gateway client get tenant ID") + } + + p, err := indexgateway.ExtractShardRequestMatchersAndAST(in.Query) + if err != nil { + return nil, errors.Wrap(err, "failure while falling back to stats for shard calculation") + + } + + stats, err := s.GetStats( + ctx, + &logproto.IndexStatsRequest{ + From: in.From, + Through: in.Through, + Matchers: (&syntax.MatchersExpr{Mts: p.Matchers}).String(), + }, + ) + if err != nil { + return nil, err + } + + var strategy sharding.PowerOfTwoSharding + shards := strategy.ShardsFor(stats.Bytes, uint64(s.limits.TSDBMaxBytesPerShard(userID))) + return &logproto.ShardsResponse{ + Shards: shards, + }, nil +} + +// TODO(owen-d): this was copied from ingester_querier.go -- move it to a shared pkg +// isUnimplementedCallError tells if the GRPC error is a gRPC error with code Unimplemented. +func isUnimplementedCallError(err error) bool { + if err == nil { + return false + } + + s, ok := status.FromError(err) + if !ok { + return false + } + return (s.Code() == codes.Unimplemented) +} + func (s *GatewayClient) doQueries(ctx context.Context, queries []index.Query, callback index.QueryPagesCallback) error { queryKeyQueryMap := make(map[string]index.Query, len(queries)) gatewayQueries := make([]*logproto.IndexQuery, 0, len(queries)) @@ -365,6 +475,14 @@ func (s *GatewayClient) clientDoQueries(ctx context.Context, gatewayQueries []*l // poolDo executes the given function for each Index Gateway instance in the ring mapping to the correct tenant in the index. // In case of callback failure, we'll try another member of the ring for that tenant ID. func (s *GatewayClient) poolDo(ctx context.Context, callback func(client logproto.IndexGatewayClient) error) error { + return s.poolDoWithStrategy(ctx, callback, func(error) bool { return true }) +} + +func (s *GatewayClient) poolDoWithStrategy( + ctx context.Context, + callback func(client logproto.IndexGatewayClient) error, + shouldRetry func(error) bool, +) error { userID, err := tenant.TenantID(ctx) if err != nil { return errors.Wrap(err, "index gateway client get tenant ID") @@ -395,6 +513,10 @@ func (s *GatewayClient) poolDo(ctx context.Context, callback func(client logprot if err := callback(client); err != nil { lastErr = err level.Error(s.logger).Log("msg", fmt.Sprintf("client do failed for instance %s", addr), "err", err) + + if !shouldRetry(err) { + return err + } continue } diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go index a2325bd5c51bb..dc2662f3f60f6 100644 --- a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway.go @@ -3,9 +3,11 @@ package indexgateway import ( "context" "fmt" + "math" "sort" "sync" + "github.com/c2h5oh/datasize" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/services" @@ -17,13 +19,17 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/logqlmodel/stats" "github.com/grafana/loki/pkg/querier/plan" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores" "github.com/grafana/loki/pkg/storage/stores/index" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" seriesindex "github.com/grafana/loki/pkg/storage/stores/series/index" + tsdb_index "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/spanlogger" ) @@ -58,6 +64,7 @@ type Gateway struct { indexQuerier IndexQuerier indexClients []IndexClientWithRange bloomQuerier BloomQuerier + metrics *Metrics cfg Config log log.Logger @@ -67,13 +74,14 @@ type Gateway struct { // // In case it is configured to be in ring mode, a Basic Service wrapping the ring client is started. // Otherwise, it starts an Idle Service that doesn't have lifecycle hooks. -func NewIndexGateway(cfg Config, log log.Logger, _ prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange, bloomQuerier BloomQuerier) (*Gateway, error) { +func NewIndexGateway(cfg Config, log log.Logger, r prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange, bloomQuerier BloomQuerier) (*Gateway, error) { g := &Gateway{ indexQuerier: indexQuerier, bloomQuerier: bloomQuerier, cfg: cfg, log: log, indexClients: indexClients, + metrics: NewMetrics(r), } // query newer periods first @@ -194,7 +202,7 @@ func buildResponses(query seriesindex.Query, batch seriesindex.ReadBatchResult, return nil } -func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequest) (*logproto.GetChunkRefResponse, error) { +func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequest) (result *logproto.GetChunkRefResponse, err error) { instanceID, err := tenant.TenantID(ctx) if err != nil { return nil, err @@ -210,7 +218,7 @@ func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequ return nil, err } - result := &logproto.GetChunkRefResponse{ + result = &logproto.GetChunkRefResponse{ Refs: make([]*logproto.ChunkRef, 0, len(chunks)), } for _, cs := range chunks { @@ -220,6 +228,12 @@ func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequ } initialChunkCount := len(result.Refs) + defer func() { + if err == nil { + g.metrics.preFilterChunks.WithLabelValues(routeChunkRefs).Observe(float64(initialChunkCount)) + g.metrics.postFilterChunks.WithLabelValues(routeChunkRefs).Observe(float64(len(result.Refs))) + } + }() // Return unfiltered results if there is no bloom querier (Bloom Gateway disabled) if g.bloomQuerier == nil { @@ -339,6 +353,298 @@ func (g *Gateway) GetVolume(ctx context.Context, req *logproto.VolumeRequest) (* return g.indexQuerier.Volume(ctx, instanceID, req.From, req.Through, req.GetLimit(), req.TargetLabels, req.AggregateBy, matchers...) } +func (g *Gateway) GetShards(request *logproto.ShardsRequest, server logproto.IndexGateway_GetShardsServer) error { + ctx := server.Context() + log, _ := spanlogger.New(context.Background(), "IndexGateway.GetShards") + defer log.Finish() + + instanceID, err := tenant.TenantID(ctx) + if err != nil { + return err + } + + p, err := ExtractShardRequestMatchersAndAST(request.Query) + if err != nil { + return err + } + + // Shards were requested, but blooms are not enabled or cannot be used due to lack of filters. + // That's ok; we can still return shard ranges without filtering + // which will be more effective than guessing power-of-2 shard ranges. + forSeries, ok := g.indexQuerier.HasForSeries(request.From, request.Through) + if g.bloomQuerier == nil || len(syntax.ExtractLineFilters(p.Plan().AST)) == 0 || !ok { + shards, err := g.indexQuerier.GetShards( + ctx, + instanceID, + request.From, request.Through, + request.TargetBytesPerShard, + p, + ) + + if err != nil { + return err + } + + return server.Send(shards) + } + + return g.getShardsWithBlooms(ctx, request, server, instanceID, p, forSeries) +} + +// getShardsWithBlooms is a helper function to get shards with blooms enabled. +func (g *Gateway) getShardsWithBlooms( + ctx context.Context, + req *logproto.ShardsRequest, + server logproto.IndexGateway_GetShardsServer, + instanceID string, + p chunk.Predicate, + forSeries sharding.ForSeries, +) error { + // TODO(owen-d): instead of using GetChunks which buffers _all_ the chunks + // (expensive when looking at the full fingerprint space), we should + // use the `ForSeries` implementation to accumulate batches of chunks to dedupe, + // but I'm leaving this as a future improvement. This may be difficult considering + // fingerprints aren't necessarily iterated in order because multiple underlying TSDBs + // can be queried independently. This could also result in the same chunks being present in + // multiple batches. However, this is all OK because we can dedupe them post-blooms and in + // many cases the majority of chunks will only be present in a single post-compacted TSDB, + // making this more of an edge case than a common occurrence (make sure to check this assumption + // as getting it _very_ wrong could harm some cache locality benefits on the bloom-gws by + // sending multiple requests to the entire keyspace). + + sp, ctx := spanlogger.NewWithLogger( + ctx, + log.With(g.log, "tenant", instanceID), + "indexgateway.getShardsWithBlooms", + ) + defer sp.Finish() + + // 1) for all bounds, get chunk refs + grps, _, err := g.indexQuerier.GetChunks(ctx, instanceID, req.From, req.Through, p) + if err != nil { + return err + } + + var ct int + for _, g := range grps { + ct += len(g) + } + // TODO(owen-d): pool + refs := make([]*logproto.ChunkRef, 0, ct) + + for _, cs := range grps { + for j := range cs { + refs = append(refs, &cs[j].ChunkRef) + } + } + + // 2) filter via blooms + filtered, err := g.bloomQuerier.FilterChunkRefs(ctx, instanceID, req.From, req.Through, refs, p.Plan()) + if err != nil { + return err + } + g.metrics.preFilterChunks.WithLabelValues(routeShards).Observe(float64(ct)) + g.metrics.postFilterChunks.WithLabelValues(routeShards).Observe(float64(len(filtered))) + + statistics := stats.Result{ + Index: stats.Index{ + TotalChunks: int64(ct), + PostFilterChunks: int64(len(filtered)), + }, + } + + resp := &logproto.ShardsResponse{ + Statistics: statistics, + } + + // Edge case: if there are no chunks after filtering, we still need to return a single shard + if len(filtered) == 0 { + resp.Shards = []logproto.Shard{ + { + Bounds: logproto.FPBounds{Min: 0, Max: math.MaxUint64}, + Stats: &logproto.IndexStatsResponse{}, + }, + } + } else { + shards, err := accumulateChunksToShards(ctx, instanceID, forSeries, req, p, filtered) + if err != nil { + return err + } + resp.Shards = shards + } + + level.Debug(g.log).Log( + "msg", "shards response", + "total_chunks", statistics.Index.TotalChunks, + "post_filter_chunks", statistics.Index.PostFilterChunks, + "shards", len(resp.Shards), + "query", req.Query, + "target_bytes_per_shard", datasize.ByteSize(req.TargetBytesPerShard).HumanReadable(), + ) + + level.Debug(sp).Log( + "msg", "shards response", + "total_chunks", statistics.Index.TotalChunks, + "post_filter_chunks", statistics.Index.PostFilterChunks, + "shards", len(resp.Shards), + "query", req.Query, + "target_bytes_per_shard", datasize.ByteSize(req.TargetBytesPerShard).HumanReadable(), + ) + + // 3) build shards + return server.Send(resp) +} + +// ExtractShardRequestMatchersAndAST extracts the matchers and AST from a query string. +// It errors if there is more than one matcher group in the AST as this is supposed to be +// split out during query planning before reaching this point. +func ExtractShardRequestMatchersAndAST(query string) (chunk.Predicate, error) { + expr, err := syntax.ParseExpr(query) + if err != nil { + return chunk.Predicate{}, err + } + + ms, err := syntax.MatcherGroups(expr) + if err != nil { + return chunk.Predicate{}, err + } + + var matchers []*labels.Matcher + switch len(ms) { + case 0: + // nothing to do + case 1: + matchers = ms[0].Matchers + default: + return chunk.Predicate{}, fmt.Errorf( + "multiple matcher groups are not supported in GetShards. This is likely an internal bug as binary operations should be dispatched separately in planning", + ) + } + + return chunk.NewPredicate(matchers, &plan.QueryPlan{ + AST: expr, + }), nil +} + +// TODO(owen-d): consider extending index impl to support returning chunkrefs _with_ sizing info +// TODO(owen-d): perf, this is expensive :( +func accumulateChunksToShards( + ctx context.Context, + user string, + forSeries sharding.ForSeries, + req *logproto.ShardsRequest, + p chunk.Predicate, + filtered []*logproto.ChunkRef, +) ([]logproto.Shard, error) { + // map for looking up post-filtered chunks in O(n) while iterating the index again for sizing info + filteredM := make(map[model.Fingerprint][]refWithSizingInfo, 1024) + for _, ref := range filtered { + x := refWithSizingInfo{ref: ref} + filteredM[model.Fingerprint(ref.Fingerprint)] = append(filteredM[model.Fingerprint(ref.Fingerprint)], x) + } + + 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) { + // check if this is a fingerprint we need + if _, ok := filteredM[fp]; !ok { + return false + } + 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 v1.Less: + // this chunk is not in the queried index, continue checking other chunks + continue outer + case v1.Greater: + // next chunk in index but didn't pass filter; continue + j++ + continue + case v1.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 + } + + return false + }, + p.Matchers..., + ); err != nil { + return nil, err + } + + collectedSeries := sharding.SizedFPs(sharding.SizedFPsPool.Get(len(filteredM))) + defer sharding.SizedFPsPool.Put(collectedSeries) + + for fp, chks := range filteredM { + x := sharding.SizedFP{Fp: fp} + x.Stats.Chunks = uint64(len(chks)) + + for _, chk := range chks { + x.Stats.Entries += uint64(chk.Entries) + x.Stats.Bytes += uint64(chk.KB << 10) + } + collectedSeries = append(collectedSeries, x) + } + sort.Sort(collectedSeries) + + return collectedSeries.ShardsFor(req.TargetBytesPerShard), nil +} + +type refWithSizingInfo struct { + ref *logproto.ChunkRef + KB uint32 + Entries uint32 +} + +// careful: only checks from,through,checksum +func (r refWithSizingInfo) Cmp(chk tsdb_index.ChunkMeta) v1.Ord { + ref := *r.ref + chkFrom := model.Time(chk.MinTime) + if ref.From != chkFrom { + if ref.From < chkFrom { + return v1.Less + } + return v1.Greater + } + + chkThrough := model.Time(chk.MaxTime) + if ref.Through != chkThrough { + if ref.Through < chkThrough { + return v1.Less + } + return v1.Greater + } + + if ref.Checksum != chk.Checksum { + if ref.Checksum < chk.Checksum { + return v1.Less + } + return v1.Greater + } + + return v1.Eq +} + type failingIndexClient struct{} func (f failingIndexClient) QueryPages(_ context.Context, _ []seriesindex.Query, _ seriesindex.QueryPagesCallback) error { diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go index 48bb4d2c8c383..52518f3995b77 100644 --- a/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/gateway_test.go @@ -15,8 +15,12 @@ import ( "google.golang.org/grpc" "github.com/grafana/loki/pkg/logproto" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/series/index" + tsdb_index "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" util_test "github.com/grafana/loki/pkg/util" util_log "github.com/grafana/loki/pkg/util/log" util_math "github.com/grafana/loki/pkg/util/math" @@ -284,3 +288,305 @@ func (i *indexQuerierMock) Volume(_ context.Context, userID string, from, throug return args.Get(0).(*logproto.VolumeResponse), args.Error(1) } + +// Tests for various cases of the `refWithSizingInfo.Cmp` function +func TestRefWithSizingInfo(t *testing.T) { + for _, tc := range []struct { + desc string + a refWithSizingInfo + b tsdb_index.ChunkMeta + exp v1.Ord + }{ + { + desc: "less by from", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + From: 1, + }, + }, + b: tsdb_index.ChunkMeta{ + MinTime: 2, + }, + exp: v1.Less, + }, + { + desc: "eq by from", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + From: 1, + }, + }, + b: tsdb_index.ChunkMeta{ + MinTime: 1, + }, + exp: v1.Eq, + }, + { + desc: "gt by from", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + From: 2, + }, + }, + b: tsdb_index.ChunkMeta{ + MinTime: 1, + }, + exp: v1.Greater, + }, + { + desc: "less by through", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Through: 1, + }, + }, + b: tsdb_index.ChunkMeta{ + MaxTime: 2, + }, + exp: v1.Less, + }, + { + desc: "eq by through", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Through: 2, + }, + }, + b: tsdb_index.ChunkMeta{ + MaxTime: 2, + }, + exp: v1.Eq, + }, + { + desc: "gt by through", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Through: 2, + }, + }, + b: tsdb_index.ChunkMeta{ + MaxTime: 1, + }, + exp: v1.Greater, + }, + { + desc: "less by checksum", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Checksum: 1, + }, + }, + b: tsdb_index.ChunkMeta{ + Checksum: 2, + }, + exp: v1.Less, + }, + { + desc: "eq by checksum", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Checksum: 2, + }, + }, + b: tsdb_index.ChunkMeta{ + Checksum: 2, + }, + exp: v1.Eq, + }, + { + desc: "gt by checksum", + a: refWithSizingInfo{ + ref: &logproto.ChunkRef{ + Checksum: 2, + }, + }, + b: tsdb_index.ChunkMeta{ + Checksum: 1, + }, + exp: v1.Greater, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, tc.a.Cmp(tc.b)) + }) + } +} + +// TODO(owen-d): more testing for specific cases +func TestAccumulateChunksToShards(t *testing.T) { + // only check eq by checksum for convenience -- we're not testing the comparison function here + mkRef := func(fp model.Fingerprint, checksum uint32) *logproto.ChunkRef { + return &logproto.ChunkRef{ + Fingerprint: uint64(fp), + Checksum: checksum, + } + } + + sized := func(ref *logproto.ChunkRef, kb, entries uint32) refWithSizingInfo { + return refWithSizingInfo{ + ref: ref, + KB: kb, + Entries: entries, + } + + } + + fsImpl := func(series [][]refWithSizingInfo) sharding.ForSeriesFunc { + return sharding.ForSeriesFunc( + func( + ctx context.Context, + _ string, + _ tsdb_index.FingerprintFilter, + _, _ model.Time, + fn func( + _ labels.Labels, + fp model.Fingerprint, + chks []tsdb_index.ChunkMeta, + ) (stop bool), matchers ...*labels.Matcher) error { + + for _, s := range series { + chks := []tsdb_index.ChunkMeta{} + for _, r := range s { + chks = append(chks, tsdb_index.ChunkMeta{ + Checksum: r.ref.Checksum, + KB: r.KB, + Entries: r.Entries, + }) + } + + if stop := fn(nil, s[0].ref.FingerprintModel(), chks); stop { + return nil + } + } + return nil + }, + ) + } + + filtered := []*logproto.ChunkRef{ + // shard 0 + mkRef(1, 0), + mkRef(1, 1), + mkRef(1, 2), + + // shard 1 + mkRef(2, 10), + mkRef(2, 20), + mkRef(2, 30), + + // shard 2 split across multiple series + mkRef(3, 10), + mkRef(4, 10), + mkRef(4, 20), + + // last shard contains leftovers + skip a few fps in between + mkRef(7, 10), + } + + series := [][]refWithSizingInfo{ + { + // first series creates one shard since a shard can't contain partial series. + // no chunks were filtered out + sized(mkRef(1, 0), 100, 1), + sized(mkRef(1, 1), 100, 1), + sized(mkRef(1, 2), 100, 1), + }, + { + // second shard also contains one series, but this series has chunks filtered out. + sized(mkRef(2, 0), 100, 1), // filtered out + sized(mkRef(2, 10), 100, 1), // included + sized(mkRef(2, 11), 100, 1), // filtered out + sized(mkRef(2, 20), 100, 1), // included + sized(mkRef(2, 21), 100, 1), // filtered out + sized(mkRef(2, 30), 100, 1), // included + sized(mkRef(2, 31), 100, 1), // filtered out + }, + + // third shard contains multiple series. + // combined they have 110kb, which is above the target of 100kb + // but closer than leaving the second series out which would create + // a shard with 50kb + { + // first series, 50kb + sized(mkRef(3, 10), 50, 1), // 50kb + sized(mkRef(3, 11), 50, 1), // 50kb, not included + }, + { + // second series + sized(mkRef(4, 10), 30, 1), // 30kb + sized(mkRef(4, 11), 30, 1), // 30kb, not included + sized(mkRef(4, 20), 30, 1), // 30kb + }, + + // Fourth shard contains a single series with 25kb, + // but iterates over non-included fp(s) before it + { + // register a series in the index which is not included in the filtered list + sized(mkRef(6, 10), 100, 1), // not included + sized(mkRef(6, 11), 100, 1), // not included + }, + { + // last shard contains leftovers + sized(mkRef(7, 10), 25, 1), + sized(mkRef(7, 11), 100, 1), // not included + }, + } + + shards, err := accumulateChunksToShards( + context.Background(), + "", + fsImpl(series), + &logproto.ShardsRequest{ + TargetBytesPerShard: 100 << 10, + }, + chunk.NewPredicate(nil, nil), // we're not checking matcher injection here + filtered, + ) + + exp := []logproto.Shard{ + { + Bounds: logproto.FPBounds{Min: 0, Max: 1}, + Stats: &logproto.IndexStatsResponse{ + Streams: 1, + Chunks: 3, + Entries: 3, + Bytes: 300 << 10, + }, + }, + { + Bounds: logproto.FPBounds{Min: 2, Max: 2}, + Stats: &logproto.IndexStatsResponse{ + Streams: 1, + Chunks: 3, + Entries: 3, + Bytes: 300 << 10, + }, + }, + { + Bounds: logproto.FPBounds{Min: 3, Max: 6}, + Stats: &logproto.IndexStatsResponse{ + Streams: 2, + Chunks: 3, + Entries: 3, + Bytes: 110 << 10, + }, + }, + { + Bounds: logproto.FPBounds{Min: 7, Max: math.MaxUint64}, + Stats: &logproto.IndexStatsResponse{ + Streams: 1, + Chunks: 1, + Entries: 1, + Bytes: 25 << 10, + }, + }, + } + + require.NoError(t, err) + + for i := range shards { + require.Equal(t, exp[i], shards[i], "invalid shard at index %d", i) + } + require.Equal(t, len(exp), len(shards)) + +} diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/metrics.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/metrics.go new file mode 100644 index 0000000000000..dcf517ea468b7 --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/metrics.go @@ -0,0 +1,37 @@ +package indexgateway + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/pkg/util/constants" +) + +const ( + routeChunkRefs = "chunk_refs" + routeShards = "shards" +) + +type Metrics struct { + preFilterChunks *prometheus.HistogramVec + postFilterChunks *prometheus.HistogramVec +} + +func NewMetrics(r prometheus.Registerer) *Metrics { + return &Metrics{ + preFilterChunks: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "index_gateway", + Name: "prefilter_chunks", + Help: "Number of chunks before filtering", + Buckets: prometheus.ExponentialBuckets(1, 4, 10), + }, []string{"route"}), + postFilterChunks: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: constants.Loki, + Subsystem: "index_gateway", + Name: "postfilter_chunks", + Help: "Number of chunks after filtering", + Buckets: prometheus.ExponentialBuckets(1, 4, 10), + }, []string{"route"}), + } +} diff --git a/pkg/storage/stores/shipper/indexshipper/indexgateway/shufflesharding.go b/pkg/storage/stores/shipper/indexshipper/indexgateway/shufflesharding.go index 50a1ae33b1390..c82efac95f025 100644 --- a/pkg/storage/stores/shipper/indexshipper/indexgateway/shufflesharding.go +++ b/pkg/storage/stores/shipper/indexshipper/indexgateway/shufflesharding.go @@ -21,6 +21,7 @@ var ( type Limits interface { IndexGatewayShardSize(tenantID string) int + TSDBMaxBytesPerShard(string) int } type ShardingStrategy interface { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go index 15ee7f1a1d675..c88b0e124c9b5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor.go @@ -53,8 +53,9 @@ 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) { + 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, "", "")) if err != nil { return nil, err @@ -212,8 +213,9 @@ 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) { + 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{})...) if err != nil { return nil, err @@ -244,8 +246,9 @@ 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) { + 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, "", "")) if err != nil { return nil, err diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go index a2dfaa2271b6a..5032f6df085f5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/compactor_test.go @@ -609,8 +609,9 @@ 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, fp model.Fingerprint, chks []index.ChunkMeta) { + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { actualChunks[lbls.String()] = chks + return false }, labels.MustNewMatcher(labels.MatchEqual, "", "")) require.NoError(t, err) @@ -823,8 +824,9 @@ 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, fp model.Fingerprint, chks []index.ChunkMeta) { + err = indexFile.(*TSDBFile).Index.(*TSDBIndex).ForSeries(context.Background(), "", nil, 0, math.MaxInt64, func(lbls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { foundChunks[lbls.String()] = append(index.ChunkMetas{}, chks...) + return false }, labels.MustNewMatcher(labels.MatchEqual, "", "")) require.NoError(t, err) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index 7342fe851c577..ad285bc32b158 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -800,6 +800,14 @@ 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 { + idx, ok := t.tenantIndex(userID, from, through) + if !ok { + return nil + } + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) +} + // helper only used in building TSDBs func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, fp uint64, chks index.ChunkMetas) error) error { for i, shard := range t.tenants { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index bb294fb13f450..475446b15090f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -8,6 +8,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" ) type Series struct { @@ -22,13 +23,22 @@ type ChunkRef struct { Checksum uint32 } -// Compares by (Start, End) +// Compares by (Fp, Start, End, checksum) // Assumes User is equivalent func (r ChunkRef) Less(x ChunkRef) bool { + if r.Fingerprint != x.Fingerprint { + return r.Fingerprint < x.Fingerprint + } + if r.Start != x.Start { return r.Start < x.Start } - return r.End <= x.End + + if r.End != x.End { + return r.End < x.End + } + + return r.Checksum < x.Checksum } type shouldIncludeChunk func(index.ChunkMeta) bool @@ -37,6 +47,7 @@ type Index interface { Bounded SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) Close() error + sharding.ForSeries // GetChunkRefs accepts an optional []ChunkRef argument. // If not nil, it will use that slice to build the result, // allowing us to avoid unnecessary allocations at the caller's discretion. @@ -84,3 +95,7 @@ func (NoopIndex) SetChunkFilterer(_ chunk.RequestChunkFilterer) {} func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAccumulator, _ index.FingerprintFilter, _ shouldIncludeChunk, _ []string, _ string, _ ...*labels.Matcher) error { 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 { + return nil +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go index b188ebbcb24ed..e32d39f167f9b 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go @@ -18,7 +18,10 @@ const ( var errDisallowedIdentityShard = errors.New("shard with factor of 1 is explicitly disallowed. It's equivalent to no sharding") type FingerprintFilter interface { + // TODO(owen-d): Match() is redundant and can be inferred from GetFromThrough() + // TODO(owen-d): GetFromThrough should just return FingerprintBounds as it's a better utility struct. Match(model.Fingerprint) bool + // GetFromThrough shows the [minimum, maximum) fingerprints. If there is no maximum, math.MaxUint64 may be used GetFromThrough() (model.Fingerprint, model.Fingerprint) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index d609dc0ed27f7..1a2115b26cdbc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -2,8 +2,12 @@ package tsdb import ( "context" + "sort" + "sync" "time" + "github.com/grafana/loki/pkg/logql" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/opentracing/opentracing-go" @@ -17,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/stats" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util" ) @@ -33,6 +38,9 @@ type IndexClientOptions struct { // duplicates when chunks are written to multiple // index buckets, which is of use in the (index-gateway|querier) // but not worth the memory costs in the ingesters. + // NB(owen-d): This is NOT the bloom-filter feature developed late 2023 onwards, + // but a smaller bloom filter used internally for probabalistic deduping of series counts + // in the index stats() method across index buckets (which can have the same series) UseBloomFilters bool } @@ -65,6 +73,20 @@ func NewIndexClient(idx Index, opts IndexClientOptions, l Limits) *IndexClient { } } +func shardFromMatchers(matchers []*labels.Matcher) (cleaned []*labels.Matcher, res logql.Shard, found bool, err error) { + for i, matcher := range matchers { + if matcher.Name == astmapper.ShardLabel && matcher.Type == labels.MatchEqual { + shard, _, err := logql.ParseShard(matcher.Value) + if err != nil { + return nil, shard, true, err + } + return append(matchers[:i], matchers[i+1:]...), shard, true, nil + } + } + + return matchers, logql.Shard{}, false, nil +} + // TODO(owen-d): This is a hack for compatibility with how the current query-mapping works. // Historically, Loki will read the index shard factor and the query planner will inject shard // labels accordingly. @@ -74,32 +96,21 @@ func NewIndexClient(idx Index, opts IndexClientOptions, l Limits) *IndexClient { func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, index.FingerprintFilter, error) { // first use withoutNameLabel to make a copy with the name label removed matchers = withoutNameLabel(matchers) - s, shardLabelIndex, err := astmapper.ShardFromMatchers(matchers) + + matchers, shard, found, err := shardFromMatchers(matchers) if err != nil { return nil, nil, err } - var fpFilter index.FingerprintFilter - if s != nil { - matchers = append(matchers[:shardLabelIndex], matchers[shardLabelIndex+1:]...) - shard := index.ShardAnnotation{ - Shard: uint32(s.Shard), - Of: uint32(s.Of), - } - fpFilter = shard - - if err := shard.Validate(); err != nil { - return nil, nil, err - } - } - if len(matchers) == 0 { // hack to query all data matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, "", "")) } - return matchers, fpFilter, err - + if found { + return matchers, &shard, nil + } + return matchers, nil, nil } // TODO(owen-d): synchronize logproto.ChunkRef and tsdb.ChunkRef so we don't have to convert. @@ -269,6 +280,45 @@ func (c *IndexClient) Volume(ctx context.Context, userID string, from, through m return acc.Volumes(), nil } +func (c *IndexClient) GetShards(ctx context.Context, userID string, from, through model.Time, targetBytesPerShard uint64, predicate chunk.Predicate) (*logproto.ShardsResponse, error) { + + // TODO(owen-d): perf, this is expensive :( + 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) { + mtx.Lock() + m[fp] = append(m[fp], chks...) + mtx.Unlock() + return false + }, predicate.Matchers...); err != nil { + return nil, err + } + + resp := &logproto.ShardsResponse{} + + series := sharding.SizedFPs(sharding.SizedFPsPool.Get(len(m))) + defer sharding.SizedFPsPool.Put(series) + + for fp, chks := range m { + x := sharding.SizedFP{Fp: fp} + deduped := chks.Finalize() + x.Stats.Chunks = uint64(len(deduped)) + resp.Statistics.Index.TotalChunks += int64(len(deduped)) + + for _, chk := range deduped { + x.Stats.Entries += uint64(chk.Entries) + x.Stats.Bytes += uint64(chk.KB << 10) + } + + series = append(series, x) + } + sort.Sort(series) + resp.Shards = series.ShardsFor(targetBytesPerShard) + + return resp, nil +} + // SetChunkFilterer sets a chunk filter to be used when retrieving chunks. // This is only used for GetSeries implementation. // Todo we might want to pass it as a parameter to GetSeries instead. @@ -293,3 +343,7 @@ func withoutNameLabel(matchers []*labels.Matcher) []*labels.Matcher { return dst } + +func (c *IndexClient) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return c.idx, true +} 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 acace60c1e4b2..dbc94b18c027a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -134,31 +134,46 @@ func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, t return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } -type resultAccumulator struct { +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 { + idx, err := i.indices(ctx, from, through, userID) + if err != nil { + return err + } + + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) +} + +type resultAccumulator[T any] struct { mtx sync.Mutex - items []interface{} - merge func(xs []interface{}) (interface{}, error) + items []T + merge func(xs []T) (T, error) } -func newResultAccumulator(merge func(xs []interface{}) (interface{}, error)) *resultAccumulator { - return &resultAccumulator{ +// TODO(owen-d): make generic to avoid casting at runtime. +func newResultAccumulator[T any](merge func(xs []T) (T, error)) *resultAccumulator[T] { + return &resultAccumulator[T]{ merge: merge, } } -func (acc *resultAccumulator) Add(item interface{}) { +func (acc *resultAccumulator[T]) Add(item T) { acc.mtx.Lock() defer acc.mtx.Unlock() acc.items = append(acc.items, item) } -func (acc *resultAccumulator) Merge() (interface{}, error) { +func (acc *resultAccumulator[T]) Merge() (res T, err error) { acc.mtx.Lock() defer acc.mtx.Unlock() - if len(acc.items) == 0 { - return nil, ErrEmptyAccumulator + ln := len(acc.items) + if ln == 0 { + return res, ErrEmptyAccumulator + } + + if ln == 1 { + return acc.items[0], nil } return acc.merge(acc.items) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index 327566f1a0ecc..db7b0e04720ed 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -80,3 +80,11 @@ 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 { + i, err := f() + if err != nil { + return err + } + return i.ForSeries(ctx, userID, fpFilter, from, through, fn, 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 08bf6bf4ff01e..c0d2080bcbe35 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -4,6 +4,7 @@ import ( "context" "math" "runtime" + "sort" "sync" "github.com/prometheus/common/model" @@ -132,7 +133,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) { - acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { + acc := newResultAccumulator(func(xs [][]ChunkRef) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() } @@ -143,9 +144,12 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro // TODO(owen-d): Do this more efficiently, // not all indices overlap each other + // TODO(owen-d): loser-tree or some other heap? + for _, group := range xs { - g := group.([]ChunkRef) + g := group for _, ref := range g { + _, ok := seen[ref] if ok { continue @@ -154,9 +158,10 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro res = append(res, ref) } ChunkRefsPool.Put(g) - } + sort.Slice(res, func(i, j int) bool { return res[i].Less(res[j]) }) + return res, nil }) @@ -183,12 +188,12 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro } return nil, err } - return merged.([]ChunkRef), nil + return merged, nil } func (i *MultiIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { - acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { + acc := newResultAccumulator(func(xs [][]Series) ([]Series, error) { if res == nil { res = SeriesPool.Get() } @@ -197,7 +202,7 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo seen := make(map[model.Fingerprint]struct{}) for _, x := range xs { - seriesSet := x.([]Series) + seriesSet := x for _, s := range seriesSet { _, ok := seen[s.Fingerprint] if ok { @@ -235,17 +240,17 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo } return nil, err } - return merged.([]Series), nil + return merged, nil } func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { + acc := newResultAccumulator(func(xs [][]string) ([]string, error) { var ( maxLn int // maximum number of lNames, assuming no duplicates lists [][]string ) for _, group := range xs { - x := group.([]string) + x := group maxLn += len(x) lists = append(lists, x) } @@ -293,17 +298,17 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug } return nil, err } - return merged.([]string), nil + return merged, nil } func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { - acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { + acc := newResultAccumulator(func(xs [][]string) ([]string, error) { var ( maxLn int // maximum number of lValues, assuming no duplicates lists [][]string ) for _, group := range xs { - x := group.([]string) + x := group maxLn += len(x) lists = append(lists, x) } @@ -351,7 +356,7 @@ func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, throu } return nil, err } - return merged.([]string), nil + return merged, nil } func (i *MultiIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { @@ -365,3 +370,9 @@ func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through mo return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) }) } + +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 { + return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { + return idx.ForSeries(ctx, userID, fpFilter, from, through, fn, matchers...) + }) +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index ec582b6e21489..52de31078673a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -96,3 +96,7 @@ func (m *MultiTenantIndex) Stats(ctx context.Context, userID string, from, throu func (m *MultiTenantIndex) 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 { 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)...) +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go new file mode 100644 index 0000000000000..362665a022776 --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/for_series.go @@ -0,0 +1,66 @@ +package sharding + +import ( + "context" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" +) + +// General purpose iteration over series. Makes it easier to build custom functionality on top of indices +// of different types without them all implementing the same feature. +// The passed callback must _not_ capture its arguments. They're reused for each call for performance. +// The passed callback may be executed concurrently, +// so any shared state must be protected by the caller. +// NB: This is a low-level API and should be used with caution. +// NB: It's possible for the callback to be called multiple times for the same series but possibly different chunks, +// 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 +} + +// function Adapter for ForSeries implementation +type ForSeriesFunc func( + 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), + matchers ...*labels.Matcher, +) error { + return f(ctx, userID, fpFilter, from, through, fn, matchers...) +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power.go new file mode 100644 index 0000000000000..299cc21ea197b --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power.go @@ -0,0 +1,117 @@ +package sharding + +import ( + "math" + + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage/stores/index/stats" +) + +const ( + DefaultTSDBMaxBytesPerShard = 600 << 20 // 600MB +) + +// PowerOfTwoSharding is a slimmed down legacy sharding implementation +// designed for use as a fallback when the newer impls aren't found +// (i.e. during a system upgrade to support the new impl) +type PowerOfTwoSharding struct { + MaxShards int +} + +func (p PowerOfTwoSharding) ShardsFor(bytes uint64, maxBytesPerShard uint64) []logproto.Shard { + factor := GuessShardFactor(bytes, maxBytesPerShard, p.MaxShards) + + if factor < 2 { + return []logproto.Shard{{ + Bounds: logproto.FPBounds{ + Min: 0, + Max: math.MaxUint64, + }, + Stats: &stats.Stats{ + Bytes: bytes, + }, + }} + } + + return LinearShards(factor, bytes) + +} + +// LinearShards is a sharding implementation that splits the data into +// equal sized shards covering the entire keyspace. It populates +// the `bytes` of each shard's stats with a proportional estimation +func LinearShards(n int, bytes uint64) []logproto.Shard { + if n < 2 { + return []logproto.Shard{ + { + Bounds: logproto.FPBounds{ + Min: 0, + Max: math.MaxUint64, + }, + Stats: &stats.Stats{ + Bytes: bytes, + }, + }, + } + } + + bytesPerShard := bytes / uint64(n) + fpPerShard := model.Fingerprint(math.MaxUint64) / model.Fingerprint(n) + + shards := make([]logproto.Shard, n) + for i := range shards { + shards[i] = logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: model.Fingerprint(i) * fpPerShard, + Max: model.Fingerprint(i+1) * fpPerShard, + }, + Stats: &stats.Stats{ + Bytes: bytesPerShard, + }, + } + + // The last shard should have the remainder of the bytes + // and the max bound should be math.MaxUint64 + // NB(owen-d): this can only happen when maxShards is used + // and the maxShards isn't a factor of 2 + shards[len(shards)-1].Stats.Bytes += bytes % uint64(n) + shards[len(shards)-1].Bounds.Max = math.MaxUint64 + } + + return shards + +} + +// Since we shard by powers of two and we increase shard factor +// once each shard surpasses maxBytesPerShard, if the shard factor +// is at least two, the range of data per shard is (maxBytesPerShard/2, maxBytesPerShard] +// For instance, for a maxBytesPerShard of 500MB and a query touching 1000MB, we split into two shards of 500MB. +// If there are 1004MB, we split into four shards of 251MB. +func GuessShardFactor(bytes, maxBytesPerShard uint64, maxShards int) int { + // If maxBytesPerShard is 0, we use the default value + // to avoid division by zero + if maxBytesPerShard < 1 { + maxBytesPerShard = DefaultTSDBMaxBytesPerShard + } + + minShards := float64(bytes) / float64(maxBytesPerShard) + + // round up to nearest power of 2 + power := math.Ceil(math.Log2(minShards)) + + // Since x^0 == 1 and we only support factors of 2 + // reset this edge case manually + factor := int(math.Pow(2, power)) + if maxShards > 0 { + factor = min(factor, maxShards) + } + + // shortcut: no need to run any sharding logic when factor=1 + // as it's the same as no sharding + if factor == 1 { + factor = 0 + } + return factor +} diff --git a/pkg/querier/queryrange/shard_resolver_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power_test.go similarity index 59% rename from pkg/querier/queryrange/shard_resolver_test.go rename to pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power_test.go index 8e963b065955f..5134b25c5cde0 100644 --- a/pkg/querier/queryrange/shard_resolver_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/power_test.go @@ -1,4 +1,4 @@ -package queryrange +package sharding import ( "fmt" @@ -7,7 +7,6 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/storage/stores/index/stats" - "github.com/grafana/loki/pkg/validation" ) func TestGuessShardFactor(t *testing.T) { @@ -23,52 +22,52 @@ func TestGuessShardFactor(t *testing.T) { { exp: 4, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard * 4, + Bytes: DefaultTSDBMaxBytesPerShard * 4, }, }, { // round up shard factor exp: 16, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard * 15, + Bytes: DefaultTSDBMaxBytesPerShard * 15, }, }, { exp: 2, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard + 1, + Bytes: DefaultTSDBMaxBytesPerShard + 1, }, }, { exp: 0, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard, + Bytes: DefaultTSDBMaxBytesPerShard, }, }, { maxShards: 8, exp: 4, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard * 4, + Bytes: DefaultTSDBMaxBytesPerShard * 4, }, }, { maxShards: 2, exp: 2, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard * 4, + Bytes: DefaultTSDBMaxBytesPerShard * 4, }, }, { maxShards: 1, exp: 0, stats: stats.Stats{ - Bytes: validation.DefaultTSDBMaxBytesPerShard * 4, + Bytes: DefaultTSDBMaxBytesPerShard * 4, }, }, } { t.Run(fmt.Sprintf("%+v", tc.stats), func(t *testing.T) { - require.Equal(t, tc.exp, guessShardFactor(tc.stats, validation.DefaultTSDBMaxBytesPerShard, tc.maxShards)) + require.Equal(t, tc.exp, GuessShardFactor(tc.stats.Bytes, uint64(DefaultTSDBMaxBytesPerShard), tc.maxShards)) }) } } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding.go new file mode 100644 index 0000000000000..284468a9de315 --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding.go @@ -0,0 +1,102 @@ +package sharding + +import ( + "math" + + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/queue" + "github.com/grafana/loki/pkg/storage/stores/index/stats" +) + +var ( + SizedFPsPool = queue.NewSlicePool[SizedFP](1<<8, 1<<16, 4) // 256->65536 +) + +type SizedFP struct { + Fp model.Fingerprint + Stats stats.Stats +} + +type SizedFPs []SizedFP + +func (xs SizedFPs) Len() int { + return len(xs) +} + +func (xs SizedFPs) Less(i, j int) bool { + return xs[i].Fp < xs[j].Fp +} + +func (xs SizedFPs) Swap(i, j int) { + xs[i], xs[j] = xs[j], xs[i] +} + +func (xs SizedFPs) newShard(minFP model.Fingerprint) logproto.Shard { + return logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: minFP, + }, + Stats: &stats.Stats{}, + } +} + +func (xs SizedFPs) ShardsFor(targetShardBytes uint64) (res []logproto.Shard) { + if len(xs) == 0 { + full := xs.newShard(0) + full.Bounds.Max = model.Fingerprint(math.MaxUint64) + return []logproto.Shard{full} + } + + var ( + cur = xs.newShard(0) + ) + + for _, x := range xs { + + // easy path, there's space -- continue + if cur.SpaceFor(&x.Stats, targetShardBytes) { + cur.Stats.Streams++ + cur.Stats.Chunks += x.Stats.Chunks + cur.Stats.Entries += x.Stats.Entries + cur.Stats.Bytes += x.Stats.Bytes + + cur.Bounds.Max = x.Fp + continue + } + + // we've hit a stream larger than the target; + // create a shard with 1 stream + if cur.Stats.Streams == 0 { + cur.Stats = &stats.Stats{ + Streams: 1, + Chunks: x.Stats.Chunks, + Bytes: x.Stats.Bytes, + Entries: x.Stats.Entries, + } + cur.Bounds.Max = x.Fp + res = append(res, cur) + cur = xs.newShard(x.Fp + 1) + continue + } + + // Otherwise we've hit a stream that's too large but the current shard isn't empty; create a new shard + cur.Bounds.Max = x.Fp - 1 + res = append(res, cur) + cur = xs.newShard(x.Fp) + cur.Stats = &stats.Stats{ + Streams: 1, + Chunks: x.Stats.Chunks, + Bytes: x.Stats.Bytes, + Entries: x.Stats.Entries, + } + } + + if cur.Stats.Streams > 0 { + res = append(res, cur) + } + + res[len(res)-1].Bounds.Max = model.Fingerprint(math.MaxUint64) + return res +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding_test.go new file mode 100644 index 0000000000000..49a3b12a8ff6e --- /dev/null +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/sharding/sharding_test.go @@ -0,0 +1,153 @@ +package sharding + +import ( + "math" + "sort" + "testing" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage/stores/index/stats" +) + +func TestSizedFPs_Sort(t *testing.T) { + xs := SizedFPs{ + {Fp: 3}, + {Fp: 1}, + {Fp: 6}, + {Fp: 10}, + {Fp: 2}, + {Fp: 0}, + {Fp: 4}, + {Fp: 5}, + {Fp: 7}, + {Fp: 9}, + {Fp: 8}, + } + + sort.Sort(xs) + exp := []model.Fingerprint{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10} + + for i, x := range xs { + require.Equal(t, exp[i], x.Fp) + } +} + +func TestSizedFPs_ShardsFor(t *testing.T) { + mkShard := func(min, max model.Fingerprint, streams, chks, entries, bytes uint64) logproto.Shard { + return logproto.Shard{ + Bounds: logproto.FPBounds{ + Min: min, + Max: max, + }, + Stats: &stats.Stats{ + Streams: streams, + Chunks: chks, + Entries: entries, + Bytes: bytes, + }, + } + } + + mkFP := func(fp model.Fingerprint, chks, entries, bytes uint64) SizedFP { + return SizedFP{ + Fp: fp, + Stats: stats.Stats{ + Chunks: chks, + Entries: entries, + Bytes: bytes, + }, + } + } + + for _, tc := range []struct { + desc string + xs SizedFPs + exp []logproto.Shard + targetShardBytes uint64 + }{ + { + desc: "empty", + targetShardBytes: 100, + xs: SizedFPs{}, + exp: []logproto.Shard{ + mkShard(0, math.MaxUint64, 0, 0, 0, 0), + }, + }, + { + desc: "single stream", + targetShardBytes: 100, + xs: SizedFPs{ + mkFP(1, 1, 1, 1), + }, + exp: []logproto.Shard{ + mkShard(0, math.MaxUint64, 1, 1, 1, 1), + }, + }, + { + desc: "single stream too large", + targetShardBytes: 100, + xs: SizedFPs{ + mkFP(1, 1, 1, 201), + }, + exp: []logproto.Shard{ + mkShard(0, math.MaxUint64, 1, 1, 1, 201), + }, + }, + { + desc: "4 streams 2 shards", + targetShardBytes: 100, + xs: SizedFPs{ + // each has 45 bytes; can only fit 2 in a shard + mkFP(1, 1, 1, 45), + mkFP(2, 1, 1, 45), + mkFP(3, 1, 1, 45), + mkFP(4, 1, 1, 45), + }, + exp: []logproto.Shard{ + mkShard(0, 2, 2, 2, 2, 90), + mkShard(3, math.MaxUint64, 2, 2, 2, 90), + }, + }, + { + desc: "5 streams 3 shards (one leftover)", + targetShardBytes: 100, + xs: SizedFPs{ + // each has 45 bytes; can only fit 2 in a shard + mkFP(1, 1, 1, 45), + mkFP(2, 1, 1, 45), + mkFP(3, 1, 1, 45), + mkFP(4, 1, 1, 45), + mkFP(5, 1, 1, 45), + }, + exp: []logproto.Shard{ + mkShard(0, 2, 2, 2, 2, 90), + mkShard(3, 4, 2, 2, 2, 90), + mkShard(5, math.MaxUint64, 1, 1, 1, 45), + }, + }, + { + desc: "allowed overflow", + targetShardBytes: 100, + xs: SizedFPs{ + // each has 40 bytes; can fit 3 in a shard + // since overflow == underflow + mkFP(1, 1, 1, 40), + mkFP(2, 1, 1, 40), + mkFP(3, 1, 1, 40), + mkFP(4, 1, 1, 40), + mkFP(5, 1, 1, 40), + }, + exp: []logproto.Shard{ + mkShard(0, 3, 3, 3, 3, 120), + mkShard(4, math.MaxUint64, 2, 2, 2, 80), + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, tc.xs.ShardsFor(tc.targetShardBytes)) + }) + } +} 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 63809f6b1356e..b7bff50e52d2d 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,9 @@ 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) { + 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, "", "")) if err != nil { return nil, err @@ -157,9 +158,10 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { // fn must NOT capture it's arguments. They're reused across series iterations and returned to // a pool after completion. -// TODO(owen-d): have callback return a bool whether to continue or not in order to short-circuit -// when applicable -func (i *TSDBIndex) ForSeries(ctx context.Context, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { +// 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 { // TODO(owen-d): use pool var ls labels.Labels @@ -187,7 +189,9 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, fpFilter index.FingerprintFil continue } - fn(ls, model.Fingerprint(hash), chks) + if stop := fn(ls, model.Fingerprint(hash), chks); stop { + break + } } return p.Err() }) @@ -214,7 +218,7 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu } res = res[:0] - if err := i.ForSeries(ctx, fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { + if err := i.ForSeries(ctx, "", fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { for _, chk := range chks { res = append(res, ChunkRef{ @@ -225,6 +229,7 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu Checksum: chk.Checksum, }) } + return false }, matchers...); err != nil { return nil, err } @@ -238,7 +243,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) { + if err := i.ForSeries(ctx, "", fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { if len(chks) == 0 { return } @@ -246,6 +251,7 @@ func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Ti Labels: ls.Copy(), Fingerprint: fp, }) + return false }, matchers...); err != nil { return nil, err } diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 759b2e336aa29..12e8168473af9 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -25,6 +25,7 @@ import ( "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores" index_stats "github.com/grafana/loki/pkg/storage/stores/index/stats" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" loki_util "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/constants" util_log "github.com/grafana/loki/pkg/util/log" @@ -270,6 +271,14 @@ func (m *mockChunkStore) Stats(_ context.Context, _ string, _, _ model.Time, _ . return nil, nil } +func (m *mockChunkStore) GetShards(_ context.Context, _ string, _, _ model.Time, _ uint64, _ chunk.Predicate) (*logproto.ShardsResponse, error) { + return nil, nil +} + +func (m *mockChunkStore) HasForSeries(_, _ model.Time) (sharding.ForSeries, bool) { + return nil, false +} + func (m *mockChunkStore) Volume(_ context.Context, _ string, _, _ model.Time, _ int32, _ []string, _ string, _ ...*labels.Matcher) (*logproto.VolumeResponse, error) { return nil, nil } diff --git a/pkg/util/marshal/legacy/marshal_test.go b/pkg/util/marshal/legacy/marshal_test.go index 88375ad842ab1..de308abb2b276 100644 --- a/pkg/util/marshal/legacy/marshal_test.go +++ b/pkg/util/marshal/legacy/marshal_test.go @@ -56,6 +56,10 @@ var queryTests = []struct { } ], "stats" : { + "index": { + "postFilterChunks": 0, + "totalChunks": 0 + }, "ingester" : { "store": { "chunksDownloadTime": 0, diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index bb961039cdd35..8b9f71ecc5782 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -155,6 +155,16 @@ func WriteIndexStatsResponseJSON(r *indexStats.Stats, w io.Writer) error { return s.Flush() } +// WriteIndexShardsResponseJSON marshals a indexgatewaypb.ShardsResponse to JSON and then +// writes it to the provided io.Writer. +func WriteIndexShardsResponseJSON(r *logproto.ShardsResponse, w io.Writer) error { + s := jsoniter.ConfigFastest.BorrowStream(w) + defer jsoniter.ConfigFastest.ReturnStream(s) + s.WriteVal(r) + s.WriteRaw("\n") + return s.Flush() +} + // WriteVolumeResponseJSON marshals a logproto.VolumeResponse to JSON and then // writes it to the provided io.Writer. func WriteVolumeResponseJSON(r *logproto.VolumeResponse, w io.Writer) error { diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index ca932064ca6c3..3a56617fb55b3 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -24,6 +24,10 @@ import ( ) const emptyStats = `{ + "index": { + "postFilterChunks": 0, + "totalChunks": 0 + }, "ingester" : { "store": { "chunksDownloadTime": 0, diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index e159fbf018f1b..e8f2ab2d994b3 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -23,9 +23,11 @@ import ( "github.com/grafana/loki/pkg/compactor/deletionmode" "github.com/grafana/loki/pkg/distributor/shardstreams" "github.com/grafana/loki/pkg/loghttp/push" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" ruler_config "github.com/grafana/loki/pkg/ruler/config" "github.com/grafana/loki/pkg/ruler/util" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/pkg/util/flagext" util_log "github.com/grafana/loki/pkg/util/log" "github.com/grafana/loki/pkg/util/validation" @@ -49,8 +51,8 @@ const ( bytesInMB = 1048576 - defaultPerStreamRateLimit = 3 << 20 // 3MB - DefaultTSDBMaxBytesPerShard = 600 << 20 // 600MB + defaultPerStreamRateLimit = 3 << 20 // 3MB + DefaultTSDBMaxBytesPerShard = sharding.DefaultTSDBMaxBytesPerShard defaultPerStreamBurstLimit = 5 * defaultPerStreamRateLimit DefaultPerTenantQueryTimeout = "1m" @@ -95,6 +97,7 @@ type Limits struct { MaxQueryParallelism int `yaml:"max_query_parallelism" json:"max_query_parallelism"` TSDBMaxQueryParallelism int `yaml:"tsdb_max_query_parallelism" json:"tsdb_max_query_parallelism"` TSDBMaxBytesPerShard flagext.ByteSize `yaml:"tsdb_max_bytes_per_shard" json:"tsdb_max_bytes_per_shard"` + TSDBShardingStrategy string `yaml:"tsdb_sharding_strategy" json:"tsdb_sharding_strategy"` CardinalityLimit int `yaml:"cardinality_limit" json:"cardinality_limit"` MaxStreamsMatchersPerQuery int `yaml:"max_streams_matchers_per_query" json:"max_streams_matchers_per_query"` MaxConcurrentTailRequests int `yaml:"max_concurrent_tail_requests" json:"max_concurrent_tail_requests"` @@ -270,7 +273,16 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.IntVar(&l.MaxQueryParallelism, "querier.max-query-parallelism", 32, "Maximum number of queries that will be scheduled in parallel by the frontend.") f.IntVar(&l.TSDBMaxQueryParallelism, "querier.tsdb-max-query-parallelism", 128, "Maximum number of queries will be scheduled in parallel by the frontend for TSDB schemas.") _ = l.TSDBMaxBytesPerShard.Set(strconv.Itoa(DefaultTSDBMaxBytesPerShard)) - f.Var(&l.TSDBMaxBytesPerShard, "querier.tsdb-max-bytes-per-shard", "Maximum number of bytes assigned to a single sharded query. Also expressible in human readable forms (1GB, etc).") + f.Var(&l.TSDBMaxBytesPerShard, "querier.tsdb-max-bytes-per-shard", "Target maximum number of bytes assigned to a single sharded query. Also expressible in human readable forms (1GB, etc). Note: This is a _target_ and not an absolute limit. The actual limit can be higher, but the query planner will try to build shards up to this limit.") + f.StringVar( + &l.TSDBShardingStrategy, + "limits.tsdb-sharding-strategy", + logql.PowerOfTwoVersion.String(), + fmt.Sprintf( + "sharding strategy to use in query planning. Suggested to use %s once all nodes can recognize it.", + logql.BoundedVersion.String(), + ), + ) f.IntVar(&l.CardinalityLimit, "store.cardinality-limit", 1e5, "Cardinality limit for index queries.") f.IntVar(&l.MaxStreamsMatchersPerQuery, "querier.max-streams-matcher-per-query", 1000, "Maximum number of stream matchers per query.") f.IntVar(&l.MaxConcurrentTailRequests, "querier.max-concurrent-tail-requests", 10, "Maximum number of concurrent tail requests.") @@ -432,6 +444,10 @@ func (l *Limits) Validate() error { return err } + if _, err := logql.ParseShardVersion(l.TSDBShardingStrategy); err != nil { + return errors.Wrap(err, "invalid tsdb sharding strategy") + } + if _, err := chunkenc.ParseEncoding(l.BloomBlockEncoding); err != nil { return err } @@ -595,6 +611,11 @@ func (o *Overrides) TSDBMaxBytesPerShard(userID string) int { return o.getOverridesForUser(userID).TSDBMaxBytesPerShard.Val() } +// TSDBShardingStrategy returns the sharding strategy to use in query planning. +func (o *Overrides) TSDBShardingStrategy(userID string) string { + return o.getOverridesForUser(userID).TSDBShardingStrategy +} + // MaxQueryParallelism returns the limit to the number of sub-queries the // frontend will process in parallel. func (o *Overrides) MaxQueryParallelism(_ context.Context, userID string) int { diff --git a/pkg/validation/limits_test.go b/pkg/validation/limits_test.go index 59626aeb8cdbe..9096d9b179444 100644 --- a/pkg/validation/limits_test.go +++ b/pkg/validation/limits_test.go @@ -15,6 +15,7 @@ import ( "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/compactor/deletionmode" "github.com/grafana/loki/pkg/loghttp/push" + "github.com/grafana/loki/pkg/logql" ) func TestLimitsTagsYamlMatchJson(t *testing.T) { @@ -338,6 +339,7 @@ func TestLimitsValidation(t *testing.T) { } { desc := fmt.Sprintf("%s/%s", tc.limits.DeletionMode, tc.limits.BloomBlockEncoding) t.Run(desc, func(t *testing.T) { + tc.limits.TSDBShardingStrategy = logql.PowerOfTwoVersion.String() // hacky but needed for test if tc.expected == nil { require.NoError(t, tc.limits.Validate()) } else { diff --git a/tools/bloom/inspector/main.go b/tools/bloom/inspector/main.go index bb81d02b260b1..d7f1a7c89bf32 100644 --- a/tools/bloom/inspector/main.go +++ b/tools/bloom/inspector/main.go @@ -17,8 +17,8 @@ func main() { fmt.Printf("Block directory: %s\n", path) r := v1.NewDirectoryBlockReader(path) - b := v1.NewBlock(r) - q := v1.NewBlockQuerier(b) + b := v1.NewBlock(r, v1.NewMetrics(nil)) + q := v1.NewBlockQuerier(b, true) md, err := q.Metadata() if err != nil { diff --git a/tools/tsdb/bloom-tester/lib.go b/tools/tsdb/bloom-tester/lib.go index 2512a3e66bee4..c7608f0046bb2 100644 --- a/tools/tsdb/bloom-tester/lib.go +++ b/tools/tsdb/bloom-tester/lib.go @@ -281,8 +281,8 @@ func analyze(metrics *Metrics, sampler Sampler, indexShipper indexshipper.IndexS casted := idx.(*tsdb.TSDBFile).Index.(*tsdb.TSDBIndex) _ = casted.ForSeries( context.Background(), - nil, model.Earliest, model.Latest, - func(ls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) { + "", nil, model.Earliest, model.Latest, + func(ls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { seriesString := ls.String() seriesStringHash := FNV32a(seriesString) pos, _ := strconv.Atoi(seriesStringHash) @@ -399,6 +399,7 @@ func analyze(metrics *Metrics, sampler Sampler, indexShipper indexshipper.IndexS )*/ } // for each series + return false }, labels.MustNewMatcher(labels.MatchEqual, "", ""), ) diff --git a/tools/tsdb/bloom-tester/readlib.go b/tools/tsdb/bloom-tester/readlib.go index 6e5c492f120de..e2a21754e865c 100644 --- a/tools/tsdb/bloom-tester/readlib.go +++ b/tools/tsdb/bloom-tester/readlib.go @@ -141,8 +141,9 @@ func analyzeRead(metrics *Metrics, sampler Sampler, shipper indexshipper.IndexSh casted := idx.(*tsdb.TSDBFile).Index.(*tsdb.TSDBIndex) _ = casted.ForSeries( context.Background(), - nil, model.Earliest, model.Latest, - func(ls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) { + "", nil, + model.Earliest, model.Latest, + func(ls labels.Labels, fp model.Fingerprint, chks []tsdbindex.ChunkMeta) (stop bool) { seriesString := ls.String() seriesStringHash := FNV32a(seriesString) pos, _ := strconv.Atoi(seriesStringHash) @@ -272,6 +273,7 @@ func analyzeRead(metrics *Metrics, sampler Sampler, shipper indexshipper.IndexSh ) */ } // For every series + return false }, labels.MustNewMatcher(labels.MatchEqual, "", ""), ) diff --git a/tools/tsdb/index-analyzer/analytics.go b/tools/tsdb/index-analyzer/analytics.go index 7558c17fc8030..d9baeedc69533 100644 --- a/tools/tsdb/index-analyzer/analytics.go +++ b/tools/tsdb/index-analyzer/analytics.go @@ -70,16 +70,17 @@ func analyze(indexShipper indexshipper.IndexShipper, tableName string, tenants [ err = casted.Index.(*tsdb.TSDBIndex).ForSeries( context.Background(), - nil, + "", nil, model.Earliest, model.Latest, - func(ls labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) { + func(ls labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { if len(chks) > maxChunksPerSeries { maxChunksPerSeries = len(chks) if len(chks) > 1000 { seriesOver1kChunks++ } } + return false }, labels.MustNewMatcher(labels.MatchEqual, "", ""), )