From bc78d13d9b736bb9313403569d0f69e85663afce Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 29 Apr 2024 12:54:05 -0700 Subject: [PATCH] feat(blooms): compute chunks once (#12664) Signed-off-by: Owen Diehl --- cmd/migrate/main.go | 2 +- docs/sources/shared/configuration.md | 6 + pkg/indexgateway/client.go | 42 +- pkg/indexgateway/gateway.go | 90 +- pkg/indexgateway/gateway_test.go | 27 +- pkg/indexgateway/shufflesharding.go | 1 + pkg/ingester/flush_test.go | 2 +- pkg/ingester/ingester.go | 2 +- pkg/ingester/ingester_test.go | 2 +- pkg/logcli/client/file.go | 2 + pkg/logcli/query/query.go | 2 + pkg/logcli/query/query_test.go | 2 +- pkg/logproto/extensions.go | 6 + pkg/logproto/indexgateway.pb.go | 185 +++- pkg/logproto/indexgateway.proto | 8 +- pkg/logproto/logproto.pb.go | 928 ++++++++++++------ pkg/logproto/logproto.proto | 10 + pkg/logql/accumulator_test.go | 6 +- pkg/logql/blocker_test.go | 2 +- pkg/logql/downstream.go | 48 +- pkg/logql/downstream_test.go | 11 +- pkg/logql/engine.go | 12 + pkg/logql/engine_test.go | 20 +- pkg/logql/evaluator.go | 39 + pkg/logql/shardmapper.go | 4 +- pkg/logql/shardmapper_test.go | 84 +- pkg/logql/shards.go | 58 +- pkg/loki/modules.go | 2 +- pkg/querier/multi_tenant_querier.go | 37 +- pkg/querier/querier_mock_test.go | 2 +- pkg/querier/queryrange/codec.go | 201 ++-- pkg/querier/queryrange/codec_test.go | 6 +- pkg/querier/queryrange/downstreamer.go | 2 + pkg/querier/queryrange/downstreamer_test.go | 3 + pkg/querier/queryrange/queryrange.pb.go | 425 +++++--- pkg/querier/queryrange/queryrange.proto | 6 + pkg/querier/queryrange/shard_resolver.go | 41 +- pkg/ruler/evaluator_local.go | 1 + pkg/storage/async_store.go | 10 +- pkg/storage/async_store_test.go | 12 +- pkg/storage/batch.go | 15 +- pkg/storage/store.go | 19 +- pkg/storage/store_test.go | 8 +- pkg/storage/stores/composite_store.go | 27 +- pkg/storage/stores/composite_store_entry.go | 46 +- pkg/storage/stores/composite_store_test.go | 76 +- .../stores/series/series_store_test.go | 4 +- .../boltdb/compactor/util_test.go | 2 +- pkg/storage/util_test.go | 2 +- pkg/validation/limits.go | 6 + 50 files changed, 1754 insertions(+), 800 deletions(-) diff --git a/cmd/migrate/main.go b/cmd/migrate/main.go index e42468e532b0..9b09a462538e 100644 --- a/cmd/migrate/main.go +++ b/cmd/migrate/main.go @@ -309,7 +309,7 @@ func (m *chunkMover) moveChunks(ctx context.Context, threadID int, syncRangeCh < start := time.Now() var totalBytes uint64 var totalChunks uint64 - schemaGroups, fetchers, err := m.source.GetChunks(m.ctx, m.sourceUser, model.TimeFromUnixNano(sr.from), model.TimeFromUnixNano(sr.to), chunk.NewPredicate(m.matchers, nil)) + schemaGroups, fetchers, err := m.source.GetChunks(m.ctx, m.sourceUser, model.TimeFromUnixNano(sr.from), model.TimeFromUnixNano(sr.to), chunk.NewPredicate(m.matchers, nil), nil) if err != nil { log.Println(threadID, "Error querying index for chunk refs:", err) errCh <- err diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index ab440d5d4f2f..99bca7d2e2cd 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -3003,6 +3003,12 @@ The `limits_config` block configures global and per-tenant limits in Loki. The v # CLI flag: -limits.tsdb-sharding-strategy [tsdb_sharding_strategy: | default = "power_of_two"] +# Precompute chunks for TSDB queries. This can improve query performance at the +# cost of increased memory usage by computing chunks once during planning, +# reducing index calls. +# CLI flag: -querier.tsdb-precompute-chunks +[tsdb_precompute_chunks: | default = false] + # Cardinality limit for index queries. # CLI flag: -store.cardinality-limit [cardinality_limit: | default = 100000] diff --git a/pkg/indexgateway/client.go b/pkg/indexgateway/client.go index 2acdad06937e..3d4abd0e79a1 100644 --- a/pkg/indexgateway/client.go +++ b/pkg/indexgateway/client.go @@ -28,9 +28,7 @@ import ( "github.com/grafana/loki/v3/pkg/distributor/clientpool" "github.com/grafana/loki/v3/pkg/logproto" - "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/storage/stores/series/index" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" "github.com/grafana/loki/v3/pkg/util/constants" "github.com/grafana/loki/v3/pkg/util/discovery" util_math "github.com/grafana/loki/v3/pkg/util/math" @@ -340,8 +338,7 @@ func (s *GatewayClient) GetShards( if err != nil { return errors.WithStack(err) } - perReplicaResult.Shards = append(perReplicaResult.Shards, resp.Shards...) - perReplicaResult.Statistics.Merge(resp.Statistics) + perReplicaResult.Merge(resp) } // Since `poolDo` retries on error, we only want to set the response if we got a successful response. @@ -355,48 +352,11 @@ func (s *GatewayClient) GetShards( 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 := 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 { diff --git a/pkg/indexgateway/gateway.go b/pkg/indexgateway/gateway.go index 27e2d9180129..0b6eda9409de 100644 --- a/pkg/indexgateway/gateway.go +++ b/pkg/indexgateway/gateway.go @@ -6,6 +6,7 @@ import ( "math" "sort" "sync" + "time" "github.com/c2h5oh/datasize" "github.com/go-kit/log" @@ -31,6 +32,7 @@ import ( seriesindex "github.com/grafana/loki/v3/pkg/storage/stores/series/index" tsdb_index "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + util_log "github.com/grafana/loki/v3/pkg/util/log" "github.com/grafana/loki/v3/pkg/util/spanlogger" ) @@ -67,19 +69,21 @@ type Gateway struct { bloomQuerier BloomQuerier metrics *Metrics - cfg Config - log log.Logger + cfg Config + limits Limits + log log.Logger } // NewIndexGateway instantiates a new Index Gateway and start its services. // // 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, r prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange, bloomQuerier BloomQuerier) (*Gateway, error) { +func NewIndexGateway(cfg Config, limits Limits, log log.Logger, r prometheus.Registerer, indexQuerier IndexQuerier, indexClients []IndexClientWithRange, bloomQuerier BloomQuerier) (*Gateway, error) { g := &Gateway{ indexQuerier: indexQuerier, bloomQuerier: bloomQuerier, cfg: cfg, + limits: limits, log: log, indexClients: indexClients, metrics: NewMetrics(r), @@ -214,7 +218,7 @@ func (g *Gateway) GetChunkRef(ctx context.Context, req *logproto.GetChunkRefRequ } predicate := chunk.NewPredicate(matchers, &req.Plan) - chunks, _, err := g.indexQuerier.GetChunks(ctx, instanceID, req.From, req.Through, predicate) + chunks, _, err := g.indexQuerier.GetChunks(ctx, instanceID, req.From, req.Through, predicate, nil) if err != nil { return nil, err } @@ -368,11 +372,12 @@ func (g *Gateway) GetShards(request *logproto.ShardsRequest, server logproto.Ind 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 { + if !ok { + sp.LogKV( + "msg", "index does not support forSeries", + "action", "falling back to indexQuerier.GetShards impl", + ) shards, err := g.indexQuerier.GetShards( ctx, instanceID, @@ -388,11 +393,11 @@ func (g *Gateway) GetShards(request *logproto.ShardsRequest, server logproto.Ind return server.Send(shards) } - return g.getShardsWithBlooms(ctx, request, server, instanceID, p, forSeries) + return g.boundedShards(ctx, request, server, instanceID, p, forSeries) } -// getShardsWithBlooms is a helper function to get shards with blooms enabled. -func (g *Gateway) getShardsWithBlooms( +// boundedShards handles bounded shard requests, optionally using blooms and/or returning precomputed chunks. +func (g *Gateway) boundedShards( ctx context.Context, req *logproto.ShardsRequest, server logproto.IndexGateway_GetShardsServer, @@ -412,12 +417,12 @@ func (g *Gateway) getShardsWithBlooms( // as getting it _very_ wrong could harm some cache locality benefits on the bloom-gws by // sending multiple requests to the entire keyspace). - logger := log.With(g.log, "tenant", instanceID) + logger := util_log.WithContext(ctx, g.log) sp, ctx := opentracing.StartSpanFromContext(ctx, "indexgateway.getShardsWithBlooms") defer sp.Finish() // 1) for all bounds, get chunk refs - grps, _, err := g.indexQuerier.GetChunks(ctx, instanceID, req.From, req.Through, p) + grps, _, err := g.indexQuerier.GetChunks(ctx, instanceID, req.From, req.Through, p, nil) if err != nil { return err } @@ -435,11 +440,16 @@ func (g *Gateway) getShardsWithBlooms( } } - // 2) filter via blooms - filtered, err := g.bloomQuerier.FilterChunkRefs(ctx, instanceID, req.From, req.Through, refs, p.Plan()) - if err != nil { - return err + filtered := refs + + // 2) filter via blooms if enabled + if g.bloomQuerier != nil && len(syntax.ExtractLineFilters(p.Plan().AST)) > 0 { + 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))) @@ -462,16 +472,29 @@ func (g *Gateway) getShardsWithBlooms( Stats: &logproto.IndexStatsResponse{}, }, } + } else { - shards, err := accumulateChunksToShards(ctx, instanceID, forSeries, req, p, filtered) + shards, chunkGrps, err := accumulateChunksToShards(ctx, instanceID, forSeries, req, p, filtered) if err != nil { return err } resp.Shards = shards + + // If the index gateway is configured to precompute chunks, we can return the chunk groups + // alongside the shards, otherwise discarding them + if g.limits.TSDBPrecomputeChunks(instanceID) { + resp.ChunkGroups = chunkGrps + } } sp.LogKV("msg", "send shards response", "shards", len(resp.Shards)) + var refCt int + for _, grp := range resp.ChunkGroups { + refCt += len(grp.Refs) + } + + ms := syntax.MatchersExpr{Mts: p.Matchers} level.Debug(logger).Log( "msg", "send shards response", "total_chunks", statistics.Index.TotalChunks, @@ -479,6 +502,12 @@ func (g *Gateway) getShardsWithBlooms( "shards", len(resp.Shards), "query", req.Query, "target_bytes_per_shard", datasize.ByteSize(req.TargetBytesPerShard).HumanReadable(), + "precomputed_refs", refCt, + "matchers", ms.String(), + "from", req.From.Time().String(), + "through", req.Through.Time().String(), + "length", req.Through.Time().Sub(req.From.Time()).String(), + "end_delta", time.Since(req.Through.Time()).String(), ) // 3) build shards @@ -525,7 +554,7 @@ func accumulateChunksToShards( req *logproto.ShardsRequest, p chunk.Predicate, filtered []*logproto.ChunkRef, -) ([]logproto.Shard, error) { +) ([]logproto.Shard, []logproto.ChunkRefGroup, 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 { @@ -541,12 +570,13 @@ func accumulateChunksToShards( v1.NewBounds(filtered[0].FingerprintModel(), filtered[len(filtered)-1].FingerprintModel()), req.From, req.Through, func(l labels.Labels, fp model.Fingerprint, chks []tsdb_index.ChunkMeta) (stop bool) { + mtx.Lock() + defer mtx.Unlock() + // check if this is a fingerprint we need if _, ok := filteredM[fp]; !ok { return false } - mtx.Lock() - defer mtx.Unlock() filteredChks := filteredM[fp] var j int @@ -579,7 +609,7 @@ func accumulateChunksToShards( }, p.Matchers..., ); err != nil { - return nil, err + return nil, nil, err } collectedSeries := sharding.SizedFPs(sharding.SizedFPsPool.Get(len(filteredM))) @@ -597,7 +627,21 @@ func accumulateChunksToShards( } sort.Sort(collectedSeries) - return collectedSeries.ShardsFor(req.TargetBytesPerShard), nil + shards := collectedSeries.ShardsFor(req.TargetBytesPerShard) + chkGrps := make([]logproto.ChunkRefGroup, 0, len(shards)) + for _, s := range shards { + from := sort.Search(len(filtered), func(i int) bool { + return filtered[i].Fingerprint >= uint64(s.Bounds.Min) + }) + through := sort.Search(len(filtered), func(i int) bool { + return filtered[i].Fingerprint > uint64(s.Bounds.Max) + }) + chkGrps = append(chkGrps, logproto.ChunkRefGroup{ + Refs: filtered[from:through], + }) + } + + return shards, chkGrps, nil } type refWithSizingInfo struct { diff --git a/pkg/indexgateway/gateway_test.go b/pkg/indexgateway/gateway_test.go index bf6af7cb5110..aa17482776e7 100644 --- a/pkg/indexgateway/gateway_test.go +++ b/pkg/indexgateway/gateway_test.go @@ -39,6 +39,18 @@ const ( valuePrefix = "value" ) +type mockLimits struct{} + +func (mockLimits) IndexGatewayShardSize(_ string) int { + return 0 +} +func (mockLimits) TSDBMaxBytesPerShard(_ string) int { + return sharding.DefaultTSDBMaxBytesPerShard +} +func (mockLimits) TSDBPrecomputeChunks(_ string) bool { + return false +} + type mockBatch struct { size int } @@ -233,7 +245,7 @@ func TestGateway_QueryIndex_multistore(t *testing.T) { }, }, }} - gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, nil, indexClients, nil) + gateway, err := NewIndexGateway(Config{}, mockLimits{}, util_log.Logger, nil, nil, indexClients, nil) require.NoError(t, err) expectedQueries = append(expectedQueries, @@ -258,7 +270,7 @@ func TestVolume(t *testing.T) { {Name: "bar", Volume: 38}, }}, nil) - gateway, err := NewIndexGateway(Config{}, util_log.Logger, nil, indexQuerier, nil, nil) + gateway, err := NewIndexGateway(Config{}, mockLimits{}, util_log.Logger, nil, indexQuerier, nil, nil) require.NoError(t, err) ctx := user.InjectOrgID(context.Background(), "test") @@ -532,7 +544,7 @@ func TestAccumulateChunksToShards(t *testing.T) { }, } - shards, err := accumulateChunksToShards( + shards, grps, err := accumulateChunksToShards( context.Background(), "", fsImpl(series), @@ -543,6 +555,12 @@ func TestAccumulateChunksToShards(t *testing.T) { filtered, ) + expectedChks := [][]*logproto.ChunkRef{ + filtered[0:3], + filtered[3:6], + filtered[6:9], + filtered[9:10], + } exp := []logproto.Shard{ { Bounds: logproto.FPBounds{Min: 0, Max: 1}, @@ -586,6 +604,9 @@ func TestAccumulateChunksToShards(t *testing.T) { for i := range shards { require.Equal(t, exp[i], shards[i], "invalid shard at index %d", i) + for j := range grps[i].Refs { + require.Equal(t, expectedChks[i][j], grps[i].Refs[j], "invalid chunk in grp %d at index %d", i, j) + } } require.Equal(t, len(exp), len(shards)) diff --git a/pkg/indexgateway/shufflesharding.go b/pkg/indexgateway/shufflesharding.go index 07bd2e8aa97b..8e73698f85ac 100644 --- a/pkg/indexgateway/shufflesharding.go +++ b/pkg/indexgateway/shufflesharding.go @@ -22,6 +22,7 @@ var ( type Limits interface { IndexGatewayShardSize(tenantID string) int TSDBMaxBytesPerShard(string) int + TSDBPrecomputeChunks(string) bool } type ShardingStrategy interface { diff --git a/pkg/ingester/flush_test.go b/pkg/ingester/flush_test.go index a4255f7510b7..e4fc748f2560 100644 --- a/pkg/ingester/flush_test.go +++ b/pkg/ingester/flush_test.go @@ -361,7 +361,7 @@ func (s *testStore) SelectSeries(_ context.Context, _ logql.SelectLogParams) ([] return nil, nil } -func (s *testStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (s *testStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate, _ *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { return nil, nil, nil } diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index e99de0d1531e..ac6a29e81d43 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -1036,7 +1036,7 @@ func (i *Ingester) GetChunkIDs(ctx context.Context, req *logproto.GetChunkIDsReq } // get chunk references - chunksGroups, _, err := i.store.GetChunks(ctx, orgID, start, end, chunk.NewPredicate(matchers, nil)) + chunksGroups, _, err := i.store.GetChunks(ctx, orgID, start, end, chunk.NewPredicate(matchers, nil), nil) if err != nil { return nil, err } diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 6722e6cfccf3..378123709a06 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -457,7 +457,7 @@ func (s *mockStore) PutOne(_ context.Context, _, _ model.Time, _ chunk.Chunk) er return nil } -func (s *mockStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (s *mockStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate, _ *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { return nil, nil, nil } diff --git a/pkg/logcli/client/file.go b/pkg/logcli/client/file.go index 34b76422d4bc..eabf29aef255 100644 --- a/pkg/logcli/client/file.go +++ b/pkg/logcli/client/file.go @@ -77,6 +77,7 @@ func (f *FileClient) Query(q string, limit int, t time.Time, direction logproto. direction, uint32(limit), nil, + nil, ) if err != nil { return nil, fmt.Errorf("failed to parse query: %w", err) @@ -118,6 +119,7 @@ func (f *FileClient) QueryRange(queryStr string, limit int, start, end time.Time direction, uint32(limit), nil, + nil, ) if err != nil { return nil, err diff --git a/pkg/logcli/query/query.go b/pkg/logcli/query/query.go index 84934e57730c..268def0bab68 100644 --- a/pkg/logcli/query/query.go +++ b/pkg/logcli/query/query.go @@ -490,6 +490,7 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string q.resultsDirection(), uint32(q.Limit), nil, + nil, ) if err != nil { return err @@ -506,6 +507,7 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string q.resultsDirection(), uint32(q.Limit), nil, + nil, ) if err != nil { return err diff --git a/pkg/logcli/query/query_test.go b/pkg/logcli/query/query_test.go index fffdcf15444b..8e52134482b7 100644 --- a/pkg/logcli/query/query_test.go +++ b/pkg/logcli/query/query_test.go @@ -426,7 +426,7 @@ func (t *testQueryClient) Query(_ string, _ int, _ time.Time, _ logproto.Directi func (t *testQueryClient) QueryRange(queryStr string, limit int, from, through time.Time, direction logproto.Direction, step, interval time.Duration, _ bool) (*loghttp.QueryResponse, error) { ctx := user.InjectOrgID(context.Background(), "fake") - params, err := logql.NewLiteralParams(queryStr, from, through, step, interval, direction, uint32(limit), nil) + params, err := logql.NewLiteralParams(queryStr, from, through, step, interval, direction, uint32(limit), nil, nil) if err != nil { return nil, err } diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index 7cd763cb5bc4..3de5c0fd7580 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -182,3 +182,9 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { func (d DetectedFieldType) String() string { return string(d) } + +func (m *ShardsResponse) Merge(other *ShardsResponse) { + m.Shards = append(m.Shards, other.Shards...) + m.ChunkGroups = append(m.ChunkGroups, other.ChunkGroups...) + m.Statistics.Merge(other.Statistics) +} diff --git a/pkg/logproto/indexgateway.pb.go b/pkg/logproto/indexgateway.pb.go index bd2650fbc01a..c9700f8cb15e 100644 --- a/pkg/logproto/indexgateway.pb.go +++ b/pkg/logproto/indexgateway.pb.go @@ -35,7 +35,7 @@ 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"` + TargetBytesPerShard uint64 `protobuf:"varint,4,opt,name=targetBytesPerShard,proto3" json:"targetBytesPerShard"` } func (m *ShardsRequest) Reset() { *m = ShardsRequest{} } @@ -87,6 +87,9 @@ func (m *ShardsRequest) GetTargetBytesPerShard() uint64 { 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"` + // If present, includes the chunk refs for each shard, + // therefore the length will be equal to the length of the `shards` field. + ChunkGroups []ChunkRefGroup `protobuf:"bytes,3,rep,name=chunkGroups,proto3" json:"chunkGroups"` } func (m *ShardsResponse) Reset() { *m = ShardsResponse{} } @@ -135,6 +138,13 @@ func (m *ShardsResponse) GetStatistics() stats.Result { return stats.Result{} } +func (m *ShardsResponse) GetChunkGroups() []ChunkRefGroup { + if m != nil { + return m.ChunkGroups + } + return nil +} + 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"` @@ -249,54 +259,54 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/indexgateway.proto", fileDescriptor_d27585148d0a52c8) } var fileDescriptor_d27585148d0a52c8 = []byte{ - // 737 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, 0x2c, 0x5f, - 0xbe, 0xc5, 0xae, 0x60, 0xa8, 0x5a, 0x09, 0x89, 0xba, 0x52, 0x22, 0x44, 0x5a, 0x51, 0x83, 0x18, - 0x2a, 0xb5, 0xd4, 0x09, 0x87, 0x63, 0x61, 0xfb, 0xc2, 0xdd, 0xb9, 0x0d, 0x5b, 0xc7, 0xaa, 0x53, - 0xd5, 0xff, 0xa0, 0x52, 0x87, 0xfe, 0x29, 0x8c, 0x8c, 0xa8, 0x83, 0x55, 0xc2, 0x52, 0x65, 0x62, - 0xee, 0x54, 0xf9, 0xce, 0x4e, 0x4c, 0x08, 0x12, 0xed, 0xe2, 0x3b, 0x3f, 0xef, 0xf3, 0x3e, 0xef, - 0x4f, 0x27, 0x50, 0xef, 0x1e, 0xb9, 0xa6, 0x4f, 0xdc, 0x2e, 0x25, 0x9c, 0x98, 0x5e, 0x78, 0x80, - 0x7b, 0xae, 0xc3, 0xf1, 0x1b, 0xe7, 0xc4, 0x10, 0x10, 0x5a, 0xc8, 0x63, 0xdd, 0x56, 0xf9, 0x6f, - 0x97, 0xb8, 0x44, 0xb2, 0x93, 0x9b, 0x64, 0x95, 0x97, 0xae, 0xc9, 0x64, 0x97, 0xd4, 0x58, 0x49, - 0x8d, 0xc7, 0x7e, 0x40, 0x0e, 0xb0, 0x6f, 0x32, 0xee, 0x70, 0x26, 0x9f, 0x92, 0x51, 0xfd, 0x34, - 0x05, 0xe7, 0x77, 0x3a, 0x0e, 0x3d, 0x60, 0x36, 0x3e, 0x8e, 0x30, 0xe3, 0x68, 0x0b, 0x16, 0x0f, - 0x29, 0x09, 0x54, 0x50, 0x01, 0xb5, 0x82, 0x75, 0xff, 0x34, 0xd6, 0x95, 0xaf, 0xb1, 0xfe, 0x9f, - 0xeb, 0xf1, 0x4e, 0xd4, 0x32, 0xda, 0x24, 0x30, 0xbb, 0x94, 0x04, 0x98, 0x77, 0x70, 0xc4, 0xcc, - 0x36, 0x09, 0x02, 0x12, 0x9a, 0x42, 0xdd, 0xd8, 0xf5, 0x02, 0x3c, 0x88, 0x75, 0xe1, 0x6e, 0x8b, - 0x27, 0xda, 0x85, 0x7f, 0xf0, 0x0e, 0x25, 0x91, 0xdb, 0x51, 0xa7, 0x84, 0xde, 0xc3, 0x5f, 0xd7, - 0xcb, 0x14, 0xec, 0xec, 0x82, 0x74, 0x58, 0x3a, 0x8e, 0x30, 0x3d, 0x51, 0x0b, 0x15, 0x50, 0x9b, - 0xb5, 0x66, 0x07, 0xb1, 0x2e, 0x01, 0x5b, 0x1e, 0xa8, 0x09, 0xff, 0xe5, 0x0e, 0x75, 0x31, 0xdf, - 0x6f, 0x9d, 0x70, 0xcc, 0xf6, 0xbb, 0x98, 0xee, 0xb3, 0xa4, 0x4a, 0xb5, 0x58, 0x01, 0xb5, 0xa2, - 0xb5, 0x38, 0x88, 0xf5, 0xbf, 0x24, 0xc3, 0x4a, 0x08, 0xdb, 0x98, 0x8a, 0x26, 0xd8, 0x93, 0xc0, - 0xea, 0x47, 0x00, 0x17, 0xb2, 0x1e, 0xb1, 0x2e, 0x09, 0x19, 0x46, 0xeb, 0x70, 0x5a, 0xe8, 0x31, - 0x15, 0x54, 0x0a, 0xb5, 0xb9, 0xd5, 0x7f, 0x8c, 0xeb, 0xc3, 0x32, 0x04, 0xdf, 0x5a, 0x48, 0xaa, - 0x1d, 0xc4, 0x7a, 0x4a, 0xb6, 0xd3, 0x13, 0x3d, 0x82, 0x30, 0x19, 0x82, 0xc7, 0xb8, 0xd7, 0x66, - 0xa2, 0x33, 0x73, 0xab, 0xf3, 0x86, 0x9c, 0x8b, 0x8d, 0x59, 0xe4, 0x73, 0x0b, 0xa5, 0xae, 0x39, - 0xa2, 0x9d, 0xbb, 0x57, 0xdf, 0x01, 0x58, 0x12, 0x41, 0xd0, 0x06, 0x9c, 0x6e, 0x91, 0x28, 0x14, - 0xb9, 0x24, 0x42, 0xea, 0x78, 0x2e, 0xf5, 0x6d, 0x4b, 0xd8, 0x47, 0xe9, 0x48, 0xbe, 0x9d, 0x9e, - 0x68, 0x1d, 0x96, 0x44, 0xec, 0x34, 0x93, 0x65, 0x63, 0xb8, 0x46, 0x9b, 0x89, 0xd2, 0x4e, 0x62, - 0xcb, 0x4a, 0x97, 0xdd, 0x16, 0x74, 0x5b, 0x1e, 0xd5, 0xcf, 0x00, 0xce, 0x64, 0x31, 0xd0, 0x16, - 0x2c, 0x04, 0x5e, 0x28, 0x52, 0x29, 0x5a, 0x0f, 0x06, 0xb1, 0x9e, 0xbc, 0xfe, 0x88, 0x75, 0xe3, - 0x0e, 0x03, 0xaf, 0x7b, 0xa1, 0x8b, 0x69, 0x97, 0x7a, 0x21, 0xb7, 0x13, 0x37, 0x21, 0xe6, 0xf4, - 0x44, 0x5a, 0x99, 0x98, 0xd3, 0xfb, 0x2d, 0x31, 0xa7, 0xb7, 0xfa, 0xbe, 0x04, 0xff, 0x14, 0xf5, - 0x34, 0x64, 0x67, 0xd0, 0x26, 0x84, 0xcf, 0x92, 0x75, 0x11, 0x20, 0x5a, 0x1a, 0x55, 0x3d, 0x42, - 0xd3, 0x8f, 0xa2, 0xbc, 0x3c, 0xd9, 0x28, 0x5b, 0x72, 0x0f, 0xa0, 0x26, 0x9c, 0x6b, 0x60, 0xfe, - 0xb8, 0x13, 0x85, 0x47, 0x36, 0x3e, 0x44, 0x39, 0x7a, 0x0e, 0xce, 0xc4, 0x56, 0x6e, 0xb1, 0x4a, - 0xb5, 0xaa, 0x82, 0xea, 0x70, 0xb6, 0x81, 0xf9, 0x0e, 0xa6, 0x1e, 0x66, 0xa8, 0x7c, 0x8d, 0x2d, - 0xc1, 0x4c, 0x69, 0x69, 0xa2, 0x6d, 0xa8, 0xf3, 0x12, 0x2e, 0x36, 0x9d, 0x16, 0xf6, 0x9f, 0x3a, - 0x01, 0x66, 0x75, 0x42, 0x9f, 0x60, 0x4e, 0xbd, 0x76, 0xf2, 0x86, 0x6a, 0x23, 0xcf, 0x5b, 0x28, - 0x59, 0x8c, 0xc5, 0x31, 0x66, 0x4e, 0xff, 0x15, 0x54, 0x05, 0xb4, 0xe7, 0xf8, 0xd1, 0x78, 0x80, - 0xff, 0xc7, 0xdc, 0x26, 0x70, 0xee, 0x10, 0xa1, 0x01, 0x67, 0x92, 0xc2, 0x92, 0x35, 0xcb, 0x0f, - 0x28, 0xbf, 0x96, 0x37, 0x06, 0x74, 0x73, 0x67, 0xab, 0x0a, 0xda, 0x10, 0x2d, 0xdd, 0x23, 0x7e, - 0x14, 0x60, 0x94, 0x0b, 0x28, 0x91, 0x4c, 0x45, 0xbd, 0x69, 0x18, 0x2a, 0x34, 0xe5, 0x50, 0xe4, - 0x07, 0xbc, 0x32, 0xf1, 0x7b, 0x1f, 0x66, 0xa3, 0xdd, 0x66, 0xce, 0x16, 0xc6, 0x7a, 0x71, 0x76, - 0xa1, 0x29, 0xe7, 0x17, 0x9a, 0x72, 0x75, 0xa1, 0x81, 0xb7, 0x7d, 0x0d, 0x7c, 0xe9, 0x6b, 0xe0, - 0xb4, 0xaf, 0x81, 0xb3, 0xbe, 0x06, 0xbe, 0xf5, 0x35, 0xf0, 0xbd, 0xaf, 0x29, 0x57, 0x7d, 0x0d, - 0x7c, 0xb8, 0xd4, 0x94, 0xb3, 0x4b, 0x4d, 0x39, 0xbf, 0xd4, 0x94, 0xe7, 0xf9, 0x5f, 0x4d, 0x97, - 0x3a, 0x87, 0x4e, 0xe8, 0x98, 0x3e, 0x39, 0xf2, 0xcc, 0xd7, 0x6b, 0x66, 0xfe, 0x7f, 0xa0, 0x35, - 0x2d, 0x8e, 0xb5, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x17, 0x36, 0xcf, 0x65, 0x06, 0x00, - 0x00, + // 751 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0xbf, 0x4f, 0xdb, 0x40, + 0x14, 0xf6, 0x91, 0x84, 0xc2, 0xa5, 0x30, 0x1c, 0xad, 0x62, 0x05, 0xb0, 0xa3, 0x2c, 0x4d, 0x17, + 0xbb, 0x82, 0xa1, 0x6a, 0x25, 0x24, 0xea, 0x4a, 0x89, 0x10, 0x14, 0x51, 0x83, 0x18, 0x2a, 0xb5, + 0xaa, 0x13, 0x0e, 0xc7, 0xc2, 0xf6, 0x05, 0xdf, 0xb9, 0x0d, 0x5b, 0xc7, 0xaa, 0x53, 0xff, 0x82, + 0x4e, 0x1d, 0xfa, 0xa7, 0x30, 0x32, 0xa2, 0x0e, 0x56, 0x31, 0x4b, 0x95, 0x89, 0xb9, 0x53, 0xe5, + 0x3b, 0x3b, 0x31, 0x21, 0x48, 0xb4, 0x8b, 0xef, 0xee, 0x7d, 0xdf, 0xfb, 0xee, 0xfd, 0xca, 0x05, + 0xaa, 0xbd, 0x23, 0x5b, 0x77, 0x89, 0xdd, 0x0b, 0x08, 0x23, 0xba, 0xe3, 0x1f, 0xe0, 0xbe, 0x6d, + 0x31, 0xfc, 0xd1, 0x3a, 0xd1, 0xb8, 0x09, 0xcd, 0xe7, 0x6d, 0xbd, 0x76, 0xf5, 0x81, 0x4d, 0x6c, + 0x22, 0xd8, 0xc9, 0x4e, 0xb0, 0xaa, 0x8b, 0xd7, 0x64, 0xb2, 0x4d, 0x0a, 0xd6, 0x52, 0xf0, 0xd8, + 0xf5, 0xc8, 0x01, 0x76, 0x75, 0xca, 0x2c, 0x46, 0xc5, 0x57, 0x30, 0xea, 0xdf, 0xa6, 0xe0, 0xdc, + 0x6e, 0xd7, 0x0a, 0x0e, 0xa8, 0x89, 0x8f, 0x43, 0x4c, 0x19, 0xda, 0x84, 0xc5, 0xc3, 0x80, 0x78, + 0x32, 0xa8, 0x81, 0x46, 0xc1, 0x78, 0x7a, 0x1a, 0xa9, 0xd2, 0xcf, 0x48, 0x7d, 0x64, 0x3b, 0xac, + 0x1b, 0xb6, 0xb5, 0x0e, 0xf1, 0xf4, 0x5e, 0x40, 0x3c, 0xcc, 0xba, 0x38, 0xa4, 0x7a, 0x87, 0x78, + 0x1e, 0xf1, 0x75, 0xae, 0xae, 0xed, 0x39, 0x1e, 0x1e, 0x44, 0x2a, 0x77, 0x37, 0xf9, 0x17, 0xed, + 0xc1, 0x7b, 0xac, 0x1b, 0x90, 0xd0, 0xee, 0xca, 0x53, 0x5c, 0xef, 0xf9, 0xbf, 0xeb, 0x65, 0x0a, + 0x66, 0xb6, 0x41, 0x2a, 0x2c, 0x1d, 0x87, 0x38, 0x38, 0x91, 0x0b, 0x35, 0xd0, 0x98, 0x35, 0x66, + 0x07, 0x91, 0x2a, 0x0c, 0xa6, 0x58, 0xd0, 0x06, 0x5c, 0x60, 0x56, 0x60, 0x63, 0x66, 0x9c, 0x30, + 0x4c, 0x77, 0x70, 0xc0, 0x53, 0x94, 0x8b, 0x35, 0xd0, 0x28, 0x1a, 0x95, 0x41, 0xa4, 0x4e, 0x82, + 0xcd, 0x49, 0xc6, 0x7a, 0x0c, 0xe0, 0x7c, 0x56, 0x20, 0xda, 0x23, 0x3e, 0xc5, 0x68, 0x0d, 0x4e, + 0x53, 0x6e, 0x91, 0x41, 0xad, 0xd0, 0x28, 0xaf, 0x3c, 0xd4, 0xae, 0x77, 0x4a, 0xe3, 0x7c, 0x63, + 0x3e, 0x49, 0x75, 0x10, 0xa9, 0x29, 0xd9, 0x4c, 0x57, 0xf4, 0x02, 0xc2, 0xa4, 0x03, 0x0e, 0x65, + 0x4e, 0x87, 0xf2, 0xb2, 0x94, 0x57, 0xe6, 0x34, 0xd1, 0x14, 0x13, 0xd3, 0xd0, 0x65, 0x06, 0x4a, + 0x5d, 0x73, 0x44, 0x33, 0xb7, 0x47, 0xdb, 0xb0, 0xdc, 0xe9, 0x86, 0xfe, 0x51, 0x2b, 0x20, 0x61, + 0x8f, 0xca, 0x05, 0x1e, 0x46, 0x45, 0x1b, 0x76, 0xff, 0x65, 0x02, 0x9a, 0xf8, 0x90, 0xe3, 0xc6, + 0x42, 0xaa, 0x96, 0xf7, 0x31, 0xf3, 0x87, 0xfa, 0x67, 0x00, 0x4b, 0x3c, 0x68, 0xb4, 0x0e, 0xa7, + 0xdb, 0x24, 0xf4, 0x79, 0x6e, 0x49, 0x60, 0xf2, 0x78, 0x6e, 0xcd, 0x1d, 0x83, 0xe3, 0xa3, 0xf4, + 0x04, 0xdf, 0x4c, 0x57, 0xb4, 0x06, 0x4b, 0x3c, 0x97, 0x34, 0xb3, 0xa5, 0x51, 0x54, 0x1b, 0x89, + 0xd2, 0x6e, 0x82, 0x65, 0xa5, 0x14, 0xad, 0xe3, 0x74, 0x53, 0x2c, 0xf5, 0xef, 0x00, 0xce, 0x64, + 0x77, 0xa0, 0x4d, 0x58, 0xf0, 0x1c, 0x9f, 0x87, 0x52, 0x34, 0x9e, 0x0d, 0x22, 0x35, 0x39, 0xfe, + 0x89, 0x54, 0xed, 0x0e, 0xd3, 0xd3, 0x74, 0x7c, 0x1b, 0x07, 0xbd, 0xc0, 0xf1, 0x99, 0x99, 0xb8, + 0x71, 0x31, 0xab, 0xcf, 0xc3, 0xca, 0xc4, 0xac, 0xfe, 0x7f, 0x89, 0x59, 0xfd, 0x95, 0x2f, 0x25, + 0x78, 0x9f, 0xe7, 0xd3, 0x12, 0x95, 0x41, 0x1b, 0x10, 0xbe, 0x4e, 0x66, 0x8f, 0x1b, 0xd1, 0xe2, + 0x28, 0xeb, 0x91, 0x35, 0xfd, 0x85, 0x55, 0x97, 0x26, 0x83, 0xa2, 0x24, 0x4f, 0x00, 0xda, 0x82, + 0xe5, 0x16, 0x66, 0x59, 0x0f, 0x51, 0x8e, 0x9e, 0x33, 0x67, 0x62, 0xcb, 0xb7, 0xa0, 0x42, 0xad, + 0x2e, 0xa1, 0x26, 0x9c, 0x6d, 0x61, 0xb6, 0x8b, 0x03, 0x07, 0x53, 0x54, 0xbd, 0xc6, 0x16, 0xc6, + 0x4c, 0x69, 0x71, 0x22, 0x36, 0xd4, 0x79, 0x07, 0x2b, 0x5b, 0x56, 0x1b, 0xbb, 0xdb, 0x96, 0x87, + 0x69, 0x93, 0x04, 0xaf, 0x30, 0x0b, 0x9c, 0x4e, 0x72, 0x42, 0x8d, 0x91, 0xe7, 0x2d, 0x94, 0xec, + 0x8e, 0xca, 0x18, 0x33, 0xa7, 0xff, 0x1e, 0xca, 0xdc, 0xb4, 0x6f, 0xb9, 0xe1, 0xf8, 0x05, 0x8f, + 0xc7, 0xdc, 0x26, 0x70, 0xee, 0x70, 0x43, 0x0b, 0xce, 0x24, 0x89, 0x25, 0x63, 0x96, 0x6f, 0x50, + 0x7e, 0x2c, 0x6f, 0x34, 0xe8, 0xe6, 0xcc, 0xd6, 0x25, 0xb4, 0xce, 0x4b, 0xba, 0x4f, 0xdc, 0xd0, + 0xc3, 0x28, 0x77, 0xa1, 0xb0, 0x64, 0x2a, 0xf2, 0x4d, 0x60, 0xa8, 0xb0, 0x25, 0x9a, 0x22, 0x1e, + 0x84, 0xe5, 0x89, 0xef, 0xc7, 0x30, 0x1a, 0xe5, 0x36, 0x38, 0x1b, 0x18, 0xe3, 0xed, 0xd9, 0x85, + 0x22, 0x9d, 0x5f, 0x28, 0xd2, 0xd5, 0x85, 0x02, 0x3e, 0xc5, 0x0a, 0xf8, 0x11, 0x2b, 0xe0, 0x34, + 0x56, 0xc0, 0x59, 0xac, 0x80, 0x5f, 0xb1, 0x02, 0x7e, 0xc7, 0x8a, 0x74, 0x15, 0x2b, 0xe0, 0xeb, + 0xa5, 0x22, 0x9d, 0x5d, 0x2a, 0xd2, 0xf9, 0xa5, 0x22, 0xbd, 0xc9, 0x3f, 0xc1, 0x76, 0x60, 0x1d, + 0x5a, 0xbe, 0xa5, 0xbb, 0xe4, 0xc8, 0xd1, 0x3f, 0xac, 0xea, 0xf9, 0x3f, 0x95, 0xf6, 0x34, 0x5f, + 0x56, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0xc1, 0xeb, 0x83, 0x96, 0xb2, 0x06, 0x00, 0x00, } func (this *ShardsRequest) Equal(that interface{}) bool { @@ -362,6 +372,14 @@ func (this *ShardsResponse) Equal(that interface{}) bool { if !this.Statistics.Equal(&that1.Statistics) { return false } + if len(this.ChunkGroups) != len(that1.ChunkGroups) { + return false + } + for i := range this.ChunkGroups { + if !this.ChunkGroups[i].Equal(&that1.ChunkGroups[i]) { + return false + } + } return true } func (this *Shard) Equal(that interface{}) bool { @@ -435,7 +453,7 @@ func (this *ShardsResponse) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) + s := make([]string, 0, 7) s = append(s, "&logproto.ShardsResponse{") if this.Shards != nil { vs := make([]*Shard, len(this.Shards)) @@ -445,6 +463,13 @@ func (this *ShardsResponse) GoString() string { s = append(s, "Shards: "+fmt.Sprintf("%#v", vs)+",\n") } s = append(s, "Statistics: "+strings.Replace(this.Statistics.GoString(), `&`, ``, 1)+",\n") + if this.ChunkGroups != nil { + vs := make([]*ChunkRefGroup, len(this.ChunkGroups)) + for i := range vs { + vs[i] = &this.ChunkGroups[i] + } + s = append(s, "ChunkGroups: "+fmt.Sprintf("%#v", vs)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -951,6 +976,20 @@ func (m *ShardsResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.ChunkGroups) > 0 { + for iNdEx := len(m.ChunkGroups) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ChunkGroups[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintIndexgateway(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } { size, err := m.Statistics.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1103,6 +1142,12 @@ func (m *ShardsResponse) Size() (n int) { } l = m.Statistics.Size() n += 1 + l + sovIndexgateway(uint64(l)) + if len(m.ChunkGroups) > 0 { + for _, e := range m.ChunkGroups { + l = e.Size() + n += 1 + l + sovIndexgateway(uint64(l)) + } + } return n } @@ -1164,9 +1209,15 @@ func (this *ShardsResponse) String() string { repeatedStringForShards += strings.Replace(strings.Replace(f.String(), "Shard", "Shard", 1), `&`, ``, 1) + "," } repeatedStringForShards += "}" + repeatedStringForChunkGroups := "[]ChunkRefGroup{" + for _, f := range this.ChunkGroups { + repeatedStringForChunkGroups += fmt.Sprintf("%v", f) + "," + } + repeatedStringForChunkGroups += "}" s := strings.Join([]string{`&ShardsResponse{`, `Shards:` + repeatedStringForShards + `,`, `Statistics:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Statistics), "Result", "stats.Result", 1), `&`, ``, 1) + `,`, + `ChunkGroups:` + repeatedStringForChunkGroups + `,`, `}`, }, "") return s @@ -1439,6 +1490,40 @@ func (m *ShardsResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkGroups", 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.ChunkGroups = append(m.ChunkGroups, ChunkRefGroup{}) + if err := m.ChunkGroups[len(m.ChunkGroups)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipIndexgateway(dAtA[iNdEx:]) diff --git a/pkg/logproto/indexgateway.proto b/pkg/logproto/indexgateway.proto index 1b54542d8f5a..a496538a5371 100644 --- a/pkg/logproto/indexgateway.proto +++ b/pkg/logproto/indexgateway.proto @@ -45,7 +45,7 @@ message ShardsRequest { (gogoproto.jsontag) = "through" ]; string query = 3 [(gogoproto.jsontag) = "query"]; - uint64 target_bytes_per_shard = 4 [(gogoproto.jsontag) = "targetBytesPerShard"]; + uint64 targetBytesPerShard = 4 [(gogoproto.jsontag) = "targetBytesPerShard"]; } message ShardsResponse { @@ -57,6 +57,12 @@ message ShardsResponse { (gogoproto.nullable) = false, (gogoproto.jsontag) = "statistics" ]; + // If present, includes the chunk refs for each shard, + // therefore the length will be equal to the length of the `shards` field. + repeated logproto.ChunkRefGroup chunkGroups = 3 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "chunkGroups" + ]; } message Shard { diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index e8c7215990bf..774af1fb709b 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -312,6 +312,9 @@ type QueryRequest struct { Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"` Deletes []*Delete `protobuf:"bytes,8,rep,name=deletes,proto3" json:"deletes,omitempty"` Plan *github_com_grafana_loki_v3_pkg_querier_plan.QueryPlan `protobuf:"bytes,9,opt,name=plan,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + StoreChunks *ChunkRefGroup `protobuf:"bytes,10,opt,name=storeChunks,proto3" json:"storeChunks"` } func (m *QueryRequest) Reset() { *m = QueryRequest{} } @@ -396,6 +399,13 @@ func (m *QueryRequest) GetDeletes() []*Delete { return nil } +func (m *QueryRequest) GetStoreChunks() *ChunkRefGroup { + if m != nil { + return m.StoreChunks + } + return nil +} + type SampleQueryRequest struct { Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` // Deprecated: Do not use. Start time.Time `protobuf:"bytes,2,opt,name=start,proto3,stdtime" json:"start"` @@ -403,6 +413,9 @@ type SampleQueryRequest struct { Shards []string `protobuf:"bytes,4,rep,name=shards,proto3" json:"shards,omitempty"` Deletes []*Delete `protobuf:"bytes,5,rep,name=deletes,proto3" json:"deletes,omitempty"` Plan *github_com_grafana_loki_v3_pkg_querier_plan.QueryPlan `protobuf:"bytes,6,opt,name=plan,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + StoreChunks *ChunkRefGroup `protobuf:"bytes,10,opt,name=storeChunks,proto3" json:"storeChunks"` } func (m *SampleQueryRequest) Reset() { *m = SampleQueryRequest{} } @@ -473,6 +486,13 @@ func (m *SampleQueryRequest) GetDeletes() []*Delete { return nil } +func (m *SampleQueryRequest) GetStoreChunks() *ChunkRefGroup { + if m != nil { + return m.StoreChunks + } + 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"` @@ -1738,6 +1758,49 @@ func (m *ChunkRef) GetChecksum() uint32 { return 0 } +type ChunkRefGroup struct { + Refs []*ChunkRef `protobuf:"bytes,1,rep,name=refs,proto3" json:"refs"` +} + +func (m *ChunkRefGroup) Reset() { *m = ChunkRefGroup{} } +func (*ChunkRefGroup) ProtoMessage() {} +func (*ChunkRefGroup) Descriptor() ([]byte, []int) { + return fileDescriptor_c28a5f14f1f4c79a, []int{30} +} +func (m *ChunkRefGroup) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChunkRefGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ChunkRefGroup.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 *ChunkRefGroup) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChunkRefGroup.Merge(m, src) +} +func (m *ChunkRefGroup) XXX_Size() int { + return m.Size() +} +func (m *ChunkRefGroup) XXX_DiscardUnknown() { + xxx_messageInfo_ChunkRefGroup.DiscardUnknown(m) +} + +var xxx_messageInfo_ChunkRefGroup proto.InternalMessageInfo + +func (m *ChunkRefGroup) GetRefs() []*ChunkRef { + if m != nil { + return m.Refs + } + return nil +} + type LabelValuesForMetricNameRequest struct { MetricName string `protobuf:"bytes,1,opt,name=metric_name,json=metricName,proto3" json:"metric_name,omitempty"` LabelName string `protobuf:"bytes,2,opt,name=label_name,json=labelName,proto3" json:"label_name,omitempty"` @@ -1749,7 +1812,7 @@ type LabelValuesForMetricNameRequest struct { func (m *LabelValuesForMetricNameRequest) Reset() { *m = LabelValuesForMetricNameRequest{} } func (*LabelValuesForMetricNameRequest) ProtoMessage() {} func (*LabelValuesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{30} + return fileDescriptor_c28a5f14f1f4c79a, []int{31} } func (m *LabelValuesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1808,7 +1871,7 @@ type LabelNamesForMetricNameRequest struct { func (m *LabelNamesForMetricNameRequest) Reset() { *m = LabelNamesForMetricNameRequest{} } func (*LabelNamesForMetricNameRequest) ProtoMessage() {} func (*LabelNamesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{31} + return fileDescriptor_c28a5f14f1f4c79a, []int{32} } func (m *LabelNamesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1852,7 +1915,7 @@ type LineFilter struct { func (m *LineFilter) Reset() { *m = LineFilter{} } func (*LineFilter) ProtoMessage() {} func (*LineFilter) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{32} + return fileDescriptor_c28a5f14f1f4c79a, []int{33} } func (m *LineFilter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1900,7 +1963,7 @@ type GetChunkRefRequest struct { func (m *GetChunkRefRequest) Reset() { *m = GetChunkRefRequest{} } func (*GetChunkRefRequest) ProtoMessage() {} func (*GetChunkRefRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{33} + return fileDescriptor_c28a5f14f1f4c79a, []int{34} } func (m *GetChunkRefRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1943,7 +2006,7 @@ type GetChunkRefResponse struct { func (m *GetChunkRefResponse) Reset() { *m = GetChunkRefResponse{} } func (*GetChunkRefResponse) ProtoMessage() {} func (*GetChunkRefResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{34} + return fileDescriptor_c28a5f14f1f4c79a, []int{35} } func (m *GetChunkRefResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1988,7 +2051,7 @@ type GetSeriesRequest struct { func (m *GetSeriesRequest) Reset() { *m = GetSeriesRequest{} } func (*GetSeriesRequest) ProtoMessage() {} func (*GetSeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{35} + return fileDescriptor_c28a5f14f1f4c79a, []int{36} } func (m *GetSeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2031,7 +2094,7 @@ type GetSeriesResponse struct { func (m *GetSeriesResponse) Reset() { *m = GetSeriesResponse{} } func (*GetSeriesResponse) ProtoMessage() {} func (*GetSeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{36} + return fileDescriptor_c28a5f14f1f4c79a, []int{37} } func (m *GetSeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2075,7 +2138,7 @@ type IndexSeries struct { func (m *IndexSeries) Reset() { *m = IndexSeries{} } func (*IndexSeries) ProtoMessage() {} func (*IndexSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{37} + return fileDescriptor_c28a5f14f1f4c79a, []int{38} } func (m *IndexSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2112,7 +2175,7 @@ type QueryIndexResponse struct { func (m *QueryIndexResponse) Reset() { *m = QueryIndexResponse{} } func (*QueryIndexResponse) ProtoMessage() {} func (*QueryIndexResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{38} + return fileDescriptor_c28a5f14f1f4c79a, []int{39} } func (m *QueryIndexResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2163,7 +2226,7 @@ type Row struct { func (m *Row) Reset() { *m = Row{} } func (*Row) ProtoMessage() {} func (*Row) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{39} + return fileDescriptor_c28a5f14f1f4c79a, []int{40} } func (m *Row) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2213,7 +2276,7 @@ type QueryIndexRequest struct { func (m *QueryIndexRequest) Reset() { *m = QueryIndexRequest{} } func (*QueryIndexRequest) ProtoMessage() {} func (*QueryIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{40} + return fileDescriptor_c28a5f14f1f4c79a, []int{41} } func (m *QueryIndexRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2260,7 +2323,7 @@ type IndexQuery struct { func (m *IndexQuery) Reset() { *m = IndexQuery{} } func (*IndexQuery) ProtoMessage() {} func (*IndexQuery) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{41} + return fileDescriptor_c28a5f14f1f4c79a, []int{42} } func (m *IndexQuery) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2333,7 +2396,7 @@ type IndexStatsRequest struct { func (m *IndexStatsRequest) Reset() { *m = IndexStatsRequest{} } func (*IndexStatsRequest) ProtoMessage() {} func (*IndexStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{42} + return fileDescriptor_c28a5f14f1f4c79a, []int{43} } func (m *IndexStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2379,7 +2442,7 @@ type IndexStatsResponse struct { func (m *IndexStatsResponse) Reset() { *m = IndexStatsResponse{} } func (*IndexStatsResponse) ProtoMessage() {} func (*IndexStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{43} + return fileDescriptor_c28a5f14f1f4c79a, []int{44} } func (m *IndexStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2449,7 +2512,7 @@ type VolumeRequest struct { func (m *VolumeRequest) Reset() { *m = VolumeRequest{} } func (*VolumeRequest) ProtoMessage() {} func (*VolumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{44} + return fileDescriptor_c28a5f14f1f4c79a, []int{45} } func (m *VolumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2521,7 +2584,7 @@ type VolumeResponse struct { func (m *VolumeResponse) Reset() { *m = VolumeResponse{} } func (*VolumeResponse) ProtoMessage() {} func (*VolumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{45} + return fileDescriptor_c28a5f14f1f4c79a, []int{46} } func (m *VolumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2572,7 +2635,7 @@ type Volume struct { func (m *Volume) Reset() { *m = Volume{} } func (*Volume) ProtoMessage() {} func (*Volume) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{46} + return fileDescriptor_c28a5f14f1f4c79a, []int{47} } func (m *Volume) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2690,7 @@ type DetectedFieldsRequest struct { func (m *DetectedFieldsRequest) Reset() { *m = DetectedFieldsRequest{} } func (*DetectedFieldsRequest) ProtoMessage() {} func (*DetectedFieldsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{47} + return fileDescriptor_c28a5f14f1f4c79a, []int{48} } func (m *DetectedFieldsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2706,7 +2769,7 @@ type DetectedFieldsResponse struct { func (m *DetectedFieldsResponse) Reset() { *m = DetectedFieldsResponse{} } func (*DetectedFieldsResponse) ProtoMessage() {} func (*DetectedFieldsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{48} + return fileDescriptor_c28a5f14f1f4c79a, []int{49} } func (m *DetectedFieldsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2761,7 +2824,7 @@ type DetectedField struct { func (m *DetectedField) Reset() { *m = DetectedField{} } func (*DetectedField) ProtoMessage() {} func (*DetectedField) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{49} + return fileDescriptor_c28a5f14f1f4c79a, []int{50} } func (m *DetectedField) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2827,7 +2890,7 @@ type DetectedLabelsRequest struct { func (m *DetectedLabelsRequest) Reset() { *m = DetectedLabelsRequest{} } func (*DetectedLabelsRequest) ProtoMessage() {} func (*DetectedLabelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{50} + return fileDescriptor_c28a5f14f1f4c79a, []int{51} } func (m *DetectedLabelsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2884,7 +2947,7 @@ type DetectedLabelsResponse struct { func (m *DetectedLabelsResponse) Reset() { *m = DetectedLabelsResponse{} } func (*DetectedLabelsResponse) ProtoMessage() {} func (*DetectedLabelsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{51} + return fileDescriptor_c28a5f14f1f4c79a, []int{52} } func (m *DetectedLabelsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2928,7 +2991,7 @@ type DetectedLabel struct { func (m *DetectedLabel) Reset() { *m = DetectedLabel{} } func (*DetectedLabel) ProtoMessage() {} func (*DetectedLabel) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{52} + return fileDescriptor_c28a5f14f1f4c79a, []int{53} } func (m *DetectedLabel) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3005,6 +3068,7 @@ func init() { proto.RegisterType((*GetChunkIDsRequest)(nil), "logproto.GetChunkIDsRequest") proto.RegisterType((*GetChunkIDsResponse)(nil), "logproto.GetChunkIDsResponse") proto.RegisterType((*ChunkRef)(nil), "logproto.ChunkRef") + proto.RegisterType((*ChunkRefGroup)(nil), "logproto.ChunkRefGroup") proto.RegisterType((*LabelValuesForMetricNameRequest)(nil), "logproto.LabelValuesForMetricNameRequest") proto.RegisterType((*LabelNamesForMetricNameRequest)(nil), "logproto.LabelNamesForMetricNameRequest") proto.RegisterType((*LineFilter)(nil), "logproto.LineFilter") @@ -3033,171 +3097,174 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) } var fileDescriptor_c28a5f14f1f4c79a = []byte{ - // 2611 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3a, 0x4d, 0x6c, 0x1b, 0xc7, - 0xd5, 0x5c, 0x72, 0xf9, 0xf7, 0x48, 0xc9, 0xf2, 0x88, 0xb6, 0x09, 0xda, 0xe1, 0x2a, 0x83, 0xef, - 0x4b, 0xdc, 0xd8, 0x11, 0x63, 0xa7, 0x49, 0x1d, 0xa7, 0x69, 0x6a, 0x4a, 0xb1, 0x63, 0x47, 0x71, - 0x9c, 0x91, 0xe2, 0xa4, 0x45, 0x83, 0x60, 0x45, 0x8e, 0xa8, 0x85, 0xc8, 0x5d, 0x7a, 0x77, 0x18, - 0x87, 0xb7, 0x02, 0x3d, 0x17, 0x0d, 0xd0, 0x43, 0xdb, 0x4b, 0xd1, 0x02, 0x05, 0x5a, 0xa4, 0xe8, - 0xa5, 0xe8, 0xb1, 0x68, 0x2f, 0x3d, 0xa4, 0xb7, 0xf4, 0x16, 0xe4, 0xc0, 0xd6, 0xca, 0xa5, 0xd0, - 0x29, 0x40, 0x6f, 0x39, 0x15, 0xf3, 0xb3, 0xbb, 0xb3, 0x2b, 0xb2, 0x0e, 0x15, 0x07, 0x81, 0x2f, - 0xe2, 0xcc, 0x9b, 0x37, 0x6f, 0xe6, 0xfd, 0xcc, 0xfb, 0x5b, 0xc1, 0xe9, 0xe1, 0x5e, 0xaf, 0xd5, - 0xf7, 0x7a, 0x43, 0xdf, 0x63, 0x5e, 0x34, 0x58, 0x15, 0x7f, 0x51, 0x29, 0x9c, 0x37, 0x6a, 0x3d, - 0xaf, 0xe7, 0x49, 0x1c, 0x3e, 0x92, 0xeb, 0x0d, 0xab, 0xe7, 0x79, 0xbd, 0x3e, 0x6d, 0x89, 0xd9, - 0xf6, 0x68, 0xa7, 0xc5, 0x9c, 0x01, 0x0d, 0x98, 0x3d, 0x18, 0x2a, 0x84, 0x15, 0x45, 0xfd, 0x4e, - 0x7f, 0xe0, 0x75, 0x69, 0xbf, 0x15, 0x30, 0x9b, 0x05, 0xf2, 0xaf, 0xc2, 0x58, 0xe6, 0x18, 0xc3, - 0x51, 0xb0, 0x2b, 0xfe, 0x48, 0x20, 0xfe, 0x93, 0x01, 0x27, 0x36, 0xec, 0x6d, 0xda, 0xdf, 0xf2, - 0x6e, 0xdb, 0xfd, 0x11, 0x0d, 0x08, 0x0d, 0x86, 0x9e, 0x1b, 0x50, 0xb4, 0x06, 0x85, 0x3e, 0x5f, - 0x08, 0xea, 0xc6, 0x4a, 0xee, 0x6c, 0xe5, 0xe2, 0xb9, 0xd5, 0xe8, 0xca, 0x53, 0x37, 0x48, 0x68, - 0xf0, 0x92, 0xcb, 0xfc, 0x31, 0x51, 0x5b, 0x1b, 0xb7, 0xa1, 0xa2, 0x81, 0xd1, 0x12, 0xe4, 0xf6, - 0xe8, 0xb8, 0x6e, 0xac, 0x18, 0x67, 0xcb, 0x84, 0x0f, 0xd1, 0x05, 0xc8, 0xbf, 0xcb, 0xc9, 0xd4, - 0xb3, 0x2b, 0xc6, 0xd9, 0xca, 0xc5, 0xd3, 0xf1, 0x21, 0x6f, 0xb8, 0xce, 0x9d, 0x11, 0x15, 0xbb, - 0xd5, 0x41, 0x12, 0xf3, 0x72, 0xf6, 0x92, 0x81, 0xcf, 0xc1, 0xf1, 0x43, 0xeb, 0xe8, 0x24, 0x14, - 0x04, 0x86, 0xbc, 0x71, 0x99, 0xa8, 0x19, 0xae, 0x01, 0xda, 0x64, 0x3e, 0xb5, 0x07, 0xc4, 0x66, - 0xfc, 0xbe, 0x77, 0x46, 0x34, 0x60, 0xf8, 0x55, 0x58, 0x4e, 0x40, 0x15, 0xdb, 0xcf, 0x42, 0x25, - 0x88, 0xc1, 0x8a, 0xf7, 0x5a, 0x7c, 0xad, 0x78, 0x0f, 0xd1, 0x11, 0xf1, 0x2f, 0x0d, 0x80, 0x78, - 0x0d, 0x35, 0x01, 0xe4, 0xea, 0xcb, 0x76, 0xb0, 0x2b, 0x18, 0x36, 0x89, 0x06, 0x41, 0xe7, 0xe1, - 0x78, 0x3c, 0xbb, 0xe9, 0x6d, 0xee, 0xda, 0x7e, 0x57, 0xc8, 0xc0, 0x24, 0x87, 0x17, 0x10, 0x02, - 0xd3, 0xb7, 0x19, 0xad, 0xe7, 0x56, 0x8c, 0xb3, 0x39, 0x22, 0xc6, 0x9c, 0x5b, 0x46, 0x5d, 0xdb, - 0x65, 0x75, 0x53, 0x88, 0x53, 0xcd, 0x38, 0x9c, 0xeb, 0x97, 0x06, 0xf5, 0xfc, 0x8a, 0x71, 0x76, - 0x81, 0xa8, 0x19, 0xfe, 0x20, 0x07, 0xd5, 0xd7, 0x47, 0xd4, 0x1f, 0x2b, 0x01, 0xa0, 0x26, 0x94, - 0x02, 0xda, 0xa7, 0x1d, 0xe6, 0xf9, 0x52, 0x23, 0xed, 0x6c, 0xdd, 0x20, 0x11, 0x0c, 0xd5, 0x20, - 0xdf, 0x77, 0x06, 0x0e, 0x13, 0xd7, 0x5a, 0x20, 0x72, 0x82, 0x2e, 0x43, 0x3e, 0x60, 0xb6, 0xcf, - 0xc4, 0x5d, 0x2a, 0x17, 0x1b, 0xab, 0xd2, 0x30, 0x57, 0x43, 0xc3, 0x5c, 0xdd, 0x0a, 0x0d, 0xb3, - 0x5d, 0xfa, 0x70, 0x62, 0x65, 0xde, 0xff, 0xa7, 0x65, 0x10, 0xb9, 0x05, 0x3d, 0x0b, 0x39, 0xea, - 0x76, 0xc5, 0x7d, 0xbf, 0xe8, 0x4e, 0xbe, 0x01, 0x5d, 0x80, 0x72, 0xd7, 0xf1, 0x69, 0x87, 0x39, - 0x9e, 0x2b, 0xb8, 0x5a, 0xbc, 0xb8, 0x1c, 0x6b, 0x64, 0x3d, 0x5c, 0x22, 0x31, 0x16, 0x3a, 0x0f, - 0x85, 0x80, 0x8b, 0x2e, 0xa8, 0x17, 0xb9, 0x2d, 0xb4, 0x6b, 0x07, 0x13, 0x6b, 0x49, 0x42, 0xce, - 0x7b, 0x03, 0x87, 0xd1, 0xc1, 0x90, 0x8d, 0x89, 0xc2, 0x41, 0x4f, 0x40, 0xb1, 0x4b, 0xfb, 0x94, - 0x2b, 0xbc, 0x24, 0x14, 0xbe, 0xa4, 0x91, 0x17, 0x0b, 0x24, 0x44, 0x40, 0x6f, 0x83, 0x39, 0xec, - 0xdb, 0x6e, 0xbd, 0x2c, 0xb8, 0x58, 0x8c, 0x11, 0x6f, 0xf5, 0x6d, 0xb7, 0xfd, 0xdc, 0x27, 0x13, - 0xeb, 0x99, 0x9e, 0xc3, 0x76, 0x47, 0xdb, 0xab, 0x1d, 0x6f, 0xd0, 0xea, 0xf9, 0xf6, 0x8e, 0xed, - 0xda, 0xad, 0xbe, 0xb7, 0xe7, 0xb4, 0xde, 0x7d, 0xba, 0xc5, 0xdf, 0xe0, 0x9d, 0x11, 0xf5, 0x1d, - 0xea, 0xb7, 0x38, 0x99, 0x55, 0xa1, 0x12, 0xbe, 0x95, 0x08, 0xb2, 0x37, 0xcc, 0x52, 0x61, 0xa9, - 0x88, 0xef, 0x65, 0x01, 0x6d, 0xda, 0x83, 0x61, 0x9f, 0xce, 0xa5, 0xb2, 0x48, 0x39, 0xd9, 0x23, - 0x2b, 0x27, 0x37, 0xaf, 0x72, 0x62, 0x49, 0x9b, 0xf3, 0x49, 0x3a, 0xff, 0x45, 0x25, 0x5d, 0xf8, - 0x4a, 0x24, 0x8d, 0xeb, 0x60, 0xf2, 0x19, 0x77, 0x4a, 0xbe, 0x7d, 0x57, 0xc8, 0xb3, 0x4a, 0xf8, - 0x10, 0x6f, 0x40, 0x41, 0xde, 0x05, 0x35, 0xd2, 0x02, 0x4f, 0xbe, 0x8f, 0x58, 0xd8, 0xb9, 0x50, - 0x8c, 0x4b, 0xb1, 0x18, 0x73, 0x42, 0x40, 0xf8, 0xcf, 0x06, 0x2c, 0x28, 0x2d, 0x2a, 0x1f, 0xb3, - 0x0d, 0x45, 0xf9, 0xc6, 0x43, 0xff, 0x72, 0x2a, 0xed, 0x5f, 0xae, 0x74, 0xed, 0x21, 0xa3, 0x7e, - 0xbb, 0xf5, 0xe1, 0xc4, 0x32, 0x3e, 0x99, 0x58, 0x8f, 0xcf, 0x62, 0x34, 0xf4, 0xe9, 0xa1, 0x5f, - 0x0a, 0x09, 0xa3, 0x73, 0xe2, 0x76, 0x2c, 0x50, 0xa6, 0x70, 0x6c, 0x55, 0x86, 0x82, 0xeb, 0x6e, - 0x8f, 0x06, 0x9c, 0xb2, 0xc9, 0xb5, 0x48, 0x24, 0x0e, 0x67, 0xf3, 0xae, 0xed, 0xbb, 0x8e, 0xdb, - 0x0b, 0xea, 0x39, 0xe1, 0x3b, 0xa3, 0x39, 0xfe, 0xb9, 0x01, 0xcb, 0x09, 0x53, 0x54, 0x4c, 0x5c, - 0x82, 0x42, 0xc0, 0xa5, 0x1b, 0xf2, 0xa0, 0x29, 0x72, 0x53, 0xc0, 0xdb, 0x8b, 0xea, 0xf2, 0x05, - 0x39, 0x27, 0x0a, 0xff, 0xc1, 0x5d, 0xed, 0x6f, 0x06, 0x54, 0x45, 0x00, 0x08, 0xdf, 0x07, 0x02, - 0xd3, 0xb5, 0x07, 0x54, 0xa9, 0x4a, 0x8c, 0xb5, 0xa8, 0xc0, 0x8f, 0x2b, 0x85, 0x51, 0x61, 0x5e, - 0x47, 0x66, 0x1c, 0xd9, 0x91, 0x19, 0xf1, 0x5b, 0xa9, 0x41, 0x9e, 0x9b, 0xe4, 0x58, 0x38, 0xb1, - 0x32, 0x91, 0x13, 0xfc, 0x38, 0x2c, 0x28, 0x2e, 0x94, 0x68, 0x67, 0x05, 0xb2, 0x01, 0x14, 0xa4, - 0x26, 0xd0, 0xff, 0x41, 0x39, 0x4a, 0x00, 0x04, 0xb7, 0xb9, 0x76, 0xe1, 0x60, 0x62, 0x65, 0x59, - 0x40, 0xe2, 0x05, 0x64, 0xe9, 0xc1, 0xd5, 0x68, 0x97, 0x0f, 0x26, 0x96, 0x04, 0xa8, 0x50, 0x8a, - 0xce, 0x80, 0xb9, 0xcb, 0xe3, 0x13, 0x17, 0x81, 0xd9, 0x2e, 0x1d, 0x4c, 0x2c, 0x31, 0x27, 0xe2, - 0x2f, 0xbe, 0x06, 0xd5, 0x0d, 0xda, 0xb3, 0x3b, 0x63, 0x75, 0x68, 0x2d, 0x24, 0xc7, 0x0f, 0x34, - 0x42, 0x1a, 0x8f, 0x42, 0x35, 0x3a, 0xf1, 0x9d, 0x41, 0xa0, 0x5e, 0x43, 0x25, 0x82, 0xbd, 0x1a, - 0xe0, 0x5f, 0x18, 0xa0, 0x6c, 0x00, 0x61, 0x2d, 0xab, 0xe0, 0xfe, 0x0b, 0x0e, 0x26, 0x96, 0x82, - 0x84, 0x49, 0x03, 0x7a, 0x1e, 0x8a, 0x81, 0x38, 0x91, 0x13, 0x4b, 0x9b, 0x96, 0x58, 0x68, 0x1f, - 0xe3, 0x26, 0x72, 0x30, 0xb1, 0x42, 0x44, 0x12, 0x0e, 0xd0, 0x6a, 0x22, 0xf0, 0x4a, 0xc6, 0x16, - 0x0f, 0x26, 0x96, 0x06, 0xd5, 0x03, 0x31, 0xfe, 0xdc, 0x80, 0xca, 0x96, 0xed, 0x44, 0x26, 0x54, - 0x0f, 0x55, 0x14, 0xfb, 0x57, 0x09, 0xe0, 0x96, 0xd8, 0xa5, 0x7d, 0x7b, 0x7c, 0xd5, 0xf3, 0x05, - 0xdd, 0x05, 0x12, 0xcd, 0xe3, 0x58, 0x69, 0x4e, 0x8d, 0x95, 0xf9, 0xf9, 0xdd, 0xf1, 0x57, 0xeb, - 0xfc, 0x6e, 0x98, 0xa5, 0xec, 0x52, 0x0e, 0xff, 0xc1, 0x80, 0xaa, 0x64, 0x5e, 0x59, 0xde, 0x0f, - 0xa0, 0x20, 0x65, 0x23, 0xd8, 0xff, 0x1f, 0x8e, 0xe9, 0xdc, 0x3c, 0x4e, 0x49, 0xd1, 0x44, 0x2f, - 0xc2, 0x62, 0xd7, 0xf7, 0x86, 0x43, 0xda, 0xdd, 0x54, 0xee, 0x2f, 0x9b, 0x76, 0x7f, 0xeb, 0xfa, - 0x3a, 0x49, 0xa1, 0xe3, 0xbf, 0x1b, 0xb0, 0xa0, 0x9c, 0x89, 0x52, 0x57, 0x24, 0x62, 0xe3, 0xc8, - 0x11, 0x2f, 0x3b, 0x6f, 0xc4, 0x3b, 0x09, 0x85, 0x9e, 0xef, 0x8d, 0x86, 0xa1, 0x43, 0x52, 0xb3, - 0xf9, 0x22, 0x21, 0xbe, 0x01, 0x8b, 0x21, 0x2b, 0x33, 0x3c, 0x6a, 0x23, 0xed, 0x51, 0xaf, 0x77, - 0xa9, 0xcb, 0x9c, 0x1d, 0x27, 0xf2, 0x91, 0x0a, 0x1f, 0xff, 0xc4, 0x80, 0xa5, 0x34, 0x0a, 0x5a, - 0x4f, 0x25, 0xf0, 0x8f, 0xcd, 0x26, 0xa7, 0xe7, 0xee, 0x21, 0x69, 0x95, 0xc1, 0x3f, 0x73, 0xbf, - 0x0c, 0xbe, 0xa6, 0x3b, 0x99, 0xb2, 0xf2, 0x0a, 0xf8, 0x67, 0x06, 0x2c, 0x24, 0x74, 0x89, 0x2e, - 0x81, 0xb9, 0xe3, 0x7b, 0x83, 0xb9, 0x14, 0x25, 0x76, 0xa0, 0x6f, 0x42, 0x96, 0x79, 0x73, 0xa9, - 0x29, 0xcb, 0x3c, 0xae, 0x25, 0xc5, 0x7e, 0x4e, 0xe6, 0xc7, 0x72, 0x86, 0x9f, 0x81, 0xb2, 0x60, - 0xe8, 0x96, 0xed, 0xf8, 0x53, 0x03, 0xc6, 0x74, 0x86, 0x9e, 0x87, 0x63, 0xd2, 0x19, 0x4e, 0xdf, - 0x5c, 0x9d, 0xb6, 0xb9, 0x1a, 0x6e, 0x3e, 0x0d, 0xf9, 0xb5, 0xdd, 0x91, 0xbb, 0xc7, 0xb7, 0x74, - 0x6d, 0x66, 0x87, 0x5b, 0xf8, 0x18, 0x9f, 0x80, 0x65, 0xfe, 0x06, 0xa9, 0x1f, 0xac, 0x79, 0x23, - 0x97, 0x85, 0xf5, 0xc9, 0x79, 0xa8, 0x25, 0xc1, 0xca, 0x4a, 0x6a, 0x90, 0xef, 0x70, 0x80, 0xa0, - 0xb1, 0x40, 0xe4, 0x04, 0xff, 0xc6, 0x00, 0x74, 0x8d, 0x32, 0x71, 0xca, 0xf5, 0xf5, 0xe8, 0x79, - 0x34, 0xa0, 0x34, 0xb0, 0x59, 0x67, 0x97, 0xfa, 0x41, 0x98, 0xbf, 0x84, 0xf3, 0xaf, 0x23, 0x59, - 0xc4, 0x17, 0x60, 0x39, 0x71, 0x4b, 0xc5, 0x53, 0x03, 0x4a, 0x1d, 0x05, 0x53, 0x21, 0x2f, 0x9a, - 0xe3, 0x3f, 0x66, 0xa1, 0x24, 0x36, 0x10, 0xba, 0x83, 0x2e, 0x40, 0x65, 0xc7, 0x71, 0x7b, 0xd4, - 0x1f, 0xfa, 0x8e, 0x12, 0x81, 0xd9, 0x3e, 0x76, 0x30, 0xb1, 0x74, 0x30, 0xd1, 0x27, 0xe8, 0x49, - 0x28, 0x8e, 0x02, 0xea, 0xbf, 0xe3, 0xc8, 0x97, 0x5e, 0x6e, 0xd7, 0xf6, 0x27, 0x56, 0xe1, 0x8d, - 0x80, 0xfa, 0xd7, 0xd7, 0x79, 0xf0, 0x19, 0x89, 0x11, 0x91, 0xbf, 0x5d, 0xf4, 0x8a, 0x32, 0x53, - 0x91, 0xc0, 0xb5, 0xbf, 0xc5, 0xaf, 0x9f, 0x72, 0x75, 0x43, 0xdf, 0x1b, 0x50, 0xb6, 0x4b, 0x47, - 0x41, 0xab, 0xe3, 0x0d, 0x06, 0x9e, 0xdb, 0x12, 0x15, 0xb7, 0x60, 0x9a, 0x47, 0x50, 0xbe, 0x5d, - 0x59, 0xee, 0x16, 0x14, 0xd9, 0xae, 0xef, 0x8d, 0x7a, 0xbb, 0x22, 0x30, 0xe4, 0xda, 0x97, 0xe7, - 0xa7, 0x17, 0x52, 0x20, 0xe1, 0x00, 0x3d, 0xca, 0xa5, 0x45, 0x3b, 0x7b, 0xc1, 0x68, 0x20, 0x6b, - 0xbc, 0x76, 0xfe, 0x60, 0x62, 0x19, 0x4f, 0x92, 0x08, 0x8c, 0x7f, 0x9c, 0x05, 0x4b, 0x2b, 0x8d, - 0xaf, 0x7a, 0xfe, 0xab, 0x94, 0xf9, 0x4e, 0xe7, 0xa6, 0x3d, 0xa0, 0xa1, 0x6d, 0x58, 0x50, 0x19, - 0x08, 0xe0, 0x3b, 0xda, 0x13, 0x80, 0x41, 0x84, 0x87, 0x1e, 0x01, 0x10, 0x6f, 0x46, 0xae, 0xcb, - 0xd7, 0x50, 0x16, 0x10, 0xb1, 0xbc, 0x96, 0x90, 0x54, 0x6b, 0x4e, 0xce, 0x94, 0x84, 0xae, 0xa7, - 0x25, 0x34, 0x37, 0x9d, 0x48, 0x2c, 0xba, 0xad, 0xe7, 0x93, 0xb6, 0x8e, 0xff, 0x61, 0x40, 0x73, - 0x23, 0xbc, 0xf9, 0x11, 0xc5, 0x11, 0xf2, 0x9b, 0x7d, 0x40, 0xfc, 0xe6, 0xbe, 0x1c, 0xbf, 0xb8, - 0x09, 0xb0, 0xe1, 0xb8, 0xf4, 0xaa, 0xd3, 0x67, 0xd4, 0x9f, 0x52, 0xc5, 0xfc, 0x34, 0x17, 0xbb, - 0x04, 0x42, 0x77, 0x42, 0x3e, 0xd7, 0x34, 0x3f, 0xfc, 0x20, 0xd8, 0xc8, 0x3e, 0x40, 0xb5, 0xe5, - 0x52, 0x2e, 0xca, 0x85, 0xe2, 0x8e, 0x60, 0x4f, 0x86, 0xd4, 0x44, 0x23, 0x26, 0xe6, 0xbd, 0xfd, - 0x1d, 0x75, 0xf8, 0xb3, 0xf7, 0xc9, 0x88, 0x44, 0x7b, 0xac, 0x15, 0x8c, 0x5d, 0x66, 0xbf, 0xa7, - 0xed, 0x27, 0xe1, 0x21, 0xc8, 0x56, 0x49, 0x57, 0x7e, 0x6a, 0xd2, 0xf5, 0x82, 0x3a, 0xe6, 0x4b, - 0x55, 0x9d, 0x2f, 0xc4, 0x1e, 0x50, 0x28, 0x45, 0x79, 0xc0, 0xc7, 0xc0, 0xf4, 0xe9, 0x4e, 0x18, - 0xaa, 0x51, 0x7c, 0x72, 0x84, 0x29, 0xd6, 0xf1, 0x5f, 0x0c, 0x58, 0xba, 0x46, 0x59, 0x32, 0x09, - 0x7a, 0x88, 0x54, 0x8a, 0x5f, 0x86, 0xe3, 0xda, 0xfd, 0x15, 0xf7, 0x4f, 0xa7, 0x32, 0x9f, 0x13, - 0x31, 0xff, 0xd7, 0xdd, 0x2e, 0x7d, 0x4f, 0x15, 0x94, 0xc9, 0xa4, 0xe7, 0x16, 0x54, 0xb4, 0x45, - 0x74, 0x25, 0x95, 0xee, 0x2c, 0xa7, 0xfa, 0x95, 0x3c, 0x64, 0xb7, 0x6b, 0x8a, 0x27, 0x59, 0x36, - 0xaa, 0x64, 0x36, 0x4a, 0x0d, 0x36, 0x01, 0x09, 0x75, 0x09, 0xb2, 0x7a, 0x70, 0x12, 0xd0, 0x57, - 0xa2, 0xbc, 0x27, 0x9a, 0xa3, 0x47, 0xc1, 0xf4, 0xbd, 0xbb, 0x61, 0x1e, 0xbb, 0x10, 0x1f, 0x49, - 0xbc, 0xbb, 0x44, 0x2c, 0xe1, 0xe7, 0x21, 0x47, 0xbc, 0xbb, 0xa8, 0x09, 0xe0, 0xdb, 0x6e, 0x8f, - 0xde, 0x8e, 0x2a, 0xa8, 0x2a, 0xd1, 0x20, 0x33, 0x12, 0x87, 0x35, 0x38, 0xae, 0xdf, 0x48, 0xaa, - 0x7b, 0x15, 0x8a, 0xaf, 0x8f, 0x74, 0x71, 0xd5, 0x52, 0xe2, 0x92, 0x85, 0x7a, 0x88, 0xc4, 0x6d, - 0x06, 0x62, 0x38, 0x3a, 0x03, 0x65, 0x66, 0x6f, 0xf7, 0xe9, 0xcd, 0xd8, 0xcd, 0xc5, 0x00, 0xbe, - 0xca, 0x8b, 0xbf, 0xdb, 0x5a, 0x06, 0x14, 0x03, 0xd0, 0x13, 0xb0, 0x14, 0xdf, 0xf9, 0x96, 0x4f, - 0x77, 0x9c, 0xf7, 0x84, 0x86, 0xab, 0xe4, 0x10, 0x1c, 0x9d, 0x85, 0x63, 0x31, 0x6c, 0x53, 0x64, - 0x1a, 0xa6, 0x40, 0x4d, 0x83, 0xb9, 0x6c, 0x04, 0xbb, 0x2f, 0xdd, 0x19, 0xd9, 0x7d, 0xf1, 0xf8, - 0xaa, 0x44, 0x83, 0xe0, 0xbf, 0x1a, 0x70, 0x5c, 0xaa, 0x9a, 0xd9, 0xec, 0xa1, 0xb4, 0xfa, 0xdf, - 0x1a, 0x80, 0x74, 0x0e, 0x94, 0x69, 0xfd, 0xbf, 0xde, 0x08, 0xe2, 0xa9, 0x4c, 0x45, 0xd4, 0xb4, - 0x12, 0x14, 0xf7, 0x72, 0x30, 0x14, 0x44, 0x3a, 0x24, 0x8b, 0x6b, 0x53, 0x16, 0xcd, 0x12, 0x42, - 0xd4, 0x2f, 0xaf, 0xf5, 0xb7, 0xc7, 0x8c, 0x06, 0xaa, 0xe4, 0x15, 0xb5, 0xbe, 0x00, 0x10, 0xf9, - 0xc3, 0xcf, 0xa2, 0x2e, 0x13, 0x56, 0x63, 0xc6, 0x67, 0x29, 0x10, 0x09, 0x07, 0xf8, 0xf7, 0x59, - 0x58, 0xb8, 0xed, 0xf5, 0x47, 0x71, 0x60, 0x7c, 0x98, 0x02, 0x46, 0xa2, 0x0e, 0xcf, 0x87, 0x75, - 0x38, 0x02, 0x33, 0x60, 0x74, 0x28, 0x2c, 0x2b, 0x47, 0xc4, 0x18, 0x61, 0xa8, 0x32, 0xdb, 0xef, - 0x51, 0x26, 0xab, 0x9b, 0x7a, 0x41, 0xa4, 0x9d, 0x09, 0x18, 0x5a, 0x81, 0x8a, 0xdd, 0xeb, 0xf9, - 0xb4, 0x67, 0x33, 0xda, 0x1e, 0xd7, 0x8b, 0xe2, 0x30, 0x1d, 0x84, 0xdf, 0x82, 0xc5, 0x50, 0x58, - 0x4a, 0xa5, 0x4f, 0x41, 0xf1, 0x5d, 0x01, 0x99, 0xd2, 0x17, 0x93, 0xa8, 0xca, 0x8d, 0x85, 0x68, - 0xc9, 0x3e, 0x7b, 0x78, 0x67, 0x7c, 0x03, 0x0a, 0x12, 0x1d, 0x9d, 0xd1, 0x6b, 0x14, 0xd9, 0xa4, - 0xe1, 0x73, 0x55, 0x70, 0x60, 0x28, 0x48, 0x42, 0x4a, 0xf1, 0xc2, 0x36, 0x24, 0x84, 0xa8, 0x5f, - 0xfc, 0x1f, 0x03, 0x4e, 0xac, 0x53, 0x46, 0x3b, 0x8c, 0x76, 0xaf, 0x3a, 0xb4, 0xdf, 0xfd, 0x5a, - 0xcb, 0xe7, 0xa8, 0x09, 0x96, 0xd3, 0x9a, 0x60, 0xdc, 0xef, 0xf4, 0x1d, 0x97, 0x6e, 0x68, 0x5d, - 0x94, 0x18, 0xc0, 0x3d, 0xc4, 0x0e, 0xbf, 0xb8, 0x5c, 0x96, 0x1f, 0x36, 0x34, 0x48, 0xa4, 0xe1, - 0x42, 0xac, 0x61, 0xfc, 0x23, 0x03, 0x4e, 0xa6, 0xb9, 0x56, 0x4a, 0x6a, 0x41, 0x41, 0x6c, 0x9e, - 0xd2, 0x7f, 0x4d, 0xec, 0x20, 0x0a, 0x0d, 0x5d, 0x4a, 0x9c, 0x2f, 0x3e, 0x88, 0xb4, 0xeb, 0x07, - 0x13, 0xab, 0x16, 0x43, 0xb5, 0x12, 0x5f, 0xc3, 0xc5, 0xbf, 0xe2, 0x85, 0xb0, 0x4e, 0x53, 0xe8, - 0x9b, 0xdb, 0x97, 0xf2, 0xbd, 0x72, 0x82, 0xbe, 0x01, 0x26, 0x1b, 0x0f, 0x95, 0xcb, 0x6d, 0x9f, - 0xf8, 0x7c, 0x62, 0x1d, 0x4f, 0x6c, 0xdb, 0x1a, 0x0f, 0x29, 0x11, 0x28, 0xdc, 0x2c, 0x3b, 0xb6, - 0xdf, 0x75, 0x5c, 0xbb, 0xef, 0x30, 0x29, 0x46, 0x93, 0xe8, 0x20, 0xd1, 0x89, 0xd8, 0xa3, 0xac, - 0x23, 0x93, 0xea, 0xaa, 0xea, 0x44, 0x08, 0x48, 0xa2, 0x13, 0x21, 0x20, 0xf8, 0xd7, 0x9a, 0x79, - 0x48, 0xcb, 0x3f, 0xa2, 0x79, 0x18, 0x47, 0x36, 0x0f, 0xe3, 0x3e, 0xe6, 0x81, 0xbf, 0x17, 0xeb, - 0x32, 0xbc, 0xa2, 0xd2, 0xe5, 0x8b, 0xb0, 0xd8, 0x4d, 0xac, 0xcc, 0xd6, 0xa9, 0xec, 0xb2, 0xa6, - 0xd0, 0xf1, 0xb5, 0x58, 0x41, 0x02, 0x32, 0x43, 0x41, 0x29, 0xa9, 0x67, 0x0f, 0x49, 0xfd, 0x89, - 0xc7, 0xa0, 0x1c, 0x7d, 0x8b, 0x42, 0x15, 0x28, 0x5e, 0x7d, 0x8d, 0xbc, 0x79, 0x85, 0xac, 0x2f, - 0x65, 0x50, 0x15, 0x4a, 0xed, 0x2b, 0x6b, 0xaf, 0x88, 0x99, 0x71, 0xf1, 0x83, 0x42, 0x18, 0xc0, - 0x7d, 0xf4, 0x6d, 0xc8, 0xcb, 0xa8, 0x7c, 0x32, 0xbe, 0xae, 0xfe, 0xc9, 0xa7, 0x71, 0xea, 0x10, - 0x5c, 0xf2, 0x8d, 0x33, 0x4f, 0x19, 0xe8, 0x26, 0x54, 0x04, 0x50, 0x35, 0x68, 0xcf, 0xa4, 0xfb, - 0xa4, 0x09, 0x4a, 0x8f, 0xcc, 0x58, 0xd5, 0xe8, 0x5d, 0x86, 0xbc, 0x14, 0xc1, 0xc9, 0x54, 0xf2, - 0x34, 0xe5, 0x36, 0x89, 0x96, 0x35, 0xce, 0xa0, 0xe7, 0xc0, 0xdc, 0xb2, 0x9d, 0x3e, 0xd2, 0x72, - 0x37, 0xad, 0xaf, 0xda, 0x38, 0x99, 0x06, 0x6b, 0xc7, 0xbe, 0x10, 0xb5, 0x87, 0x4f, 0xa5, 0x7b, - 0x54, 0xe1, 0xf6, 0xfa, 0xe1, 0x85, 0xe8, 0xe4, 0xd7, 0x64, 0x13, 0x33, 0xec, 0x94, 0xa0, 0x47, - 0x92, 0x47, 0xa5, 0x1a, 0x2b, 0x8d, 0xe6, 0xac, 0xe5, 0x88, 0xe0, 0x06, 0x54, 0xb4, 0x2e, 0x85, - 0x2e, 0xd6, 0xc3, 0x2d, 0x16, 0x5d, 0xac, 0x53, 0x5a, 0x1b, 0x38, 0x83, 0xae, 0x41, 0x89, 0x67, - 0xbc, 0xe2, 0x6b, 0xc6, 0xe9, 0x74, 0x62, 0xab, 0x25, 0x34, 0x8d, 0x33, 0xd3, 0x17, 0x23, 0x42, - 0xdf, 0x85, 0xf2, 0x35, 0xca, 0x54, 0x54, 0x38, 0x95, 0x0e, 0x2b, 0x53, 0x24, 0x95, 0x0c, 0x4d, - 0x38, 0x83, 0xde, 0x12, 0xc9, 0x77, 0xd2, 0x29, 0x22, 0x6b, 0x86, 0xf3, 0x8b, 0xee, 0xb5, 0x32, - 0x1b, 0x21, 0xa2, 0xfc, 0x66, 0x82, 0xb2, 0x8a, 0x9f, 0xd6, 0x8c, 0x27, 0x18, 0x51, 0xb6, 0xee, - 0xf3, 0x3f, 0x05, 0x38, 0x73, 0xf1, 0xed, 0xf0, 0xb3, 0xfa, 0xba, 0xcd, 0x6c, 0xf4, 0x1a, 0x2c, - 0x0a, 0x59, 0x46, 0xdf, 0xdd, 0x13, 0x36, 0x7f, 0xe8, 0x23, 0x7f, 0xc2, 0xe6, 0x0f, 0x7f, 0xec, - 0xc7, 0x99, 0xf6, 0xdb, 0x1f, 0xdd, 0x6b, 0x66, 0x3e, 0xbe, 0xd7, 0xcc, 0x7c, 0x76, 0xaf, 0x69, - 0xfc, 0x70, 0xbf, 0x69, 0xfc, 0x6e, 0xbf, 0x69, 0x7c, 0xb8, 0xdf, 0x34, 0x3e, 0xda, 0x6f, 0x1a, - 0xff, 0xda, 0x6f, 0x1a, 0xff, 0xde, 0x6f, 0x66, 0x3e, 0xdb, 0x6f, 0x1a, 0xef, 0x7f, 0xda, 0xcc, - 0x7c, 0xf4, 0x69, 0x33, 0xf3, 0xf1, 0xa7, 0xcd, 0xcc, 0xf7, 0x1f, 0xbf, 0x7f, 0xa1, 0x29, 0x1d, - 0x5d, 0x41, 0xfc, 0x3c, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf0, 0xfa, 0x1a, 0x95, 0xfc, - 0x21, 0x00, 0x00, + // 2658 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x1a, 0x4d, 0x8c, 0x5b, 0x47, + 0xd9, 0xcf, 0x7e, 0xf6, 0xda, 0x9f, 0xbd, 0x9b, 0xcd, 0xac, 0x93, 0x58, 0x9b, 0xd4, 0x6f, 0x3b, + 0x82, 0x36, 0x34, 0xe9, 0xba, 0x49, 0x69, 0x49, 0x53, 0x4a, 0x89, 0x77, 0x9b, 0x6d, 0xd2, 0x6d, + 0x9a, 0xce, 0xa6, 0x69, 0x41, 0x54, 0xd5, 0x8b, 0x3d, 0xeb, 0x7d, 0x8a, 0xfd, 0x9e, 0xf3, 0xde, + 0xb8, 0xe9, 0xde, 0x90, 0x38, 0x23, 0x2a, 0x71, 0x00, 0x2e, 0x08, 0x24, 0x24, 0x10, 0xa8, 0x17, + 0xc4, 0x11, 0xc1, 0x85, 0x43, 0xb9, 0x95, 0x5b, 0xd5, 0x83, 0xa1, 0xdb, 0x0b, 0xda, 0x53, 0x25, + 0x24, 0x0e, 0x3d, 0xa1, 0xf9, 0x7b, 0x6f, 0xde, 0x5b, 0x9b, 0xd4, 0xdb, 0xa0, 0x92, 0x8b, 0x3d, + 0xf3, 0xcd, 0x37, 0xdf, 0xcc, 0xf7, 0x33, 0xdf, 0x9f, 0x0d, 0x27, 0x87, 0xb7, 0x7b, 0xad, 0x7e, + 0xd0, 0x1b, 0x86, 0x01, 0x0b, 0xe2, 0xc1, 0xaa, 0xf8, 0x44, 0x65, 0x3d, 0x5f, 0xae, 0xf7, 0x82, + 0x5e, 0x20, 0x71, 0xf8, 0x48, 0xae, 0x2f, 0x3b, 0xbd, 0x20, 0xe8, 0xf5, 0x69, 0x4b, 0xcc, 0x6e, + 0x8d, 0xb6, 0x5b, 0xcc, 0x1b, 0xd0, 0x88, 0xb9, 0x83, 0xa1, 0x42, 0x58, 0x51, 0xd4, 0xef, 0xf4, + 0x07, 0x41, 0x97, 0xf6, 0x5b, 0x11, 0x73, 0x59, 0x24, 0x3f, 0x15, 0xc6, 0x12, 0xc7, 0x18, 0x8e, + 0xa2, 0x1d, 0xf1, 0x21, 0x81, 0xf8, 0x0f, 0x16, 0x1c, 0xdb, 0x74, 0x6f, 0xd1, 0xfe, 0x8d, 0xe0, + 0xa6, 0xdb, 0x1f, 0xd1, 0x88, 0xd0, 0x68, 0x18, 0xf8, 0x11, 0x45, 0x6b, 0x50, 0xea, 0xf3, 0x85, + 0xa8, 0x61, 0xad, 0x14, 0x4e, 0x57, 0xcf, 0x9f, 0x59, 0x8d, 0xaf, 0x3c, 0x71, 0x83, 0x84, 0x46, + 0x2f, 0xf8, 0x2c, 0xdc, 0x25, 0x6a, 0xeb, 0xf2, 0x4d, 0xa8, 0x1a, 0x60, 0xb4, 0x08, 0x85, 0xdb, + 0x74, 0xb7, 0x61, 0xad, 0x58, 0xa7, 0x2b, 0x84, 0x0f, 0xd1, 0x39, 0x28, 0xbe, 0xcd, 0xc9, 0x34, + 0xf2, 0x2b, 0xd6, 0xe9, 0xea, 0xf9, 0x93, 0xc9, 0x21, 0xaf, 0xf9, 0xde, 0x9d, 0x11, 0x15, 0xbb, + 0xd5, 0x41, 0x12, 0xf3, 0x62, 0xfe, 0x82, 0x85, 0xcf, 0xc0, 0xd1, 0x03, 0xeb, 0xe8, 0x38, 0x94, + 0x04, 0x86, 0xbc, 0x71, 0x85, 0xa8, 0x19, 0xae, 0x03, 0xda, 0x62, 0x21, 0x75, 0x07, 0xc4, 0x65, + 0xfc, 0xbe, 0x77, 0x46, 0x34, 0x62, 0xf8, 0x65, 0x58, 0x4a, 0x41, 0x15, 0xdb, 0x4f, 0x43, 0x35, + 0x4a, 0xc0, 0x8a, 0xf7, 0x7a, 0x72, 0xad, 0x64, 0x0f, 0x31, 0x11, 0xf1, 0xcf, 0x2d, 0x80, 0x64, + 0x0d, 0x35, 0x01, 0xe4, 0xea, 0x8b, 0x6e, 0xb4, 0x23, 0x18, 0xb6, 0x89, 0x01, 0x41, 0x67, 0xe1, + 0x68, 0x32, 0xbb, 0x16, 0x6c, 0xed, 0xb8, 0x61, 0x57, 0xc8, 0xc0, 0x26, 0x07, 0x17, 0x10, 0x02, + 0x3b, 0x74, 0x19, 0x6d, 0x14, 0x56, 0xac, 0xd3, 0x05, 0x22, 0xc6, 0x9c, 0x5b, 0x46, 0x7d, 0xd7, + 0x67, 0x0d, 0x5b, 0x88, 0x53, 0xcd, 0x38, 0x9c, 0xeb, 0x97, 0x46, 0x8d, 0xe2, 0x8a, 0x75, 0x7a, + 0x9e, 0xa8, 0x19, 0xfe, 0x77, 0x01, 0x6a, 0xaf, 0x8e, 0x68, 0xb8, 0xab, 0x04, 0x80, 0x9a, 0x50, + 0x8e, 0x68, 0x9f, 0x76, 0x58, 0x10, 0x4a, 0x8d, 0xb4, 0xf3, 0x0d, 0x8b, 0xc4, 0x30, 0x54, 0x87, + 0x62, 0xdf, 0x1b, 0x78, 0x4c, 0x5c, 0x6b, 0x9e, 0xc8, 0x09, 0xba, 0x08, 0xc5, 0x88, 0xb9, 0x21, + 0x13, 0x77, 0xa9, 0x9e, 0x5f, 0x5e, 0x95, 0x86, 0xb9, 0xaa, 0x0d, 0x73, 0xf5, 0x86, 0x36, 0xcc, + 0x76, 0xf9, 0xfd, 0xb1, 0x93, 0x7b, 0xf7, 0xef, 0x8e, 0x45, 0xe4, 0x16, 0xf4, 0x34, 0x14, 0xa8, + 0xdf, 0x15, 0xf7, 0xfd, 0xbc, 0x3b, 0xf9, 0x06, 0x74, 0x0e, 0x2a, 0x5d, 0x2f, 0xa4, 0x1d, 0xe6, + 0x05, 0xbe, 0xe0, 0x6a, 0xe1, 0xfc, 0x52, 0xa2, 0x91, 0x75, 0xbd, 0x44, 0x12, 0x2c, 0x74, 0x16, + 0x4a, 0x11, 0x17, 0x5d, 0xd4, 0x98, 0xe3, 0xb6, 0xd0, 0xae, 0xef, 0x8f, 0x9d, 0x45, 0x09, 0x39, + 0x1b, 0x0c, 0x3c, 0x46, 0x07, 0x43, 0xb6, 0x4b, 0x14, 0x0e, 0x7a, 0x0c, 0xe6, 0xba, 0xb4, 0x4f, + 0xb9, 0xc2, 0xcb, 0x42, 0xe1, 0x8b, 0x06, 0x79, 0xb1, 0x40, 0x34, 0x02, 0x7a, 0x13, 0xec, 0x61, + 0xdf, 0xf5, 0x1b, 0x15, 0xc1, 0xc5, 0x42, 0x82, 0x78, 0xbd, 0xef, 0xfa, 0xed, 0x67, 0x3e, 0x1a, + 0x3b, 0x4f, 0xf5, 0x3c, 0xb6, 0x33, 0xba, 0xb5, 0xda, 0x09, 0x06, 0xad, 0x5e, 0xe8, 0x6e, 0xbb, + 0xbe, 0xdb, 0xea, 0x07, 0xb7, 0xbd, 0xd6, 0xdb, 0x4f, 0xb6, 0xf8, 0x1b, 0xbc, 0x33, 0xa2, 0xa1, + 0x47, 0xc3, 0x16, 0x27, 0xb3, 0x2a, 0x54, 0xc2, 0xb7, 0x12, 0x41, 0x16, 0x5d, 0xe5, 0xf6, 0x17, + 0x84, 0x74, 0x6d, 0x67, 0xe4, 0xdf, 0x8e, 0x1a, 0x20, 0x4e, 0x39, 0x91, 0x9c, 0x22, 0xe0, 0x84, + 0x6e, 0x6f, 0x84, 0xc1, 0x68, 0xd8, 0x3e, 0xb2, 0x3f, 0x76, 0x4c, 0x7c, 0x62, 0x4e, 0xae, 0xda, + 0xe5, 0xd2, 0xe2, 0x1c, 0x7e, 0xaf, 0x00, 0x68, 0xcb, 0x1d, 0x0c, 0xfb, 0x74, 0x26, 0xf5, 0xc7, + 0x8a, 0xce, 0x1f, 0x5a, 0xd1, 0x85, 0x59, 0x15, 0x9d, 0x68, 0xcd, 0x9e, 0x4d, 0x6b, 0xc5, 0xcf, + 0xab, 0xb5, 0xd2, 0xff, 0xbd, 0xd6, 0x70, 0x03, 0x6c, 0x4e, 0x99, 0x3b, 0xcb, 0xd0, 0xbd, 0x2b, + 0x74, 0x53, 0x23, 0x7c, 0x88, 0x37, 0xa1, 0x24, 0xf9, 0x42, 0xcb, 0x59, 0xe5, 0xa5, 0xdf, 0x6d, + 0xa2, 0xb8, 0x82, 0x56, 0xc9, 0x62, 0xa2, 0x92, 0x82, 0x10, 0x36, 0xfe, 0xa3, 0x05, 0xf3, 0xca, + 0x22, 0x94, 0xef, 0xbb, 0x05, 0x73, 0xd2, 0xf7, 0x68, 0xbf, 0x77, 0x22, 0xeb, 0xf7, 0x2e, 0x75, + 0xdd, 0x21, 0xa3, 0x61, 0xbb, 0xf5, 0xfe, 0xd8, 0xb1, 0x3e, 0x1a, 0x3b, 0x8f, 0x4e, 0x13, 0x9a, + 0x8e, 0x35, 0xda, 0x5f, 0x6a, 0xc2, 0xe8, 0x8c, 0xb8, 0x1d, 0x8b, 0x94, 0x59, 0x1d, 0x59, 0x95, + 0x21, 0xea, 0x8a, 0xdf, 0xa3, 0x11, 0xa7, 0x6c, 0x73, 0x8b, 0x20, 0x12, 0x87, 0xb3, 0x79, 0xd7, + 0x0d, 0x7d, 0xcf, 0xef, 0x45, 0x8d, 0x82, 0xf0, 0xe9, 0xf1, 0x1c, 0xff, 0xd4, 0x82, 0xa5, 0x94, + 0x59, 0x2b, 0x26, 0x2e, 0x40, 0x29, 0xe2, 0x9a, 0xd2, 0x3c, 0x18, 0x46, 0xb1, 0x25, 0xe0, 0xed, + 0x05, 0x75, 0xf9, 0x92, 0x9c, 0x13, 0x85, 0x7f, 0xff, 0xae, 0xf6, 0x17, 0x0b, 0x6a, 0x22, 0x30, + 0xe9, 0xb7, 0x86, 0xc0, 0xf6, 0xdd, 0x01, 0x55, 0xaa, 0x12, 0x63, 0x23, 0x5a, 0xf1, 0xe3, 0xca, + 0x3a, 0x5a, 0xcd, 0xea, 0x60, 0xad, 0x43, 0x3b, 0x58, 0x2b, 0x79, 0x77, 0x75, 0x28, 0x72, 0xf3, + 0xde, 0x15, 0xce, 0xb5, 0x42, 0xe4, 0x04, 0x3f, 0x0a, 0xf3, 0x8a, 0x0b, 0x25, 0xda, 0x69, 0x01, + 0x76, 0x00, 0x25, 0xa9, 0x09, 0xf4, 0x15, 0xa8, 0xc4, 0x89, 0x89, 0xe0, 0xb6, 0xd0, 0x2e, 0xed, + 0x8f, 0x9d, 0x3c, 0x8b, 0x48, 0xb2, 0x80, 0x1c, 0x33, 0xe8, 0x5b, 0xed, 0xca, 0xfe, 0xd8, 0x91, + 0x00, 0x15, 0xe2, 0xd1, 0x29, 0xb0, 0x77, 0x78, 0xdc, 0xe4, 0x22, 0xb0, 0xdb, 0xe5, 0xfd, 0xb1, + 0x23, 0xe6, 0x44, 0x7c, 0xe2, 0x0d, 0xa8, 0x6d, 0xd2, 0x9e, 0xdb, 0xd9, 0x55, 0x87, 0xd6, 0x35, + 0x39, 0x7e, 0xa0, 0xa5, 0x69, 0x3c, 0x0c, 0xb5, 0xf8, 0xc4, 0xb7, 0x06, 0x91, 0x7a, 0x0d, 0xd5, + 0x18, 0xf6, 0x72, 0x84, 0x7f, 0x66, 0x81, 0xb2, 0x01, 0x84, 0x8d, 0x6c, 0x87, 0xfb, 0x42, 0xd8, + 0x1f, 0x3b, 0x0a, 0xa2, 0x93, 0x19, 0xf4, 0x2c, 0xcc, 0x45, 0xe2, 0x44, 0x4e, 0x2c, 0x6b, 0x5a, + 0x62, 0xa1, 0x7d, 0x84, 0x9b, 0xc8, 0xfe, 0xd8, 0xd1, 0x88, 0x44, 0x0f, 0xd0, 0x6a, 0x2a, 0x21, + 0x90, 0x8c, 0x2d, 0xec, 0x8f, 0x1d, 0x03, 0x6a, 0x26, 0x08, 0xf8, 0x33, 0x0b, 0xaa, 0x37, 0x5c, + 0x2f, 0x36, 0xa1, 0x86, 0x56, 0x51, 0xe2, 0xab, 0x25, 0x80, 0x5b, 0x62, 0x97, 0xf6, 0xdd, 0xdd, + 0xcb, 0x41, 0x28, 0xe8, 0xce, 0x93, 0x78, 0x9e, 0xc4, 0x70, 0x7b, 0x62, 0x0c, 0x2f, 0xce, 0xee, + 0xda, 0xff, 0xb7, 0x8e, 0xf4, 0xaa, 0x5d, 0xce, 0x2f, 0x16, 0xf0, 0x7b, 0x16, 0xd4, 0x24, 0xf3, + 0xca, 0xf2, 0xbe, 0x07, 0x25, 0x29, 0x1b, 0xc1, 0xfe, 0x7f, 0x71, 0x4c, 0x67, 0x66, 0x71, 0x4a, + 0x8a, 0x26, 0x7a, 0x1e, 0x16, 0xba, 0x61, 0x30, 0x1c, 0xd2, 0xee, 0x96, 0x72, 0x7f, 0xf9, 0xac, + 0xfb, 0x5b, 0x37, 0xd7, 0x49, 0x06, 0x1d, 0xff, 0xd5, 0x82, 0x79, 0xe5, 0x4c, 0x94, 0xba, 0x62, + 0x11, 0x5b, 0x87, 0x8e, 0x9e, 0xf9, 0x59, 0xa3, 0xe7, 0x71, 0x28, 0xf5, 0x78, 0x7c, 0xd1, 0x0e, + 0x49, 0xcd, 0x66, 0x8b, 0xaa, 0xf8, 0x2a, 0x2c, 0x68, 0x56, 0xa6, 0x78, 0xd4, 0xe5, 0xac, 0x47, + 0xbd, 0xd2, 0xa5, 0x3e, 0xf3, 0xb6, 0xbd, 0xd8, 0x47, 0x2a, 0x7c, 0xfc, 0x23, 0x0b, 0x16, 0xb3, + 0x28, 0x68, 0x3d, 0x53, 0x58, 0x3c, 0x32, 0x9d, 0x9c, 0x59, 0x53, 0x68, 0xd2, 0xaa, 0xb2, 0x78, + 0xea, 0x5e, 0x95, 0x45, 0xdd, 0x74, 0x32, 0x15, 0xe5, 0x15, 0xf0, 0x4f, 0x2c, 0x98, 0x4f, 0xe9, + 0x12, 0x5d, 0x00, 0x7b, 0x3b, 0x0c, 0x06, 0x33, 0x29, 0x4a, 0xec, 0x40, 0x5f, 0x87, 0x3c, 0x0b, + 0x66, 0x52, 0x53, 0x9e, 0x05, 0x5c, 0x4b, 0x8a, 0xfd, 0x82, 0xcc, 0xdb, 0xe5, 0x0c, 0x3f, 0x05, + 0x15, 0xc1, 0xd0, 0x75, 0xd7, 0x0b, 0x27, 0x06, 0x8c, 0xc9, 0x0c, 0x3d, 0x0b, 0x47, 0xa4, 0x33, + 0x9c, 0xbc, 0xb9, 0x36, 0x69, 0x73, 0x4d, 0x6f, 0x3e, 0x09, 0x45, 0x91, 0x74, 0xf0, 0x2d, 0x5d, + 0x97, 0xb9, 0x7a, 0x0b, 0x1f, 0xe3, 0x63, 0xb0, 0xc4, 0xdf, 0x20, 0x0d, 0xa3, 0xb5, 0x60, 0xe4, + 0x33, 0x5d, 0x37, 0x9d, 0x85, 0x7a, 0x1a, 0xac, 0xac, 0xa4, 0x0e, 0xc5, 0x0e, 0x07, 0x08, 0x1a, + 0xf3, 0x44, 0x4e, 0xf0, 0xaf, 0x2c, 0x40, 0x1b, 0x94, 0x89, 0x53, 0xae, 0xac, 0xc7, 0xcf, 0x63, + 0x19, 0xca, 0x03, 0x97, 0x75, 0x76, 0x68, 0x18, 0xe9, 0xfc, 0x45, 0xcf, 0xbf, 0x8c, 0xc4, 0x13, + 0x9f, 0x83, 0xa5, 0xd4, 0x2d, 0x15, 0x4f, 0xcb, 0x50, 0xee, 0x28, 0x98, 0x0a, 0x79, 0xf1, 0x1c, + 0xff, 0x3e, 0x0f, 0x65, 0x9d, 0xd6, 0xa1, 0x73, 0x50, 0xdd, 0xf6, 0xfc, 0x1e, 0x0d, 0x87, 0xa1, + 0xa7, 0x44, 0x60, 0xcb, 0x34, 0xcf, 0x00, 0x13, 0x73, 0x82, 0x1e, 0x87, 0xb9, 0x51, 0x44, 0xc3, + 0xb7, 0x3c, 0xf9, 0xd2, 0x2b, 0xed, 0xfa, 0xde, 0xd8, 0x29, 0xbd, 0x16, 0xd1, 0xf0, 0xca, 0x3a, + 0x0f, 0x3e, 0x23, 0x31, 0x22, 0xf2, 0xbb, 0x8b, 0x5e, 0x52, 0x66, 0x2a, 0x12, 0xb8, 0xf6, 0x37, + 0xf8, 0xf5, 0x33, 0xae, 0x6e, 0x18, 0x06, 0x03, 0xca, 0x76, 0xe8, 0x28, 0x6a, 0x75, 0x82, 0xc1, + 0x20, 0xf0, 0x5b, 0xa2, 0x13, 0x20, 0x98, 0xe6, 0x11, 0x94, 0x6f, 0x57, 0x96, 0x7b, 0x03, 0xe6, + 0xd8, 0x4e, 0x18, 0x8c, 0x7a, 0x3b, 0x22, 0x30, 0x14, 0xda, 0x17, 0x67, 0xa7, 0xa7, 0x29, 0x10, + 0x3d, 0x40, 0x0f, 0x73, 0x69, 0xd1, 0xce, 0xed, 0x68, 0x34, 0x90, 0xb5, 0x67, 0xbb, 0xb8, 0x3f, + 0x76, 0xac, 0xc7, 0x49, 0x0c, 0xc6, 0x97, 0x60, 0x3e, 0x95, 0x0a, 0xa3, 0x27, 0xc0, 0x0e, 0xe9, + 0xb6, 0x76, 0x05, 0xe8, 0x60, 0xc6, 0x2c, 0xa3, 0x3f, 0xc7, 0x21, 0xe2, 0x13, 0xff, 0x30, 0x0f, + 0x8e, 0x51, 0xf5, 0x5f, 0x0e, 0xc2, 0x97, 0x29, 0x0b, 0xbd, 0xce, 0x35, 0x77, 0x40, 0xb5, 0x79, + 0x39, 0x50, 0x1d, 0x08, 0xe0, 0x5b, 0xc6, 0x2b, 0x82, 0x41, 0x8c, 0x87, 0x1e, 0x02, 0x10, 0xcf, + 0x4e, 0xae, 0xcb, 0x07, 0x55, 0x11, 0x10, 0xb1, 0xbc, 0x96, 0x12, 0x76, 0x6b, 0x46, 0xe1, 0x28, + 0x21, 0x5f, 0xc9, 0x0a, 0x79, 0x66, 0x3a, 0xb1, 0x64, 0xcd, 0xe7, 0x52, 0x4c, 0x3f, 0x17, 0xfc, + 0x37, 0x0b, 0x9a, 0x9b, 0xfa, 0xe6, 0x87, 0x14, 0x87, 0xe6, 0x37, 0x7f, 0x9f, 0xf8, 0x2d, 0x7c, + 0x31, 0x7e, 0x71, 0x13, 0x60, 0xd3, 0xf3, 0xe9, 0x65, 0xaf, 0xcf, 0x68, 0x38, 0xa1, 0x10, 0xfa, + 0x71, 0x21, 0xf1, 0x2a, 0x84, 0x6e, 0x6b, 0x3e, 0xd7, 0x0c, 0x57, 0x7e, 0x3f, 0xd8, 0xc8, 0xdf, + 0x47, 0xb5, 0x15, 0x32, 0x5e, 0xce, 0x87, 0xb9, 0x6d, 0xc1, 0x9e, 0x8c, 0xca, 0xa9, 0x1e, 0x53, + 0xc2, 0x7b, 0xfb, 0x5b, 0xea, 0xf0, 0xa7, 0xef, 0x91, 0x54, 0x89, 0xce, 0x5f, 0x2b, 0xda, 0xf5, + 0x99, 0xfb, 0x8e, 0xb1, 0x9f, 0xe8, 0x43, 0x90, 0xab, 0xf2, 0xb6, 0xe2, 0xc4, 0xbc, 0xed, 0x39, + 0x75, 0xcc, 0x17, 0xc9, 0xdd, 0xf0, 0x73, 0x89, 0x13, 0x15, 0x4a, 0x51, 0x4e, 0xf4, 0x91, 0x7b, + 0x3d, 0x71, 0xf5, 0xb0, 0xff, 0x64, 0xc1, 0xe2, 0x06, 0x65, 0xe9, 0x3c, 0xea, 0x01, 0x52, 0x29, + 0x7e, 0x11, 0x8e, 0x1a, 0xf7, 0x57, 0xdc, 0x3f, 0x99, 0x49, 0x9e, 0x8e, 0x25, 0xfc, 0x5f, 0xf1, + 0xbb, 0xf4, 0x1d, 0x55, 0x93, 0xa6, 0xf3, 0xa6, 0xeb, 0x50, 0x35, 0x16, 0xd1, 0xa5, 0x4c, 0xc6, + 0xb4, 0x94, 0x69, 0xc5, 0xf2, 0xa8, 0xdf, 0xae, 0x2b, 0x9e, 0x64, 0xe5, 0xa9, 0xf2, 0xe1, 0x38, + 0xbb, 0xd8, 0x02, 0x24, 0xd4, 0x25, 0xc8, 0x9a, 0xf1, 0x4d, 0x40, 0x5f, 0x8a, 0x53, 0xa7, 0x78, + 0x8e, 0x1e, 0x06, 0x3b, 0x0c, 0xee, 0xea, 0x54, 0x78, 0x3e, 0x39, 0x92, 0x04, 0x77, 0x89, 0x58, + 0xc2, 0xcf, 0x42, 0x81, 0x04, 0x77, 0x51, 0x13, 0x20, 0x74, 0xfd, 0x1e, 0xbd, 0x19, 0x17, 0x61, + 0x35, 0x62, 0x40, 0xa6, 0xe4, 0x1e, 0x6b, 0x70, 0xd4, 0xbc, 0x91, 0x54, 0xf7, 0x2a, 0xcc, 0xbd, + 0x3a, 0x32, 0xc5, 0x55, 0xcf, 0x88, 0x4b, 0xd6, 0xfa, 0x1a, 0x89, 0xdb, 0x0c, 0x24, 0x70, 0x74, + 0x0a, 0x2a, 0xcc, 0xbd, 0xd5, 0xa7, 0xd7, 0x12, 0x37, 0x97, 0x00, 0xf8, 0x2a, 0xaf, 0x1f, 0x6f, + 0x1a, 0x49, 0x54, 0x02, 0x40, 0x8f, 0xc1, 0x62, 0x72, 0xe7, 0xeb, 0x21, 0xdd, 0xf6, 0xde, 0x11, + 0x1a, 0xae, 0x91, 0x03, 0x70, 0x74, 0x1a, 0x8e, 0x24, 0xb0, 0x2d, 0x91, 0xac, 0xd8, 0x02, 0x35, + 0x0b, 0xe6, 0xb2, 0x11, 0xec, 0xbe, 0x70, 0x67, 0xe4, 0xf6, 0xc5, 0xe3, 0xab, 0x11, 0x03, 0x82, + 0xff, 0x6c, 0xc1, 0x51, 0xa9, 0x6a, 0xe6, 0xb2, 0x07, 0xd2, 0xea, 0x7f, 0x6d, 0x01, 0x32, 0x39, + 0x50, 0xa6, 0xf5, 0x55, 0xb3, 0x97, 0xc4, 0xb3, 0xa1, 0xaa, 0x28, 0x8b, 0x25, 0x28, 0x69, 0x07, + 0x61, 0x28, 0x75, 0x64, 0xcf, 0x4c, 0x34, 0xbf, 0x65, 0xdd, 0x2d, 0x21, 0x44, 0x7d, 0x23, 0x07, + 0x8a, 0xb7, 0x76, 0x19, 0x8d, 0x54, 0xd5, 0x2c, 0xda, 0x05, 0x02, 0x40, 0xe4, 0x17, 0x3f, 0x8b, + 0xfa, 0x4c, 0x58, 0x8d, 0x9d, 0x9c, 0xa5, 0x40, 0x44, 0x0f, 0xf0, 0xef, 0xf2, 0x30, 0x7f, 0x33, + 0xe8, 0x8f, 0x92, 0xc0, 0xf8, 0x20, 0x05, 0x8c, 0x54, 0x29, 0x5f, 0xd4, 0xa5, 0x3c, 0x02, 0x3b, + 0x62, 0x74, 0x28, 0x2c, 0xab, 0x40, 0xc4, 0x18, 0x61, 0xa8, 0x31, 0x37, 0xec, 0x51, 0x26, 0x0b, + 0xa4, 0x46, 0x49, 0x64, 0xae, 0x29, 0x18, 0x5a, 0x81, 0xaa, 0xdb, 0xeb, 0x85, 0xb4, 0xe7, 0x32, + 0xda, 0xde, 0x6d, 0xcc, 0x89, 0xc3, 0x4c, 0x10, 0x7e, 0x03, 0x16, 0xb4, 0xb0, 0x94, 0x4a, 0x9f, + 0x80, 0xb9, 0xb7, 0x05, 0x64, 0x42, 0x6b, 0x4d, 0xa2, 0x2a, 0x37, 0xa6, 0xd1, 0xd2, 0x3f, 0x21, + 0xe8, 0x3b, 0xe3, 0xab, 0x50, 0x92, 0xe8, 0xe8, 0x94, 0x59, 0xe6, 0xc8, 0x4c, 0x8f, 0xcf, 0x55, + 0xcd, 0x82, 0xa1, 0x24, 0x09, 0x29, 0xc5, 0x0b, 0xdb, 0x90, 0x10, 0xa2, 0xbe, 0xf1, 0xbf, 0x2c, + 0x38, 0xb6, 0x4e, 0x19, 0xed, 0x30, 0xda, 0xbd, 0xec, 0xd1, 0x7e, 0xf7, 0x4b, 0xad, 0xc0, 0xe3, + 0x3e, 0x5a, 0xc1, 0xe8, 0xa3, 0x71, 0xbf, 0xd3, 0xf7, 0x7c, 0xba, 0x69, 0x34, 0x62, 0x12, 0x00, + 0xf7, 0x10, 0xdb, 0xfc, 0xe2, 0x72, 0x59, 0xfe, 0x66, 0x63, 0x40, 0x62, 0x0d, 0x97, 0x12, 0x0d, + 0xe3, 0x1f, 0x58, 0x70, 0x3c, 0xcb, 0xb5, 0x52, 0x52, 0x0b, 0x4a, 0x62, 0xf3, 0x84, 0x16, 0x6e, + 0x6a, 0x07, 0x51, 0x68, 0xe8, 0x42, 0xea, 0x7c, 0xf1, 0x5b, 0x4f, 0xbb, 0xb1, 0x3f, 0x76, 0xea, + 0x09, 0xd4, 0xe8, 0x12, 0x18, 0xb8, 0xf8, 0x17, 0xbc, 0x96, 0x36, 0x69, 0x0a, 0x7d, 0x73, 0xfb, + 0x52, 0xbe, 0x57, 0x4e, 0xd0, 0xd7, 0xc0, 0x66, 0xbb, 0x43, 0xe5, 0x72, 0xdb, 0xc7, 0x3e, 0x1b, + 0x3b, 0x47, 0x53, 0xdb, 0x6e, 0xec, 0x0e, 0x29, 0x11, 0x28, 0xdc, 0x2c, 0x3b, 0x6e, 0xd8, 0xf5, + 0x7c, 0xb7, 0xef, 0x31, 0x29, 0x46, 0x9b, 0x98, 0x20, 0xd1, 0xcc, 0xb8, 0x4d, 0x59, 0x47, 0x26, + 0xd5, 0x35, 0xd5, 0xcc, 0x10, 0x90, 0x54, 0x33, 0x43, 0x40, 0xf0, 0x2f, 0x0d, 0xf3, 0x90, 0x96, + 0x7f, 0x48, 0xf3, 0xb0, 0x0e, 0x6d, 0x1e, 0xd6, 0x3d, 0xcc, 0x03, 0x7f, 0x27, 0xd1, 0xa5, 0xbe, + 0xa2, 0xd2, 0xe5, 0xf3, 0xb0, 0xd0, 0x4d, 0xad, 0x4c, 0xd7, 0xa9, 0x6c, 0xd4, 0x66, 0xd0, 0xf1, + 0x46, 0xa2, 0x20, 0x01, 0x99, 0xa2, 0xa0, 0x8c, 0xd4, 0xf3, 0x07, 0xa4, 0xfe, 0xd8, 0x23, 0x50, + 0x89, 0x7f, 0x66, 0x43, 0x55, 0x98, 0xbb, 0xfc, 0x0a, 0x79, 0xfd, 0x12, 0x59, 0x5f, 0xcc, 0xa1, + 0x1a, 0x94, 0xdb, 0x97, 0xd6, 0x5e, 0x12, 0x33, 0xeb, 0xfc, 0x6f, 0x4b, 0x3a, 0x80, 0x87, 0xe8, + 0x9b, 0x50, 0x94, 0x51, 0xf9, 0x78, 0x72, 0x5d, 0xf3, 0x17, 0xa8, 0xe5, 0x13, 0x07, 0xe0, 0x92, + 0x6f, 0x9c, 0x7b, 0xc2, 0x42, 0xd7, 0xa0, 0x2a, 0x80, 0xaa, 0xc7, 0x7b, 0x2a, 0xdb, 0x6a, 0x4d, + 0x51, 0x7a, 0x68, 0xca, 0xaa, 0x41, 0xef, 0x22, 0x14, 0xa5, 0x08, 0x8e, 0x67, 0x92, 0xa7, 0x09, + 0xb7, 0x49, 0x75, 0xbd, 0x71, 0x0e, 0x3d, 0x03, 0xf6, 0x0d, 0xd7, 0xeb, 0x23, 0x23, 0x77, 0x33, + 0x5a, 0xb3, 0xcb, 0xc7, 0xb3, 0x60, 0xe3, 0xd8, 0xe7, 0xe2, 0x0e, 0xf3, 0x89, 0x6c, 0x9b, 0x4b, + 0x6f, 0x6f, 0x1c, 0x5c, 0x88, 0x4f, 0x7e, 0x45, 0xf6, 0x41, 0x75, 0xb3, 0x05, 0x3d, 0x94, 0x3e, + 0x2a, 0xd3, 0x9b, 0x59, 0x6e, 0x4e, 0x5b, 0x8e, 0x09, 0x6e, 0x42, 0xd5, 0x68, 0x74, 0x98, 0x62, + 0x3d, 0xd8, 0xa5, 0x31, 0xc5, 0x3a, 0xa1, 0x3b, 0x82, 0x73, 0x68, 0x03, 0xca, 0x3c, 0xe3, 0x15, + 0x3f, 0x88, 0x9c, 0xcc, 0x26, 0xb6, 0x46, 0x42, 0xb3, 0x7c, 0x6a, 0xf2, 0x62, 0x4c, 0xe8, 0xdb, + 0x50, 0xd9, 0xa0, 0x4c, 0x45, 0x85, 0x13, 0xd9, 0xb0, 0x32, 0x41, 0x52, 0xe9, 0xd0, 0x84, 0x73, + 0xe8, 0x0d, 0x91, 0x7c, 0xa7, 0x9d, 0x22, 0x72, 0xa6, 0x38, 0xbf, 0xf8, 0x5e, 0x2b, 0xd3, 0x11, + 0x62, 0xca, 0xaf, 0xa7, 0x28, 0xab, 0xf8, 0xe9, 0x4c, 0x79, 0x82, 0x31, 0x65, 0xe7, 0x1e, 0x7f, + 0x97, 0xc0, 0xb9, 0xf3, 0x6f, 0xea, 0x7f, 0x0c, 0xac, 0xbb, 0xcc, 0x45, 0xaf, 0xc0, 0x82, 0x90, + 0x65, 0xfc, 0x97, 0x82, 0x94, 0xcd, 0x1f, 0xf8, 0xff, 0x42, 0xca, 0xe6, 0x0f, 0xfe, 0x8f, 0x01, + 0xe7, 0xda, 0x6f, 0x7e, 0xf0, 0x71, 0x33, 0xf7, 0xe1, 0xc7, 0xcd, 0xdc, 0xa7, 0x1f, 0x37, 0xad, + 0xef, 0xef, 0x35, 0xad, 0xdf, 0xec, 0x35, 0xad, 0xf7, 0xf7, 0x9a, 0xd6, 0x07, 0x7b, 0x4d, 0xeb, + 0x1f, 0x7b, 0x4d, 0xeb, 0x9f, 0x7b, 0xcd, 0xdc, 0xa7, 0x7b, 0x4d, 0xeb, 0xdd, 0x4f, 0x9a, 0xb9, + 0x0f, 0x3e, 0x69, 0xe6, 0x3e, 0xfc, 0xa4, 0x99, 0xfb, 0xee, 0xa3, 0xf7, 0x2e, 0x34, 0xa5, 0xa3, + 0x2b, 0x89, 0xaf, 0x27, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0x2d, 0xd2, 0x02, 0x97, 0xd7, 0x22, + 0x00, 0x00, } func (x Direction) String() string { @@ -3408,6 +3475,9 @@ func (this *QueryRequest) Equal(that interface{}) bool { } else if !this.Plan.Equal(*that1.Plan) { return false } + if !this.StoreChunks.Equal(that1.StoreChunks) { + return false + } return true } func (this *SampleQueryRequest) Equal(that interface{}) bool { @@ -3461,6 +3531,9 @@ func (this *SampleQueryRequest) Equal(that interface{}) bool { } else if !this.Plan.Equal(*that1.Plan) { return false } + if !this.StoreChunks.Equal(that1.StoreChunks) { + return false + } return true } func (this *Plan) Equal(that interface{}) bool { @@ -4214,6 +4287,35 @@ func (this *ChunkRef) Equal(that interface{}) bool { } return true } +func (this *ChunkRefGroup) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ChunkRefGroup) + if !ok { + that2, ok := that.(ChunkRefGroup) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Refs) != len(that1.Refs) { + return false + } + for i := range this.Refs { + if !this.Refs[i].Equal(that1.Refs[i]) { + return false + } + } + return true +} func (this *LabelValuesForMetricNameRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -5024,7 +5126,7 @@ func (this *QueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 12) + s := make([]string, 0, 13) s = append(s, "&logproto.QueryRequest{") s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") @@ -5036,6 +5138,9 @@ func (this *QueryRequest) GoString() string { s = append(s, "Deletes: "+fmt.Sprintf("%#v", this.Deletes)+",\n") } s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") + if this.StoreChunks != nil { + s = append(s, "StoreChunks: "+fmt.Sprintf("%#v", this.StoreChunks)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -5043,7 +5148,7 @@ func (this *SampleQueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 10) + s := make([]string, 0, 11) s = append(s, "&logproto.SampleQueryRequest{") s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n") s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") @@ -5053,6 +5158,9 @@ func (this *SampleQueryRequest) GoString() string { s = append(s, "Deletes: "+fmt.Sprintf("%#v", this.Deletes)+",\n") } s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") + if this.StoreChunks != nil { + s = append(s, "StoreChunks: "+fmt.Sprintf("%#v", this.StoreChunks)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -5349,6 +5457,18 @@ func (this *ChunkRef) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *ChunkRefGroup) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&logproto.ChunkRefGroup{") + if this.Refs != nil { + s = append(s, "Refs: "+fmt.Sprintf("%#v", this.Refs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} func (this *LabelValuesForMetricNameRequest) GoString() string { if this == nil { return "nil" @@ -6466,6 +6586,18 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.StoreChunks != nil { + { + size, err := m.StoreChunks.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x52 + } if m.Plan != nil { { size := m.Plan.Size() @@ -6506,21 +6638,21 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x28 } - n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err3 != nil { - return 0, err3 - } - i -= n3 - i = encodeVarintLogproto(dAtA, i, uint64(n3)) - i-- - dAtA[i] = 0x22 - n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) if err4 != nil { return 0, err4 } i -= n4 i = encodeVarintLogproto(dAtA, i, uint64(n4)) i-- + dAtA[i] = 0x22 + n5, err5 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err5 != nil { + return 0, err5 + } + i -= n5 + i = encodeVarintLogproto(dAtA, i, uint64(n5)) + i-- dAtA[i] = 0x1a if m.Limit != 0 { i = encodeVarintLogproto(dAtA, i, uint64(m.Limit)) @@ -6557,6 +6689,18 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.StoreChunks != nil { + { + size, err := m.StoreChunks.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x52 + } if m.Plan != nil { { size := m.Plan.Size() @@ -6592,20 +6736,20 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x22 } } - n6, err6 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err6 != nil { - return 0, err6 + n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err8 != nil { + return 0, err8 } - i -= n6 - i = encodeVarintLogproto(dAtA, i, uint64(n6)) + i -= n8 + i = encodeVarintLogproto(dAtA, i, uint64(n8)) i-- dAtA[i] = 0x1a - n7, err7 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err7 != nil { - return 0, err7 + n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err9 != nil { + return 0, err9 } - i -= n7 - i = encodeVarintLogproto(dAtA, i, uint64(n7)) + i -= n9 + i = encodeVarintLogproto(dAtA, i, uint64(n9)) i-- dAtA[i] = 0x12 if len(m.Selector) > 0 { @@ -6828,22 +6972,22 @@ func (m *LabelRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x2a } if m.End != nil { - n10, err10 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) - if err10 != nil { - return 0, err10 + n12, err12 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) + if err12 != nil { + return 0, err12 } - i -= n10 - i = encodeVarintLogproto(dAtA, i, uint64(n10)) + i -= n12 + i = encodeVarintLogproto(dAtA, i, uint64(n12)) i-- dAtA[i] = 0x22 } if m.Start != nil { - n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) - if err11 != nil { - return 0, err11 + n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) + if err13 != nil { + return 0, err13 } - i -= n11 - i = encodeVarintLogproto(dAtA, i, uint64(n11)) + i -= n13 + i = encodeVarintLogproto(dAtA, i, uint64(n13)) i-- dAtA[i] = 0x1a } @@ -7053,12 +7197,12 @@ func (m *TailRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x32 } - n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err13 != nil { - return 0, err13 + n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err15 != nil { + return 0, err15 } - i -= n13 - i = encodeVarintLogproto(dAtA, i, uint64(n13)) + i -= n15 + i = encodeVarintLogproto(dAtA, i, uint64(n15)) i-- dAtA[i] = 0x2a if m.Limit != 0 { @@ -7168,20 +7312,20 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x1a } } - n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err15 != nil { - return 0, err15 + n17, err17 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err17 != nil { + return 0, err17 } - i -= n15 - i = encodeVarintLogproto(dAtA, i, uint64(n15)) + i -= n17 + i = encodeVarintLogproto(dAtA, i, uint64(n17)) i-- dAtA[i] = 0x12 - n16, err16 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err16 != nil { - return 0, err16 + n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err18 != nil { + return 0, err18 } - i -= n16 - i = encodeVarintLogproto(dAtA, i, uint64(n16)) + i -= n18 + i = encodeVarintLogproto(dAtA, i, uint64(n18)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -7325,20 +7469,20 @@ func (m *DroppedStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n17, err17 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) - if err17 != nil { - return 0, err17 + n19, err19 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) + if err19 != nil { + return 0, err19 } - i -= n17 - i = encodeVarintLogproto(dAtA, i, uint64(n17)) + i -= n19 + i = encodeVarintLogproto(dAtA, i, uint64(n19)) i-- dAtA[i] = 0x12 - n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) - if err18 != nil { - return 0, err18 + n20, err20 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) + if err20 != nil { + return 0, err20 } - i -= n18 - i = encodeVarintLogproto(dAtA, i, uint64(n18)) + i -= n20 + i = encodeVarintLogproto(dAtA, i, uint64(n20)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -7519,20 +7663,20 @@ func (m *GetChunkIDsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n19, err19 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err19 != nil { - return 0, err19 + n21, err21 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err21 != nil { + return 0, err21 } - i -= n19 - i = encodeVarintLogproto(dAtA, i, uint64(n19)) + i -= n21 + i = encodeVarintLogproto(dAtA, i, uint64(n21)) i-- dAtA[i] = 0x1a - n20, err20 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err20 != nil { - return 0, err20 + n22, err22 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err22 != nil { + return 0, err22 } - i -= n20 - i = encodeVarintLogproto(dAtA, i, uint64(n20)) + i -= n22 + i = encodeVarintLogproto(dAtA, i, uint64(n22)) i-- dAtA[i] = 0x12 if len(m.Matchers) > 0 { @@ -7627,6 +7771,43 @@ func (m *ChunkRef) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ChunkRefGroup) 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 *ChunkRefGroup) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ChunkRefGroup) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Refs) > 0 { + for iNdEx := len(m.Refs) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Refs[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func (m *LabelValuesForMetricNameRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -8410,20 +8591,20 @@ func (m *DetectedFieldsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n22, err22 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err22 != nil { - return 0, err22 + n24, err24 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err24 != nil { + return 0, err24 } - i -= n22 - i = encodeVarintLogproto(dAtA, i, uint64(n22)) + i -= n24 + i = encodeVarintLogproto(dAtA, i, uint64(n24)) i-- dAtA[i] = 0x12 - n23, err23 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err23 != nil { - return 0, err23 + n25, err25 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err25 != nil { + return 0, err25 } - i -= n23 - i = encodeVarintLogproto(dAtA, i, uint64(n23)) + i -= n25 + i = encodeVarintLogproto(dAtA, i, uint64(n25)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -8548,22 +8729,22 @@ func (m *DetectedLabelsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x1a } if m.End != nil { - n24, err24 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) - if err24 != nil { - return 0, err24 + n26, err26 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) + if err26 != nil { + return 0, err26 } - i -= n24 - i = encodeVarintLogproto(dAtA, i, uint64(n24)) + i -= n26 + i = encodeVarintLogproto(dAtA, i, uint64(n26)) i-- dAtA[i] = 0x12 } if m.Start != nil { - n25, err25 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) - if err25 != nil { - return 0, err25 + n27, err27 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) + if err27 != nil { + return 0, err27 } - i -= n25 - i = encodeVarintLogproto(dAtA, i, uint64(n25)) + i -= n27 + i = encodeVarintLogproto(dAtA, i, uint64(n27)) i-- dAtA[i] = 0xa } @@ -8775,6 +8956,10 @@ func (m *QueryRequest) Size() (n int) { l = m.Plan.Size() n += 1 + l + sovLogproto(uint64(l)) } + if m.StoreChunks != nil { + l = m.StoreChunks.Size() + n += 1 + l + sovLogproto(uint64(l)) + } return n } @@ -8808,6 +8993,10 @@ func (m *SampleQueryRequest) Size() (n int) { l = m.Plan.Size() n += 1 + l + sovLogproto(uint64(l)) } + if m.StoreChunks != nil { + l = m.StoreChunks.Size() + n += 1 + l + sovLogproto(uint64(l)) + } return n } @@ -9245,6 +9434,21 @@ func (m *ChunkRef) Size() (n int) { return n } +func (m *ChunkRefGroup) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Refs) > 0 { + for _, e := range m.Refs { + l = e.Size() + n += 1 + l + sovLogproto(uint64(l)) + } + } + return n +} + func (m *LabelValuesForMetricNameRequest) Size() (n int) { if m == nil { return 0 @@ -9796,6 +10000,7 @@ func (this *QueryRequest) String() string { `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Deletes:` + repeatedStringForDeletes + `,`, `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, + `StoreChunks:` + strings.Replace(this.StoreChunks.String(), "ChunkRefGroup", "ChunkRefGroup", 1) + `,`, `}`, }, "") return s @@ -9816,6 +10021,7 @@ func (this *SampleQueryRequest) String() string { `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Deletes:` + repeatedStringForDeletes + `,`, `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, + `StoreChunks:` + strings.Replace(this.StoreChunks.String(), "ChunkRefGroup", "ChunkRefGroup", 1) + `,`, `}`, }, "") return s @@ -10113,6 +10319,21 @@ func (this *ChunkRef) String() string { }, "") return s } +func (this *ChunkRefGroup) String() string { + if this == nil { + return "nil" + } + repeatedStringForRefs := "[]*ChunkRef{" + for _, f := range this.Refs { + repeatedStringForRefs += strings.Replace(f.String(), "ChunkRef", "ChunkRef", 1) + "," + } + repeatedStringForRefs += "}" + s := strings.Join([]string{`&ChunkRefGroup{`, + `Refs:` + repeatedStringForRefs + `,`, + `}`, + }, "") + return s +} func (this *LabelValuesForMetricNameRequest) String() string { if this == nil { return "nil" @@ -11264,6 +11485,42 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreChunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StoreChunks == nil { + m.StoreChunks = &ChunkRefGroup{} + } + if err := m.StoreChunks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) @@ -11517,6 +11774,42 @@ func (m *SampleQueryRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreChunks", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StoreChunks == nil { + m.StoreChunks = &ChunkRefGroup{} + } + if err := m.StoreChunks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) @@ -14459,6 +14752,93 @@ func (m *ChunkRef) Unmarshal(dAtA []byte) error { } return nil } +func (m *ChunkRefGroup) 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 ErrIntOverflowLogproto + } + 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: ChunkRefGroup: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChunkRefGroup: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Refs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Refs = append(m.Refs, &ChunkRef{}) + if err := m.Refs[len(m.Refs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *LabelValuesForMetricNameRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 830fbfe627ab..a29e38df01af 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -78,6 +78,9 @@ message QueryRequest { repeated string shards = 7 [(gogoproto.jsontag) = "shards,omitempty"]; repeated Delete deletes = 8; Plan plan = 9 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan"]; + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + ChunkRefGroup storeChunks = 10 [(gogoproto.jsontag) = "storeChunks"]; } message SampleQueryRequest { @@ -93,6 +96,9 @@ message SampleQueryRequest { repeated string shards = 4 [(gogoproto.jsontag) = "shards,omitempty"]; repeated Delete deletes = 5; Plan plan = 6 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan"]; + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + ChunkRefGroup storeChunks = 10 [(gogoproto.jsontag) = "storeChunks"]; } // TODO(owen-d): fix. This will break rollouts as soon as the internal repr is changed. @@ -286,6 +292,10 @@ message ChunkRef { uint32 checksum = 5 [(gogoproto.jsontag) = "-"]; } +message ChunkRefGroup { + repeated ChunkRef refs = 1 [(gogoproto.jsontag) = "refs"]; +} + message LabelValuesForMetricNameRequest { string metric_name = 1; string label_name = 2; diff --git a/pkg/logql/accumulator_test.go b/pkg/logql/accumulator_test.go index b9b8b86760d0..0975ea4789d2 100644 --- a/pkg/logql/accumulator_test.go +++ b/pkg/logql/accumulator_test.go @@ -51,7 +51,7 @@ func TestDownstreamAccumulatorSimple(t *testing.T) { } // dummy params. Only need to populate direction & limit params, err := NewLiteralParams( - `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, + `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, nil, ) require.NoError(t, err) @@ -110,7 +110,7 @@ func TestDownstreamAccumulatorMultiMerge(t *testing.T) { // dummy params. Only need to populate direction & limit params, err := NewLiteralParams( - `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, + `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, nil, ) require.NoError(t, err) @@ -154,7 +154,7 @@ func BenchmarkAccumulator(b *testing.B) { // dummy params. Only need to populate direction & limit lim := 30 params, err := NewLiteralParams( - `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, logproto.BACKWARD, uint32(lim), nil, + `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, logproto.BACKWARD, uint32(lim), nil, nil, ) require.NoError(b, err) diff --git a/pkg/logql/blocker_test.go b/pkg/logql/blocker_test.go index c39d77c07403..00eb80b86c3c 100644 --- a/pkg/logql/blocker_test.go +++ b/pkg/logql/blocker_test.go @@ -145,7 +145,7 @@ func TestEngine_ExecWithBlockedQueries(t *testing.T) { t.Run(test.name, func(t *testing.T) { limits.blockedQueries = test.blocked - params, err := NewLiteralParams(test.q, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, logproto.FORWARD, 1000, nil) + params, err := NewLiteralParams(test.q, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, logproto.FORWARD, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) _, err = q.Exec(user.InjectOrgID(context.Background(), "fake")) diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 0640f00307e0..2e3dbfd014a1 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -74,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 *Shard + shard *ShardWithChunkRefs syntax.SampleExpr } @@ -107,7 +107,7 @@ func (d DownstreamSampleExpr) Pretty(level int) string { // DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation type DownstreamLogSelectorExpr struct { - shard *Shard + shard *ShardWithChunkRefs syntax.LogSelectorExpr } @@ -394,15 +394,11 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( case DownstreamSampleExpr: // downstream to a querier - var shards Shards - if e.shard != nil { - shards = append(shards, *e.shard) - } acc := NewBufferedAccumulator(1) results, err := ev.Downstream(ctx, []DownstreamQuery{{ - Params: ParamsWithShardsOverride{ - Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: e.SampleExpr}, - ShardsOverride: shards.Encode(), + Params: ParamsWithExpressionOverride{ + Params: ParamOverridesFromShard(params, e.shard), + ExpressionOverride: e.SampleExpr, }, }}, acc) if err != nil { @@ -415,10 +411,10 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( var queries []DownstreamQuery for cur != nil { qry := DownstreamQuery{ - Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: cur.DownstreamSampleExpr.SampleExpr}, - } - if shard := cur.DownstreamSampleExpr.shard; shard != nil { - qry.Params = ParamsWithShardsOverride{Params: qry.Params, ShardsOverride: Shards{*shard}.Encode()} + Params: ParamsWithExpressionOverride{ + Params: ParamOverridesFromShard(params, cur.DownstreamSampleExpr.shard), + ExpressionOverride: cur.DownstreamSampleExpr.SampleExpr, + }, } queries = append(queries, qry) cur = cur.next @@ -451,16 +447,10 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( for _, d := range e.quantileMergeExpr.downstreams { qry := DownstreamQuery{ Params: ParamsWithExpressionOverride{ - Params: params, + Params: ParamOverridesFromShard(params, d.shard), ExpressionOverride: d.SampleExpr, }, } - if shard := d.shard; shard != nil { - qry.Params = ParamsWithShardsOverride{ - Params: qry.Params, - ShardsOverride: Shards{*shard}.Encode(), - } - } queries = append(queries, qry) } } @@ -496,15 +486,11 @@ func (ev *DownstreamEvaluator) NewIterator( switch e := expr.(type) { case DownstreamLogSelectorExpr: // downstream to a querier - var shards Shards - if e.shard != nil { - shards = append(shards, *e.shard) - } acc := NewStreamAccumulator(params) results, err := ev.Downstream(ctx, []DownstreamQuery{{ - Params: ParamsWithShardsOverride{ - Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: e.LogSelectorExpr}, - ShardsOverride: shards.Encode(), + Params: ParamsWithExpressionOverride{ + Params: ParamOverridesFromShard(params, e.shard), + ExpressionOverride: e.LogSelectorExpr, }, }}, acc) if err != nil { @@ -517,10 +503,10 @@ func (ev *DownstreamEvaluator) NewIterator( var queries []DownstreamQuery for cur != nil { qry := DownstreamQuery{ - Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: cur.DownstreamLogSelectorExpr.LogSelectorExpr}, - } - if shard := cur.DownstreamLogSelectorExpr.shard; shard != nil { - qry.Params = ParamsWithShardsOverride{Params: qry.Params, ShardsOverride: Shards{*shard}.Encode()} + Params: ParamsWithExpressionOverride{ + Params: ParamOverridesFromShard(params, cur.DownstreamLogSelectorExpr.shard), + ExpressionOverride: cur.DownstreamLogSelectorExpr.LogSelectorExpr, + }, } queries = append(queries, qry) cur = cur.next diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index fa179502d6b7..c33f97ed74a5 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -90,6 +90,7 @@ func TestMappingEquivalence(t *testing.T) { logproto.FORWARD, uint32(limit), nil, + nil, ) require.NoError(t, err) @@ -162,6 +163,7 @@ func TestMappingEquivalenceSketches(t *testing.T) { logproto.FORWARD, uint32(limit), nil, + nil, ) require.NoError(t, err) qry := regular.Query(params) @@ -197,6 +199,7 @@ func TestMappingEquivalenceSketches(t *testing.T) { logproto.FORWARD, uint32(limit), nil, + nil, ) require.NoError(t, err) qry := regular.Query(params) @@ -264,6 +267,7 @@ func TestShardCounter(t *testing.T) { logproto.FORWARD, uint32(limit), nil, + nil, ) require.NoError(t, err) ctx := user.InjectOrgID(context.Background(), "fake") @@ -524,6 +528,7 @@ func TestRangeMappingEquivalence(t *testing.T) { logproto.FORWARD, uint32(limit), nil, + nil, ) require.NoError(t, err) @@ -627,7 +632,7 @@ func TestFormat_ShardedExpr(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 3, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -643,7 +648,7 @@ func TestFormat_ShardedExpr(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 3, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -659,7 +664,7 @@ func TestFormat_ShardedExpr(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 3, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 64ea77bb115d..eabbf1c43323 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -73,6 +73,12 @@ type SelectLogParams struct { *logproto.QueryRequest } +func (s SelectLogParams) WithStoreChunks(chunkRefGroup *logproto.ChunkRefGroup) SelectLogParams { + cpy := *s.QueryRequest + cpy.StoreChunks = chunkRefGroup + return SelectLogParams{&cpy} +} + func (s SelectLogParams) String() string { if s.QueryRequest != nil { return fmt.Sprintf("selector=%s, direction=%s, start=%s, end=%s, limit=%d, shards=%s", @@ -98,6 +104,12 @@ type SelectSampleParams struct { *logproto.SampleQueryRequest } +func (s SelectSampleParams) WithStoreChunks(chunkRefGroup *logproto.ChunkRefGroup) SelectSampleParams { + cpy := *s.SampleQueryRequest + cpy.StoreChunks = chunkRefGroup + return SelectSampleParams{&cpy} +} + // Expr returns the SampleExpr from the SelectSampleParams. // The `LogSelectorExpr` can then returns all matchers and filters to use for that request. func (s SelectSampleParams) Expr() (syntax.SampleExpr, error) { diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index 2e354bdf5b8d..48e0e2832303 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -151,7 +151,7 @@ func TestEngine_LogsRateUnwrap(t *testing.T) { t.Parallel() eng := NewEngine(EngineOpts{}, newQuerierRecorder(t, test.data, test.params), NoLimits, log.NewNopLogger()) - params, err := NewLiteralParams(test.qs, test.ts, test.ts, 0, 0, test.direction, test.limit, nil) + params, err := NewLiteralParams(test.qs, test.ts, test.ts, 0, 0, test.direction, test.limit, nil, nil) require.NoError(t, err) q := eng.Query(params) res, err := q.Exec(user.InjectOrgID(context.Background(), "fake")) @@ -979,7 +979,7 @@ func TestEngine_LogsInstantQuery(t *testing.T) { eng := NewEngine(EngineOpts{}, newQuerierRecorder(t, test.data, test.params), NoLimits, log.NewNopLogger()) - params, err := NewLiteralParams(test.qs, test.ts, test.ts, 0, 0, test.direction, test.limit, nil) + params, err := NewLiteralParams(test.qs, test.ts, test.ts, 0, 0, test.direction, test.limit, nil, nil) require.NoError(t, err) q := eng.Query(params) res, err := q.Exec(user.InjectOrgID(context.Background(), "fake")) @@ -2281,7 +2281,7 @@ func TestEngine_RangeQuery(t *testing.T) { eng := NewEngine(EngineOpts{}, newQuerierRecorder(t, test.data, test.params), NoLimits, log.NewNopLogger()) - params, err := NewLiteralParams(test.qs, test.start, test.end, test.step, test.interval, test.direction, test.limit, nil) + params, err := NewLiteralParams(test.qs, test.start, test.end, test.step, test.interval, test.direction, test.limit, nil, nil) require.NoError(t, err) q := eng.Query(params) res, err := q.Exec(user.InjectOrgID(context.Background(), "fake")) @@ -2312,7 +2312,7 @@ func TestEngine_Stats(t *testing.T) { queueTime := 2 * time.Nanosecond - params, err := NewLiteralParams(`{foo="bar"}`, time.Now(), time.Now(), 0, 0, logproto.FORWARD, 1000, nil) + params, err := NewLiteralParams(`{foo="bar"}`, time.Now(), time.Now(), 0, 0, logproto.FORWARD, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) @@ -2345,7 +2345,7 @@ func (metaQuerier) SelectSamples(ctx context.Context, _ SelectSampleParams) (ite func TestEngine_Metadata(t *testing.T) { eng := NewEngine(EngineOpts{}, &metaQuerier{}, NoLimits, log.NewNopLogger()) - params, err := NewLiteralParams(`{foo="bar"}`, time.Now(), time.Now(), 0, 0, logproto.BACKWARD, 1000, nil) + params, err := NewLiteralParams(`{foo="bar"}`, time.Now(), time.Now(), 0, 0, logproto.BACKWARD, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) @@ -2362,7 +2362,7 @@ func TestEngine_LogsInstantQuery_Vector(t *testing.T) { queueTime := 2 * time.Nanosecond logqlVector := `vector(5)` - params, err := NewLiteralParams(logqlVector, now, now, 0, time.Second*30, logproto.BACKWARD, 1000, nil) + params, err := NewLiteralParams(logqlVector, now, now, 0, time.Second*30, logproto.BACKWARD, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) ctx := context.WithValue(context.Background(), httpreq.QueryQueueTimeHTTPHeader, queueTime) @@ -2448,7 +2448,7 @@ func TestStepEvaluator_Error(t *testing.T) { t.Run(tc.name, func(t *testing.T) { eng := NewEngine(EngineOpts{}, tc.querier, NoLimits, log.NewNopLogger()) - params, err := NewLiteralParams(tc.qs, time.Unix(0, 0), time.Unix(180, 0), 1*time.Second, 0, logproto.BACKWARD, 1, nil) + params, err := NewLiteralParams(tc.qs, time.Unix(0, 0), time.Unix(180, 0), 1*time.Second, 0, logproto.BACKWARD, 1, nil, nil) require.NoError(t, err) q := eng.Query(params) _, err = q.Exec(user.InjectOrgID(context.Background(), "fake")) @@ -2474,7 +2474,7 @@ func TestEngine_MaxSeries(t *testing.T) { {`avg(count_over_time({app=~"foo|bar"} |~".+bar" [1m]))`, logproto.FORWARD, false}, } { t.Run(test.qs, func(t *testing.T) { - params, err := NewLiteralParams(test.qs, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, test.direction, 1000, nil) + params, err := NewLiteralParams(test.qs, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, test.direction, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) _, err = q.Exec(user.InjectOrgID(context.Background(), "fake")) @@ -2501,7 +2501,7 @@ func TestEngine_MaxRangeInterval(t *testing.T) { {`topk(1,rate({app=~"foo|bar"}[12h]) / (rate({app="baz"}[23h]) + rate({app="fiz"}[25h])))`, logproto.FORWARD, true}, } { t.Run(test.qs, func(t *testing.T) { - params, err := NewLiteralParams(test.qs, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, test.direction, 1000, nil) + params, err := NewLiteralParams(test.qs, time.Unix(0, 0), time.Unix(100000, 0), 60*time.Second, 0, test.direction, 1000, nil, nil) require.NoError(t, err) q := eng.Query(params) @@ -2568,7 +2568,7 @@ func benchmarkRangeQuery(testsize int64, b *testing.B) { {`bottomk(2,rate(({app=~"foo|bar"} |~".+bar")[1m]))`, logproto.FORWARD}, {`bottomk(3,rate(({app=~"foo|bar"} |~".+bar")[1m])) without (app)`, logproto.FORWARD}, } { - params, err := NewLiteralParams(test.qs, start, end, 60*time.Second, 0, logproto.BACKWARD, 1000, nil) + params, err := NewLiteralParams(test.qs, start, end, 60*time.Second, 0, logproto.BACKWARD, 1000, nil, nil) require.NoError(b, err) q := eng.Query(params) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index eb7958691ac6..7c1f45021731 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -41,6 +41,7 @@ type Params interface { Direction() logproto.Direction Shards() []string GetExpression() syntax.Expr + GetStoreChunks() *logproto.ChunkRefGroup } func NewLiteralParams( @@ -50,6 +51,7 @@ func NewLiteralParams( direction logproto.Direction, limit uint32, shards []string, + storeChunks *logproto.ChunkRefGroup, ) (LiteralParams, error) { p := LiteralParams{ queryString: qs, @@ -60,6 +62,7 @@ func NewLiteralParams( direction: direction, limit: limit, shards: shards, + storeChunks: storeChunks, } var err error p.queryExpr, err = syntax.ParseExpr(qs) @@ -76,6 +79,7 @@ type LiteralParams struct { limit uint32 shards []string queryExpr syntax.Expr + storeChunks *logproto.ChunkRefGroup } func (p LiteralParams) Copy() LiteralParams { return p } @@ -107,6 +111,9 @@ func (p LiteralParams) Direction() logproto.Direction { return p.direction } // Shards impls Params func (p LiteralParams) Shards() []string { return p.shards } +// StoreChunks impls Params +func (p LiteralParams) GetStoreChunks() *logproto.ChunkRefGroup { return p.storeChunks } + // GetRangeType returns whether a query is an instant query or range query func GetRangeType(q Params) QueryRangeType { if q.Start() == q.End() && q.Step() == 0 { @@ -141,6 +148,35 @@ func (p ParamsWithShardsOverride) Shards() []string { return p.ShardsOverride } +type ParamsWithChunkOverrides struct { + Params + StoreChunksOverride *logproto.ChunkRefGroup +} + +func (p ParamsWithChunkOverrides) GetStoreChunks() *logproto.ChunkRefGroup { + return p.StoreChunksOverride +} + +func ParamOverridesFromShard(base Params, shard *ShardWithChunkRefs) (result Params) { + if shard == nil { + return base + } + + result = ParamsWithShardsOverride{ + Params: base, + ShardsOverride: Shards{shard.Shard}.Encode(), + } + + if shard.chunks != nil { + result = ParamsWithChunkOverrides{ + Params: result, + StoreChunksOverride: shard.chunks, + } + } + + return result +} + // Sortable logql contain sort or sort_desc. func Sortable(q Params) (bool, error) { var sortable bool @@ -214,6 +250,7 @@ func (ev *DefaultEvaluator) NewIterator(ctx context.Context, expr syntax.LogSele Plan: &plan.QueryPlan{ AST: expr, }, + StoreChunks: q.GetStoreChunks(), }, } @@ -245,6 +282,7 @@ func (ev *DefaultEvaluator) NewStepEvaluator( Plan: &plan.QueryPlan{ AST: expr, }, + StoreChunks: q.GetStoreChunks(), }, }) if err != nil { @@ -264,6 +302,7 @@ func (ev *DefaultEvaluator) NewStepEvaluator( Plan: &plan.QueryPlan{ AST: expr, }, + StoreChunks: q.GetStoreChunks(), }, }) if err != nil { diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index df7c62a895bb..d965676c278f 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -457,7 +457,9 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, Of: uint32(shards), }) downstreams = append(downstreams, DownstreamSampleExpr{ - shard: &s, + shard: &ShardWithChunkRefs{ + Shard: s, + }, SampleExpr: expr, }) } diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index f81f90a13778..784301928583 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -24,7 +24,7 @@ func TestShardedStringer(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -34,7 +34,7 @@ func TestShardedStringer(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -75,7 +75,7 @@ func TestMapSampleExpr(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -91,7 +91,7 @@ func TestMapSampleExpr(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -508,7 +508,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -518,7 +518,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, }, @@ -534,7 +534,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.PipelineExpr{ Left: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, @@ -555,7 +555,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), LogSelectorExpr: &syntax.PipelineExpr{ Left: &syntax.MatchersExpr{ Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, @@ -582,7 +582,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -598,7 +598,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -620,7 +620,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeCount, Left: &syntax.LogRange{ @@ -636,7 +636,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeCount, Left: &syntax.LogRange{ @@ -661,7 +661,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -681,7 +681,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -712,7 +712,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -728,7 +728,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.RangeAggregationExpr{ Operation: syntax.OpRangeTypeRate, Left: &syntax.LogRange{ @@ -754,7 +754,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -774,7 +774,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -806,7 +806,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -826,7 +826,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -853,7 +853,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -873,7 +873,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -913,7 +913,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -935,7 +935,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -975,7 +975,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -995,7 +995,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeSum, @@ -1030,7 +1030,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1050,7 +1050,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1092,7 +1092,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1114,7 +1114,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1144,7 +1144,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1164,7 +1164,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1206,7 +1206,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1228,7 +1228,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1257,7 +1257,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1277,7 +1277,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{}, Operation: syntax.OpTypeCount, @@ -1312,7 +1312,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1337,7 +1337,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1371,7 +1371,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1393,7 +1393,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Grouping: &syntax.Grouping{ Groups: []string{"cluster"}, @@ -1482,7 +1482,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 0, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Left: &syntax.RangeAggregationExpr{ Left: &syntax.LogRange{ @@ -1505,7 +1505,7 @@ func TestMapping(t *testing.T) { shard: NewPowerOfTwoShard(index.ShardAnnotation{ Shard: 1, Of: 2, - }).Ptr(), + }).Bind(nil), SampleExpr: &syntax.VectorAggregationExpr{ Left: &syntax.RangeAggregationExpr{ Left: &syntax.LogRange{ diff --git a/pkg/logql/shards.go b/pkg/logql/shards.go index 75281aa3c95b..f2a18c0053ab 100644 --- a/pkg/logql/shards.go +++ b/pkg/logql/shards.go @@ -20,6 +20,9 @@ type Shards []Shard type ShardVersion uint8 +// TODO(owen-d): refactor this file. There's too many layers (sharding strategies, sharding resolvers). +// Eventually we should have a single strategy (bounded) and a single resolver (dynamic). +// It's likely this could be refactored anyway -- I was in a rush writing it the first time around. const ( PowerOfTwoVersion ShardVersion = iota BoundedVersion @@ -62,20 +65,24 @@ func ParseShardVersion(s string) (ShardVersion, error) { type ShardResolver interface { Shards(expr syntax.Expr) (int, uint64, error) - ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ([]logproto.Shard, error) + // ShardingRanges returns shards and optionally a set of precomputed chunk refs for each group. If present, + // they will be used in lieu of resolving chunk refs from the index durin evaluation. + // If chunks are present, the number of shards returned must match the number of chunk ref groups. + ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ([]logproto.Shard, []logproto.ChunkRefGroup, 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) ShardingRanges(_ syntax.Expr, _ uint64) ([]logproto.Shard, []logproto.ChunkRefGroup, error) { + return sharding.LinearShards(int(s), 0), nil, 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) + // The chunks for each shard are optional and are used to precompute chunk refs for each group + Shards(expr syntax.Expr) (shards []ShardWithChunkRefs, maxBytesPerShard uint64, err error) Resolver() ShardResolver } @@ -84,19 +91,25 @@ type DynamicBoundsStrategy struct { targetBytesPerShard uint64 } -func (s DynamicBoundsStrategy) Shards(expr syntax.Expr) (Shards, uint64, error) { - shards, err := s.resolver.ShardingRanges(expr, s.targetBytesPerShard) +func (s DynamicBoundsStrategy) Shards(expr syntax.Expr) ([]ShardWithChunkRefs, uint64, error) { + shards, chunks, 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 { + res := make([]ShardWithChunkRefs, 0, len(shards)) + for i, shard := range shards { + x := ShardWithChunkRefs{ + Shard: NewBoundedShard(shard), + } if shard.Stats != nil { maxBytes = max(maxBytes, shard.Stats.Bytes) } - res = append(res, NewBoundedShard(shard)) + if len(chunks) > 0 { + x.chunks = &chunks[i] + } + res = append(res, x) } return res, maxBytes, nil @@ -122,7 +135,8 @@ func (s PowerOfTwoStrategy) Resolver() ShardResolver { return s.resolver } -func (s PowerOfTwoStrategy) Shards(expr syntax.Expr) (Shards, uint64, error) { +// PowerOfTwo strategy does not support precomputed chunk refs +func (s PowerOfTwoStrategy) Shards(expr syntax.Expr) ([]ShardWithChunkRefs, uint64, error) { factor, bytesPerShard, err := s.resolver.Shards(expr) if err != nil { return nil, 0, err @@ -132,13 +146,26 @@ func (s PowerOfTwoStrategy) Shards(expr syntax.Expr) (Shards, uint64, error) { return nil, bytesPerShard, nil } - res := make(Shards, 0, factor) + res := make([]ShardWithChunkRefs, 0, factor) for i := 0; i < factor; i++ { - res = append(res, NewPowerOfTwoShard(index.ShardAnnotation{Of: uint32(factor), Shard: uint32(i)})) + res = append( + res, + ShardWithChunkRefs{ + Shard: NewPowerOfTwoShard(index.ShardAnnotation{Of: uint32(factor), Shard: uint32(i)}), + }, + ) } return res, bytesPerShard, nil } +// ShardWithChunkRefs is a convenience type for passing around shards with associated chunk refs. +// The chunk refs are optional as determined by their contents (zero chunks means no precomputed refs) +// and are used to precompute chunk refs for each group +type ShardWithChunkRefs struct { + Shard + chunks *logproto.ChunkRefGroup +} + // Shard represents a shard annotation // It holds either a power of two shard (legacy) or a bounded shard type Shard struct { @@ -176,6 +203,13 @@ func (s Shard) Ptr() *Shard { return &s } +func (s Shard) Bind(chunks *logproto.ChunkRefGroup) *ShardWithChunkRefs { + return &ShardWithChunkRefs{ + Shard: s, + chunks: chunks, + } +} + func NewBoundedShard(shard logproto.Shard) Shard { return Shard{Bounded: &shard} } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 1473a5616fd3..a4690a779f0b 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1440,7 +1440,7 @@ func (t *Loki) initIndexGateway() (services.Service, error) { bloomQuerier = bloomgateway.NewQuerier(bloomGatewayClient, t.Overrides, resolver, prometheus.DefaultRegisterer, logger) } - gateway, err := indexgateway.NewIndexGateway(t.Cfg.IndexGateway, logger, prometheus.DefaultRegisterer, t.Store, indexClients, bloomQuerier) + gateway, err := indexgateway.NewIndexGateway(t.Cfg.IndexGateway, t.Overrides, logger, prometheus.DefaultRegisterer, t.Store, indexClients, bloomQuerier) if err != nil { return nil, err } diff --git a/pkg/querier/multi_tenant_querier.go b/pkg/querier/multi_tenant_querier.go index 654ae7c2deb2..961b35a91651 100644 --- a/pkg/querier/multi_tenant_querier.go +++ b/pkg/querier/multi_tenant_querier.go @@ -67,11 +67,24 @@ func (q *MultiTenantQuerier) SelectLogs(ctx context.Context, params logql.Select AST: parsed, } + // in case of multiple tenants, we need to filter the store chunks by tenant if they are provided + storeOverridesByTenant := make(map[string][]*logproto.ChunkRef) + if overrides := params.GetStoreChunks(); overrides != nil { + storeOverridesByTenant = partitionChunkRefsByTenant(overrides.Refs) + } + iters := make([]iter.EntryIterator, len(matchedTenants)) i := 0 for id := range matchedTenants { singleContext := user.InjectOrgID(ctx, id) - iter, err := q.Querier.SelectLogs(singleContext, params) + + tenantParams := params + + if tenantChunkOverrides, ok := storeOverridesByTenant[id]; ok { + tenantParams = tenantParams.WithStoreChunks(&logproto.ChunkRefGroup{Refs: tenantChunkOverrides}) + } + + iter, err := q.Querier.SelectLogs(singleContext, tenantParams) if err != nil { return nil, err } @@ -98,11 +111,23 @@ func (q *MultiTenantQuerier) SelectSamples(ctx context.Context, params logql.Sel } params.Selector = updatedSelector.String() + // in case of multiple tenants, we need to filter the store chunks by tenant if they are provided + storeOverridesByTenant := make(map[string][]*logproto.ChunkRef) + if overrides := params.GetStoreChunks(); overrides != nil { + storeOverridesByTenant = partitionChunkRefsByTenant(params.GetStoreChunks().Refs) + } + iters := make([]iter.SampleIterator, len(matchedTenants)) i := 0 for id := range matchedTenants { singleContext := user.InjectOrgID(ctx, id) - iter, err := q.Querier.SelectSamples(singleContext, params) + tenantParams := params + + if tenantChunkOverrides, ok := storeOverridesByTenant[id]; ok { + tenantParams = tenantParams.WithStoreChunks(&logproto.ChunkRefGroup{Refs: tenantChunkOverrides}) + } + + iter, err := q.Querier.SelectSamples(singleContext, tenantParams) if err != nil { return nil, err } @@ -448,3 +473,11 @@ func NewTenantSampleIterator(iter iter.SampleIterator, id string) *TenantSampleI func (i *TenantSampleIterator) Labels() string { return i.relabel.relabel(i.SampleIterator.Labels()) } + +func partitionChunkRefsByTenant(refs []*logproto.ChunkRef) map[string][]*logproto.ChunkRef { + filtered := make(map[string][]*logproto.ChunkRef) + for _, ref := range refs { + filtered[ref.UserID] = append(filtered[ref.UserID], ref) + } + return filtered +} diff --git a/pkg/querier/querier_mock_test.go b/pkg/querier/querier_mock_test.go index 6d025a9e0db5..51e4a6a9d8dd 100644 --- a/pkg/querier/querier_mock_test.go +++ b/pkg/querier/querier_mock_test.go @@ -334,7 +334,7 @@ func (s *storeMock) SelectSamples(ctx context.Context, req logql.SelectSamplePar return res.(iter.SampleIterator), args.Error(1) } -func (s *storeMock) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (s *storeMock) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate, _ *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { args := s.Called(ctx, userID, from, through, predicate) return args.Get(0).([][]chunk.Chunk), args.Get(0).([]*fetcher.Fetcher), args.Error(2) } diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 99392878ef9b..6626d08f8775 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -4,7 +4,7 @@ import ( "bytes" "container/heap" "context" - "errors" + "encoding/json" "fmt" "io" "net/http" @@ -14,19 +14,14 @@ import ( "strings" "time" - "golang.org/x/exp/maps" - - "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" - "github.com/grafana/loki/v3/pkg/storage/detected" - "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" - "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/user" - json "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" + "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/timestamp" + "golang.org/x/exp/maps" "github.com/grafana/loki/v3/pkg/loghttp" "github.com/grafana/loki/v3/pkg/logproto" @@ -36,6 +31,9 @@ import ( "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/querier/plan" "github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" + "github.com/grafana/loki/v3/pkg/storage/detected" + "github.com/grafana/loki/v3/pkg/storage/stores/index/seriesvolume" indexStats "github.com/grafana/loki/v3/pkg/storage/stores/index/stats" "github.com/grafana/loki/v3/pkg/util" "github.com/grafana/loki/v3/pkg/util/httpreq" @@ -333,52 +331,19 @@ func (Codec) DecodeRequest(_ context.Context, r *http.Request, _ []string) (quer switch op := getOperation(r.URL.Path); op { case QueryRangeOp: - rangeQuery, err := loghttp.ParseRangeQuery(r) - if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - - parsed, err := syntax.ParseExpr(rangeQuery.Query) + req, err := parseRangeQuery(r) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - return &LokiRequest{ - Query: rangeQuery.Query, - Limit: rangeQuery.Limit, - Direction: rangeQuery.Direction, - StartTs: rangeQuery.Start.UTC(), - EndTs: rangeQuery.End.UTC(), - Step: rangeQuery.Step.Milliseconds(), - Interval: rangeQuery.Interval.Milliseconds(), - Path: r.URL.Path, - Shards: rangeQuery.Shards, - Plan: &plan.QueryPlan{ - AST: parsed, - }, - }, nil + return req, nil case InstantQueryOp: - req, err := loghttp.ParseInstantQuery(r) - if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - - parsed, err := syntax.ParseExpr(req.Query) + req, err := parseInstantQuery(r) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - return &LokiInstantRequest{ - Query: req.Query, - Limit: req.Limit, - Direction: req.Direction, - TimeTs: req.Ts.UTC(), - Path: r.URL.Path, - Shards: req.Shards, - Plan: &plan.QueryPlan{ - AST: parsed, - }, - }, nil + return req, nil case SeriesOp: req, err := loghttp.ParseAndValidateSeriesQuery(r) if err != nil { @@ -548,52 +513,19 @@ func (Codec) DecodeHTTPGrpcRequest(ctx context.Context, r *httpgrpc.HTTPRequest) switch op := getOperation(httpReq.URL.Path); op { case QueryRangeOp: - req, err := loghttp.ParseRangeQuery(httpReq) - if err != nil { - return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - - parsed, err := syntax.ParseExpr(req.Query) + req, err := parseRangeQuery(httpReq) if err != nil { return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - return &LokiRequest{ - Query: req.Query, - Limit: req.Limit, - Direction: req.Direction, - StartTs: req.Start.UTC(), - EndTs: req.End.UTC(), - Step: req.Step.Milliseconds(), - Interval: req.Interval.Milliseconds(), - Path: r.Url, - Shards: req.Shards, - Plan: &plan.QueryPlan{ - AST: parsed, - }, - }, ctx, nil + return req, ctx, nil case InstantQueryOp: - req, err := loghttp.ParseInstantQuery(httpReq) - if err != nil { - return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - - parsed, err := syntax.ParseExpr(req.Query) + req, err := parseInstantQuery(httpReq) if err != nil { return nil, ctx, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - return &LokiInstantRequest{ - Query: req.Query, - Limit: req.Limit, - Direction: req.Direction, - TimeTs: req.Ts.UTC(), - Path: r.Url, - Shards: req.Shards, - Plan: &plan.QueryPlan{ - AST: parsed, - }, - }, ctx, nil + return req, ctx, nil case SeriesOp: req, err := loghttp.ParseAndValidateSeriesQuery(httpReq) if err != nil { @@ -810,6 +742,17 @@ func (c Codec) EncodeRequest(ctx context.Context, r queryrangebase.Request) (*ht if request.Interval != 0 { params["interval"] = []string{fmt.Sprintf("%f", float64(request.Interval)/float64(1e3))} } + // undocumented param to allow specifying store chunks for a request, + // used in bounded tsdb sharding + // TODO(owen-d): version & encode in body instead? We're experiencing the limits + // using the same reprs for internal vs external APIs and maybe we should handle that. + if request.StoreChunks != nil { + b, err := request.StoreChunks.Marshal() + if err != nil { + return nil, errors.Wrap(err, "marshaling store chunks") + } + params["storeChunks"] = []string{string(b)} + } u := &url.URL{ // the request could come /api/prom/query but we want to only use the new api. Path: "/loki/api/v1/query_range", @@ -1906,6 +1849,10 @@ func (p paramsSeriesWrapper) Shards() []string { return p.GetShards() } +func (p paramsSeriesWrapper) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} + type paramsLabelWrapper struct { *LabelRequest } @@ -1938,6 +1885,10 @@ func (p paramsLabelWrapper) Shards() []string { return make([]string, 0) } +func (p paramsLabelWrapper) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} + type paramsStatsWrapper struct { *logproto.IndexStatsRequest } @@ -1970,6 +1921,10 @@ func (p paramsStatsWrapper) Shards() []string { return make([]string, 0) } +func (p paramsStatsWrapper) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} + type paramsDetectedFieldsWrapper struct { *DetectedFieldsRequest } @@ -2006,11 +1961,17 @@ func (p paramsDetectedFieldsWrapper) Interval() time.Duration { func (p paramsDetectedFieldsWrapper) Direction() logproto.Direction { return logproto.BACKWARD } + func (p paramsDetectedFieldsWrapper) Limit() uint32 { return p.DetectedFieldsRequest.LineLimit } + func (p paramsDetectedFieldsWrapper) Shards() []string { return make([]string, 0) } +func (p paramsDetectedFieldsWrapper) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} + func httpResponseHeadersToPromResponseHeaders(httpHeaders http.Header) []queryrangebase.PrometheusResponseHeader { var promHeaders []queryrangebase.PrometheusResponseHeader for h, hv := range httpHeaders { @@ -2126,7 +2087,81 @@ func mergeLokiResponse(responses ...queryrangebase.Response) *LokiResponse { } } -// In some other world LabelRequest could implement queryrangebase.Request. +func parseRangeQuery(r *http.Request) (*LokiRequest, error) { + rangeQuery, err := loghttp.ParseRangeQuery(r) + if err != nil { + return nil, err + } + + parsed, err := syntax.ParseExpr(rangeQuery.Query) + if err != nil { + return nil, err + } + + storeChunks, err := parseStoreChunks(r) + if err != nil { + return nil, err + } + + return &LokiRequest{ + Query: rangeQuery.Query, + Limit: rangeQuery.Limit, + Direction: rangeQuery.Direction, + StartTs: rangeQuery.Start.UTC(), + EndTs: rangeQuery.End.UTC(), + Step: rangeQuery.Step.Milliseconds(), + Interval: rangeQuery.Interval.Milliseconds(), + Path: r.URL.Path, + Shards: rangeQuery.Shards, + StoreChunks: storeChunks, + Plan: &plan.QueryPlan{ + AST: parsed, + }, + }, nil +} + +func parseInstantQuery(r *http.Request) (*LokiInstantRequest, error) { + req, err := loghttp.ParseInstantQuery(r) + if err != nil { + return nil, err + } + + parsed, err := syntax.ParseExpr(req.Query) + if err != nil { + return nil, err + } + + storeChunks, err := parseStoreChunks(r) + if err != nil { + return nil, err + } + + return &LokiInstantRequest{ + Query: req.Query, + Limit: req.Limit, + Direction: req.Direction, + TimeTs: req.Ts.UTC(), + Path: r.URL.Path, + Shards: req.Shards, + StoreChunks: storeChunks, + Plan: &plan.QueryPlan{ + AST: parsed, + }, + }, nil +} + +// escape hatch for including store chunks in the request +func parseStoreChunks(r *http.Request) (*logproto.ChunkRefGroup, error) { + if s := r.Form.Get("storeChunks"); s != "" { + storeChunks := &logproto.ChunkRefGroup{} + if err := storeChunks.Unmarshal([]byte(s)); err != nil { + return nil, errors.Wrap(err, "unmarshaling storeChunks") + } + return storeChunks, nil + } + return nil, nil +} + type DetectedFieldsRequest struct { logproto.DetectedFieldsRequest path string diff --git a/pkg/querier/queryrange/codec_test.go b/pkg/querier/queryrange/codec_test.go index 833132d5c2b9..2a67ac512bce 100644 --- a/pkg/querier/queryrange/codec_test.go +++ b/pkg/querier/queryrange/codec_test.go @@ -400,17 +400,17 @@ func Test_codec_DecodeResponse(t *testing.T) { { "series error wrong key type", &http.Response{StatusCode: 200, Body: io.NopCloser(strings.NewReader(`{"status":"success","data": [{42: "some string"}]}`))}, &LokiSeriesRequest{Path: "/loki/api/v1/series"}, - nil, "error decoding response: ReadObjectCB", + nil, "error decoding response: invalid character", }, { "series error key decode", &http.Response{StatusCode: 200, Body: io.NopCloser(strings.NewReader(`{"status":"success","data": [{"\x": "some string"}]}`))}, &LokiSeriesRequest{Path: "/loki/api/v1/series"}, - nil, "invalid escape char after", + nil, "invalid character 'x' in string escape code", }, { "series error value decode", &http.Response{StatusCode: 200, Body: io.NopCloser(strings.NewReader(`{"status":"success","data": [{"label": "some string\x"}]}`))}, &LokiSeriesRequest{Path: "/loki/api/v1/series"}, - nil, "invalid escape char after", + nil, "invalid character 'x' in string escape code", }, } for _, tt := range tests { diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index 3d1485d5a77e..7103a38ab376 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -45,6 +45,7 @@ func ParamsToLokiRequest(params logql.Params) queryrangebase.Request { Plan: &plan.QueryPlan{ AST: params.GetExpression(), }, + StoreChunks: params.GetStoreChunks(), } } return &LokiRequest{ @@ -60,6 +61,7 @@ func ParamsToLokiRequest(params logql.Params) queryrangebase.Request { Plan: &plan.QueryPlan{ AST: params.GetExpression(), }, + StoreChunks: params.GetStoreChunks(), } } diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index 0dddddab9b00..979cc5a04e00 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -236,6 +236,7 @@ func TestInstanceFor(t *testing.T) { logproto.BACKWARD, 1000, nil, + nil, ) require.NoError(t, err) return params @@ -340,6 +341,7 @@ func TestInstanceDownstream(t *testing.T) { logproto.BACKWARD, 1000, nil, + nil, ) require.NoError(t, err) expr, err := syntax.ParseExpr(`{foo="bar"}`) @@ -412,6 +414,7 @@ func TestInstanceDownstream(t *testing.T) { logproto.BACKWARD, 1000, nil, + nil, ) require.NoError(t, err) diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index e78a1054643a..e295fac92c6d 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -52,6 +52,9 @@ type LokiRequest struct { Path string `protobuf:"bytes,7,opt,name=path,proto3" json:"path,omitempty"` Shards []string `protobuf:"bytes,8,rep,name=shards,proto3" json:"shards"` Plan *github_com_grafana_loki_v3_pkg_querier_plan.QueryPlan `protobuf:"bytes,10,opt,name=plan,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + StoreChunks *logproto.ChunkRefGroup `protobuf:"bytes,11,opt,name=storeChunks,proto3" json:"storeChunks"` } func (m *LokiRequest) Reset() { *m = LokiRequest{} } @@ -149,6 +152,13 @@ func (m *LokiRequest) GetShards() []string { return nil } +func (m *LokiRequest) GetStoreChunks() *logproto.ChunkRefGroup { + if m != nil { + return m.StoreChunks + } + return nil +} + type LokiInstantRequest struct { Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` Limit uint32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` @@ -157,6 +167,9 @@ type LokiInstantRequest struct { Path string `protobuf:"bytes,5,opt,name=path,proto3" json:"path,omitempty"` Shards []string `protobuf:"bytes,6,rep,name=shards,proto3" json:"shards"` Plan *github_com_grafana_loki_v3_pkg_querier_plan.QueryPlan `protobuf:"bytes,7,opt,name=plan,proto3,customtype=github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + StoreChunks *logproto.ChunkRefGroup `protobuf:"bytes,8,opt,name=storeChunks,proto3" json:"storeChunks"` } func (m *LokiInstantRequest) Reset() { *m = LokiInstantRequest{} } @@ -233,6 +246,13 @@ func (m *LokiInstantRequest) GetShards() []string { return nil } +func (m *LokiInstantRequest) GetStoreChunks() *logproto.ChunkRefGroup { + if m != nil { + return m.StoreChunks + } + return nil +} + type Plan struct { Raw []byte `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` } @@ -1451,122 +1471,125 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1836 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0x1c, 0x49, - 0x15, 0x9f, 0x9e, 0x4f, 0xcf, 0xf3, 0xc7, 0x9a, 0x8a, 0x71, 0x1a, 0xef, 0xee, 0xf4, 0x30, 0x12, - 0xbb, 0x06, 0xc1, 0x0c, 0x19, 0xef, 0x86, 0x5d, 0x13, 0xa2, 0x4d, 0xe3, 0x44, 0x0e, 0x64, 0x21, - 0xdb, 0xb6, 0x38, 0x70, 0x41, 0x65, 0x4f, 0x65, 0xdc, 0x78, 0xa6, 0xbb, 0xd3, 0x5d, 0xe3, 0xc4, - 0x12, 0x42, 0xfb, 0x0f, 0xac, 0xd8, 0xbf, 0x02, 0x71, 0x43, 0x48, 0x9c, 0x38, 0x71, 0x0c, 0x48, - 0x48, 0x39, 0xae, 0x46, 0xa2, 0x21, 0x0e, 0x42, 0xc8, 0xa7, 0x48, 0x5c, 0x39, 0xa0, 0xfa, 0xe8, - 0x9e, 0xaa, 0xe9, 0x36, 0x99, 0x09, 0xe2, 0x60, 0xe0, 0xe2, 0xa9, 0xae, 0x7a, 0xbf, 0xea, 0x57, - 0xbf, 0xf7, 0x7e, 0xaf, 0xab, 0xca, 0xf0, 0x76, 0x70, 0xdc, 0xef, 0x3c, 0x1c, 0x91, 0xd0, 0x25, - 0x21, 0xff, 0x3d, 0x0d, 0xb1, 0xd7, 0x27, 0x4a, 0xb3, 0x1d, 0x84, 0x3e, 0xf5, 0x11, 0x4c, 0x7a, - 0x36, 0xba, 0x7d, 0x97, 0x1e, 0x8d, 0x0e, 0xda, 0x87, 0xfe, 0xb0, 0xd3, 0xf7, 0xfb, 0x7e, 0xa7, - 0xef, 0xfb, 0xfd, 0x01, 0xc1, 0x81, 0x1b, 0xc9, 0x66, 0x27, 0x0c, 0x0e, 0x3b, 0x11, 0xc5, 0x74, - 0x14, 0x09, 0xfc, 0xc6, 0x1a, 0x33, 0xe4, 0x4d, 0x0e, 0x91, 0xbd, 0x96, 0x34, 0xe7, 0x4f, 0x07, - 0xa3, 0x07, 0x1d, 0xea, 0x0e, 0x49, 0x44, 0xf1, 0x30, 0x48, 0x0c, 0x98, 0x7f, 0x03, 0xbf, 0x2f, - 0x90, 0xae, 0xd7, 0x23, 0x8f, 0xfb, 0x98, 0x92, 0x47, 0xf8, 0x54, 0x1a, 0xbc, 0xae, 0x19, 0x24, - 0x0d, 0x39, 0xb8, 0xa1, 0x0d, 0x06, 0x98, 0x52, 0x12, 0x7a, 0x72, 0xec, 0x0b, 0xda, 0x58, 0x74, - 0x4c, 0xe8, 0xe1, 0x91, 0x1c, 0x6a, 0xca, 0xa1, 0x87, 0x83, 0xa1, 0xdf, 0x23, 0x03, 0xbe, 0x90, - 0x48, 0xfc, 0x95, 0x16, 0x57, 0x98, 0x45, 0x30, 0x8a, 0x8e, 0xf8, 0x1f, 0xd9, 0xf9, 0xed, 0x97, - 0x72, 0x79, 0x80, 0x23, 0xd2, 0xe9, 0x91, 0x07, 0xae, 0xe7, 0x52, 0xd7, 0xf7, 0x22, 0xb5, 0x2d, - 0x27, 0xb9, 0x3e, 0xdb, 0x24, 0xd3, 0xf1, 0x69, 0xfd, 0xaa, 0x04, 0x8b, 0xf7, 0xfc, 0x63, 0xd7, - 0x21, 0x0f, 0x47, 0x24, 0xa2, 0x68, 0x0d, 0x2a, 0xdc, 0xc6, 0x34, 0x9a, 0xc6, 0x66, 0xdd, 0x11, - 0x0f, 0xac, 0x77, 0xe0, 0x0e, 0x5d, 0x6a, 0x16, 0x9b, 0xc6, 0xe6, 0xb2, 0x23, 0x1e, 0x10, 0x82, - 0x72, 0x44, 0x49, 0x60, 0x96, 0x9a, 0xc6, 0x66, 0xc9, 0xe1, 0x6d, 0xb4, 0x01, 0x0b, 0xae, 0x47, - 0x49, 0x78, 0x82, 0x07, 0x66, 0x9d, 0xf7, 0xa7, 0xcf, 0xe8, 0x26, 0xd4, 0x22, 0x8a, 0x43, 0xba, - 0x1f, 0x99, 0xe5, 0xa6, 0xb1, 0xb9, 0xd8, 0xdd, 0x68, 0x8b, 0x38, 0xb6, 0x93, 0x38, 0xb6, 0xf7, - 0x93, 0x38, 0xda, 0x0b, 0x4f, 0x62, 0xab, 0xf0, 0xe9, 0x9f, 0x2c, 0xc3, 0x49, 0x40, 0x68, 0x1b, - 0x2a, 0xc4, 0xeb, 0xed, 0x47, 0x66, 0x65, 0x0e, 0xb4, 0x80, 0xa0, 0x6b, 0x50, 0xef, 0xb9, 0x21, - 0x39, 0x64, 0x9c, 0x99, 0xd5, 0xa6, 0xb1, 0xb9, 0xd2, 0xbd, 0xd2, 0x4e, 0xc3, 0xbe, 0x93, 0x0c, - 0x39, 0x13, 0x2b, 0xb6, 0xbc, 0x00, 0xd3, 0x23, 0xb3, 0xc6, 0x99, 0xe0, 0x6d, 0xd4, 0x82, 0x6a, - 0x74, 0x84, 0xc3, 0x5e, 0x64, 0x2e, 0x34, 0x4b, 0x9b, 0x75, 0x1b, 0xce, 0x63, 0x4b, 0xf6, 0x38, - 0xf2, 0x17, 0xfd, 0x08, 0xca, 0xc1, 0x00, 0x7b, 0x26, 0x70, 0x2f, 0x57, 0xdb, 0x0a, 0xe7, 0xf7, - 0x07, 0xd8, 0xb3, 0xdf, 0x1f, 0xc7, 0xd6, 0xbb, 0xaa, 0x14, 0x42, 0xfc, 0x00, 0x7b, 0xb8, 0x33, - 0xf0, 0x8f, 0xdd, 0xce, 0xc9, 0x56, 0x47, 0x8d, 0x24, 0x9b, 0xa8, 0xfd, 0x11, 0x9b, 0x80, 0x41, - 0x1d, 0x3e, 0x71, 0xeb, 0xf7, 0x45, 0x40, 0x2c, 0x66, 0x77, 0xbd, 0x88, 0x62, 0x8f, 0xbe, 0x4a, - 0xe8, 0x6e, 0x40, 0x95, 0x49, 0x66, 0x3f, 0xe2, 0xc1, 0x9b, 0x95, 0x4b, 0x89, 0xd1, 0xc9, 0x2c, - 0xcf, 0x45, 0x66, 0x25, 0x97, 0xcc, 0xea, 0x4b, 0xc9, 0xac, 0xfd, 0xa7, 0xc8, 0x34, 0xa1, 0xcc, - 0x9e, 0xd0, 0x2a, 0x94, 0x42, 0xfc, 0x88, 0x73, 0xb7, 0xe4, 0xb0, 0x66, 0xeb, 0xac, 0x0c, 0x4b, - 0x42, 0x1a, 0x51, 0xe0, 0x7b, 0x11, 0x61, 0xfe, 0xee, 0xf1, 0xda, 0x24, 0x18, 0x96, 0xfe, 0xf2, - 0x1e, 0x47, 0x8e, 0xa0, 0x0f, 0xa0, 0xbc, 0x83, 0x29, 0xe6, 0x6c, 0x2f, 0x76, 0xd7, 0x54, 0x7f, - 0xd9, 0x5c, 0x6c, 0xcc, 0x5e, 0x67, 0x84, 0x9e, 0xc7, 0xd6, 0x4a, 0x0f, 0x53, 0xfc, 0x55, 0x7f, - 0xe8, 0x52, 0x32, 0x0c, 0xe8, 0xa9, 0xc3, 0x91, 0xe8, 0x5d, 0xa8, 0xdf, 0x0e, 0x43, 0x3f, 0xdc, - 0x3f, 0x0d, 0x08, 0x8f, 0x4e, 0xdd, 0xbe, 0x7a, 0x1e, 0x5b, 0x57, 0x48, 0xd2, 0xa9, 0x20, 0x26, - 0x96, 0xe8, 0xcb, 0x50, 0xe1, 0x0f, 0x3c, 0x1e, 0x75, 0xfb, 0xca, 0x79, 0x6c, 0xbd, 0xc6, 0x21, - 0x8a, 0xb9, 0xb0, 0xd0, 0xc3, 0x57, 0x99, 0x29, 0x7c, 0x69, 0x16, 0x55, 0xd5, 0x2c, 0x32, 0xa1, - 0x76, 0x42, 0xc2, 0x88, 0x4d, 0x53, 0xe3, 0xfd, 0xc9, 0x23, 0xba, 0x05, 0xc0, 0x88, 0x71, 0x23, - 0xea, 0x1e, 0x32, 0xad, 0x30, 0x32, 0x96, 0xdb, 0xa2, 0x14, 0x3a, 0x24, 0x1a, 0x0d, 0xa8, 0x8d, - 0x24, 0x0b, 0x8a, 0xa1, 0xa3, 0xb4, 0xd1, 0x2f, 0x0d, 0xa8, 0xed, 0x12, 0xdc, 0x23, 0x61, 0x64, - 0xd6, 0x9b, 0xa5, 0xcd, 0xc5, 0xee, 0x97, 0xda, 0x6a, 0xdd, 0xbb, 0x1f, 0xfa, 0x43, 0x42, 0x8f, - 0xc8, 0x28, 0x4a, 0x02, 0x24, 0xac, 0x6d, 0x6f, 0x1c, 0x5b, 0x64, 0xc6, 0x94, 0x98, 0xa9, 0xdc, - 0x5e, 0xf8, 0xaa, 0xf3, 0xd8, 0x32, 0xbe, 0xe6, 0x24, 0x5e, 0xa2, 0x2e, 0x2c, 0x3c, 0xc2, 0xa1, - 0xe7, 0x7a, 0xfd, 0xc8, 0x04, 0x9e, 0xd1, 0xeb, 0xe7, 0xb1, 0x85, 0x92, 0x3e, 0x25, 0x10, 0xa9, - 0x5d, 0xeb, 0x8f, 0x06, 0x7c, 0x8e, 0x25, 0xc6, 0x1e, 0xf3, 0x27, 0x52, 0xa4, 0x3c, 0xc4, 0xf4, - 0xf0, 0xc8, 0x34, 0xd8, 0x34, 0x8e, 0x78, 0x50, 0xeb, 0x67, 0xf1, 0xdf, 0xaa, 0x9f, 0xa5, 0xf9, - 0xeb, 0x67, 0xa2, 0xdf, 0x72, 0xae, 0x7e, 0x2b, 0x17, 0xe9, 0xb7, 0xf5, 0xb3, 0x92, 0xa8, 0x55, - 0xc9, 0xfa, 0xe6, 0x90, 0xd2, 0x9d, 0x54, 0x4a, 0x25, 0xee, 0x6d, 0x9a, 0xa1, 0x62, 0xae, 0xbb, - 0x3d, 0xe2, 0x51, 0xf7, 0x81, 0x4b, 0xc2, 0x97, 0x08, 0x4a, 0xc9, 0xd2, 0x92, 0x9e, 0xa5, 0x6a, - 0x8a, 0x95, 0x2f, 0x45, 0x8a, 0xe9, 0xba, 0xaa, 0xbc, 0x82, 0xae, 0x5a, 0x7f, 0x2f, 0xc2, 0x3a, - 0x8b, 0xc8, 0x3d, 0x7c, 0x40, 0x06, 0xdf, 0xc3, 0xc3, 0x39, 0xa3, 0xf2, 0x96, 0x12, 0x95, 0xba, - 0x8d, 0xfe, 0xcf, 0xfa, 0x6c, 0xac, 0xff, 0xdc, 0x80, 0x85, 0xe4, 0x03, 0x80, 0xda, 0x00, 0x02, - 0xc6, 0x6b, 0xbc, 0xe0, 0x7a, 0x85, 0x81, 0xc3, 0xb4, 0xd7, 0x51, 0x2c, 0xd0, 0x8f, 0xa1, 0x2a, - 0x9e, 0xa4, 0x16, 0xae, 0x2a, 0x5a, 0xa0, 0x21, 0xc1, 0xc3, 0x5b, 0x3d, 0x1c, 0x50, 0x12, 0xda, - 0xef, 0x33, 0x2f, 0xc6, 0xb1, 0xf5, 0xf6, 0x45, 0x2c, 0x25, 0xfb, 0x4f, 0x89, 0x63, 0xf1, 0x15, - 0xef, 0x74, 0xe4, 0x1b, 0x5a, 0x9f, 0x18, 0xb0, 0xca, 0x1c, 0x65, 0xd4, 0xa4, 0x89, 0xb1, 0x03, - 0x0b, 0xa1, 0x6c, 0x73, 0x77, 0x17, 0xbb, 0xad, 0xb6, 0x4e, 0x6b, 0x0e, 0x95, 0x76, 0xf9, 0x49, - 0x6c, 0x19, 0x4e, 0x8a, 0x44, 0x5b, 0x1a, 0x8d, 0xc5, 0x3c, 0x1a, 0x19, 0xa4, 0xa0, 0x11, 0xf7, - 0xdb, 0x22, 0xa0, 0xbb, 0x6c, 0xff, 0xce, 0xf2, 0x6f, 0x92, 0xaa, 0x8f, 0x33, 0x1e, 0xbd, 0x31, - 0x21, 0x25, 0x6b, 0x6f, 0xdf, 0x1c, 0xc7, 0xd6, 0xf6, 0x4b, 0x72, 0xe7, 0x5f, 0xe0, 0x95, 0x55, - 0xa8, 0xe9, 0x5b, 0xbc, 0x0c, 0xe9, 0xdb, 0xfa, 0x75, 0x11, 0x56, 0x7e, 0xe0, 0x0f, 0x46, 0x43, - 0x92, 0xd2, 0x17, 0x64, 0xe8, 0x33, 0x27, 0xf4, 0xe9, 0xb6, 0xf6, 0xf6, 0x38, 0xb6, 0xae, 0xcf, - 0x4a, 0x9d, 0x8e, 0xbd, 0xd4, 0xb4, 0xfd, 0xb5, 0x08, 0x6b, 0xfb, 0x7e, 0xf0, 0xdd, 0x3d, 0x7e, - 0xc6, 0x53, 0xca, 0xe4, 0x51, 0x86, 0xbc, 0xb5, 0x09, 0x79, 0x0c, 0xf1, 0x21, 0xa6, 0xa1, 0xfb, - 0xd8, 0xbe, 0x3e, 0x8e, 0xad, 0xee, 0xac, 0xc4, 0x4d, 0x70, 0x97, 0x99, 0x34, 0x6d, 0x0f, 0x54, - 0x9a, 0x71, 0x0f, 0xf4, 0x8f, 0x22, 0xac, 0x7f, 0x34, 0xc2, 0x1e, 0x75, 0x07, 0x44, 0x90, 0x9d, - 0x52, 0xfd, 0x93, 0x0c, 0xd5, 0x8d, 0x09, 0xd5, 0x3a, 0x46, 0x92, 0xfe, 0xc1, 0x38, 0xb6, 0x6e, - 0xcc, 0x4a, 0x7a, 0xde, 0x0c, 0xff, 0x73, 0xf4, 0xff, 0xa6, 0x08, 0x2b, 0x7b, 0x62, 0xd7, 0x96, - 0x2c, 0xfc, 0x24, 0x87, 0x76, 0xf5, 0x12, 0x25, 0x38, 0x68, 0xeb, 0x88, 0xf9, 0x8a, 0x84, 0x8e, - 0xbd, 0xd4, 0x45, 0xe2, 0x0f, 0x45, 0x58, 0xdf, 0x21, 0x94, 0x1c, 0x52, 0xd2, 0xbb, 0xe3, 0x92, - 0x81, 0x42, 0xe2, 0xc7, 0x46, 0x86, 0xc5, 0xa6, 0x72, 0xcc, 0xca, 0x05, 0xd9, 0xf6, 0x38, 0xb6, - 0x6e, 0xce, 0xca, 0x63, 0xfe, 0x1c, 0x97, 0x9a, 0xcf, 0xdf, 0x15, 0xe1, 0xf3, 0xe2, 0x88, 0x2e, - 0x6e, 0xdd, 0x26, 0x74, 0xfe, 0x34, 0xc3, 0xa6, 0xa5, 0x96, 0x82, 0x1c, 0x88, 0x7d, 0x6b, 0x1c, - 0x5b, 0xdf, 0x9a, 0xbd, 0x16, 0xe4, 0x4c, 0xf1, 0x5f, 0x93, 0x9b, 0x7c, 0xb7, 0x3f, 0x6f, 0x6e, - 0xea, 0xa0, 0x57, 0xcb, 0x4d, 0x7d, 0x8e, 0x4b, 0xcd, 0xe7, 0x5f, 0xaa, 0xb0, 0xcc, 0xb3, 0x24, - 0xa5, 0xf1, 0x2b, 0x20, 0x8f, 0x47, 0x92, 0x43, 0x94, 0x1c, 0xa9, 0xc3, 0xe0, 0xb0, 0xbd, 0x27, - 0x0f, 0x4e, 0xc2, 0x02, 0xbd, 0x07, 0xd5, 0x88, 0x1f, 0x5c, 0xe5, 0xce, 0xb7, 0x31, 0x7d, 0x37, - 0xa4, 0x1f, 0x91, 0x77, 0x0b, 0x8e, 0xb4, 0x47, 0x37, 0xa0, 0x3a, 0xe0, 0x2c, 0xca, 0x83, 0x7b, - 0x6b, 0x1a, 0x99, 0x3d, 0xca, 0x31, 0xb4, 0xc0, 0xa0, 0xeb, 0x50, 0xe1, 0x5b, 0x6c, 0x79, 0xe7, - 0xaa, 0xbd, 0x36, 0xbb, 0xd1, 0xdd, 0x2d, 0x38, 0xc2, 0x1c, 0x75, 0xa1, 0x1c, 0x84, 0xfe, 0x50, - 0x1e, 0x77, 0xde, 0x98, 0x7e, 0xa7, 0x7a, 0x3e, 0xd8, 0x2d, 0x38, 0xdc, 0x16, 0xbd, 0x03, 0xb5, - 0x88, 0x1f, 0x2c, 0x22, 0x7e, 0x51, 0xc4, 0x76, 0x95, 0x53, 0x30, 0x05, 0x92, 0x98, 0xa2, 0x77, - 0xa0, 0x7a, 0xc2, 0xb7, 0x8d, 0xf2, 0x96, 0x6f, 0x43, 0x05, 0xe9, 0x1b, 0x4a, 0xb6, 0x2e, 0x61, - 0x8b, 0xee, 0xc0, 0x12, 0xf5, 0x83, 0xe3, 0x64, 0x77, 0x26, 0x2f, 0x99, 0x9a, 0x2a, 0x36, 0x6f, - 0xf7, 0xb6, 0x5b, 0x70, 0x34, 0x1c, 0xba, 0x0f, 0xab, 0x0f, 0xb5, 0x6d, 0x00, 0x89, 0xf8, 0xcd, - 0xf5, 0x14, 0xcf, 0xf9, 0x1b, 0x94, 0xdd, 0x82, 0x93, 0x41, 0xa3, 0x1d, 0x58, 0x89, 0xb4, 0x2f, - 0x9c, 0xbc, 0x0a, 0xd6, 0xd6, 0xa5, 0x7f, 0x03, 0x77, 0x0b, 0xce, 0x14, 0x06, 0xdd, 0x83, 0x95, - 0x9e, 0x56, 0xdf, 0xcd, 0xc5, 0xac, 0x57, 0xf9, 0x5f, 0x00, 0x36, 0x9b, 0x8e, 0x45, 0xdf, 0x87, - 0xd5, 0x60, 0xaa, 0xb6, 0x99, 0x4b, 0x7c, 0xbe, 0x2f, 0xea, 0xab, 0xcc, 0x29, 0x82, 0x6c, 0x91, - 0xd3, 0x60, 0xd5, 0x3d, 0x21, 0x71, 0x73, 0xf9, 0x62, 0xf7, 0xf4, 0x22, 0xa0, 0xba, 0x27, 0x46, - 0x6c, 0x98, 0x94, 0xa3, 0xd6, 0x27, 0x55, 0x58, 0x92, 0x32, 0x13, 0xb7, 0x61, 0xdf, 0x48, 0x95, - 0x23, 0x54, 0xf6, 0xe6, 0x45, 0xca, 0xe1, 0xe6, 0x8a, 0x70, 0xbe, 0x9e, 0x0a, 0x47, 0x48, 0x6e, - 0x7d, 0x52, 0xe2, 0xf8, 0x7b, 0x15, 0x84, 0x14, 0xcb, 0x56, 0x22, 0x16, 0xa1, 0xb4, 0xd7, 0xf3, - 0xcf, 0x94, 0x09, 0x4a, 0x2a, 0x65, 0x1b, 0x6a, 0xae, 0xb8, 0x8a, 0xcf, 0xd3, 0x58, 0xf6, 0xa6, - 0x9e, 0xe5, 0xbe, 0x04, 0xa0, 0xad, 0x89, 0x62, 0x84, 0xd0, 0xae, 0x66, 0x15, 0x93, 0x82, 0x12, - 0xc1, 0x5c, 0x4b, 0x05, 0x53, 0x95, 0x98, 0xcc, 0xf9, 0x2b, 0x5d, 0x98, 0x54, 0xcb, 0x6d, 0x58, - 0x4e, 0xf2, 0x8b, 0x0f, 0x49, 0xb9, 0xbc, 0x79, 0xd1, 0xb6, 0x2e, 0xc1, 0xeb, 0x28, 0x74, 0x37, - 0x93, 0x94, 0xf5, 0xe9, 0x4f, 0xf1, 0x74, 0x4a, 0x26, 0x33, 0x4d, 0x67, 0xe4, 0x77, 0xe0, 0xb5, - 0x49, 0x52, 0x09, 0x9f, 0x20, 0xbb, 0xc3, 0xd7, 0xd2, 0x31, 0x99, 0x6a, 0x1a, 0xa8, 0xba, 0x25, - 0x93, 0x71, 0xf1, 0x22, 0xb7, 0x92, 0x54, 0xcc, 0xb8, 0x25, 0x06, 0xd0, 0x2e, 0x2c, 0x0c, 0x09, - 0xc5, 0x3d, 0x4c, 0xb1, 0x59, 0xe3, 0x9f, 0xa5, 0xb7, 0x32, 0x02, 0x91, 0xe8, 0xf6, 0x87, 0xd2, - 0xf0, 0xb6, 0x47, 0xc3, 0x53, 0x79, 0x77, 0x91, 0xa2, 0x37, 0xbe, 0x09, 0xcb, 0x9a, 0x01, 0x5a, - 0x85, 0xd2, 0x31, 0x49, 0xfe, 0x3d, 0xc3, 0x9a, 0x68, 0x0d, 0x2a, 0x27, 0x78, 0x30, 0x22, 0x3c, - 0x3f, 0xeb, 0x8e, 0x78, 0xd8, 0x2e, 0xbe, 0x67, 0xd8, 0x75, 0xa8, 0x85, 0xe2, 0x2d, 0x76, 0xff, - 0xe9, 0xb3, 0x46, 0xe1, 0xb3, 0x67, 0x8d, 0xc2, 0x8b, 0x67, 0x0d, 0xe3, 0xe3, 0xb3, 0x86, 0xf1, - 0x8b, 0xb3, 0x86, 0xf1, 0xe4, 0xac, 0x61, 0x3c, 0x3d, 0x6b, 0x18, 0x7f, 0x3e, 0x6b, 0x18, 0x7f, - 0x3b, 0x6b, 0x14, 0x5e, 0x9c, 0x35, 0x8c, 0x4f, 0x9f, 0x37, 0x0a, 0x4f, 0x9f, 0x37, 0x0a, 0x9f, - 0x3d, 0x6f, 0x14, 0x7e, 0x78, 0x6d, 0xee, 0x2f, 0xe4, 0x41, 0x95, 0x33, 0xb5, 0xf5, 0xcf, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x96, 0xad, 0x71, 0x3d, 0xdd, 0x1d, 0x00, 0x00, + // 1879 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcb, 0x6f, 0x1c, 0x49, + 0x19, 0x9f, 0x9e, 0xa7, 0xe7, 0xf3, 0x23, 0xa6, 0x62, 0xbc, 0x8d, 0x77, 0x77, 0x7a, 0x18, 0x89, + 0x5d, 0x83, 0x60, 0x86, 0xd8, 0xbb, 0x61, 0xd7, 0x84, 0x68, 0xd3, 0xeb, 0x04, 0x67, 0xc9, 0x42, + 0xb6, 0x6d, 0x71, 0xe0, 0x82, 0xca, 0x9e, 0xf2, 0x4c, 0xe3, 0x99, 0xee, 0x4e, 0x57, 0x8d, 0x13, + 0x4b, 0x08, 0xed, 0x3f, 0xb0, 0x62, 0x4f, 0xfc, 0x09, 0x88, 0x1b, 0x17, 0x4e, 0x9c, 0x38, 0x86, + 0x03, 0x52, 0x8e, 0xab, 0x91, 0x68, 0x88, 0x83, 0x10, 0xf2, 0x69, 0x25, 0xae, 0x20, 0xa1, 0x7a, + 0x74, 0x4f, 0xd5, 0xcc, 0x98, 0x8c, 0x13, 0x09, 0xc9, 0xc0, 0xc5, 0x53, 0x8f, 0xef, 0xf7, 0x75, + 0xd5, 0xef, 0x7b, 0xd4, 0x57, 0x65, 0x78, 0x33, 0x3a, 0xea, 0xb4, 0x1e, 0x0c, 0x48, 0xec, 0x93, + 0x58, 0xfc, 0x9e, 0xc4, 0x38, 0xe8, 0x10, 0xad, 0xd9, 0x8c, 0xe2, 0x90, 0x85, 0x08, 0x46, 0x23, + 0x6b, 0x1b, 0x1d, 0x9f, 0x75, 0x07, 0xfb, 0xcd, 0x83, 0xb0, 0xdf, 0xea, 0x84, 0x9d, 0xb0, 0xd5, + 0x09, 0xc3, 0x4e, 0x8f, 0xe0, 0xc8, 0xa7, 0xaa, 0xd9, 0x8a, 0xa3, 0x83, 0x16, 0x65, 0x98, 0x0d, + 0xa8, 0xc4, 0xaf, 0xad, 0x70, 0x41, 0xd1, 0x14, 0x10, 0x35, 0xea, 0x28, 0x71, 0xd1, 0xdb, 0x1f, + 0x1c, 0xb6, 0x98, 0xdf, 0x27, 0x94, 0xe1, 0x7e, 0x94, 0x0a, 0xf0, 0xf5, 0xf5, 0xc2, 0x8e, 0x44, + 0xfa, 0x41, 0x9b, 0x3c, 0xea, 0x60, 0x46, 0x1e, 0xe2, 0x13, 0x25, 0xf0, 0xaa, 0x21, 0x90, 0x36, + 0xd4, 0xe4, 0x9a, 0x31, 0x19, 0x61, 0xc6, 0x48, 0x1c, 0xa8, 0xb9, 0x2f, 0x19, 0x73, 0xf4, 0x88, + 0xb0, 0x83, 0xae, 0x9a, 0xaa, 0xab, 0xa9, 0x07, 0xbd, 0x7e, 0xd8, 0x26, 0x3d, 0xb1, 0x11, 0x2a, + 0xff, 0x2a, 0x89, 0xab, 0x5c, 0x22, 0x1a, 0xd0, 0xae, 0xf8, 0xa3, 0x06, 0xdf, 0x7f, 0x2e, 0x97, + 0xfb, 0x98, 0x92, 0x56, 0x9b, 0x1c, 0xfa, 0x81, 0xcf, 0xfc, 0x30, 0xa0, 0x7a, 0x5b, 0x29, 0xb9, + 0x3e, 0x9b, 0x92, 0x71, 0xfb, 0x34, 0xfe, 0x59, 0x80, 0xf9, 0x7b, 0xe1, 0x91, 0xef, 0x91, 0x07, + 0x03, 0x42, 0x19, 0x5a, 0x81, 0x92, 0x90, 0xb1, 0xad, 0xba, 0xb5, 0x5e, 0xf5, 0x64, 0x87, 0x8f, + 0xf6, 0xfc, 0xbe, 0xcf, 0xec, 0x7c, 0xdd, 0x5a, 0x5f, 0xf4, 0x64, 0x07, 0x21, 0x28, 0x52, 0x46, + 0x22, 0xbb, 0x50, 0xb7, 0xd6, 0x0b, 0x9e, 0x68, 0xa3, 0x35, 0x98, 0xf3, 0x03, 0x46, 0xe2, 0x63, + 0xdc, 0xb3, 0xab, 0x62, 0x3c, 0xeb, 0xa3, 0x9b, 0x50, 0xa1, 0x0c, 0xc7, 0x6c, 0x8f, 0xda, 0xc5, + 0xba, 0xb5, 0x3e, 0xbf, 0xb1, 0xd6, 0x94, 0x76, 0x6c, 0xa6, 0x76, 0x6c, 0xee, 0xa5, 0x76, 0x74, + 0xe7, 0x1e, 0x27, 0x4e, 0xee, 0xd3, 0x3f, 0x39, 0x96, 0x97, 0x82, 0xd0, 0x16, 0x94, 0x48, 0xd0, + 0xde, 0xa3, 0x76, 0xe9, 0x02, 0x68, 0x09, 0x41, 0xd7, 0xa0, 0xda, 0xf6, 0x63, 0x72, 0xc0, 0x39, + 0xb3, 0xcb, 0x75, 0x6b, 0x7d, 0x69, 0xe3, 0x6a, 0x33, 0x33, 0xfb, 0x76, 0x3a, 0xe5, 0x8d, 0xa4, + 0xf8, 0xf6, 0x22, 0xcc, 0xba, 0x76, 0x45, 0x30, 0x21, 0xda, 0xa8, 0x01, 0x65, 0xda, 0xc5, 0x71, + 0x9b, 0xda, 0x73, 0xf5, 0xc2, 0x7a, 0xd5, 0x85, 0xb3, 0xc4, 0x51, 0x23, 0x9e, 0xfa, 0x45, 0x3f, + 0x86, 0x62, 0xd4, 0xc3, 0x81, 0x0d, 0x62, 0x95, 0xcb, 0x4d, 0x8d, 0xf3, 0xfb, 0x3d, 0x1c, 0xb8, + 0xef, 0x0e, 0x13, 0xe7, 0x6d, 0x3d, 0x14, 0x62, 0x7c, 0x88, 0x03, 0xdc, 0xea, 0x85, 0x47, 0x7e, + 0xeb, 0x78, 0xb3, 0xa5, 0x5b, 0x92, 0x2b, 0x6a, 0x7e, 0xc4, 0x15, 0x70, 0xa8, 0x27, 0x14, 0xa3, + 0x0f, 0x60, 0x9e, 0xb2, 0x30, 0x26, 0xef, 0x77, 0x07, 0xc1, 0x11, 0xb5, 0xe7, 0xc5, 0x77, 0x5e, + 0x19, 0xed, 0x46, 0x8c, 0x7b, 0xe4, 0xf0, 0xbb, 0x71, 0x38, 0x88, 0xdc, 0x2b, 0x67, 0x89, 0xa3, + 0xcb, 0x7b, 0x7a, 0xa7, 0xf1, 0x8b, 0x02, 0x20, 0x6e, 0xff, 0xbb, 0x01, 0x65, 0x38, 0x60, 0x2f, + 0xe2, 0x06, 0x37, 0xa0, 0xcc, 0xc3, 0x6f, 0x8f, 0x0a, 0x47, 0x98, 0xd5, 0x2e, 0x0a, 0x63, 0x1a, + 0xa6, 0x78, 0x21, 0xc3, 0x94, 0xa6, 0x1a, 0xa6, 0xfc, 0x5c, 0xc3, 0x54, 0xfe, 0x43, 0x86, 0x99, + 0x7b, 0x19, 0xc3, 0xd8, 0x50, 0xe4, 0x9a, 0xd1, 0x32, 0x14, 0x62, 0xfc, 0x50, 0xd8, 0x61, 0xc1, + 0xe3, 0xcd, 0xc6, 0x69, 0x11, 0x16, 0x64, 0xc8, 0xd2, 0x28, 0x0c, 0x28, 0xe1, 0x7b, 0xdf, 0x15, + 0x39, 0x53, 0x5a, 0x4b, 0xed, 0x5d, 0x8c, 0x78, 0x6a, 0x06, 0xbd, 0x07, 0xc5, 0x6d, 0xcc, 0xb0, + 0xb0, 0xdc, 0xfc, 0xc6, 0x8a, 0xbe, 0x77, 0xae, 0x8b, 0xcf, 0xb9, 0xab, 0xdc, 0x38, 0x67, 0x89, + 0xb3, 0xd4, 0xc6, 0x0c, 0x7f, 0x3d, 0xec, 0xfb, 0x8c, 0xf4, 0x23, 0x76, 0xe2, 0x09, 0x24, 0x7a, + 0x1b, 0xaa, 0xb7, 0xe3, 0x38, 0x8c, 0xf7, 0x4e, 0x22, 0x22, 0x2c, 0x5d, 0x75, 0x5f, 0x39, 0x4b, + 0x9c, 0xab, 0x24, 0x1d, 0xd4, 0x10, 0x23, 0x49, 0xf4, 0x55, 0x28, 0x89, 0x8e, 0xb0, 0x6d, 0xd5, + 0xbd, 0x7a, 0x96, 0x38, 0x57, 0x04, 0x44, 0x13, 0x97, 0x12, 0xa6, 0x2b, 0x94, 0x66, 0x72, 0x85, + 0xcc, 0x23, 0xcb, 0xba, 0x47, 0xda, 0x50, 0x39, 0x26, 0x31, 0xe5, 0x6a, 0x2a, 0x62, 0x3c, 0xed, + 0xa2, 0x5b, 0x00, 0x9c, 0x18, 0x9f, 0x32, 0xff, 0x20, 0x35, 0xd0, 0x62, 0x53, 0xa6, 0x68, 0x8f, + 0xd0, 0x41, 0x8f, 0xb9, 0x48, 0xb1, 0xa0, 0x09, 0x7a, 0x5a, 0x1b, 0xfd, 0xda, 0x82, 0xca, 0x0e, + 0xc1, 0x6d, 0x12, 0x53, 0xbb, 0x5a, 0x2f, 0xac, 0xcf, 0x6f, 0x7c, 0xa5, 0xa9, 0xe7, 0xe3, 0xfb, + 0x71, 0xd8, 0x27, 0xac, 0x4b, 0x06, 0x34, 0x35, 0x90, 0x94, 0x76, 0x83, 0x61, 0xe2, 0x90, 0x19, + 0xdd, 0x6b, 0xa6, 0x63, 0xe0, 0xdc, 0x4f, 0x9d, 0x25, 0x8e, 0xf5, 0x0d, 0x2f, 0x5d, 0x25, 0xda, + 0x80, 0xb9, 0x87, 0x38, 0x0e, 0xfc, 0xa0, 0x43, 0x6d, 0x10, 0xd1, 0xb1, 0x7a, 0x96, 0x38, 0x28, + 0x1d, 0xd3, 0x0c, 0x91, 0xc9, 0x35, 0xfe, 0x68, 0xc1, 0x17, 0xb8, 0x63, 0xec, 0xf2, 0xf5, 0x50, + 0x2d, 0x2d, 0xf4, 0x31, 0x3b, 0xe8, 0xda, 0x16, 0x57, 0xe3, 0xc9, 0x8e, 0x9e, 0xd7, 0xf3, 0x2f, + 0x95, 0xd7, 0x0b, 0x17, 0xcf, 0xeb, 0x69, 0x2e, 0x28, 0x4e, 0xcd, 0x05, 0xa5, 0xf3, 0x72, 0x41, + 0xe3, 0xe7, 0x2a, 0xef, 0xa5, 0xfb, 0xbb, 0x40, 0x28, 0xdd, 0xc9, 0x42, 0xa9, 0x20, 0x56, 0x9b, + 0x79, 0xa8, 0xd4, 0x75, 0xb7, 0x4d, 0x02, 0xe6, 0x1f, 0xfa, 0x24, 0x7e, 0x4e, 0x40, 0x69, 0x5e, + 0x5a, 0x30, 0xbd, 0x54, 0x77, 0xb1, 0xe2, 0xa5, 0x70, 0x31, 0x33, 0xae, 0x4a, 0x2f, 0x10, 0x57, + 0x8d, 0xbf, 0xe7, 0x61, 0x95, 0x5b, 0xe4, 0x1e, 0xde, 0x27, 0xbd, 0xef, 0xe3, 0xfe, 0x05, 0xad, + 0xf2, 0x86, 0x66, 0x95, 0xaa, 0x8b, 0xfe, 0xcf, 0xfa, 0x6c, 0xac, 0xff, 0xd2, 0x82, 0xb9, 0xf4, + 0x00, 0x40, 0x4d, 0x00, 0x09, 0x13, 0x39, 0x5e, 0x72, 0xbd, 0xc4, 0xc1, 0x71, 0x36, 0xea, 0x69, + 0x12, 0xe8, 0x27, 0x50, 0x96, 0x3d, 0x15, 0x0b, 0xda, 0x51, 0xb7, 0xcb, 0x62, 0x82, 0xfb, 0xb7, + 0xda, 0x38, 0x62, 0x24, 0x76, 0xdf, 0xe5, 0xab, 0x18, 0x26, 0xce, 0x9b, 0xe7, 0xb1, 0x94, 0xd6, + 0xc5, 0x0a, 0xc7, 0xed, 0x2b, 0xbf, 0xe9, 0xa9, 0x2f, 0x34, 0x3e, 0xb1, 0x60, 0x99, 0x2f, 0x94, + 0x53, 0x93, 0x39, 0xc6, 0x36, 0xcc, 0xc5, 0xaa, 0x2d, 0x96, 0x3b, 0xbf, 0xd1, 0x68, 0x9a, 0xb4, + 0x4e, 0xa1, 0xd2, 0x2d, 0x3e, 0x4e, 0x1c, 0xcb, 0xcb, 0x90, 0x68, 0xd3, 0xa0, 0x31, 0x3f, 0x8d, + 0x46, 0x0e, 0xc9, 0x19, 0xc4, 0xfd, 0x2e, 0x0f, 0xe8, 0x2e, 0xbf, 0x57, 0x70, 0xff, 0x1b, 0xb9, + 0xea, 0xa3, 0x89, 0x15, 0xbd, 0x36, 0x22, 0x65, 0x52, 0xde, 0xbd, 0x39, 0x4c, 0x9c, 0xad, 0xe7, + 0xf8, 0xce, 0xbf, 0xc1, 0x6b, 0xbb, 0xd0, 0xdd, 0x37, 0x7f, 0x19, 0xdc, 0xb7, 0xf1, 0x9b, 0x3c, + 0x2c, 0xfd, 0x30, 0xec, 0x0d, 0xfa, 0x24, 0xa3, 0x2f, 0x9a, 0xa0, 0xcf, 0x1e, 0xd1, 0x67, 0xca, + 0xba, 0x5b, 0xc3, 0xc4, 0xb9, 0x3e, 0x2b, 0x75, 0x26, 0xf6, 0x52, 0xd3, 0xf6, 0xd7, 0x3c, 0xac, + 0xec, 0x85, 0xd1, 0xf7, 0x76, 0xc5, 0xdd, 0x53, 0x4b, 0x93, 0xdd, 0x09, 0xf2, 0x56, 0x46, 0xe4, + 0x71, 0xc4, 0x87, 0x98, 0xc5, 0xfe, 0x23, 0xf7, 0xfa, 0x30, 0x71, 0x36, 0x66, 0x25, 0x6e, 0x84, + 0xbb, 0xcc, 0xa4, 0x19, 0x35, 0x50, 0x61, 0xc6, 0x1a, 0xe8, 0x1f, 0x79, 0x58, 0xfd, 0x68, 0x80, + 0x03, 0xe6, 0xf7, 0x88, 0x24, 0x3b, 0xa3, 0xfa, 0xa7, 0x13, 0x54, 0xd7, 0x46, 0x54, 0x9b, 0x18, + 0x45, 0xfa, 0x7b, 0xc3, 0xc4, 0xb9, 0x31, 0x2b, 0xe9, 0xd3, 0x34, 0xfc, 0xcf, 0xd1, 0xff, 0xdb, + 0x3c, 0x2c, 0xed, 0xca, 0xaa, 0x2d, 0xdd, 0xf8, 0xf1, 0x14, 0xda, 0xf5, 0xc7, 0x9d, 0x68, 0xbf, + 0x69, 0x22, 0x2e, 0x96, 0x24, 0x4c, 0xec, 0xa5, 0x4e, 0x12, 0x7f, 0xc8, 0xc3, 0xea, 0x36, 0x61, + 0xe4, 0x80, 0x91, 0xf6, 0x1d, 0x9f, 0xf4, 0x34, 0x12, 0x3f, 0xb6, 0x26, 0x58, 0xac, 0x6b, 0xd7, + 0xac, 0xa9, 0x20, 0xd7, 0x1d, 0x26, 0xce, 0xcd, 0x59, 0x79, 0x9c, 0xae, 0xe3, 0x52, 0xf3, 0xf9, + 0xfb, 0x3c, 0x7c, 0x51, 0x5e, 0xf7, 0xe5, 0x6b, 0xe0, 0x88, 0xce, 0x9f, 0x4d, 0xb0, 0xe9, 0xe8, + 0xa9, 0x60, 0x0a, 0xc4, 0xbd, 0x35, 0x4c, 0x9c, 0xef, 0xcc, 0x9e, 0x0b, 0xa6, 0xa8, 0xf8, 0xaf, + 0xf1, 0x4d, 0x51, 0xed, 0x5f, 0xd4, 0x37, 0x4d, 0xd0, 0x8b, 0xf9, 0xa6, 0xa9, 0xe3, 0x52, 0xf3, + 0xf9, 0x97, 0x32, 0x2c, 0x0a, 0x2f, 0xc9, 0x68, 0xfc, 0x1a, 0xa8, 0xeb, 0x91, 0xe2, 0x10, 0xa5, + 0x57, 0xea, 0x38, 0x3a, 0x68, 0xee, 0xaa, 0x8b, 0x93, 0x94, 0x40, 0xef, 0x40, 0x99, 0x8a, 0x8b, + 0xab, 0xaa, 0x7c, 0x6b, 0xe3, 0x6f, 0x43, 0xe6, 0x15, 0x79, 0x27, 0xe7, 0x29, 0x79, 0x74, 0x03, + 0xca, 0x3d, 0xc1, 0xa2, 0xba, 0xb8, 0x37, 0xc6, 0x91, 0x93, 0x57, 0x39, 0x8e, 0x96, 0x18, 0x74, + 0x1d, 0x4a, 0xa2, 0xc4, 0x56, 0x6f, 0xc1, 0xc6, 0x67, 0x27, 0x0b, 0xdd, 0x9d, 0x9c, 0x27, 0xc5, + 0xd1, 0x06, 0x14, 0xa3, 0x38, 0xec, 0xab, 0xeb, 0xce, 0x6b, 0xe3, 0xdf, 0xd4, 0xef, 0x07, 0x3b, + 0x39, 0x4f, 0xc8, 0xa2, 0xb7, 0xa0, 0x42, 0xc5, 0xc5, 0x82, 0x8a, 0x87, 0x22, 0x5e, 0x55, 0x8e, + 0xc1, 0x34, 0x48, 0x2a, 0x8a, 0xde, 0x82, 0xf2, 0xb1, 0x28, 0x1b, 0xd5, 0x8b, 0xe1, 0x9a, 0x0e, + 0x32, 0x0b, 0x4a, 0xbe, 0x2f, 0x29, 0x8b, 0xee, 0xc0, 0x02, 0x0b, 0xa3, 0xa3, 0xb4, 0x3a, 0x53, + 0x8f, 0x4c, 0x75, 0x1d, 0x3b, 0xad, 0x7a, 0xdb, 0xc9, 0x79, 0x06, 0x0e, 0xdd, 0x87, 0xe5, 0x07, + 0x46, 0x19, 0x40, 0xa8, 0x78, 0x51, 0x1f, 0xe3, 0x79, 0x7a, 0x81, 0xb2, 0x93, 0xf3, 0x26, 0xd0, + 0x68, 0x1b, 0x96, 0xa8, 0x71, 0xc2, 0xa9, 0x27, 0x6a, 0x63, 0x5f, 0xe6, 0x19, 0xb8, 0x93, 0xf3, + 0xc6, 0x30, 0xe8, 0x1e, 0x2c, 0xb5, 0x8d, 0xfc, 0xae, 0x1e, 0xa0, 0x8d, 0x55, 0x4d, 0x3f, 0x01, + 0xb8, 0x36, 0x13, 0x8b, 0x7e, 0x00, 0xcb, 0xd1, 0x58, 0x6e, 0xb3, 0x17, 0x84, 0xbe, 0x2f, 0x9b, + 0xbb, 0x9c, 0x92, 0x04, 0xf9, 0x26, 0xc7, 0xc1, 0xfa, 0xf2, 0x64, 0x88, 0xdb, 0x8b, 0xe7, 0x2f, + 0xcf, 0x4c, 0x02, 0xfa, 0xf2, 0xe4, 0x8c, 0x0b, 0xa3, 0x74, 0xd4, 0xf8, 0xa4, 0x0c, 0x0b, 0x2a, + 0xcc, 0xe4, 0x6b, 0xd8, 0xb7, 0xb2, 0xc8, 0x91, 0x51, 0xf6, 0xfa, 0x79, 0x91, 0x23, 0xc4, 0xb5, + 0xc0, 0xf9, 0x66, 0x16, 0x38, 0x32, 0xe4, 0x56, 0x47, 0x29, 0x4e, 0x7c, 0x57, 0x43, 0xa8, 0x60, + 0xd9, 0x4c, 0x83, 0x45, 0x46, 0xda, 0xab, 0xd3, 0xef, 0x94, 0x29, 0x4a, 0x45, 0xca, 0x16, 0x54, + 0x7c, 0xf9, 0xac, 0x3f, 0x2d, 0xc6, 0x26, 0x5f, 0xfd, 0xb9, 0xef, 0x2b, 0x00, 0xda, 0x1c, 0x45, + 0x4c, 0x49, 0x3d, 0x63, 0x4f, 0x44, 0x4c, 0x06, 0x4a, 0x03, 0xe6, 0x5a, 0x16, 0x30, 0xe5, 0xf1, + 0xa7, 0xef, 0x34, 0x5c, 0xb2, 0x8d, 0xa9, 0x68, 0xb9, 0x0d, 0x8b, 0xa9, 0x7f, 0x89, 0x29, 0x15, + 0x2e, 0xaf, 0x9f, 0x57, 0xd6, 0xa5, 0x78, 0x13, 0x85, 0xee, 0x4e, 0x38, 0x65, 0x75, 0xfc, 0x28, + 0x1e, 0x77, 0xc9, 0x54, 0xd3, 0xb8, 0x47, 0x7e, 0x00, 0x57, 0x46, 0x4e, 0x25, 0xd7, 0x04, 0x93, + 0x15, 0xbe, 0xe1, 0x8e, 0xa9, 0xaa, 0x71, 0xa0, 0xbe, 0x2c, 0xe5, 0x8c, 0xf3, 0xe7, 0x2d, 0x2b, + 0x75, 0xc5, 0x89, 0x65, 0xc9, 0x09, 0xb4, 0x03, 0x73, 0x7d, 0xc2, 0x70, 0x1b, 0x33, 0x6c, 0x57, + 0xc4, 0xb1, 0xf4, 0xc6, 0x44, 0x80, 0x28, 0x74, 0xf3, 0x43, 0x25, 0x78, 0x3b, 0x60, 0xf1, 0x89, + 0x7a, 0xbb, 0xc8, 0xd0, 0x6b, 0xdf, 0x86, 0x45, 0x43, 0x00, 0x2d, 0x43, 0xe1, 0x88, 0xa4, 0xff, + 0xea, 0xe1, 0x4d, 0xb4, 0x02, 0xa5, 0x63, 0xdc, 0x1b, 0x10, 0xe1, 0x9f, 0x55, 0x4f, 0x76, 0xb6, + 0xf2, 0xef, 0x58, 0x6e, 0x15, 0x2a, 0xb1, 0xfc, 0x8a, 0xdb, 0x79, 0xf2, 0xb4, 0x96, 0xfb, 0xec, + 0x69, 0x2d, 0xf7, 0xf9, 0xd3, 0x9a, 0xf5, 0xf1, 0x69, 0xcd, 0xfa, 0xd5, 0x69, 0xcd, 0x7a, 0x7c, + 0x5a, 0xb3, 0x9e, 0x9c, 0xd6, 0xac, 0x3f, 0x9f, 0xd6, 0xac, 0xbf, 0x9d, 0xd6, 0x72, 0x9f, 0x9f, + 0xd6, 0xac, 0x4f, 0x9f, 0xd5, 0x72, 0x4f, 0x9e, 0xd5, 0x72, 0x9f, 0x3d, 0xab, 0xe5, 0x7e, 0x74, + 0xed, 0xc2, 0x27, 0xe4, 0x7e, 0x59, 0x30, 0xb5, 0xf9, 0xaf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd2, + 0x2b, 0x01, 0x83, 0x75, 0x1e, 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -1627,6 +1650,9 @@ func (this *LokiRequest) Equal(that interface{}) bool { } else if !this.Plan.Equal(*that1.Plan) { return false } + if !this.StoreChunks.Equal(that1.StoreChunks) { + return false + } return true } func (this *LokiInstantRequest) Equal(that interface{}) bool { @@ -1678,6 +1704,9 @@ func (this *LokiInstantRequest) Equal(that interface{}) bool { } else if !this.Plan.Equal(*that1.Plan) { return false } + if !this.StoreChunks.Equal(that1.StoreChunks) { + return false + } return true } func (this *Plan) Equal(that interface{}) bool { @@ -2869,7 +2898,7 @@ func (this *LokiRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 14) + s := make([]string, 0, 15) s = append(s, "&queryrange.LokiRequest{") s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") @@ -2881,6 +2910,9 @@ func (this *LokiRequest) GoString() string { s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n") s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") + if this.StoreChunks != nil { + s = append(s, "StoreChunks: "+fmt.Sprintf("%#v", this.StoreChunks)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -2888,7 +2920,7 @@ func (this *LokiInstantRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&queryrange.LokiInstantRequest{") s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") @@ -2897,6 +2929,9 @@ func (this *LokiInstantRequest) GoString() string { s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n") s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") + if this.StoreChunks != nil { + s = append(s, "StoreChunks: "+fmt.Sprintf("%#v", this.StoreChunks)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -3335,6 +3370,18 @@ func (m *LokiRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.StoreChunks != nil { + { + size, err := m.StoreChunks.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } if m.Plan != nil { { size := m.Plan.Size() @@ -3373,21 +3420,21 @@ func (m *LokiRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x30 } - n2, err2 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) - if err2 != nil { - return 0, err2 - } - i -= n2 - i = encodeVarintQueryrange(dAtA, i, uint64(n2)) - i-- - dAtA[i] = 0x2a - n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) + n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) if err3 != nil { return 0, err3 } i -= n3 i = encodeVarintQueryrange(dAtA, i, uint64(n3)) i-- + dAtA[i] = 0x2a + n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) + if err4 != nil { + return 0, err4 + } + i -= n4 + i = encodeVarintQueryrange(dAtA, i, uint64(n4)) + i-- dAtA[i] = 0x22 if m.Step != 0 { i = encodeVarintQueryrange(dAtA, i, uint64(m.Step)) @@ -3429,6 +3476,18 @@ func (m *LokiInstantRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.StoreChunks != nil { + { + size, err := m.StoreChunks.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQueryrange(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } if m.Plan != nil { { size := m.Plan.Size() @@ -3462,12 +3521,12 @@ func (m *LokiInstantRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x20 } - n5, err5 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.TimeTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.TimeTs):]) - if err5 != nil { - return 0, err5 + n7, err7 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.TimeTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.TimeTs):]) + if err7 != nil { + return 0, err7 } - i -= n5 - i = encodeVarintQueryrange(dAtA, i, uint64(n5)) + i -= n7 + i = encodeVarintQueryrange(dAtA, i, uint64(n7)) i-- dAtA[i] = 0x1a if m.Limit != 0 { @@ -3653,20 +3712,20 @@ func (m *LokiSeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x22 } - n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) - if err8 != nil { - return 0, err8 + n10, err10 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) + if err10 != nil { + return 0, err10 } - i -= n8 - i = encodeVarintQueryrange(dAtA, i, uint64(n8)) + i -= n10 + i = encodeVarintQueryrange(dAtA, i, uint64(n10)) i-- dAtA[i] = 0x1a - n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) - if err9 != nil { - return 0, err9 + n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) + if err11 != nil { + return 0, err11 } - i -= n9 - i = encodeVarintQueryrange(dAtA, i, uint64(n9)) + i -= n11 + i = encodeVarintQueryrange(dAtA, i, uint64(n11)) i-- dAtA[i] = 0x12 if len(m.Match) > 0 { @@ -4907,6 +4966,10 @@ func (m *LokiRequest) Size() (n int) { l = m.Plan.Size() n += 1 + l + sovQueryrange(uint64(l)) } + if m.StoreChunks != nil { + l = m.StoreChunks.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } return n } @@ -4942,6 +5005,10 @@ func (m *LokiInstantRequest) Size() (n int) { l = m.Plan.Size() n += 1 + l + sovQueryrange(uint64(l)) } + if m.StoreChunks != nil { + l = m.StoreChunks.Size() + n += 1 + l + sovQueryrange(uint64(l)) + } return n } @@ -5613,6 +5680,7 @@ func (this *LokiRequest) String() string { `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Interval:` + fmt.Sprintf("%v", this.Interval) + `,`, `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, + `StoreChunks:` + strings.Replace(fmt.Sprintf("%v", this.StoreChunks), "ChunkRefGroup", "logproto.ChunkRefGroup", 1) + `,`, `}`, }, "") return s @@ -5629,6 +5697,7 @@ func (this *LokiInstantRequest) String() string { `Path:` + fmt.Sprintf("%v", this.Path) + `,`, `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, + `StoreChunks:` + strings.Replace(fmt.Sprintf("%v", this.StoreChunks), "ChunkRefGroup", "logproto.ChunkRefGroup", 1) + `,`, `}`, }, "") return s @@ -6384,6 +6453,42 @@ func (m *LokiRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreChunks", 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.StoreChunks == nil { + m.StoreChunks = &logproto.ChunkRefGroup{} + } + if err := m.StoreChunks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) @@ -6640,6 +6745,42 @@ func (m *LokiInstantRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreChunks", 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.StoreChunks == nil { + m.StoreChunks = &logproto.ChunkRefGroup{} + } + if err := m.StoreChunks.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + 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 91d248842624..46513c9f4e1e 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -36,6 +36,9 @@ message LokiRequest { string path = 7; repeated string shards = 8 [(gogoproto.jsontag) = "shards"]; Plan plan = 10 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan"]; + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + logproto.ChunkRefGroup storeChunks = 11 [(gogoproto.jsontag) = "storeChunks"]; } message LokiInstantRequest { @@ -49,6 +52,9 @@ message LokiInstantRequest { string path = 5; repeated string shards = 6 [(gogoproto.jsontag) = "shards"]; Plan plan = 7 [(gogoproto.customtype) = "github.com/grafana/loki/v3/pkg/querier/plan.QueryPlan"]; + // If populated, these represent the chunk references that the querier should + // use to fetch the data, plus any other chunks reported by ingesters. + logproto.ChunkRefGroup storeChunks = 8 [(gogoproto.jsontag) = "storeChunks"]; } message Plan { diff --git a/pkg/querier/queryrange/shard_resolver.go b/pkg/querier/queryrange/shard_resolver.go index fbfeb6b8cfd3..9950190ae0eb 100644 --- a/pkg/querier/queryrange/shard_resolver.go +++ b/pkg/querier/queryrange/shard_resolver.go @@ -3,7 +3,6 @@ package queryrange import ( "context" "fmt" - "net/http" strings "strings" "time" @@ -12,7 +11,6 @@ import ( "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" @@ -218,7 +216,11 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) { return factor, bytesPerShard, nil } -func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ([]logproto.Shard, error) { +func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerShard uint64) ( + []logproto.Shard, + []logproto.ChunkRefGroup, + error, +) { sp, ctx := opentracing.StartSpanFromContext(r.ctx, "dynamicShardResolver.ShardingRanges") defer sp.Finish() log := spanlogger.FromContext(ctx) @@ -231,7 +233,7 @@ func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerSh // of binary ops, but I'm putting in the loop for completion grps, err := syntax.MatcherGroups(expr) if err != nil { - return nil, err + return nil, nil, err } for _, grp := range grps { @@ -260,42 +262,31 @@ func (r *dynamicShardResolver) ShardingRanges(expr syntax.Expr, targetBytesPerSh }) 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) - + return nil, 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 nil, nil, fmt.Errorf("expected *ShardsResponse while querying index, got %T", resp) } // accumulate stats logqlstats.JoinResults(ctx, casted.Response.Statistics) + var refs int + for _, x := range casted.Response.ChunkGroups { + refs += len(x.Refs) + } + 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, + "post_filter_chunks", casted.Response.Statistics.Index.PostFilterChunks, + "precomputed_refs", refs, ) - return casted.Response.Shards, err + return casted.Response.Shards, casted.Response.ChunkGroups, err } diff --git a/pkg/ruler/evaluator_local.go b/pkg/ruler/evaluator_local.go index dbbd8c813e1f..00faabd7ea49 100644 --- a/pkg/ruler/evaluator_local.go +++ b/pkg/ruler/evaluator_local.go @@ -37,6 +37,7 @@ func (l *LocalEvaluator) Eval(ctx context.Context, qs string, now time.Time) (*l logproto.FORWARD, 0, nil, + nil, ) if err != nil { return nil, err diff --git a/pkg/storage/async_store.go b/pkg/storage/async_store.go index 31fea0ced088..8d104d702b8b 100644 --- a/pkg/storage/async_store.go +++ b/pkg/storage/async_store.go @@ -63,7 +63,13 @@ func (a *AsyncStore) shouldQueryIngesters(through, now model.Time) bool { return a.queryIngestersWithin == 0 || through.After(now.Add(-a.queryIngestersWithin)) } -func (a *AsyncStore) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (a *AsyncStore) GetChunks(ctx context.Context, + userID string, + from, + through model.Time, + predicate chunk.Predicate, + storeChunksOverride *logproto.ChunkRefGroup, +) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { spanLogger := spanlogger.FromContext(ctx) errs := make(chan error) @@ -72,7 +78,7 @@ func (a *AsyncStore) GetChunks(ctx context.Context, userID string, from, through var fetchers []*fetcher.Fetcher go func() { var err error - storeChunks, fetchers, err = a.Store.GetChunks(ctx, userID, from, through, predicate) + storeChunks, fetchers, err = a.Store.GetChunks(ctx, userID, from, through, predicate, storeChunksOverride) errs <- err }() diff --git a/pkg/storage/async_store_test.go b/pkg/storage/async_store_test.go index 366a7d6f1f1d..a85b33ecccef 100644 --- a/pkg/storage/async_store_test.go +++ b/pkg/storage/async_store_test.go @@ -29,8 +29,8 @@ func newStoreMock() *storeMock { return &storeMock{} } -func (s *storeMock) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { - args := s.Called(ctx, userID, from, through, predicate) +func (s *storeMock) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate, storeChunksOverride *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { + args := s.Called(ctx, userID, from, through, predicate, storeChunksOverride) return args.Get(0).([][]chunk.Chunk), args.Get(1).([]*fetcher.Fetcher), args.Error(2) } @@ -233,7 +233,7 @@ func TestAsyncStore_mergeIngesterAndStoreChunks(t *testing.T) { } { t.Run(tc.name, func(t *testing.T) { store := newStoreMock() - store.On("GetChunks", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(tc.storeChunks, tc.storeFetcher, nil) + store.On("GetChunks", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(tc.storeChunks, tc.storeFetcher, nil) store.On("GetChunkFetcher", mock.Anything).Return(tc.ingesterFetcher) ingesterQuerier := newIngesterQuerierMock() @@ -242,7 +242,7 @@ func TestAsyncStore_mergeIngesterAndStoreChunks(t *testing.T) { asyncStoreCfg := AsyncStoreCfg{IngesterQuerier: ingesterQuerier} asyncStore := NewAsyncStore(asyncStoreCfg, store, config.SchemaConfig{}) - chunks, fetchers, err := asyncStore.GetChunks(context.Background(), "fake", model.Now(), model.Now(), chunk.NewPredicate(nil, nil)) + chunks, fetchers, err := asyncStore.GetChunks(context.Background(), "fake", model.Now(), model.Now(), chunk.NewPredicate(nil, nil), nil) require.NoError(t, err) require.Equal(t, tc.expectedChunks, chunks) @@ -293,7 +293,7 @@ func TestAsyncStore_QueryIngestersWithin(t *testing.T) { t.Run(tc.name, func(t *testing.T) { store := newStoreMock() - store.On("GetChunks", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([][]chunk.Chunk{}, []*fetcher.Fetcher{}, nil) + store.On("GetChunks", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([][]chunk.Chunk{}, []*fetcher.Fetcher{}, nil) ingesterQuerier := newIngesterQuerierMock() ingesterQuerier.On("GetChunkIDs", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]string{}, nil) @@ -304,7 +304,7 @@ func TestAsyncStore_QueryIngestersWithin(t *testing.T) { } asyncStore := NewAsyncStore(asyncStoreCfg, store, config.SchemaConfig{}) - _, _, err := asyncStore.GetChunks(context.Background(), "fake", tc.queryFrom, tc.queryThrough, chunk.NewPredicate(nil, nil)) + _, _, err := asyncStore.GetChunks(context.Background(), "fake", tc.queryFrom, tc.queryThrough, chunk.NewPredicate(nil, nil), nil) require.NoError(t, err) expectedNumCalls := 0 diff --git a/pkg/storage/batch.go b/pkg/storage/batch.go index 21f321a6f8a5..ff47345670ac 100644 --- a/pkg/storage/batch.go +++ b/pkg/storage/batch.go @@ -28,10 +28,11 @@ import ( ) type ChunkMetrics struct { - refs *prometheus.CounterVec - series *prometheus.CounterVec - chunks *prometheus.CounterVec - batches *prometheus.HistogramVec + refs *prometheus.CounterVec + refsBypassed prometheus.Counter + series *prometheus.CounterVec + chunks *prometheus.CounterVec + batches *prometheus.HistogramVec } const ( @@ -52,6 +53,12 @@ func NewChunkMetrics(r prometheus.Registerer, maxBatchSize int) *ChunkMetrics { Name: "chunk_refs_total", Help: "Number of chunks refs downloaded, partitioned by whether they intersect the query bounds.", }, []string{"status"}), + refsBypassed: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: constants.Loki, + Subsystem: "store", + Name: "chunk_ref_lookups_bypassed_total", + Help: "Number of chunk refs that were bypassed due to store overrides: computed during planning to avoid lookups", + }), series: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: constants.Loki, Subsystem: "store", diff --git a/pkg/storage/store.go b/pkg/storage/store.go index ecf34dee8c79..1900803637d7 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -366,6 +366,7 @@ func decodeReq(req logql.QueryParams) ([]*labels.Matcher, model.Time, model.Time // 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 +// or a way to transform the base store into a more specialized variant. func injectShardLabel(shards []string, matchers []*labels.Matcher) ([]*labels.Matcher, error) { if shards != nil { parsed, _, err := logql.ParseShards(shards) @@ -401,8 +402,13 @@ func (s *LokiStore) SetPipelineWrapper(wrapper lokilog.PipelineWrapper) { s.pipelineWrapper = wrapper } -// lazyChunks is an internal function used to resolve a set of lazy chunks from the store without actually loading them. It's used internally by `LazyQuery` and `GetSeries` -func (s *LokiStore) lazyChunks(ctx context.Context, from, through model.Time, predicate chunk.Predicate) ([]*LazyChunk, error) { +// lazyChunks is an internal function used to resolve a set of lazy chunks from the store without actually loading them. +func (s *LokiStore) lazyChunks( + ctx context.Context, + from, through model.Time, + predicate chunk.Predicate, + storeChunksOverride *logproto.ChunkRefGroup, +) ([]*LazyChunk, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, err @@ -411,7 +417,7 @@ func (s *LokiStore) lazyChunks(ctx context.Context, from, through model.Time, pr stats := stats.FromContext(ctx) start := time.Now() - chks, fetchers, err := s.GetChunks(ctx, userID, from, through, predicate) + chks, fetchers, err := s.GetChunks(ctx, userID, from, through, predicate, storeChunksOverride) stats.AddChunkRefsFetchTime(time.Since(start)) if err != nil { @@ -427,6 +433,9 @@ func (s *LokiStore) lazyChunks(ctx context.Context, from, through model.Time, pr filtered += len(chks[i]) } + if storeChunksOverride != nil { + s.chunkMetrics.refsBypassed.Add(float64(len(storeChunksOverride.Refs))) + } s.chunkMetrics.refs.WithLabelValues(statusDiscarded).Add(float64(prefiltered - filtered)) s.chunkMetrics.refs.WithLabelValues(statusMatched).Add(float64(filtered)) @@ -487,7 +496,7 @@ func (s *LokiStore) SelectLogs(ctx context.Context, req logql.SelectLogParams) ( return nil, err } - lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan)) + lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan), req.GetStoreChunks()) if err != nil { return nil, err } @@ -534,7 +543,7 @@ func (s *LokiStore) SelectSamples(ctx context.Context, req logql.SelectSamplePar return nil, err } - lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan)) + lazyChunks, err := s.lazyChunks(ctx, from, through, chunk.NewPredicate(matchers, req.Plan), req.GetStoreChunks()) if err != nil { return nil, err } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 45a505be3f43..51f04538cc18 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -1338,7 +1338,7 @@ func TestStore_indexPrefixChange(t *testing.T) { // get all the chunks from the first period predicate := chunk.NewPredicate(newMatchers(fooLabelsWithName.String()), nil) - chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(firstPeriodDate), timeToModelTime(secondPeriodDate), predicate) + chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(firstPeriodDate), timeToModelTime(secondPeriodDate), predicate, nil) require.NoError(t, err) var totalChunks int for _, chks := range chunks { @@ -1409,7 +1409,7 @@ func TestStore_indexPrefixChange(t *testing.T) { // get all the chunks from both the stores predicate = chunk.NewPredicate(newMatchers(fooLabelsWithName.String()), nil) - chunks, _, err = store.GetChunks(ctx, "fake", timeToModelTime(firstPeriodDate), timeToModelTime(secondPeriodDate.Add(24*time.Hour)), predicate) + chunks, _, err = store.GetChunks(ctx, "fake", timeToModelTime(firstPeriodDate), timeToModelTime(secondPeriodDate.Add(24*time.Hour)), predicate, nil) require.NoError(t, err) totalChunks = 0 @@ -1543,7 +1543,7 @@ func TestStore_MultiPeriod(t *testing.T) { // get all the chunks from both the stores predicate := chunk.NewPredicate(newMatchers(fooLabelsWithName.String()), nil) - chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(firstStoreDate), timeToModelTime(secondStoreDate.Add(24*time.Hour)), predicate) + chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(firstStoreDate), timeToModelTime(secondStoreDate.Add(24*time.Hour)), predicate, nil) require.NoError(t, err) var totalChunks int for _, chks := range chunks { @@ -1914,7 +1914,7 @@ func TestStore_BoltdbTsdbSameIndexPrefix(t *testing.T) { // get all the chunks from both the stores predicate := chunk.NewPredicate(newMatchers(fooLabelsWithName.String()), nil) - chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(boltdbShipperStartDate), timeToModelTime(tsdbStartDate.Add(24*time.Hour)), predicate) + chunks, _, err := store.GetChunks(ctx, "fake", timeToModelTime(boltdbShipperStartDate), timeToModelTime(tsdbStartDate.Add(24*time.Hour)), predicate, nil) require.NoError(t, err) var totalChunks int for _, chks := range chunks { diff --git a/pkg/storage/stores/composite_store.go b/pkg/storage/stores/composite_store.go index 212cd9408283..62ba7f12df8a 100644 --- a/pkg/storage/stores/composite_store.go +++ b/pkg/storage/stores/composite_store.go @@ -28,7 +28,14 @@ type ChunkFetcherProvider interface { } type ChunkFetcher interface { - GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) + GetChunks( + ctx context.Context, + userID string, + from, + through model.Time, + predicate chunk.Predicate, + storeChunksOverride *logproto.ChunkRefGroup, + ) ([][]chunk.Chunk, []*fetcher.Fetcher, error) } type Store interface { @@ -155,11 +162,18 @@ func (c CompositeStore) LabelNamesForMetricName(ctx context.Context, userID stri return result.Strings(), err } -func (c CompositeStore) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (c CompositeStore) GetChunks( + ctx context.Context, + userID string, + from, + through model.Time, + predicate chunk.Predicate, + storeChunksOverride *logproto.ChunkRefGroup, +) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { chunkIDs := [][]chunk.Chunk{} fetchers := []*fetcher.Fetcher{} err := c.forStores(ctx, from, through, func(innerCtx context.Context, from, through model.Time, store Store) error { - ids, fetcher, err := store.GetChunks(innerCtx, userID, from, through, predicate) + ids, fetcher, err := store.GetChunks(innerCtx, userID, from, through, predicate, storeChunksOverride) if err != nil { return err } @@ -329,13 +343,6 @@ func (c CompositeStore) forStores(ctx context.Context, from, through model.Time, return c.stores[j].start > through }) - min := func(a, b model.Time) model.Time { - if a < b { - return a - } - return b - } - start := from for ; i < j; i++ { nextSchemaStarts := model.Latest diff --git a/pkg/storage/stores/composite_store_entry.go b/pkg/storage/stores/composite_store_entry.go index d4590d199d9e..fd214543ef03 100644 --- a/pkg/storage/stores/composite_store_entry.go +++ b/pkg/storage/stores/composite_store_entry.go @@ -43,7 +43,16 @@ type storeEntry struct { ChunkWriter } -func (c *storeEntry) GetChunks(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (c *storeEntry) GetChunks( + ctx context.Context, + userID string, + from, + through model.Time, + predicate chunk.Predicate, + storeChunksOverride *logproto.ChunkRefGroup, +) ([][]chunk.Chunk, + []*fetcher.Fetcher, + error) { if ctx.Err() != nil { return nil, nil, ctx.Err() } @@ -55,18 +64,41 @@ func (c *storeEntry) GetChunks(ctx context.Context, userID string, from, through return nil, nil, nil } - refs, err := c.indexReader.GetChunkRefs(ctx, userID, from, through, predicate) - - chunks := make([]chunk.Chunk, len(refs)) - for i, ref := range refs { - chunks[i] = chunk.Chunk{ - ChunkRef: ref, + var refs []*logproto.ChunkRef + if storeChunksOverride != nil { + refs = storeChunksOverride.Refs + } else { + // TODO(owen-d): fix needless O(n) conversions that stem from difference in store impls (value) + // vs proto impls (reference) + var values []logproto.ChunkRef + values, err = c.indexReader.GetChunkRefs(ctx, userID, from, through, predicate) + // convert to refs + refs = make([]*logproto.ChunkRef, 0, len(values)) + for i := range values { + refs = append(refs, &values[i]) } } + // Store overrides are passed through from the parent and can reference chunks not owned by this particular store, + // so we filter them out based on the requested timerange passed, which is guaranteed to be within the store's timerange. + // Otherwise, we'd return chunks that do not belong to the store, which would error during fetching. + chunks := filterForTimeRange(refs, from, through) + return [][]chunk.Chunk{chunks}, []*fetcher.Fetcher{c.fetcher}, err } +func filterForTimeRange(refs []*logproto.ChunkRef, from, through model.Time) []chunk.Chunk { + filtered := make([]chunk.Chunk, 0, len(refs)) + for _, ref := range refs { + if through >= ref.From && from < ref.Through { + filtered = append(filtered, chunk.Chunk{ + ChunkRef: *ref, + }) + } + } + return filtered +} + func (c *storeEntry) GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) { return c.indexReader.GetSeries(ctx, userID, from, through, matchers...) } diff --git a/pkg/storage/stores/composite_store_test.go b/pkg/storage/stores/composite_store_test.go index 3836243f3878..203057c60845 100644 --- a/pkg/storage/stores/composite_store_test.go +++ b/pkg/storage/stores/composite_store_test.go @@ -37,7 +37,7 @@ func (m mockStore) LabelValuesForMetricName(_ context.Context, _ string, _, _ mo func (m mockStore) SetChunkFilterer(_ chunk.RequestChunkFilterer) {} -func (m mockStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (m mockStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate, _ *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { return nil, nil, nil } @@ -354,3 +354,77 @@ func TestVolume(t *testing.T) { }) } + +func TestFilterForTimeRange(t *testing.T) { + mkRefs := func(from, through model.Time) (res []*logproto.ChunkRef) { + for i := from; i <= through; i++ { + res = append(res, &logproto.ChunkRef{ + From: i, + Through: i + 1, + }) + } + return res + } + + mkChks := func(from, through model.Time) (res []chunk.Chunk) { + for _, ref := range mkRefs(from, through) { + res = append(res, chunk.Chunk{ChunkRef: *ref}) + } + return res + } + + for _, tc := range []struct { + desc string + input []*logproto.ChunkRef + from, through model.Time + exp []chunk.Chunk + }{ + { + desc: "no refs", + input: nil, + from: 0, + through: 10, + exp: []chunk.Chunk{}, + }, + { + desc: "no refs in range", + input: mkRefs(0, 5), + from: 10, + through: 15, + exp: []chunk.Chunk{}, + }, + { + desc: "all refs in range", + input: mkRefs(0, 5), + from: 0, + through: 5, + exp: mkChks(0, 5), + }, + { + desc: "some refs in range", + input: mkRefs(0, 5), + from: 2, + through: 3, + exp: mkChks(2, 3), + }, + { + desc: "left overlap", + input: mkRefs(0, 5), + from: 3, + through: 7, + exp: mkChks(3, 5), + }, + { + desc: "right overlap", + input: mkRefs(5, 10), + from: 3, + through: 7, + exp: mkChks(5, 7), + }, + } { + t.Run(tc.desc, func(t *testing.T) { + got := filterForTimeRange(tc.input, tc.from, tc.through) + require.Equal(t, tc.exp, got) + }) + } +} diff --git a/pkg/storage/stores/series/series_store_test.go b/pkg/storage/stores/series/series_store_test.go index 2f1146a1d737..f5948e8ef60d 100644 --- a/pkg/storage/stores/series/series_store_test.go +++ b/pkg/storage/stores/series/series_store_test.go @@ -393,7 +393,7 @@ func TestChunkStore_getMetricNameChunks(t *testing.T) { t.Fatal(err) } - chunks, fetchers, err := store.GetChunks(ctx, userID, now.Add(-time.Hour), now, chunk.NewPredicate(matchers, nil)) + chunks, fetchers, err := store.GetChunks(ctx, userID, now.Add(-time.Hour), now, chunk.NewPredicate(matchers, nil), nil) require.NoError(t, err) fetchedChunk := []chunk.Chunk{} for _, f := range fetchers { @@ -652,7 +652,7 @@ func TestChunkStoreError(t *testing.T) { require.NoError(t, err) // Query with ordinary time-range - _, _, err = store.GetChunks(ctx, userID, tc.from, tc.through, chunk.NewPredicate(matchers, nil)) + _, _, err = store.GetChunks(ctx, userID, tc.from, tc.through, chunk.NewPredicate(matchers, nil), nil) require.EqualError(t, err, tc.err) }) } diff --git a/pkg/storage/stores/shipper/indexshipper/boltdb/compactor/util_test.go b/pkg/storage/stores/shipper/indexshipper/boltdb/compactor/util_test.go index 7e6be7bb1414..38bd567ff4a3 100644 --- a/pkg/storage/stores/shipper/indexshipper/boltdb/compactor/util_test.go +++ b/pkg/storage/stores/shipper/indexshipper/boltdb/compactor/util_test.go @@ -177,7 +177,7 @@ func (t *testStore) GetChunks(userID string, from, through model.Time, metric la matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, l.Name, l.Value)) } ctx := user.InjectOrgID(context.Background(), userID) - chunks, fetchers, err := t.Store.GetChunks(ctx, userID, from, through, chunk.NewPredicate(matchers, nil)) + chunks, fetchers, err := t.Store.GetChunks(ctx, userID, from, through, chunk.NewPredicate(matchers, nil), nil) require.NoError(t.t, err) fetchedChunk := []chunk.Chunk{} for _, f := range fetchers { diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index db71f70af5a8..23f89b7a9dfe 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -244,7 +244,7 @@ func (m *mockChunkStore) GetChunkFetcher(_ model.Time) *fetcher.Fetcher { return nil } -func (m *mockChunkStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { +func (m *mockChunkStore) GetChunks(_ context.Context, _ string, _, _ model.Time, _ chunk.Predicate, _ *logproto.ChunkRefGroup) ([][]chunk.Chunk, []*fetcher.Fetcher, error) { refs := make([]chunk.Chunk, 0, len(m.chunks)) // transform real chunks into ref chunks. for _, c := range m.chunks { diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 9a6f6a1a8841..015919783164 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -101,6 +101,7 @@ type Limits struct { 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"` + TSDBPrecomputeChunks bool `yaml:"tsdb_precompute_chunks" json:"tsdb_precompute_chunks"` 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"` @@ -301,6 +302,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { logql.BoundedVersion.String(), ), ) + f.BoolVar(&l.TSDBPrecomputeChunks, "querier.tsdb-precompute-chunks", false, "Precompute chunks for TSDB queries. This can improve query performance at the cost of increased memory usage by computing chunks once during planning, reducing index calls.") 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.") @@ -642,6 +644,10 @@ func (o *Overrides) TSDBShardingStrategy(userID string) string { return o.getOverridesForUser(userID).TSDBShardingStrategy } +func (o *Overrides) TSDBPrecomputeChunks(userID string) bool { + return o.getOverridesForUser(userID).TSDBPrecomputeChunks +} + // 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 {