feat(blooms): compute chunks once (#12664)

Signed-off-by: Owen Diehl <ow.diehl@gmail.com>
pull/12790/head^2
Owen Diehl 2 years ago committed by GitHub
parent c3a3bc396f
commit bc78d13d9b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 2
      cmd/migrate/main.go
  2. 6
      docs/sources/shared/configuration.md
  3. 42
      pkg/indexgateway/client.go
  4. 90
      pkg/indexgateway/gateway.go
  5. 27
      pkg/indexgateway/gateway_test.go
  6. 1
      pkg/indexgateway/shufflesharding.go
  7. 2
      pkg/ingester/flush_test.go
  8. 2
      pkg/ingester/ingester.go
  9. 2
      pkg/ingester/ingester_test.go
  10. 2
      pkg/logcli/client/file.go
  11. 2
      pkg/logcli/query/query.go
  12. 2
      pkg/logcli/query/query_test.go
  13. 6
      pkg/logproto/extensions.go
  14. 185
      pkg/logproto/indexgateway.pb.go
  15. 8
      pkg/logproto/indexgateway.proto
  16. 928
      pkg/logproto/logproto.pb.go
  17. 10
      pkg/logproto/logproto.proto
  18. 6
      pkg/logql/accumulator_test.go
  19. 2
      pkg/logql/blocker_test.go
  20. 48
      pkg/logql/downstream.go
  21. 11
      pkg/logql/downstream_test.go
  22. 12
      pkg/logql/engine.go
  23. 20
      pkg/logql/engine_test.go
  24. 39
      pkg/logql/evaluator.go
  25. 4
      pkg/logql/shardmapper.go
  26. 84
      pkg/logql/shardmapper_test.go
  27. 58
      pkg/logql/shards.go
  28. 2
      pkg/loki/modules.go
  29. 37
      pkg/querier/multi_tenant_querier.go
  30. 2
      pkg/querier/querier_mock_test.go
  31. 201
      pkg/querier/queryrange/codec.go
  32. 6
      pkg/querier/queryrange/codec_test.go
  33. 2
      pkg/querier/queryrange/downstreamer.go
  34. 3
      pkg/querier/queryrange/downstreamer_test.go
  35. 425
      pkg/querier/queryrange/queryrange.pb.go
  36. 6
      pkg/querier/queryrange/queryrange.proto
  37. 41
      pkg/querier/queryrange/shard_resolver.go
  38. 1
      pkg/ruler/evaluator_local.go
  39. 10
      pkg/storage/async_store.go
  40. 12
      pkg/storage/async_store_test.go
  41. 15
      pkg/storage/batch.go
  42. 19
      pkg/storage/store.go
  43. 8
      pkg/storage/store_test.go
  44. 27
      pkg/storage/stores/composite_store.go
  45. 46
      pkg/storage/stores/composite_store_entry.go
  46. 76
      pkg/storage/stores/composite_store_test.go
  47. 4
      pkg/storage/stores/series/series_store_test.go
  48. 2
      pkg/storage/stores/shipper/indexshipper/boltdb/compactor/util_test.go
  49. 2
      pkg/storage/util_test.go
  50. 6
      pkg/validation/limits.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

@ -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: <string> | 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: <boolean> | default = false]
# Cardinality limit for index queries.
# CLI flag: -store.cardinality-limit
[cardinality_limit: <int> | default = 100000]

@ -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 {

@ -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 {

@ -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))

@ -22,6 +22,7 @@ var (
type Limits interface {
IndexGatewayShardSize(tenantID string) int
TSDBMaxBytesPerShard(string) int
TSDBPrecomputeChunks(string) bool
}
type ShardingStrategy interface {

@ -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
}

@ -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
}

@ -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
}

@ -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

@ -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

@ -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
}

@ -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)
}

@ -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:])

@ -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 {

File diff suppressed because it is too large Load Diff

@ -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;

@ -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)

@ -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"))

@ -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

@ -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{

@ -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) {

@ -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)

@ -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 {

@ -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,
})
}

@ -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{

@ -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}
}

@ -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
}

@ -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
}

@ -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)
}

@ -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

@ -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 {

@ -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(),
}
}

@ -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)

@ -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:])

@ -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 {

@ -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
}

@ -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

@ -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
}()

@ -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

@ -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",

@ -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
}

@ -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 {

@ -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

@ -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...)
}

@ -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)
})
}
}

@ -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)
})
}

@ -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 {

@ -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 {

@ -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 {

Loading…
Cancel
Save