diff --git a/CHANGELOG.md b/CHANGELOG.md index 8a93537fe6..8975f31b20 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -38,6 +38,7 @@ * [10727](https://github.com/grafana/loki/pull/10727) **sandeepsukhani** Native otlp ingestion support * [11051](https://github.com/grafana/loki/pull/11051) Refactor to not use global logger in modules * [10956](https://github.com/grafana/loki/pull/10956) **jeschkies** do not wrap requests but send pure Protobuf from frontend v2 via scheduler to querier when `-frontend.encoding=protobuf`. +* [10417](https://github.com/grafana/loki/pull/10417) **jeschkies** shard `quantile_over_time` range queries using probabilistic data structures. * [11284](https://github.com/grafana/loki/pull/11284) **ashwanthgoli** Config: Adds `frontend.max-query-capacity` to tune per-tenant query capacity. ##### Fixes diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 0323f0c652..3cc898005d 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -842,6 +842,11 @@ results_cache: # CLI flag: -querier.parallelise-shardable-queries [parallelise_shardable_queries: | default = true] +# A comma-separated list of LogQL vector and range aggregations that should be +# sharded +# CLI flag: -querier.shard-aggregation +[shard_aggregations: | default = ""] + # Cache index stats query results. # CLI flag: -querier.cache-index-stats-results [cache_index_stats_results: | default = false] diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 147262ff22..29875c4847 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -37,6 +37,7 @@ import ( "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/runtime" "github.com/grafana/loki/pkg/storage" "github.com/grafana/loki/pkg/storage/chunk" @@ -851,6 +852,16 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie // initialize stats collection for ingester queries. _, ctx := stats.NewContext(queryServer.Context()) + if req.Plan == nil { + parsed, err := syntax.ParseLogSelector(req.Selector, true) + if err != nil { + return err + } + req.Plan = &plan.QueryPlan{ + AST: parsed, + } + } + instanceID, err := tenant.TenantID(ctx) if err != nil { return err @@ -874,6 +885,7 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie Limit: req.Limit, Shards: req.Shards, Deletes: req.Deletes, + Plan: req.Plan, }} storeItr, err := i.store.SelectLogs(ctx, storeReq) if err != nil { @@ -900,6 +912,17 @@ func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer log _, ctx := stats.NewContext(queryServer.Context()) sp := opentracing.SpanFromContext(ctx) + // If the plan is empty we want all series to be returned. + if req.Plan == nil { + parsed, err := syntax.ParseSampleExpr(req.Selector) + if err != nil { + return err + } + req.Plan = &plan.QueryPlan{ + AST: parsed, + } + } + instanceID, err := tenant.TenantID(ctx) if err != nil { return err @@ -925,6 +948,7 @@ func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer log Selector: req.Selector, Shards: req.Shards, Deletes: req.Deletes, + Plan: req.Plan, }} storeItr, err := i.store.SelectSamples(ctx, storeReq) if err != nil { @@ -1234,6 +1258,16 @@ func (i *Ingester) Tail(req *logproto.TailRequest, queryServer logproto.Querier_ default: } + if req.Plan == nil { + parsed, err := syntax.ParseLogSelector(req.Query, true) + if err != nil { + return err + } + req.Plan = &plan.QueryPlan{ + AST: parsed, + } + } + instanceID, err := tenant.TenantID(queryServer.Context()) if err != nil { return err @@ -1243,7 +1277,13 @@ func (i *Ingester) Tail(req *logproto.TailRequest, queryServer logproto.Querier_ if err != nil { return err } - tailer, err := newTailer(instanceID, req.Query, queryServer, i.cfg.MaxDroppedStreams) + + expr, ok := req.Plan.AST.(syntax.LogSelectorExpr) + if !ok { + return fmt.Errorf("unsupported query expression: want (LogSelectorExpr), got (%T)", req.Plan.AST) + } + + tailer, err := newTailer(instanceID, expr, queryServer, i.cfg.MaxDroppedStreams) if err != nil { return err } diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 3da4cd356d..0e43daf2cd 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -35,6 +35,8 @@ import ( "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/runtime" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/fetcher" @@ -812,6 +814,9 @@ func Test_DedupeIngester(t *testing.T) { End: time.Unix(0, requests+1), Limit: uint32(requests * streamCount), Direction: logproto.BACKWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"} | label_format bar=""`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewQueryClientIterator(stream, logproto.BACKWARD)) @@ -870,6 +875,9 @@ func Test_DedupeIngester(t *testing.T) { Selector: `sum(rate({foo="bar"}[1m])) by (bar)`, Start: time.Unix(0, 0), End: time.Unix(0, requests+1), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(rate({foo="bar"}[1m])) by (bar)`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewSampleQueryClientIterator(stream)) @@ -905,6 +913,9 @@ func Test_DedupeIngester(t *testing.T) { Selector: `sum(rate({foo="bar"}[1m]))`, Start: time.Unix(0, 0), End: time.Unix(0, requests+1), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum(rate({foo="bar"}[1m]))`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewSampleQueryClientIterator(stream)) @@ -965,6 +976,9 @@ func Test_DedupeIngesterParser(t *testing.T) { End: time.Unix(0, int64(requests+1)), Limit: uint32(requests * streamCount * 2), Direction: logproto.BACKWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"} | json`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewQueryClientIterator(stream, logproto.BACKWARD)) @@ -992,6 +1006,9 @@ func Test_DedupeIngesterParser(t *testing.T) { End: time.Unix(0, int64(requests+1)), Limit: uint32(requests * streamCount * 2), Direction: logproto.FORWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"} | json`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewQueryClientIterator(stream, logproto.FORWARD)) @@ -1016,6 +1033,9 @@ func Test_DedupeIngesterParser(t *testing.T) { Selector: `rate({foo="bar"} | json [1m])`, Start: time.Unix(0, 0), End: time.Unix(0, int64(requests+1)), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate({foo="bar"} | json [1m])`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewSampleQueryClientIterator(stream)) @@ -1041,6 +1061,9 @@ func Test_DedupeIngesterParser(t *testing.T) { Selector: `sum by (c,d,e,foo) (rate({foo="bar"} | json [1m]))`, Start: time.Unix(0, 0), End: time.Unix(0, int64(requests+1)), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`sum by (c,d,e,foo) (rate({foo="bar"} | json [1m]))`), + }, }) require.NoError(t, err) iterators = append(iterators, iter.NewSampleQueryClientIterator(stream)) diff --git a/pkg/ingester/instance_test.go b/pkg/ingester/instance_test.go index ac29f3516d..492f78404c 100644 --- a/pkg/ingester/instance_test.go +++ b/pkg/ingester/instance_test.go @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/querier/plan" loki_runtime "github.com/grafana/loki/pkg/runtime" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/config" @@ -537,7 +538,9 @@ func Benchmark_instance_addNewTailer(b *testing.B) { ctx := context.Background() inst, _ := newInstance(&Config{}, defaultPeriodConfigs, "test", limiter, loki_runtime.DefaultTenantConfigs(), noopWAL{}, NilMetrics, &OnceSwitch{}, nil, NewStreamRateCalculator(), nil) - t, err := newTailer("foo", `{namespace="foo",pod="bar",instance=~"10.*"}`, nil, 10) + expr, err := syntax.ParseLogSelector(`{namespace="foo",pod="bar",instance=~"10.*"}`, true) + require.NoError(b, err) + t, err := newTailer("foo", expr, nil, 10) require.NoError(b, err) for i := 0; i < 10000; i++ { require.NoError(b, inst.Push(ctx, &logproto.PushRequest{ @@ -596,6 +599,9 @@ func Test_Iterator(t *testing.T) { Start: time.Unix(0, 0), End: time.Unix(0, 100000000), Direction: logproto.BACKWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{job="3"} | logfmt`), + }, }, }, ) @@ -648,6 +654,9 @@ func Test_ChunkFilter(t *testing.T) { Start: time.Unix(0, 0), End: time.Unix(0, 100000000), Direction: logproto.BACKWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{job="3"}`), + }, }, }, ) @@ -690,6 +699,9 @@ func Test_QueryWithDelete(t *testing.T) { End: 10 * 1e6, }, }, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{job="3"}`), + }, }, }, ) @@ -730,6 +742,9 @@ func Test_QuerySampleWithDelete(t *testing.T) { End: 10 * 1e6, }, }, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`count_over_time({job="3"}[5m])`), + }, }, }, ) diff --git a/pkg/ingester/stream_test.go b/pkg/ingester/stream_test.go index 641fd1c926..d1b01f2274 100644 --- a/pkg/ingester/stream_test.go +++ b/pkg/ingester/stream_test.go @@ -18,6 +18,7 @@ import ( "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/log" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/util/flagext" "github.com/grafana/loki/pkg/validation" ) @@ -524,7 +525,9 @@ func Benchmark_PushStream(b *testing.B) { chunkfmt, headfmt := defaultChunkFormat(b) s := newStream(chunkfmt, headfmt, &Config{MaxChunkAge: 24 * time.Hour}, limiter, "fake", model.Fingerprint(0), ls, true, NewStreamRateCalculator(), NilMetrics, nil) - t, err := newTailer("foo", `{namespace="loki-dev"}`, &fakeTailServer{}, 10) + expr, err := syntax.ParseLogSelector(`{namespace="loki-dev"}`, true) + require.NoError(b, err) + t, err := newTailer("foo", expr, &fakeTailServer{}, 10) require.NoError(b, err) go t.loop() diff --git a/pkg/ingester/tailer.go b/pkg/ingester/tailer.go index 72e7026e81..3e9a8a64cf 100644 --- a/pkg/ingester/tailer.go +++ b/pkg/ingester/tailer.go @@ -46,11 +46,7 @@ type tailer struct { conn TailServer } -func newTailer(orgID, query string, conn TailServer, maxDroppedStreams int) (*tailer, error) { - expr, err := syntax.ParseLogSelector(query, true) - if err != nil { - return nil, err - } +func newTailer(orgID string, expr syntax.LogSelectorExpr, conn TailServer, maxDroppedStreams int) (*tailer, error) { // Make sure we can build a pipeline. The stream processing code doesn't have a place to handle // this error so make sure we handle it here. pipeline, err := expr.Pipeline() @@ -66,7 +62,7 @@ func newTailer(orgID, query string, conn TailServer, maxDroppedStreams int) (*ta conn: conn, droppedStreams: make([]*logproto.DroppedStream, 0, maxDroppedStreams), maxDroppedStreams: maxDroppedStreams, - id: generateUniqueID(orgID, query), + id: generateUniqueID(orgID, expr.String()), closeChan: make(chan struct{}), pipeline: pipeline, }, nil diff --git a/pkg/ingester/tailer_test.go b/pkg/ingester/tailer_test.go index 5929335203..674dde3df8 100644 --- a/pkg/ingester/tailer_test.go +++ b/pkg/ingester/tailer_test.go @@ -12,6 +12,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" ) func TestTailer_sendRaceConditionOnSendWhileClosing(t *testing.T) { @@ -26,7 +27,9 @@ func TestTailer_sendRaceConditionOnSendWhileClosing(t *testing.T) { } for run := 0; run < runs; run++ { - tailer, err := newTailer("org-id", stream.Labels, nil, 10) + expr, err := syntax.ParseLogSelector(stream.Labels, true) + require.NoError(t, err) + tailer, err := newTailer("org-id", expr, nil, 10) require.NoError(t, err) require.NotNil(t, tailer) @@ -78,7 +81,9 @@ func Test_dropstream(t *testing.T) { for _, c := range cases { t.Run(c.name, func(t *testing.T) { - tail, err := newTailer("foo", `{app="foo"} |= "foo"`, &fakeTailServer{}, maxDroppedStreams) + expr, err := syntax.ParseLogSelector(`{app="foo"} |= "foo"`, true) + require.NoError(t, err) + tail, err := newTailer("foo", expr, &fakeTailServer{}, maxDroppedStreams) require.NoError(t, err) for i := 0; i < c.drop; i++ { @@ -114,7 +119,9 @@ func (f *fakeTailServer) Reset() { } func Test_TailerSendRace(t *testing.T) { - tail, err := newTailer("foo", `{app="foo"} |= "foo"`, &fakeTailServer{}, 10) + expr, err := syntax.ParseLogSelector(`{app="foo"} |= "foo"`, true) + require.NoError(t, err) + tail, err := newTailer("foo", expr, &fakeTailServer{}, 10) require.NoError(t, err) var wg sync.WaitGroup @@ -250,7 +257,9 @@ func Test_StructuredMetadata(t *testing.T) { } { t.Run(tc.name, func(t *testing.T) { var server fakeTailServer - tail, err := newTailer("foo", tc.query, &server, 10) + expr, err := syntax.ParseLogSelector(tc.query, true) + require.NoError(t, err) + tail, err := newTailer("foo", expr, &server, 10) require.NoError(t, err) var wg sync.WaitGroup diff --git a/pkg/loghttp/tail.go b/pkg/loghttp/tail.go index 6b9b5ad7d1..9ad2219b10 100644 --- a/pkg/loghttp/tail.go +++ b/pkg/loghttp/tail.go @@ -11,6 +11,8 @@ import ( "github.com/grafana/dskit/httpgrpc" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/plan" ) const ( @@ -67,8 +69,16 @@ func (s *DroppedStream) UnmarshalJSON(data []byte) error { // ParseTailQuery parses a TailRequest request from an http request. func ParseTailQuery(r *http.Request) (*logproto.TailRequest, error) { var err error + qs := query(r) + parsed, err := syntax.ParseExpr(qs) + if err != nil { + return nil, err + } req := logproto.TailRequest{ - Query: query(r), + Query: qs, + Plan: &plan.QueryPlan{ + AST: parsed, + }, } req.Query, err = parseRegexQuery(r) diff --git a/pkg/loghttp/tail_test.go b/pkg/loghttp/tail_test.go index f5b2039723..6fe7163116 100644 --- a/pkg/loghttp/tail_test.go +++ b/pkg/loghttp/tail_test.go @@ -9,6 +9,8 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/plan" ) func TestParseTailQuery(t *testing.T) { @@ -38,6 +40,9 @@ func TestParseTailQuery(t *testing.T) { DelayFor: 5, Start: time.Date(2017, 06, 10, 21, 42, 24, 760738998, time.UTC), Limit: 1000, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"}`), + }, }, false}, } for _, tt := range tests { diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index 6aa905ab98..6a4bc06d26 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -16,6 +16,7 @@ import ( stats "github.com/grafana/loki/pkg/logqlmodel/stats" _ "github.com/grafana/loki/pkg/push" github_com_grafana_loki_pkg_push "github.com/grafana/loki/pkg/push" + github_com_grafana_loki_pkg_querier_plan "github.com/grafana/loki/pkg/querier/plan" github_com_prometheus_common_model "github.com/prometheus/common/model" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" @@ -216,13 +217,14 @@ func (m *StreamRate) GetPushes() uint32 { } type QueryRequest struct { - Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` - Limit uint32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` - Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` - End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"` - Direction Direction `protobuf:"varint,5,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"` - Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"` - Deletes []*Delete `protobuf:"bytes,8,rep,name=deletes,proto3" json:"deletes,omitempty"` + Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` // Deprecated: Do not use. + Limit uint32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` + Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` + End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"` + Direction Direction `protobuf:"varint,5,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"` + Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"` + Deletes []*Delete `protobuf:"bytes,8,rep,name=deletes,proto3" json:"deletes,omitempty"` + Plan *github_com_grafana_loki_pkg_querier_plan.QueryPlan `protobuf:"bytes,9,opt,name=plan,proto3,customtype=github.com/grafana/loki/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` } func (m *QueryRequest) Reset() { *m = QueryRequest{} } @@ -257,6 +259,7 @@ func (m *QueryRequest) XXX_DiscardUnknown() { var xxx_messageInfo_QueryRequest proto.InternalMessageInfo +// Deprecated: Do not use. func (m *QueryRequest) GetSelector() string { if m != nil { return m.Selector @@ -307,11 +310,12 @@ func (m *QueryRequest) GetDeletes() []*Delete { } type SampleQueryRequest struct { - Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` - Start time.Time `protobuf:"bytes,2,opt,name=start,proto3,stdtime" json:"start"` - End time.Time `protobuf:"bytes,3,opt,name=end,proto3,stdtime" json:"end"` - Shards []string `protobuf:"bytes,4,rep,name=shards,proto3" json:"shards,omitempty"` - Deletes []*Delete `protobuf:"bytes,5,rep,name=deletes,proto3" json:"deletes,omitempty"` + Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` // Deprecated: Do not use. + Start time.Time `protobuf:"bytes,2,opt,name=start,proto3,stdtime" json:"start"` + End time.Time `protobuf:"bytes,3,opt,name=end,proto3,stdtime" json:"end"` + Shards []string `protobuf:"bytes,4,rep,name=shards,proto3" json:"shards,omitempty"` + Deletes []*Delete `protobuf:"bytes,5,rep,name=deletes,proto3" json:"deletes,omitempty"` + Plan *github_com_grafana_loki_pkg_querier_plan.QueryPlan `protobuf:"bytes,6,opt,name=plan,proto3,customtype=github.com/grafana/loki/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` } func (m *SampleQueryRequest) Reset() { *m = SampleQueryRequest{} } @@ -346,6 +350,7 @@ func (m *SampleQueryRequest) XXX_DiscardUnknown() { var xxx_messageInfo_SampleQueryRequest proto.InternalMessageInfo +// Deprecated: Do not use. func (m *SampleQueryRequest) GetSelector() string { if m != nil { return m.Selector @@ -381,6 +386,49 @@ func (m *SampleQueryRequest) GetDeletes() []*Delete { return nil } +type Plan struct { + Raw []byte `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` +} + +func (m *Plan) Reset() { *m = Plan{} } +func (*Plan) ProtoMessage() {} +func (*Plan) Descriptor() ([]byte, []int) { + return fileDescriptor_c28a5f14f1f4c79a, []int{5} +} +func (m *Plan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Plan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Plan.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Plan) XXX_Merge(src proto.Message) { + xxx_messageInfo_Plan.Merge(m, src) +} +func (m *Plan) XXX_Size() int { + return m.Size() +} +func (m *Plan) XXX_DiscardUnknown() { + xxx_messageInfo_Plan.DiscardUnknown(m) +} + +var xxx_messageInfo_Plan proto.InternalMessageInfo + +func (m *Plan) GetRaw() []byte { + if m != nil { + return m.Raw + } + return nil +} + type Delete struct { Selector string `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector,omitempty"` Start int64 `protobuf:"varint,2,opt,name=start,proto3" json:"start,omitempty"` @@ -390,7 +438,7 @@ type Delete struct { func (m *Delete) Reset() { *m = Delete{} } func (*Delete) ProtoMessage() {} func (*Delete) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{5} + return fileDescriptor_c28a5f14f1f4c79a, []int{6} } func (m *Delete) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -448,7 +496,7 @@ type QueryResponse struct { func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{6} + return fileDescriptor_c28a5f14f1f4c79a, []int{7} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -492,7 +540,7 @@ type SampleQueryResponse struct { func (m *SampleQueryResponse) Reset() { *m = SampleQueryResponse{} } func (*SampleQueryResponse) ProtoMessage() {} func (*SampleQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{7} + return fileDescriptor_c28a5f14f1f4c79a, []int{8} } func (m *SampleQueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -539,7 +587,7 @@ type LabelRequest struct { func (m *LabelRequest) Reset() { *m = LabelRequest{} } func (*LabelRequest) ProtoMessage() {} func (*LabelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{8} + return fileDescriptor_c28a5f14f1f4c79a, []int{9} } func (m *LabelRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -610,7 +658,7 @@ type LabelResponse struct { func (m *LabelResponse) Reset() { *m = LabelResponse{} } func (*LabelResponse) ProtoMessage() {} func (*LabelResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{9} + return fileDescriptor_c28a5f14f1f4c79a, []int{10} } func (m *LabelResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -655,7 +703,7 @@ type Sample struct { func (m *Sample) Reset() { *m = Sample{} } func (*Sample) ProtoMessage() {} func (*Sample) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{10} + return fileDescriptor_c28a5f14f1f4c79a, []int{11} } func (m *Sample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,7 +762,7 @@ type LegacySample struct { func (m *LegacySample) Reset() { *m = LegacySample{} } func (*LegacySample) ProtoMessage() {} func (*LegacySample) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{11} + return fileDescriptor_c28a5f14f1f4c79a, []int{12} } func (m *LegacySample) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +814,7 @@ type Series struct { func (m *Series) Reset() { *m = Series{} } func (*Series) ProtoMessage() {} func (*Series) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{12} + return fileDescriptor_c28a5f14f1f4c79a, []int{13} } func (m *Series) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -817,16 +865,17 @@ func (m *Series) GetStreamHash() uint64 { } type TailRequest struct { - Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` - DelayFor uint32 `protobuf:"varint,3,opt,name=delayFor,proto3" json:"delayFor,omitempty"` - Limit uint32 `protobuf:"varint,4,opt,name=limit,proto3" json:"limit,omitempty"` - Start time.Time `protobuf:"bytes,5,opt,name=start,proto3,stdtime" json:"start"` + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` // Deprecated: Do not use. + DelayFor uint32 `protobuf:"varint,3,opt,name=delayFor,proto3" json:"delayFor,omitempty"` + Limit uint32 `protobuf:"varint,4,opt,name=limit,proto3" json:"limit,omitempty"` + Start time.Time `protobuf:"bytes,5,opt,name=start,proto3,stdtime" json:"start"` + Plan *github_com_grafana_loki_pkg_querier_plan.QueryPlan `protobuf:"bytes,6,opt,name=plan,proto3,customtype=github.com/grafana/loki/pkg/querier/plan.QueryPlan" json:"plan,omitempty"` } func (m *TailRequest) Reset() { *m = TailRequest{} } func (*TailRequest) ProtoMessage() {} func (*TailRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{13} + return fileDescriptor_c28a5f14f1f4c79a, []int{14} } func (m *TailRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -855,6 +904,7 @@ func (m *TailRequest) XXX_DiscardUnknown() { var xxx_messageInfo_TailRequest proto.InternalMessageInfo +// Deprecated: Do not use. func (m *TailRequest) GetQuery() string { if m != nil { return m.Query @@ -891,7 +941,7 @@ type TailResponse struct { func (m *TailResponse) Reset() { *m = TailResponse{} } func (*TailResponse) ProtoMessage() {} func (*TailResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{14} + return fileDescriptor_c28a5f14f1f4c79a, []int{15} } func (m *TailResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -937,7 +987,7 @@ type SeriesRequest struct { func (m *SeriesRequest) Reset() { *m = SeriesRequest{} } func (*SeriesRequest) ProtoMessage() {} func (*SeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{15} + return fileDescriptor_c28a5f14f1f4c79a, []int{16} } func (m *SeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1001,7 +1051,7 @@ type SeriesResponse struct { func (m *SeriesResponse) Reset() { *m = SeriesResponse{} } func (*SeriesResponse) ProtoMessage() {} func (*SeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{16} + return fileDescriptor_c28a5f14f1f4c79a, []int{17} } func (m *SeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1044,7 +1094,7 @@ type SeriesIdentifier struct { func (m *SeriesIdentifier) Reset() { *m = SeriesIdentifier{} } func (*SeriesIdentifier) ProtoMessage() {} func (*SeriesIdentifier) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{17} + return fileDescriptor_c28a5f14f1f4c79a, []int{18} } func (m *SeriesIdentifier) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1089,7 +1139,7 @@ type DroppedStream struct { func (m *DroppedStream) Reset() { *m = DroppedStream{} } func (*DroppedStream) ProtoMessage() {} func (*DroppedStream) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{18} + return fileDescriptor_c28a5f14f1f4c79a, []int{19} } func (m *DroppedStream) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1147,7 +1197,7 @@ type LabelPair struct { func (m *LabelPair) Reset() { *m = LabelPair{} } func (*LabelPair) ProtoMessage() {} func (*LabelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{19} + return fileDescriptor_c28a5f14f1f4c79a, []int{20} } func (m *LabelPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1200,7 +1250,7 @@ type LegacyLabelPair struct { func (m *LegacyLabelPair) Reset() { *m = LegacyLabelPair{} } func (*LegacyLabelPair) ProtoMessage() {} func (*LegacyLabelPair) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{20} + return fileDescriptor_c28a5f14f1f4c79a, []int{21} } func (m *LegacyLabelPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1250,7 +1300,7 @@ type Chunk struct { func (m *Chunk) Reset() { *m = Chunk{} } func (*Chunk) ProtoMessage() {} func (*Chunk) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{21} + return fileDescriptor_c28a5f14f1f4c79a, []int{22} } func (m *Chunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1292,7 +1342,7 @@ type TailersCountRequest struct { func (m *TailersCountRequest) Reset() { *m = TailersCountRequest{} } func (*TailersCountRequest) ProtoMessage() {} func (*TailersCountRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{22} + return fileDescriptor_c28a5f14f1f4c79a, []int{23} } func (m *TailersCountRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1328,7 +1378,7 @@ type TailersCountResponse struct { func (m *TailersCountResponse) Reset() { *m = TailersCountResponse{} } func (*TailersCountResponse) ProtoMessage() {} func (*TailersCountResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{23} + return fileDescriptor_c28a5f14f1f4c79a, []int{24} } func (m *TailersCountResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1373,7 +1423,7 @@ type GetChunkIDsRequest struct { func (m *GetChunkIDsRequest) Reset() { *m = GetChunkIDsRequest{} } func (*GetChunkIDsRequest) ProtoMessage() {} func (*GetChunkIDsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{24} + return fileDescriptor_c28a5f14f1f4c79a, []int{25} } func (m *GetChunkIDsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1430,7 +1480,7 @@ type GetChunkIDsResponse struct { func (m *GetChunkIDsResponse) Reset() { *m = GetChunkIDsResponse{} } func (*GetChunkIDsResponse) ProtoMessage() {} func (*GetChunkIDsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{25} + return fileDescriptor_c28a5f14f1f4c79a, []int{26} } func (m *GetChunkIDsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1483,7 +1533,7 @@ type ChunkRef struct { func (m *ChunkRef) Reset() { *m = ChunkRef{} } func (*ChunkRef) ProtoMessage() {} func (*ChunkRef) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{26} + return fileDescriptor_c28a5f14f1f4c79a, []int{27} } func (m *ChunkRef) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1544,7 +1594,7 @@ type LabelValuesForMetricNameRequest struct { func (m *LabelValuesForMetricNameRequest) Reset() { *m = LabelValuesForMetricNameRequest{} } func (*LabelValuesForMetricNameRequest) ProtoMessage() {} func (*LabelValuesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{27} + return fileDescriptor_c28a5f14f1f4c79a, []int{28} } func (m *LabelValuesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1603,7 +1653,7 @@ type LabelNamesForMetricNameRequest struct { func (m *LabelNamesForMetricNameRequest) Reset() { *m = LabelNamesForMetricNameRequest{} } func (*LabelNamesForMetricNameRequest) ProtoMessage() {} func (*LabelNamesForMetricNameRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{28} + return fileDescriptor_c28a5f14f1f4c79a, []int{29} } func (m *LabelNamesForMetricNameRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1647,7 +1697,7 @@ type LineFilterExpression struct { func (m *LineFilterExpression) Reset() { *m = LineFilterExpression{} } func (*LineFilterExpression) ProtoMessage() {} func (*LineFilterExpression) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{29} + return fileDescriptor_c28a5f14f1f4c79a, []int{30} } func (m *LineFilterExpression) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1700,7 +1750,7 @@ type GetChunkRefRequest struct { func (m *GetChunkRefRequest) Reset() { *m = GetChunkRefRequest{} } func (*GetChunkRefRequest) ProtoMessage() {} func (*GetChunkRefRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{30} + return fileDescriptor_c28a5f14f1f4c79a, []int{31} } func (m *GetChunkRefRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1750,7 +1800,7 @@ type GetChunkRefResponse struct { func (m *GetChunkRefResponse) Reset() { *m = GetChunkRefResponse{} } func (*GetChunkRefResponse) ProtoMessage() {} func (*GetChunkRefResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{31} + return fileDescriptor_c28a5f14f1f4c79a, []int{32} } func (m *GetChunkRefResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1795,7 +1845,7 @@ type GetSeriesRequest struct { func (m *GetSeriesRequest) Reset() { *m = GetSeriesRequest{} } func (*GetSeriesRequest) ProtoMessage() {} func (*GetSeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{32} + return fileDescriptor_c28a5f14f1f4c79a, []int{33} } func (m *GetSeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1838,7 +1888,7 @@ type GetSeriesResponse struct { func (m *GetSeriesResponse) Reset() { *m = GetSeriesResponse{} } func (*GetSeriesResponse) ProtoMessage() {} func (*GetSeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{33} + return fileDescriptor_c28a5f14f1f4c79a, []int{34} } func (m *GetSeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1882,7 +1932,7 @@ type IndexSeries struct { func (m *IndexSeries) Reset() { *m = IndexSeries{} } func (*IndexSeries) ProtoMessage() {} func (*IndexSeries) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{34} + return fileDescriptor_c28a5f14f1f4c79a, []int{35} } func (m *IndexSeries) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1919,7 +1969,7 @@ type QueryIndexResponse struct { func (m *QueryIndexResponse) Reset() { *m = QueryIndexResponse{} } func (*QueryIndexResponse) ProtoMessage() {} func (*QueryIndexResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{35} + return fileDescriptor_c28a5f14f1f4c79a, []int{36} } func (m *QueryIndexResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1970,7 +2020,7 @@ type Row struct { func (m *Row) Reset() { *m = Row{} } func (*Row) ProtoMessage() {} func (*Row) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{36} + return fileDescriptor_c28a5f14f1f4c79a, []int{37} } func (m *Row) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2020,7 +2070,7 @@ type QueryIndexRequest struct { func (m *QueryIndexRequest) Reset() { *m = QueryIndexRequest{} } func (*QueryIndexRequest) ProtoMessage() {} func (*QueryIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{37} + return fileDescriptor_c28a5f14f1f4c79a, []int{38} } func (m *QueryIndexRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2067,7 +2117,7 @@ type IndexQuery struct { func (m *IndexQuery) Reset() { *m = IndexQuery{} } func (*IndexQuery) ProtoMessage() {} func (*IndexQuery) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{38} + return fileDescriptor_c28a5f14f1f4c79a, []int{39} } func (m *IndexQuery) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2140,7 +2190,7 @@ type IndexStatsRequest struct { func (m *IndexStatsRequest) Reset() { *m = IndexStatsRequest{} } func (*IndexStatsRequest) ProtoMessage() {} func (*IndexStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{39} + return fileDescriptor_c28a5f14f1f4c79a, []int{40} } func (m *IndexStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2186,7 +2236,7 @@ type IndexStatsResponse struct { func (m *IndexStatsResponse) Reset() { *m = IndexStatsResponse{} } func (*IndexStatsResponse) ProtoMessage() {} func (*IndexStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{40} + return fileDescriptor_c28a5f14f1f4c79a, []int{41} } func (m *IndexStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2256,7 +2306,7 @@ type VolumeRequest struct { func (m *VolumeRequest) Reset() { *m = VolumeRequest{} } func (*VolumeRequest) ProtoMessage() {} func (*VolumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{41} + return fileDescriptor_c28a5f14f1f4c79a, []int{42} } func (m *VolumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2328,7 +2378,7 @@ type VolumeResponse struct { func (m *VolumeResponse) Reset() { *m = VolumeResponse{} } func (*VolumeResponse) ProtoMessage() {} func (*VolumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{42} + return fileDescriptor_c28a5f14f1f4c79a, []int{43} } func (m *VolumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2379,7 +2429,7 @@ type Volume struct { func (m *Volume) Reset() { *m = Volume{} } func (*Volume) ProtoMessage() {} func (*Volume) Descriptor() ([]byte, []int) { - return fileDescriptor_c28a5f14f1f4c79a, []int{43} + return fileDescriptor_c28a5f14f1f4c79a, []int{44} } func (m *Volume) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2429,6 +2479,7 @@ func init() { proto.RegisterType((*StreamRate)(nil), "logproto.StreamRate") proto.RegisterType((*QueryRequest)(nil), "logproto.QueryRequest") proto.RegisterType((*SampleQueryRequest)(nil), "logproto.SampleQueryRequest") + proto.RegisterType((*Plan)(nil), "logproto.Plan") proto.RegisterType((*Delete)(nil), "logproto.Delete") proto.RegisterType((*QueryResponse)(nil), "logproto.QueryResponse") proto.RegisterType((*SampleQueryResponse)(nil), "logproto.SampleQueryResponse") @@ -2474,145 +2525,150 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) } var fileDescriptor_c28a5f14f1f4c79a = []byte{ - // 2202 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0x4d, 0x8f, 0x1b, 0x49, - 0xd5, 0x6d, 0xb7, 0xbf, 0x9e, 0x3d, 0x93, 0x49, 0x8d, 0x37, 0xb1, 0x9c, 0xc4, 0x9e, 0x94, 0x96, - 0xec, 0x28, 0x9b, 0xb5, 0x37, 0xb3, 0xb0, 0x64, 0x13, 0x16, 0x88, 0x67, 0xf2, 0x31, 0xc9, 0xe4, - 0x83, 0x9a, 0x10, 0xd0, 0x0a, 0x14, 0xf5, 0xd8, 0x65, 0xbb, 0x15, 0xb7, 0xdb, 0xe9, 0x2e, 0x6f, - 0x32, 0x12, 0x07, 0xfe, 0xc0, 0x4a, 0x7b, 0x43, 0x5c, 0x10, 0x07, 0x24, 0x90, 0x10, 0x17, 0x7e, - 0x00, 0x5c, 0x90, 0x08, 0xb7, 0x70, 0x5b, 0x71, 0x30, 0x64, 0x72, 0x41, 0x73, 0xda, 0x1b, 0x12, - 0x07, 0x84, 0xea, 0xab, 0xbb, 0xec, 0xf1, 0xec, 0xae, 0x43, 0x24, 0x94, 0x8b, 0xbb, 0xde, 0xab, - 0x57, 0xaf, 0xde, 0x77, 0xd5, 0x2b, 0xc3, 0x89, 0xe1, 0xc3, 0x6e, 0xa3, 0xef, 0x77, 0x87, 0x81, - 0xcf, 0xfc, 0x68, 0x50, 0x17, 0xbf, 0x28, 0xa7, 0xe1, 0x4a, 0xa9, 0xeb, 0x77, 0x7d, 0x49, 0xc3, - 0x47, 0x72, 0xbe, 0x52, 0xeb, 0xfa, 0x7e, 0xb7, 0x4f, 0x1b, 0x02, 0xda, 0x19, 0x75, 0x1a, 0xcc, - 0xf5, 0x68, 0xc8, 0x1c, 0x6f, 0xa8, 0x08, 0x56, 0x14, 0xf7, 0x47, 0x7d, 0xcf, 0x6f, 0xd3, 0x7e, - 0x23, 0x64, 0x0e, 0x0b, 0xe5, 0xaf, 0xa2, 0x58, 0xe6, 0x14, 0xc3, 0x51, 0xd8, 0x13, 0x3f, 0x12, - 0x89, 0x4b, 0x80, 0xb6, 0x59, 0x40, 0x1d, 0x8f, 0x38, 0x8c, 0x86, 0x84, 0x3e, 0x1a, 0xd1, 0x90, - 0xe1, 0x5b, 0xb0, 0x3c, 0x81, 0x0d, 0x87, 0xfe, 0x20, 0xa4, 0xe8, 0x7d, 0x28, 0x84, 0x31, 0xba, - 0x6c, 0xad, 0xa4, 0x56, 0x0b, 0x6b, 0xa5, 0x7a, 0xa4, 0x4a, 0xbc, 0x86, 0x98, 0x84, 0xf8, 0x17, - 0x16, 0x40, 0x3c, 0x87, 0xaa, 0x00, 0x72, 0xf6, 0xba, 0x13, 0xf6, 0xca, 0xd6, 0x8a, 0xb5, 0x6a, - 0x13, 0x03, 0x83, 0xce, 0xc1, 0xd1, 0x18, 0xba, 0xed, 0x6f, 0xf7, 0x9c, 0xa0, 0x5d, 0x4e, 0x0a, - 0xb2, 0x83, 0x13, 0x08, 0x81, 0x1d, 0x38, 0x8c, 0x96, 0x53, 0x2b, 0xd6, 0x6a, 0x8a, 0x88, 0x31, - 0x3a, 0x06, 0x19, 0x46, 0x07, 0xce, 0x80, 0x95, 0xed, 0x15, 0x6b, 0x35, 0x4f, 0x14, 0xc4, 0xf1, - 0x5c, 0x77, 0x1a, 0x96, 0xd3, 0x2b, 0xd6, 0xea, 0x02, 0x51, 0x10, 0xfe, 0x73, 0x12, 0x8a, 0xdf, - 0x1b, 0xd1, 0x60, 0x57, 0x19, 0x00, 0x55, 0x20, 0x17, 0xd2, 0x3e, 0x6d, 0x31, 0x3f, 0x10, 0x02, - 0xe6, 0x49, 0x04, 0xa3, 0x12, 0xa4, 0xfb, 0xae, 0xe7, 0x32, 0x21, 0xd2, 0x02, 0x91, 0x00, 0xba, - 0x08, 0xe9, 0x90, 0x39, 0x01, 0x13, 0x72, 0x14, 0xd6, 0x2a, 0x75, 0xe9, 0xb0, 0xba, 0x76, 0x58, - 0xfd, 0x9e, 0x76, 0x58, 0x33, 0xf7, 0x74, 0x5c, 0x4b, 0x7c, 0xfa, 0xf7, 0x9a, 0x45, 0xe4, 0x12, - 0xf4, 0x3e, 0xa4, 0xe8, 0xa0, 0x2d, 0x64, 0xfd, 0xaa, 0x2b, 0xf9, 0x02, 0x74, 0x1e, 0xf2, 0x6d, - 0x37, 0xa0, 0x2d, 0xe6, 0xfa, 0x03, 0xa1, 0xd1, 0xe2, 0xda, 0x72, 0xec, 0x8d, 0x0d, 0x3d, 0x45, - 0x62, 0x2a, 0x74, 0x0e, 0x32, 0x21, 0x37, 0x5b, 0x58, 0xce, 0xae, 0xa4, 0x56, 0xf3, 0xcd, 0xd2, - 0xfe, 0xb8, 0xb6, 0x24, 0x31, 0xe7, 0x7c, 0xcf, 0x65, 0xd4, 0x1b, 0xb2, 0x5d, 0xa2, 0x68, 0xd0, - 0x59, 0xc8, 0xb6, 0x69, 0x9f, 0x72, 0x67, 0xe7, 0x84, 0xb3, 0x97, 0x0c, 0xf6, 0x62, 0x82, 0x68, - 0x82, 0x1b, 0x76, 0x2e, 0xb3, 0x94, 0xc5, 0xff, 0xb1, 0x00, 0x6d, 0x3b, 0xde, 0xb0, 0x4f, 0xbf, - 0xb2, 0x3d, 0x23, 0xcb, 0x25, 0x5f, 0xda, 0x72, 0xa9, 0x79, 0x2d, 0x17, 0x9b, 0xc1, 0x9e, 0xcf, - 0x0c, 0xe9, 0x2f, 0x31, 0x03, 0xde, 0x82, 0x8c, 0x44, 0x7d, 0x59, 0x0c, 0xc5, 0x3a, 0xa7, 0xb4, - 0x36, 0x4b, 0xb1, 0x36, 0x29, 0x21, 0x27, 0xfe, 0xa5, 0x05, 0x0b, 0xca, 0x90, 0x2a, 0x07, 0x77, - 0x20, 0x2b, 0x73, 0x40, 0xe7, 0xdf, 0xf1, 0xe9, 0xfc, 0xbb, 0xdc, 0x76, 0x86, 0x8c, 0x06, 0xcd, - 0xc6, 0xd3, 0x71, 0xcd, 0xfa, 0xdb, 0xb8, 0xf6, 0x56, 0xd7, 0x65, 0xbd, 0xd1, 0x4e, 0xbd, 0xe5, - 0x7b, 0x8d, 0x6e, 0xe0, 0x74, 0x9c, 0x81, 0xd3, 0xe8, 0xfb, 0x0f, 0xdd, 0x86, 0xae, 0x07, 0x3a, - 0x6f, 0x35, 0x63, 0xf4, 0xb6, 0x90, 0x8e, 0x85, 0xca, 0x23, 0x47, 0xea, 0xb2, 0x8c, 0x6c, 0x0e, - 0xba, 0x34, 0xe4, 0x9c, 0x6d, 0x6e, 0x4c, 0x22, 0x69, 0xf0, 0x4f, 0x60, 0x79, 0xc2, 0xe1, 0x4a, - 0xce, 0x0b, 0x90, 0x09, 0x69, 0xe0, 0x46, 0x65, 0xc2, 0x30, 0xd9, 0xb6, 0xc0, 0x37, 0x17, 0x95, - 0x7c, 0x19, 0x09, 0x13, 0x45, 0x3f, 0xdf, 0xee, 0x7f, 0xb2, 0xa0, 0xb8, 0xe5, 0xec, 0xd0, 0xbe, - 0x8e, 0x34, 0x04, 0xf6, 0xc0, 0xf1, 0xa8, 0xb2, 0xb8, 0x18, 0xf3, 0xb4, 0xff, 0xd8, 0xe9, 0x8f, - 0xa8, 0x64, 0x99, 0x23, 0x0a, 0x9a, 0x37, 0x67, 0xad, 0x97, 0xce, 0x59, 0x2b, 0x8e, 0xbc, 0x12, - 0xa4, 0x1f, 0x71, 0x43, 0x89, 0x7c, 0xcd, 0x13, 0x09, 0xe0, 0xb7, 0x60, 0x41, 0x69, 0xa1, 0xcc, - 0x17, 0x8b, 0xcc, 0xcd, 0x97, 0xd7, 0x22, 0x63, 0x0f, 0x32, 0xd2, 0xda, 0xe8, 0x4d, 0xc8, 0x47, - 0x67, 0x80, 0xd0, 0x36, 0xd5, 0xcc, 0xec, 0x8f, 0x6b, 0x49, 0x16, 0x92, 0x78, 0x02, 0xd5, 0x20, - 0x2d, 0x56, 0x0a, 0xcd, 0xad, 0x66, 0x7e, 0x7f, 0x5c, 0x93, 0x08, 0x22, 0x3f, 0xe8, 0x24, 0xd8, - 0x3d, 0x5e, 0x86, 0xb9, 0x09, 0xec, 0x66, 0x6e, 0x7f, 0x5c, 0x13, 0x30, 0x11, 0xbf, 0xf8, 0x1a, - 0x14, 0xb7, 0x68, 0xd7, 0x69, 0xed, 0xaa, 0x4d, 0x4b, 0x9a, 0x1d, 0xdf, 0xd0, 0xd2, 0x3c, 0x4e, - 0x43, 0x31, 0xda, 0xf1, 0x81, 0x17, 0xaa, 0xa0, 0x2e, 0x44, 0xb8, 0x5b, 0x21, 0xfe, 0xb9, 0x05, - 0xca, 0xcf, 0x08, 0x43, 0xa6, 0xcf, 0x75, 0x0d, 0xa5, 0x8f, 0x9a, 0xb0, 0x3f, 0xae, 0x29, 0x0c, - 0x51, 0x5f, 0x74, 0x09, 0xb2, 0xa1, 0xd8, 0x91, 0x33, 0x9b, 0x0e, 0x1f, 0x31, 0xd1, 0x3c, 0xc2, - 0xc3, 0x60, 0x7f, 0x5c, 0xd3, 0x84, 0x44, 0x0f, 0x50, 0x7d, 0xe2, 0x7c, 0x91, 0x8a, 0x2d, 0xee, - 0x8f, 0x6b, 0x06, 0xd6, 0x3c, 0x6f, 0xf0, 0xcf, 0x2c, 0x28, 0xdc, 0x73, 0xdc, 0x28, 0x84, 0x22, - 0x17, 0x59, 0x86, 0x8b, 0x78, 0x3a, 0xb7, 0x69, 0xdf, 0xd9, 0xbd, 0xea, 0x07, 0x82, 0xe7, 0x02, - 0x89, 0xe0, 0xf8, 0x48, 0xb0, 0x67, 0x1e, 0x09, 0xe9, 0xb9, 0x0b, 0xdb, 0x0d, 0x3b, 0x97, 0x5c, - 0x4a, 0xe1, 0xdf, 0x59, 0x50, 0x94, 0x92, 0xa9, 0xb0, 0xf8, 0x11, 0x64, 0xa4, 0xe0, 0x42, 0xb6, - 0x2f, 0x48, 0xfe, 0xb7, 0xe7, 0x49, 0x7c, 0xc5, 0x13, 0x7d, 0x07, 0x16, 0xdb, 0x81, 0x3f, 0x1c, - 0xd2, 0xf6, 0xb6, 0x2a, 0x31, 0xc9, 0xe9, 0x12, 0xb3, 0x61, 0xce, 0x93, 0x29, 0x72, 0xfc, 0x17, - 0x0b, 0x16, 0x54, 0x36, 0x2b, 0x5b, 0x46, 0x36, 0xb0, 0x5e, 0xba, 0xb8, 0x27, 0xe7, 0x2d, 0xee, - 0xc7, 0x20, 0xd3, 0x0d, 0xfc, 0xd1, 0x30, 0x2c, 0xa7, 0x64, 0xee, 0x48, 0x68, 0xbe, 0xa2, 0x8f, - 0x6f, 0xc0, 0xa2, 0x56, 0xe5, 0x90, 0x92, 0x56, 0x99, 0x2e, 0x69, 0x9b, 0x6d, 0x3a, 0x60, 0x6e, - 0xc7, 0x8d, 0x8a, 0x94, 0xa2, 0xc7, 0x9f, 0x58, 0xb0, 0x34, 0x4d, 0x82, 0xbe, 0x6d, 0xe4, 0x01, - 0x67, 0x77, 0xe6, 0x70, 0x76, 0x75, 0x51, 0x1c, 0xc2, 0x2b, 0x03, 0x16, 0xec, 0xea, 0x1c, 0xa9, - 0x7c, 0x00, 0x05, 0x03, 0xcd, 0x0f, 0x8f, 0x87, 0x54, 0xc7, 0x2c, 0x1f, 0xc6, 0xc9, 0x9a, 0x94, - 0x71, 0x2c, 0x80, 0x8b, 0xc9, 0x0b, 0x16, 0x8f, 0xf8, 0x85, 0x09, 0x4f, 0xa2, 0x0b, 0x60, 0x77, - 0x02, 0xdf, 0x9b, 0xcb, 0x4d, 0x62, 0x05, 0xfa, 0x3a, 0x24, 0x99, 0x3f, 0x97, 0x93, 0x92, 0xcc, - 0xe7, 0x3e, 0x52, 0xca, 0xa7, 0xe4, 0x0d, 0x4d, 0x42, 0xf8, 0x1b, 0x90, 0x17, 0x4a, 0xdd, 0x75, - 0xdc, 0x60, 0x66, 0x2d, 0x9f, 0xa9, 0x14, 0xbe, 0x04, 0x47, 0x64, 0x9d, 0x9a, 0xbd, 0xb8, 0x38, - 0x6b, 0x71, 0x51, 0x2f, 0x3e, 0x01, 0xe9, 0xf5, 0xde, 0x68, 0xf0, 0x90, 0x2f, 0x69, 0x3b, 0xcc, - 0xd1, 0x4b, 0xf8, 0x18, 0xbf, 0x01, 0xcb, 0x3c, 0x03, 0x69, 0x10, 0xae, 0xfb, 0xa3, 0x01, 0xd3, - 0x37, 0xe4, 0x73, 0x50, 0x9a, 0x44, 0xab, 0x18, 0x29, 0x41, 0xba, 0xc5, 0x11, 0x82, 0xc7, 0x02, - 0x91, 0x00, 0xfe, 0x95, 0x05, 0xe8, 0x1a, 0x65, 0x62, 0x97, 0xcd, 0x8d, 0xd0, 0xb8, 0x15, 0x79, - 0x0e, 0x6b, 0xf5, 0x68, 0x10, 0xea, 0x1b, 0x82, 0x86, 0xff, 0x1f, 0xb7, 0x22, 0x7c, 0x1e, 0x96, - 0x27, 0xa4, 0x54, 0x3a, 0x55, 0x20, 0xd7, 0x52, 0x38, 0x75, 0x1a, 0x45, 0x30, 0xfe, 0x7d, 0x12, - 0x72, 0x62, 0x01, 0xa1, 0x1d, 0x74, 0x1e, 0x0a, 0x1d, 0x77, 0xd0, 0xa5, 0xc1, 0x30, 0x70, 0x95, - 0x09, 0xec, 0xe6, 0x91, 0xfd, 0x71, 0xcd, 0x44, 0x13, 0x13, 0x40, 0xef, 0x40, 0x76, 0x14, 0xd2, - 0xe0, 0x81, 0x2b, 0xf3, 0x3c, 0xdf, 0x2c, 0xed, 0x8d, 0x6b, 0x99, 0xef, 0x87, 0x34, 0xd8, 0xdc, - 0xe0, 0xe7, 0xc2, 0x48, 0x8c, 0x88, 0xfc, 0xb6, 0xd1, 0x4d, 0x15, 0xa6, 0xe2, 0x8a, 0xd4, 0xfc, - 0x26, 0x17, 0x7f, 0xaa, 0xd0, 0x0d, 0x03, 0xdf, 0xa3, 0xac, 0x47, 0x47, 0x61, 0xa3, 0xe5, 0x7b, - 0x9e, 0x3f, 0x68, 0x88, 0x7e, 0x48, 0x28, 0xcd, 0x0f, 0x37, 0xbe, 0x5c, 0x45, 0xee, 0x3d, 0xc8, - 0xb2, 0x5e, 0xe0, 0x8f, 0xba, 0x3d, 0x51, 0xb7, 0x53, 0xcd, 0x8b, 0xf3, 0xf3, 0xd3, 0x1c, 0x88, - 0x1e, 0xa0, 0xd3, 0xdc, 0x5a, 0xb4, 0xf5, 0x30, 0x1c, 0x79, 0xb2, 0xcb, 0x68, 0xa6, 0xf7, 0xc7, - 0x35, 0xeb, 0x1d, 0x12, 0xa1, 0xf1, 0x27, 0x49, 0xa8, 0x89, 0x40, 0xbd, 0x2f, 0x0e, 0xf5, 0xab, - 0x7e, 0x70, 0x8b, 0xb2, 0xc0, 0x6d, 0xdd, 0x76, 0x3c, 0xaa, 0x63, 0xa3, 0x06, 0x05, 0x4f, 0x20, - 0x1f, 0x18, 0x29, 0x00, 0x5e, 0x44, 0x87, 0x4e, 0x01, 0x88, 0x9c, 0x91, 0xf3, 0x32, 0x1b, 0xf2, - 0x02, 0x23, 0xa6, 0xd7, 0x27, 0x2c, 0xd5, 0x98, 0x53, 0x33, 0x65, 0xa1, 0xcd, 0x69, 0x0b, 0xcd, - 0xcd, 0x27, 0x32, 0x8b, 0x19, 0xeb, 0xe9, 0xc9, 0x58, 0xc7, 0x7f, 0xb5, 0xa0, 0xba, 0xa5, 0x25, - 0x7f, 0x49, 0x73, 0x68, 0x7d, 0x93, 0xaf, 0x48, 0xdf, 0xd4, 0xff, 0xa6, 0x2f, 0xbe, 0x0e, 0xa5, - 0x2d, 0x77, 0x40, 0xaf, 0xba, 0x7d, 0x46, 0x83, 0x2b, 0x4f, 0x86, 0x01, 0x0d, 0x43, 0xde, 0x80, - 0x55, 0x20, 0xe7, 0x0f, 0x69, 0xe0, 0xe8, 0xae, 0x20, 0x45, 0x22, 0x98, 0x17, 0x0f, 0x61, 0x13, - 0x5d, 0xdb, 0x04, 0x80, 0xff, 0x6d, 0x14, 0x0f, 0x42, 0x3b, 0xda, 0x22, 0xeb, 0x46, 0xc5, 0x7e, - 0x15, 0x0a, 0x27, 0x5f, 0xa1, 0x83, 0x53, 0x53, 0xc5, 0xec, 0x02, 0x64, 0x3b, 0xc2, 0x10, 0xf2, - 0xe8, 0x2d, 0xac, 0x55, 0xe3, 0xb3, 0x6e, 0x96, 0x95, 0x88, 0x26, 0xc7, 0x1f, 0xc6, 0x25, 0x49, - 0xe8, 0xae, 0x4a, 0xd2, 0x19, 0xb0, 0x03, 0xda, 0xd1, 0x27, 0x27, 0x8a, 0xb9, 0x45, 0x94, 0x62, - 0x1e, 0xff, 0xc1, 0x82, 0xa5, 0x6b, 0x94, 0x4d, 0xde, 0x49, 0x5e, 0x23, 0xcb, 0xe1, 0xeb, 0x70, - 0xd4, 0x90, 0x5f, 0x69, 0xff, 0xde, 0xd4, 0x45, 0xe4, 0x8d, 0x58, 0xff, 0xcd, 0x41, 0x9b, 0x3e, - 0x51, 0x0d, 0xd6, 0xe4, 0x1d, 0xe4, 0x2e, 0x14, 0x8c, 0x49, 0x74, 0x79, 0xea, 0xf6, 0x61, 0x3c, - 0x1c, 0x44, 0x67, 0x68, 0xb3, 0xa4, 0x74, 0x92, 0x2d, 0x96, 0xba, 0x5b, 0x46, 0x67, 0xf5, 0x36, - 0x20, 0xd1, 0xf3, 0x09, 0xb6, 0xe6, 0x69, 0x21, 0xb0, 0x37, 0xa3, 0xcb, 0x48, 0x04, 0xa3, 0xd3, - 0x60, 0x07, 0xfe, 0x63, 0x7d, 0xad, 0x5c, 0x88, 0xb7, 0x24, 0xfe, 0x63, 0x22, 0xa6, 0xf0, 0x25, - 0x48, 0x11, 0xff, 0x31, 0xaa, 0x02, 0x04, 0xce, 0xa0, 0x4b, 0xef, 0x47, 0xdd, 0x46, 0x91, 0x18, - 0x98, 0x43, 0x4e, 0xf2, 0x75, 0x38, 0x6a, 0x4a, 0x24, 0xdd, 0x5d, 0x87, 0x2c, 0x47, 0xba, 0xb3, - 0x5e, 0xac, 0x04, 0xa1, 0x6c, 0x5c, 0x35, 0x11, 0x8f, 0x19, 0x88, 0xf1, 0xe8, 0x24, 0xe4, 0x99, - 0xb3, 0xd3, 0xa7, 0xb7, 0xe3, 0xba, 0x13, 0x23, 0xf8, 0x2c, 0x6f, 0x94, 0xee, 0x1b, 0x57, 0x92, - 0x18, 0x81, 0xce, 0xc2, 0x52, 0x2c, 0xf3, 0xdd, 0x80, 0x76, 0xdc, 0x27, 0xc2, 0xc3, 0x45, 0x72, - 0x00, 0x8f, 0x56, 0xe1, 0x48, 0x8c, 0xdb, 0x16, 0x47, 0xbf, 0x2d, 0x48, 0xa7, 0xd1, 0xdc, 0x36, - 0x42, 0xdd, 0x2b, 0x8f, 0x46, 0x4e, 0x5f, 0x14, 0xd3, 0x22, 0x31, 0x30, 0xf8, 0x8f, 0x16, 0x1c, - 0x95, 0xae, 0xe6, 0x2d, 0xf2, 0xeb, 0x18, 0xf5, 0xbf, 0xb6, 0x00, 0x99, 0x1a, 0xa8, 0xd0, 0xfa, - 0x9a, 0xf9, 0xf6, 0xc1, 0xef, 0x16, 0x05, 0xd1, 0xff, 0x49, 0x54, 0xfc, 0x7c, 0x81, 0x21, 0x23, - 0xee, 0x27, 0xb2, 0x11, 0xb5, 0x65, 0x83, 0x29, 0x31, 0x44, 0x7d, 0x79, 0x5f, 0xbc, 0xb3, 0xcb, - 0x68, 0xa8, 0xda, 0x43, 0xd1, 0x17, 0x0b, 0x04, 0x91, 0x1f, 0xbe, 0x17, 0x1d, 0x30, 0x11, 0x35, - 0x76, 0xbc, 0x97, 0x42, 0x11, 0x3d, 0xc0, 0xbf, 0x4d, 0xc2, 0xc2, 0x7d, 0xbf, 0x3f, 0x8a, 0x4f, - 0xaa, 0xd7, 0xa9, 0x2e, 0x4f, 0xf4, 0xad, 0x69, 0xdd, 0xb7, 0x22, 0xb0, 0x43, 0x46, 0x87, 0x22, - 0xb2, 0x52, 0x44, 0x8c, 0x11, 0x86, 0x22, 0x73, 0x82, 0x2e, 0x65, 0xb2, 0xe5, 0x28, 0x67, 0xc4, - 0x3d, 0x70, 0x02, 0x87, 0x56, 0xa0, 0xe0, 0x74, 0xbb, 0x01, 0xed, 0x3a, 0x8c, 0x36, 0x77, 0xcb, - 0x59, 0xb1, 0x99, 0x89, 0xc2, 0x3f, 0x84, 0x45, 0x6d, 0x2c, 0xe5, 0xd2, 0x77, 0x21, 0xfb, 0xb1, - 0xc0, 0xcc, 0x78, 0x27, 0x92, 0xa4, 0xaa, 0x8c, 0x69, 0xb2, 0xc9, 0xe7, 0x57, 0x2d, 0x33, 0xbe, - 0x01, 0x19, 0x49, 0x8e, 0x4e, 0x9a, 0x4d, 0x83, 0x7c, 0xd0, 0xe0, 0xb0, 0xea, 0x00, 0x30, 0x64, - 0x24, 0x23, 0xe5, 0x78, 0x11, 0x1b, 0x12, 0x43, 0xd4, 0xf7, 0xec, 0x19, 0xc8, 0x47, 0x6f, 0xa7, - 0xa8, 0x00, 0xd9, 0xab, 0x77, 0xc8, 0x0f, 0x2e, 0x93, 0x8d, 0xa5, 0x04, 0x2a, 0x42, 0xae, 0x79, - 0x79, 0xfd, 0xa6, 0x80, 0xac, 0xb5, 0x7f, 0xd9, 0xba, 0xb2, 0x04, 0xe8, 0x5b, 0x90, 0x96, 0xe5, - 0xe2, 0x58, 0x2c, 0xbf, 0xf9, 0x02, 0x5a, 0x39, 0x7e, 0x00, 0x2f, 0x2d, 0x80, 0x13, 0xef, 0x5a, - 0xe8, 0x36, 0x14, 0x04, 0x52, 0xbd, 0xb2, 0x9c, 0x9c, 0x7e, 0xec, 0x98, 0xe0, 0x74, 0xea, 0x90, - 0x59, 0x83, 0xdf, 0x45, 0x48, 0x0b, 0x9f, 0x98, 0xd2, 0x98, 0xaf, 0x64, 0xa6, 0x34, 0x13, 0xef, - 0x4e, 0x38, 0x81, 0x3e, 0x00, 0x9b, 0x77, 0x36, 0xc8, 0x38, 0x54, 0x8c, 0xc7, 0x91, 0xca, 0xb1, - 0x69, 0xb4, 0xb1, 0xed, 0x87, 0xd1, 0x1b, 0xcf, 0xf1, 0xe9, 0x5e, 0x56, 0x2f, 0x2f, 0x1f, 0x9c, - 0x88, 0x76, 0xbe, 0x23, 0x1f, 0x3b, 0x74, 0x4f, 0x85, 0x4e, 0x4d, 0x6e, 0x35, 0xd5, 0x82, 0x55, - 0xaa, 0x87, 0x4d, 0x47, 0x0c, 0xb7, 0xa0, 0x60, 0xf4, 0x33, 0xa6, 0x59, 0x0f, 0x36, 0x63, 0xa6, - 0x59, 0x67, 0x34, 0x41, 0x38, 0x81, 0xae, 0x41, 0x8e, 0x1f, 0xc5, 0xbc, 0x22, 0xa1, 0x13, 0xd3, - 0x27, 0xae, 0x51, 0x69, 0x2b, 0x27, 0x67, 0x4f, 0x46, 0x8c, 0xbe, 0x0b, 0xf9, 0x6b, 0x94, 0xa9, - 0x70, 0x3d, 0x3e, 0x1d, 0xef, 0x33, 0x2c, 0x35, 0x99, 0x33, 0x38, 0xb1, 0xf6, 0x63, 0xfd, 0x7f, - 0xca, 0x86, 0xc3, 0x1c, 0x74, 0x07, 0x16, 0x85, 0x60, 0xd1, 0x1f, 0x2e, 0x13, 0x01, 0x74, 0xe0, - 0xdf, 0x9d, 0x89, 0x00, 0x3a, 0xf8, 0x2f, 0x0f, 0x4e, 0x34, 0x3f, 0x7a, 0xf6, 0xbc, 0x9a, 0xf8, - 0xec, 0x79, 0x35, 0xf1, 0xf9, 0xf3, 0xaa, 0xf5, 0xd3, 0xbd, 0xaa, 0xf5, 0x9b, 0xbd, 0xaa, 0xf5, - 0x74, 0xaf, 0x6a, 0x3d, 0xdb, 0xab, 0x5a, 0xff, 0xd8, 0xab, 0x5a, 0xff, 0xdc, 0xab, 0x26, 0x3e, - 0xdf, 0xab, 0x5a, 0x9f, 0xbe, 0xa8, 0x26, 0x9e, 0xbd, 0xa8, 0x26, 0x3e, 0x7b, 0x51, 0x4d, 0x7c, - 0xf4, 0xe6, 0x17, 0x3d, 0x37, 0xe9, 0x1d, 0x77, 0x32, 0xe2, 0xf3, 0xde, 0x7f, 0x03, 0x00, 0x00, - 0xff, 0xff, 0xc7, 0xff, 0x87, 0xf1, 0x0e, 0x1b, 0x00, 0x00, + // 2281 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x19, 0x4d, 0x6f, 0x1b, 0xc7, + 0x95, 0x4b, 0x2e, 0xbf, 0x1e, 0x29, 0x59, 0x1e, 0x31, 0x36, 0x41, 0xdb, 0xa4, 0x3c, 0x48, 0x1d, + 0xc1, 0x71, 0xc8, 0x58, 0x69, 0x5d, 0xc7, 0x6e, 0xda, 0x9a, 0x92, 0x3f, 0x64, 0xcb, 0x1f, 0x19, + 0xb9, 0x6e, 0x61, 0xb4, 0x30, 0x56, 0xe2, 0x88, 0x22, 0xcc, 0xe5, 0xd2, 0xbb, 0xc3, 0xd8, 0x02, + 0x7a, 0xe8, 0x1f, 0x08, 0x90, 0x5b, 0xd1, 0x4b, 0xd1, 0x43, 0x81, 0x14, 0x28, 0x7a, 0xe9, 0x0f, + 0x68, 0x2f, 0x3d, 0xb8, 0x37, 0xf7, 0x16, 0xe4, 0xc0, 0xd6, 0xf2, 0xa1, 0x85, 0x4e, 0xb9, 0x15, + 0xe8, 0xa9, 0x98, 0xaf, 0xdd, 0xd9, 0x15, 0x65, 0x84, 0x8e, 0x81, 0xc0, 0x17, 0xee, 0xbc, 0x37, + 0x6f, 0xde, 0xbc, 0xaf, 0x79, 0x6f, 0xde, 0x10, 0x8e, 0x0d, 0x1f, 0x76, 0x5b, 0x7d, 0xaf, 0x3b, + 0xf4, 0x3d, 0xe6, 0x85, 0x83, 0xa6, 0xf8, 0x45, 0x05, 0x0d, 0xd7, 0x2a, 0x5d, 0xaf, 0xeb, 0x49, + 0x1a, 0x3e, 0x92, 0xf3, 0xb5, 0x46, 0xd7, 0xf3, 0xba, 0x7d, 0xda, 0x12, 0xd0, 0xc6, 0x68, 0xab, + 0xc5, 0x7a, 0x2e, 0x0d, 0x98, 0xe3, 0x0e, 0x15, 0xc1, 0x82, 0xe2, 0xfe, 0xa8, 0xef, 0x7a, 0x1d, + 0xda, 0x6f, 0x05, 0xcc, 0x61, 0x81, 0xfc, 0x55, 0x14, 0xf3, 0x9c, 0x62, 0x38, 0x0a, 0xb6, 0xc5, + 0x8f, 0x44, 0xe2, 0x0a, 0xa0, 0x75, 0xe6, 0x53, 0xc7, 0x25, 0x0e, 0xa3, 0x01, 0xa1, 0x8f, 0x46, + 0x34, 0x60, 0xf8, 0x26, 0xcc, 0xc7, 0xb0, 0xc1, 0xd0, 0x1b, 0x04, 0x14, 0x9d, 0x83, 0x52, 0x10, + 0xa1, 0xab, 0xd6, 0x42, 0x66, 0xb1, 0xb4, 0x54, 0x69, 0x86, 0xaa, 0x44, 0x6b, 0x88, 0x49, 0x88, + 0x7f, 0x6b, 0x01, 0x44, 0x73, 0xa8, 0x0e, 0x20, 0x67, 0xaf, 0x39, 0xc1, 0x76, 0xd5, 0x5a, 0xb0, + 0x16, 0x6d, 0x62, 0x60, 0xd0, 0x19, 0x38, 0x1c, 0x41, 0xb7, 0xbc, 0xf5, 0x6d, 0xc7, 0xef, 0x54, + 0xd3, 0x82, 0x6c, 0xff, 0x04, 0x42, 0x60, 0xfb, 0x0e, 0xa3, 0xd5, 0xcc, 0x82, 0xb5, 0x98, 0x21, + 0x62, 0x8c, 0x8e, 0x40, 0x8e, 0xd1, 0x81, 0x33, 0x60, 0x55, 0x7b, 0xc1, 0x5a, 0x2c, 0x12, 0x05, + 0x71, 0x3c, 0xd7, 0x9d, 0x06, 0xd5, 0xec, 0x82, 0xb5, 0x38, 0x43, 0x14, 0x84, 0x3f, 0xcf, 0x40, + 0xf9, 0xe3, 0x11, 0xf5, 0x77, 0x94, 0x01, 0x50, 0x1d, 0x0a, 0x01, 0xed, 0xd3, 0x4d, 0xe6, 0xf9, + 0x42, 0xc0, 0x62, 0x3b, 0x5d, 0xb5, 0x48, 0x88, 0x43, 0x15, 0xc8, 0xf6, 0x7b, 0x6e, 0x8f, 0x09, + 0xb1, 0x66, 0x88, 0x04, 0xd0, 0x05, 0xc8, 0x06, 0xcc, 0xf1, 0x99, 0x90, 0xa5, 0xb4, 0x54, 0x6b, + 0x4a, 0xa7, 0x35, 0xb5, 0xd3, 0x9a, 0x77, 0xb5, 0xd3, 0xda, 0x85, 0xa7, 0xe3, 0x46, 0xea, 0xb3, + 0x7f, 0x36, 0x2c, 0x22, 0x97, 0xa0, 0x73, 0x90, 0xa1, 0x83, 0x8e, 0x90, 0xf7, 0xeb, 0xae, 0xe4, + 0x0b, 0xd0, 0x59, 0x28, 0x76, 0x7a, 0x3e, 0xdd, 0x64, 0x3d, 0x6f, 0x20, 0xb4, 0x9a, 0x5d, 0x9a, + 0x8f, 0x3c, 0xb2, 0xa2, 0xa7, 0x48, 0x44, 0x85, 0xce, 0x40, 0x2e, 0xe0, 0xa6, 0x0b, 0xaa, 0xf9, + 0x85, 0xcc, 0x62, 0xb1, 0x5d, 0xd9, 0x1b, 0x37, 0xe6, 0x24, 0xe6, 0x8c, 0xe7, 0xf6, 0x18, 0x75, + 0x87, 0x6c, 0x87, 0x28, 0x1a, 0x74, 0x1a, 0xf2, 0x1d, 0xda, 0xa7, 0xdc, 0xe1, 0x05, 0xe1, 0xf0, + 0x39, 0x83, 0xbd, 0x98, 0x20, 0x9a, 0x00, 0xdd, 0x07, 0x7b, 0xd8, 0x77, 0x06, 0xd5, 0xa2, 0xd0, + 0x62, 0x36, 0x22, 0xbc, 0xd3, 0x77, 0x06, 0xed, 0x73, 0x5f, 0x8e, 0x1b, 0x4b, 0xdd, 0x1e, 0xdb, + 0x1e, 0x6d, 0x34, 0x37, 0x3d, 0xb7, 0xd5, 0xf5, 0x9d, 0x2d, 0x67, 0xe0, 0xb4, 0xfa, 0xde, 0xc3, + 0x5e, 0x8b, 0x07, 0xe7, 0xa3, 0x11, 0xf5, 0x7b, 0xd4, 0x6f, 0x71, 0x1e, 0x4d, 0xe1, 0x0f, 0xbe, + 0x8e, 0x08, 0x9e, 0xd7, 0xed, 0x42, 0x6e, 0x2e, 0x8f, 0xc7, 0x69, 0x40, 0xeb, 0x8e, 0x3b, 0xec, + 0xd3, 0xa9, 0xfc, 0x15, 0x7a, 0x26, 0xfd, 0xca, 0x9e, 0xc9, 0x4c, 0xeb, 0x99, 0xc8, 0xcc, 0xf6, + 0x74, 0x66, 0xce, 0x7e, 0x5d, 0x33, 0xe7, 0x5e, 0xbf, 0x99, 0x71, 0x15, 0x6c, 0x0e, 0xa1, 0x39, + 0xc8, 0xf8, 0xce, 0x63, 0x61, 0xcc, 0x32, 0xe1, 0x43, 0xbc, 0x06, 0x39, 0x29, 0x08, 0xaa, 0x25, + 0xad, 0x1d, 0x3f, 0x19, 0x91, 0xa5, 0x33, 0xda, 0x86, 0x73, 0x91, 0x0d, 0x33, 0xc2, 0x3a, 0xf8, + 0x77, 0x16, 0xcc, 0x28, 0x17, 0xaa, 0xec, 0xb2, 0x01, 0x79, 0x79, 0xba, 0x75, 0x66, 0x39, 0x9a, + 0xcc, 0x2c, 0x97, 0x3a, 0xce, 0x90, 0x51, 0xbf, 0xdd, 0x7a, 0x3a, 0x6e, 0x58, 0x5f, 0x8e, 0x1b, + 0xef, 0xbc, 0x4c, 0x4b, 0x91, 0xe4, 0x54, 0xd6, 0xd1, 0x8c, 0xd1, 0xbb, 0x42, 0x3a, 0x16, 0xa8, + 0x38, 0x38, 0xd4, 0x94, 0x09, 0x72, 0x75, 0xd0, 0xa5, 0x01, 0xe7, 0x6c, 0x73, 0x17, 0x12, 0x49, + 0x83, 0x7f, 0x09, 0xf3, 0xb1, 0x50, 0x53, 0x72, 0x9e, 0x87, 0x5c, 0xc0, 0x0d, 0xa8, 0xc5, 0x34, + 0x1c, 0xb5, 0x2e, 0xf0, 0xed, 0x59, 0x25, 0x5f, 0x4e, 0xc2, 0x44, 0xd1, 0x4f, 0xb7, 0xfb, 0xdf, + 0x2c, 0x28, 0xaf, 0x39, 0x1b, 0xb4, 0xaf, 0x63, 0x1c, 0x81, 0x3d, 0x70, 0x5c, 0xaa, 0x2c, 0x2e, + 0xc6, 0x3c, 0xa1, 0x7d, 0xe2, 0xf4, 0x47, 0x54, 0xb2, 0x2c, 0x10, 0x05, 0x4d, 0x9b, 0x89, 0xac, + 0x57, 0xce, 0x44, 0x56, 0x14, 0xef, 0x15, 0xc8, 0xf2, 0xc8, 0xda, 0x11, 0x59, 0xa8, 0x48, 0x24, + 0x80, 0xdf, 0x81, 0x19, 0xa5, 0x85, 0x32, 0x5f, 0x24, 0x32, 0x37, 0x5f, 0x51, 0x8b, 0x8c, 0x5d, + 0xc8, 0x49, 0x6b, 0xa3, 0xb7, 0xa1, 0x18, 0x56, 0x37, 0xa1, 0x6d, 0xa6, 0x9d, 0xdb, 0x1b, 0x37, + 0xd2, 0x2c, 0x20, 0xd1, 0x04, 0x6a, 0x40, 0x56, 0xac, 0x14, 0x9a, 0x5b, 0xed, 0xe2, 0xde, 0xb8, + 0x21, 0x11, 0x44, 0x7e, 0xd0, 0x71, 0xb0, 0xb7, 0x79, 0x81, 0xe1, 0x26, 0xb0, 0xdb, 0x85, 0xbd, + 0x71, 0x43, 0xc0, 0x44, 0xfc, 0xe2, 0xab, 0x50, 0x5e, 0xa3, 0x5d, 0x67, 0x73, 0x47, 0x6d, 0x5a, + 0xd1, 0xec, 0xf8, 0x86, 0x96, 0xe6, 0x71, 0x12, 0xca, 0xe1, 0x8e, 0x0f, 0xdc, 0x40, 0x05, 0x75, + 0x29, 0xc4, 0xdd, 0x0c, 0xf0, 0x6f, 0x2c, 0x50, 0x7e, 0x46, 0x18, 0x72, 0x7d, 0xae, 0x6b, 0xa0, + 0x72, 0x10, 0xec, 0x8d, 0x1b, 0x0a, 0x43, 0xd4, 0x17, 0x5d, 0x84, 0x7c, 0x20, 0x76, 0xe4, 0xcc, + 0x92, 0xe1, 0x23, 0x26, 0xda, 0x87, 0x78, 0x18, 0xec, 0x8d, 0x1b, 0x9a, 0x90, 0xe8, 0x01, 0x6a, + 0xc6, 0x2a, 0xa7, 0x54, 0x6c, 0x76, 0x6f, 0xdc, 0x30, 0xb0, 0x66, 0x25, 0xc5, 0xff, 0xb6, 0xa0, + 0x74, 0xd7, 0xe9, 0x85, 0x21, 0x54, 0xd5, 0x2e, 0x8a, 0x72, 0xa4, 0x44, 0xf0, 0x23, 0xdd, 0xa1, + 0x7d, 0x67, 0xe7, 0x8a, 0xe7, 0x0b, 0xbe, 0x33, 0x24, 0x84, 0xa3, 0x62, 0x67, 0x4f, 0x2c, 0x76, + 0xd9, 0xe9, 0x53, 0xea, 0x75, 0x23, 0x81, 0x95, 0xbf, 0x71, 0x5d, 0x48, 0xcf, 0x65, 0xf0, 0x9f, + 0x2c, 0x28, 0x4b, 0x4d, 0x55, 0x98, 0xfd, 0x1c, 0x72, 0xd2, 0x10, 0x42, 0xd7, 0x97, 0x24, 0x93, + 0x77, 0xa7, 0x49, 0x24, 0x8a, 0x27, 0xfa, 0x11, 0xcc, 0x76, 0x7c, 0x6f, 0x38, 0xa4, 0x9d, 0x75, + 0x95, 0xb2, 0xd2, 0xc9, 0x94, 0xb5, 0x62, 0xce, 0x93, 0x04, 0x39, 0xfe, 0xbb, 0x05, 0x33, 0x2a, + 0x3b, 0x28, 0xdf, 0x84, 0xf6, 0xb4, 0x5e, 0xb9, 0x44, 0xa5, 0xa7, 0x2d, 0x51, 0x47, 0x20, 0xd7, + 0xf5, 0xbd, 0xd1, 0x30, 0xa8, 0x66, 0xe4, 0x59, 0x94, 0xd0, 0x74, 0xa5, 0x0b, 0x5f, 0x87, 0x59, + 0xad, 0xca, 0x01, 0x29, 0xb2, 0x96, 0x4c, 0x91, 0xab, 0x1d, 0x3a, 0x60, 0xbd, 0xad, 0x5e, 0x98, + 0xf4, 0x14, 0x3d, 0xfe, 0xd4, 0x82, 0xb9, 0x24, 0x09, 0xfa, 0xa1, 0x71, 0xae, 0x38, 0xbb, 0x53, + 0x07, 0xb3, 0x6b, 0x8a, 0x64, 0x13, 0x5c, 0x1e, 0x30, 0x7f, 0x47, 0x9f, 0xb9, 0xda, 0x87, 0x50, + 0x32, 0xd0, 0xbc, 0x18, 0x3d, 0xa4, 0xea, 0x0c, 0x10, 0x3e, 0x8c, 0x0e, 0x7f, 0x5a, 0xa6, 0x2e, + 0x01, 0x5c, 0x48, 0x9f, 0xb7, 0xf0, 0xaf, 0x2d, 0x98, 0x89, 0x79, 0x12, 0x9d, 0x07, 0x7b, 0xcb, + 0xf7, 0xdc, 0xa9, 0xdc, 0x24, 0x56, 0xa0, 0xef, 0x42, 0x9a, 0x79, 0x53, 0x39, 0x29, 0xcd, 0x3c, + 0xee, 0x23, 0xa5, 0x7c, 0x46, 0xde, 0x65, 0x25, 0x84, 0xbf, 0x07, 0x45, 0xa1, 0xd4, 0x1d, 0xa7, + 0xe7, 0x4f, 0xac, 0x0d, 0x13, 0x95, 0xc2, 0x17, 0xe1, 0x90, 0xcc, 0x7b, 0x93, 0x17, 0x97, 0x27, + 0x2d, 0x2e, 0xeb, 0xc5, 0xc7, 0x20, 0xbb, 0xbc, 0x3d, 0x1a, 0x3c, 0xe4, 0x4b, 0x3a, 0x0e, 0x73, + 0xf4, 0x12, 0x3e, 0xc6, 0x6f, 0xc1, 0x3c, 0x3f, 0x81, 0xd4, 0x0f, 0x96, 0xbd, 0xd1, 0x80, 0xe9, + 0x5e, 0xe2, 0x0c, 0x54, 0xe2, 0x68, 0x15, 0x23, 0x15, 0xc8, 0x6e, 0x72, 0x84, 0xe0, 0x31, 0x43, + 0x24, 0x80, 0x7f, 0x6f, 0x01, 0xba, 0x4a, 0x99, 0xd8, 0x65, 0x75, 0x25, 0x3c, 0x1c, 0x35, 0x28, + 0xb8, 0x0e, 0xdb, 0xdc, 0xa6, 0x7e, 0xa0, 0x6f, 0x1c, 0x1a, 0xfe, 0x36, 0xee, 0x76, 0xf8, 0x2c, + 0xcc, 0xc7, 0xa4, 0x54, 0x3a, 0xd5, 0xa0, 0xb0, 0xa9, 0x70, 0xaa, 0xba, 0x85, 0x30, 0xfe, 0x73, + 0x1a, 0x0a, 0x62, 0x01, 0xa1, 0x5b, 0xe8, 0x2c, 0x94, 0xb6, 0x7a, 0x83, 0x2e, 0xf5, 0x87, 0x7e, + 0x4f, 0x99, 0xc0, 0x6e, 0x1f, 0xda, 0x1b, 0x37, 0x4c, 0x34, 0x31, 0x01, 0xf4, 0x1e, 0xe4, 0x47, + 0x01, 0xf5, 0x1f, 0xf4, 0xe4, 0x39, 0x2f, 0xb6, 0x2b, 0xbb, 0xe3, 0x46, 0xee, 0x27, 0x01, 0xf5, + 0x57, 0x57, 0x78, 0x9d, 0x19, 0x89, 0x11, 0x91, 0xdf, 0x0e, 0xba, 0xa1, 0xc2, 0x54, 0x5c, 0xb9, + 0xda, 0xdf, 0xe7, 0xe2, 0x27, 0x12, 0xdd, 0xd0, 0xf7, 0x5c, 0xca, 0xb6, 0xe9, 0x28, 0x68, 0x6d, + 0x7a, 0xae, 0xeb, 0x0d, 0x5a, 0xa2, 0x73, 0x14, 0x4a, 0xf3, 0x62, 0xc9, 0x97, 0xab, 0xc8, 0xbd, + 0x0b, 0x79, 0xb6, 0xed, 0x7b, 0xa3, 0xee, 0xb6, 0xa8, 0x01, 0x99, 0xf6, 0x85, 0xe9, 0xf9, 0x69, + 0x0e, 0x44, 0x0f, 0xd0, 0x49, 0x6e, 0x2d, 0xba, 0xf9, 0x30, 0x18, 0xb9, 0xb2, 0x1f, 0x6b, 0x67, + 0xf7, 0xc6, 0x0d, 0xeb, 0x3d, 0x12, 0xa2, 0xf1, 0xa7, 0x69, 0x68, 0x88, 0x40, 0xbd, 0x27, 0x2e, + 0x09, 0x57, 0x3c, 0xff, 0x26, 0x65, 0x7e, 0x6f, 0xf3, 0x96, 0xe3, 0x52, 0x1d, 0x1b, 0x0d, 0x28, + 0xb9, 0x02, 0xf9, 0xc0, 0x38, 0x02, 0xe0, 0x86, 0x74, 0xe8, 0x04, 0x80, 0x38, 0x33, 0x72, 0x5e, + 0x9e, 0x86, 0xa2, 0xc0, 0x88, 0xe9, 0xe5, 0x98, 0xa5, 0x5a, 0x53, 0x6a, 0xa6, 0x2c, 0xb4, 0x9a, + 0xb4, 0xd0, 0xd4, 0x7c, 0x42, 0xb3, 0x98, 0xb1, 0x9e, 0x8d, 0xc7, 0x3a, 0xfe, 0x87, 0x05, 0xf5, + 0x35, 0x2d, 0xf9, 0x2b, 0x9a, 0x43, 0xeb, 0x9b, 0x7e, 0x4d, 0xfa, 0x66, 0xbe, 0x99, 0xbe, 0xf8, + 0x1a, 0x54, 0xd6, 0x7a, 0x03, 0x7a, 0xa5, 0xd7, 0x67, 0xd4, 0xbf, 0xfc, 0x64, 0xe8, 0xd3, 0x20, + 0xe0, 0x6d, 0x6a, 0x0d, 0x0a, 0xde, 0x90, 0xfa, 0x8e, 0xee, 0x32, 0x32, 0x24, 0x84, 0x79, 0xf2, + 0x10, 0x36, 0xd1, 0xb9, 0x4d, 0x00, 0xf8, 0x7f, 0x46, 0xf2, 0x20, 0x74, 0x4b, 0x5b, 0x64, 0xd9, + 0xc8, 0xd8, 0xaf, 0x43, 0xe1, 0xf4, 0x6b, 0x74, 0x70, 0x26, 0x91, 0xcc, 0xce, 0x43, 0x7e, 0x4b, + 0x18, 0x42, 0x96, 0xde, 0xd2, 0x52, 0x3d, 0xaa, 0x75, 0x93, 0xac, 0x44, 0x34, 0x39, 0xfe, 0x28, + 0x4a, 0x49, 0x42, 0x77, 0x95, 0x92, 0x4e, 0x81, 0xed, 0xd3, 0x2d, 0x5d, 0x39, 0x51, 0xc4, 0x2d, + 0xa4, 0x14, 0xf3, 0xf8, 0x2f, 0x16, 0xcc, 0x5d, 0xa5, 0x2c, 0x7e, 0x27, 0x79, 0x83, 0x2c, 0x87, + 0xaf, 0xc1, 0x61, 0x43, 0x7e, 0xa5, 0xfd, 0x07, 0x89, 0x8b, 0xc8, 0x5b, 0x91, 0xfe, 0xab, 0x83, + 0x0e, 0x7d, 0xa2, 0x1a, 0xb6, 0xf8, 0x1d, 0xe4, 0x0e, 0x94, 0x8c, 0x49, 0x74, 0x29, 0x71, 0xfb, + 0x30, 0x9e, 0x57, 0xc2, 0x1a, 0xda, 0xae, 0x28, 0x9d, 0x64, 0xcb, 0xa6, 0xee, 0x96, 0x61, 0xad, + 0x5e, 0x07, 0x24, 0xae, 0xad, 0x82, 0xad, 0x59, 0x2d, 0x04, 0xf6, 0x46, 0x78, 0x19, 0x09, 0x61, + 0x74, 0x12, 0x6c, 0xdf, 0x7b, 0xac, 0xaf, 0x95, 0x33, 0xd1, 0x96, 0xc4, 0x7b, 0x4c, 0xc4, 0x14, + 0xbe, 0x08, 0x19, 0xe2, 0x3d, 0x46, 0x75, 0x00, 0xdf, 0x19, 0x74, 0xe9, 0xbd, 0xb0, 0x7b, 0x29, + 0x13, 0x03, 0x73, 0x40, 0x25, 0x5f, 0x86, 0xc3, 0xa6, 0x44, 0xd2, 0xdd, 0x4d, 0xc8, 0x7f, 0x3c, + 0x32, 0xcd, 0x55, 0x49, 0x98, 0x4b, 0x36, 0xc2, 0x9a, 0x88, 0xc7, 0x0c, 0x44, 0x78, 0x74, 0x1c, + 0x8a, 0xcc, 0xd9, 0xe8, 0xd3, 0x5b, 0x51, 0xde, 0x89, 0x10, 0x7c, 0x96, 0x37, 0x5e, 0xf7, 0x8c, + 0x2b, 0x49, 0x84, 0x40, 0xa7, 0x61, 0x2e, 0x92, 0xf9, 0x8e, 0x4f, 0xb7, 0x7a, 0x4f, 0x84, 0x87, + 0xcb, 0x64, 0x1f, 0x1e, 0x2d, 0xc2, 0xa1, 0x08, 0xb7, 0x2e, 0x4a, 0xbf, 0x2d, 0x48, 0x93, 0x68, + 0x6e, 0x1b, 0xa1, 0xee, 0xe5, 0x47, 0x23, 0xa7, 0x2f, 0x92, 0x69, 0x99, 0x18, 0x18, 0xfc, 0x57, + 0x0b, 0x0e, 0x4b, 0x57, 0xf3, 0x96, 0xfb, 0x4d, 0x8c, 0xfa, 0xcf, 0x2d, 0x40, 0xa6, 0x06, 0x2a, + 0xb4, 0xbe, 0x63, 0xbe, 0xa5, 0xf0, 0xbb, 0x45, 0x49, 0xf4, 0x93, 0x12, 0x15, 0x3d, 0x87, 0x60, + 0xc8, 0x89, 0xfb, 0x89, 0x6c, 0x6c, 0x6d, 0xd9, 0xb0, 0x4a, 0x0c, 0x51, 0x5f, 0xde, 0x67, 0x6f, + 0xec, 0x30, 0x1a, 0xa8, 0x76, 0x53, 0xf4, 0xd9, 0x02, 0x41, 0xe4, 0x87, 0xef, 0x45, 0x07, 0x4c, + 0x44, 0x8d, 0x1d, 0xed, 0xa5, 0x50, 0x44, 0x0f, 0xf0, 0x1f, 0xd3, 0x30, 0x73, 0xcf, 0xeb, 0x8f, + 0xa2, 0x4a, 0xf5, 0x26, 0xe5, 0xe5, 0x58, 0x0f, 0x9c, 0xd5, 0x3d, 0x30, 0x02, 0x3b, 0x60, 0x74, + 0x28, 0x22, 0x2b, 0x43, 0xc4, 0x18, 0x61, 0x28, 0x33, 0xc7, 0xef, 0x52, 0x26, 0x5b, 0x8e, 0x6a, + 0x4e, 0xdc, 0x03, 0x63, 0x38, 0xb4, 0x00, 0x25, 0xa7, 0xdb, 0xf5, 0x69, 0xd7, 0x61, 0xb4, 0xbd, + 0x53, 0xcd, 0x8b, 0xcd, 0x4c, 0x14, 0xfe, 0x19, 0xcc, 0x6a, 0x63, 0x29, 0x97, 0xbe, 0x0f, 0xf9, + 0x4f, 0x04, 0x66, 0xc2, 0xbb, 0x93, 0x24, 0x55, 0x69, 0x4c, 0x93, 0xc5, 0x1f, 0xa9, 0xb5, 0xcc, + 0xf8, 0x3a, 0xe4, 0x24, 0x39, 0x3a, 0x6e, 0x36, 0x0d, 0xf2, 0x81, 0x84, 0xc3, 0xaa, 0x03, 0xc0, + 0x90, 0x93, 0x8c, 0x94, 0xe3, 0x45, 0x6c, 0x48, 0x0c, 0x51, 0xdf, 0xd3, 0xa7, 0xa0, 0x18, 0xbe, + 0x30, 0xa3, 0x12, 0xe4, 0xaf, 0xdc, 0x26, 0x3f, 0xbd, 0x44, 0x56, 0xe6, 0x52, 0xa8, 0x0c, 0x85, + 0xf6, 0xa5, 0xe5, 0x1b, 0x02, 0xb2, 0x96, 0xfe, 0x6b, 0xeb, 0xcc, 0xe2, 0xa3, 0x1f, 0x40, 0x56, + 0xa6, 0x8b, 0x23, 0x91, 0xfc, 0xe6, 0x5b, 0x6e, 0xed, 0xe8, 0x3e, 0xbc, 0xb4, 0x00, 0x4e, 0xbd, + 0x6f, 0xa1, 0x5b, 0x50, 0x12, 0x48, 0xf5, 0x6a, 0x73, 0x3c, 0xf9, 0x78, 0x12, 0xe3, 0x74, 0xe2, + 0x80, 0x59, 0x83, 0xdf, 0x05, 0xc8, 0x0a, 0x9f, 0x98, 0xd2, 0x98, 0xaf, 0x6e, 0xa6, 0x34, 0xb1, + 0x77, 0x2c, 0x9c, 0x42, 0x1f, 0x82, 0xcd, 0x3b, 0x1b, 0x64, 0x14, 0x15, 0xe3, 0xb1, 0xa5, 0x76, + 0x24, 0x89, 0x36, 0xb6, 0xfd, 0x28, 0x7c, 0x33, 0x3a, 0x9a, 0xec, 0x65, 0xf5, 0xf2, 0xea, 0xfe, + 0x89, 0x70, 0xe7, 0xdb, 0xf2, 0xb1, 0x43, 0xf7, 0x54, 0xe8, 0x44, 0x7c, 0xab, 0x44, 0x0b, 0x56, + 0xab, 0x1f, 0x34, 0x1d, 0x32, 0x5c, 0x83, 0x92, 0xd1, 0xcf, 0x98, 0x66, 0xdd, 0xdf, 0x8c, 0x99, + 0x66, 0x9d, 0xd0, 0x04, 0xe1, 0x14, 0xba, 0x0a, 0x05, 0x5e, 0x8a, 0x79, 0x46, 0x42, 0xc7, 0x92, + 0x15, 0xd7, 0xc8, 0xb4, 0xb5, 0xe3, 0x93, 0x27, 0x43, 0x46, 0x3f, 0x86, 0xe2, 0x55, 0xca, 0x54, + 0xb8, 0x1e, 0x4d, 0xc6, 0xfb, 0x04, 0x4b, 0xc5, 0xcf, 0x0c, 0x4e, 0x2d, 0xfd, 0x42, 0xff, 0xf3, + 0xb4, 0xe2, 0x30, 0x07, 0xdd, 0x86, 0x59, 0x21, 0x58, 0xf8, 0xd7, 0x54, 0x2c, 0x80, 0xf6, 0xfd, + 0x0f, 0x16, 0x0b, 0xa0, 0xfd, 0xff, 0x87, 0xe1, 0x54, 0xfb, 0xfe, 0xb3, 0xe7, 0xf5, 0xd4, 0x17, + 0xcf, 0xeb, 0xa9, 0xaf, 0x9e, 0xd7, 0xad, 0x5f, 0xed, 0xd6, 0xad, 0x3f, 0xec, 0xd6, 0xad, 0xa7, + 0xbb, 0x75, 0xeb, 0xd9, 0x6e, 0xdd, 0xfa, 0xd7, 0x6e, 0xdd, 0xfa, 0xcf, 0x6e, 0x3d, 0xf5, 0xd5, + 0x6e, 0xdd, 0xfa, 0xec, 0x45, 0x3d, 0xf5, 0xec, 0x45, 0x3d, 0xf5, 0xc5, 0x8b, 0x7a, 0xea, 0xfe, + 0xdb, 0x2f, 0x7b, 0x6e, 0xd2, 0x3b, 0x6e, 0xe4, 0xc4, 0xe7, 0x83, 0xff, 0x07, 0x00, 0x00, 0xff, + 0xff, 0xd9, 0x28, 0xaf, 0x8d, 0x38, 0x1c, 0x00, 0x00, } func (x Direction) String() string { @@ -2758,6 +2814,13 @@ func (this *QueryRequest) Equal(that interface{}) bool { return false } } + if that1.Plan == nil { + if this.Plan != nil { + return false + } + } else if !this.Plan.Equal(*that1.Plan) { + return false + } return true } func (this *SampleQueryRequest) Equal(that interface{}) bool { @@ -2804,6 +2867,37 @@ func (this *SampleQueryRequest) Equal(that interface{}) bool { return false } } + if that1.Plan == nil { + if this.Plan != nil { + return false + } + } else if !this.Plan.Equal(*that1.Plan) { + return false + } + return true +} +func (this *Plan) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Plan) + if !ok { + that2, ok := that.(Plan) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !bytes.Equal(this.Raw, that1.Raw) { + return false + } return true } func (this *Delete) Equal(that interface{}) bool { @@ -3096,6 +3190,13 @@ func (this *TailRequest) Equal(that interface{}) bool { if !this.Start.Equal(that1.Start) { return false } + if that1.Plan == nil { + if this.Plan != nil { + return false + } + } else if !this.Plan.Equal(*that1.Plan) { + return false + } return true } func (this *TailResponse) Equal(that interface{}) bool { @@ -4063,7 +4164,7 @@ func (this *QueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&logproto.QueryRequest{") s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") @@ -4074,6 +4175,7 @@ func (this *QueryRequest) GoString() string { if this.Deletes != nil { s = append(s, "Deletes: "+fmt.Sprintf("%#v", this.Deletes)+",\n") } + s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -4081,7 +4183,7 @@ func (this *SampleQueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 10) s = append(s, "&logproto.SampleQueryRequest{") s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n") s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") @@ -4090,6 +4192,17 @@ func (this *SampleQueryRequest) GoString() string { if this.Deletes != nil { s = append(s, "Deletes: "+fmt.Sprintf("%#v", this.Deletes)+",\n") } + s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Plan) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&logproto.Plan{") + s = append(s, "Raw: "+fmt.Sprintf("%#v", this.Raw)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -4196,12 +4309,13 @@ func (this *TailRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 8) + s := make([]string, 0, 9) s = append(s, "&logproto.TailRequest{") s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") s = append(s, "DelayFor: "+fmt.Sprintf("%#v", this.DelayFor)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") + s = append(s, "Plan: "+fmt.Sprintf("%#v", this.Plan)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -5258,6 +5372,18 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Plan != nil { + { + size := m.Plan.Size() + i -= size + if _, err := m.Plan.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } if len(m.Deletes) > 0 { for iNdEx := len(m.Deletes) - 1; iNdEx >= 0; iNdEx-- { { @@ -5286,21 +5412,21 @@ func (m *QueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x28 } - n1, err1 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err1 != nil { - return 0, err1 - } - i -= n1 - i = encodeVarintLogproto(dAtA, i, uint64(n1)) - i-- - dAtA[i] = 0x22 - n2, err2 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + n2, err2 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) if err2 != nil { return 0, err2 } i -= n2 i = encodeVarintLogproto(dAtA, i, uint64(n2)) i-- + dAtA[i] = 0x22 + n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err3 != nil { + return 0, err3 + } + i -= n3 + i = encodeVarintLogproto(dAtA, i, uint64(n3)) + i-- dAtA[i] = 0x1a if m.Limit != 0 { i = encodeVarintLogproto(dAtA, i, uint64(m.Limit)) @@ -5337,6 +5463,18 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Plan != nil { + { + size := m.Plan.Size() + i -= size + if _, err := m.Plan.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } if len(m.Deletes) > 0 { for iNdEx := len(m.Deletes) - 1; iNdEx >= 0; iNdEx-- { { @@ -5360,20 +5498,20 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x22 } } - n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err3 != nil { - return 0, err3 + n5, err5 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err5 != nil { + return 0, err5 } - i -= n3 - i = encodeVarintLogproto(dAtA, i, uint64(n3)) + i -= n5 + i = encodeVarintLogproto(dAtA, i, uint64(n5)) i-- dAtA[i] = 0x1a - n4, err4 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err4 != nil { - return 0, err4 + n6, err6 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err6 != nil { + return 0, err6 } - i -= n4 - i = encodeVarintLogproto(dAtA, i, uint64(n4)) + i -= n6 + i = encodeVarintLogproto(dAtA, i, uint64(n6)) i-- dAtA[i] = 0x12 if len(m.Selector) > 0 { @@ -5386,6 +5524,36 @@ func (m *SampleQueryRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *Plan) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Plan) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Plan) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Raw) > 0 { + i -= len(m.Raw) + copy(dAtA[i:], m.Raw) + i = encodeVarintLogproto(dAtA, i, uint64(len(m.Raw))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *Delete) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -5548,22 +5716,22 @@ func (m *LabelRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x2a } if m.End != nil { - n7, err7 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) - if err7 != nil { - return 0, err7 + n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.End):]) + if err9 != nil { + return 0, err9 } - i -= n7 - i = encodeVarintLogproto(dAtA, i, uint64(n7)) + i -= n9 + i = encodeVarintLogproto(dAtA, i, uint64(n9)) i-- dAtA[i] = 0x22 } if m.Start != nil { - n8, err8 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) - if err8 != nil { - return 0, err8 + n10, err10 := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(*m.Start):]) + if err10 != nil { + return 0, err10 } - i -= n8 - i = encodeVarintLogproto(dAtA, i, uint64(n8)) + i -= n10 + i = encodeVarintLogproto(dAtA, i, uint64(n10)) i-- dAtA[i] = 0x1a } @@ -5761,12 +5929,24 @@ func (m *TailRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n9, err9 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err9 != nil { - return 0, err9 + if m.Plan != nil { + { + size := m.Plan.Size() + i -= size + if _, err := m.Plan.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintLogproto(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err11 != nil { + return 0, err11 } - i -= n9 - i = encodeVarintLogproto(dAtA, i, uint64(n9)) + i -= n11 + i = encodeVarintLogproto(dAtA, i, uint64(n11)) i-- dAtA[i] = 0x2a if m.Limit != 0 { @@ -5876,20 +6056,20 @@ func (m *SeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x1a } } - n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err11 != nil { - return 0, err11 + n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err13 != nil { + return 0, err13 } - i -= n11 - i = encodeVarintLogproto(dAtA, i, uint64(n11)) + i -= n13 + i = encodeVarintLogproto(dAtA, i, uint64(n13)) i-- dAtA[i] = 0x12 - n12, err12 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err12 != nil { - return 0, err12 + n14, err14 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err14 != nil { + return 0, err14 } - i -= n12 - i = encodeVarintLogproto(dAtA, i, uint64(n12)) + i -= n14 + i = encodeVarintLogproto(dAtA, i, uint64(n14)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -6001,20 +6181,20 @@ func (m *DroppedStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x1a } - n13, err13 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) - if err13 != nil { - return 0, err13 + n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.To, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.To):]) + if err15 != nil { + return 0, err15 } - i -= n13 - i = encodeVarintLogproto(dAtA, i, uint64(n13)) + i -= n15 + i = encodeVarintLogproto(dAtA, i, uint64(n15)) i-- dAtA[i] = 0x12 - n14, err14 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) - if err14 != nil { - return 0, err14 + n16, err16 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.From, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.From):]) + if err16 != nil { + return 0, err16 } - i -= n14 - i = encodeVarintLogproto(dAtA, i, uint64(n14)) + i -= n16 + i = encodeVarintLogproto(dAtA, i, uint64(n16)) i-- dAtA[i] = 0xa return len(dAtA) - i, nil @@ -6195,20 +6375,20 @@ func (m *GetChunkIDsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n15, err15 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) - if err15 != nil { - return 0, err15 + n17, err17 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.End, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.End):]) + if err17 != nil { + return 0, err17 } - i -= n15 - i = encodeVarintLogproto(dAtA, i, uint64(n15)) + i -= n17 + i = encodeVarintLogproto(dAtA, i, uint64(n17)) i-- dAtA[i] = 0x1a - n16, err16 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) - if err16 != nil { - return 0, err16 + n18, err18 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Start):]) + if err18 != nil { + return 0, err18 } - i -= n16 - i = encodeVarintLogproto(dAtA, i, uint64(n16)) + i -= n18 + i = encodeVarintLogproto(dAtA, i, uint64(n18)) i-- dAtA[i] = 0x12 if len(m.Matchers) > 0 { @@ -7131,6 +7311,10 @@ func (m *QueryRequest) Size() (n int) { n += 1 + l + sovLogproto(uint64(l)) } } + if m.Plan != nil { + l = m.Plan.Size() + n += 1 + l + sovLogproto(uint64(l)) + } return n } @@ -7160,6 +7344,23 @@ func (m *SampleQueryRequest) Size() (n int) { n += 1 + l + sovLogproto(uint64(l)) } } + if m.Plan != nil { + l = m.Plan.Size() + n += 1 + l + sovLogproto(uint64(l)) + } + return n +} + +func (m *Plan) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Raw) + if l > 0 { + n += 1 + l + sovLogproto(uint64(l)) + } return n } @@ -7332,6 +7533,10 @@ func (m *TailRequest) Size() (n int) { } l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Start) n += 1 + l + sovLogproto(uint64(l)) + if m.Plan != nil { + l = m.Plan.Size() + n += 1 + l + sovLogproto(uint64(l)) + } return n } @@ -7954,6 +8159,7 @@ func (this *QueryRequest) String() string { `Direction:` + fmt.Sprintf("%v", this.Direction) + `,`, `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Deletes:` + repeatedStringForDeletes + `,`, + `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, `}`, }, "") return s @@ -7973,6 +8179,17 @@ func (this *SampleQueryRequest) String() string { `End:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.End), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `Deletes:` + repeatedStringForDeletes + `,`, + `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, + `}`, + }, "") + return s +} +func (this *Plan) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Plan{`, + `Raw:` + fmt.Sprintf("%v", this.Raw) + `,`, `}`, }, "") return s @@ -8084,6 +8301,7 @@ func (this *TailRequest) String() string { `DelayFor:` + fmt.Sprintf("%v", this.DelayFor) + `,`, `Limit:` + fmt.Sprintf("%v", this.Limit) + `,`, `Start:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Start), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, + `Plan:` + fmt.Sprintf("%v", this.Plan) + `,`, `}`, }, "") return s @@ -9022,6 +9240,42 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Plan", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Plan == nil { + m.Plan = &github_com_grafana_loki_pkg_querier_plan.QueryPlan{} + } + if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) @@ -9239,6 +9493,129 @@ func (m *SampleQueryRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Plan", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Plan == nil { + m.Plan = &github_com_grafana_loki_pkg_querier_plan.QueryPlan{} + } + if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipLogproto(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthLogproto + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Plan) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Plan: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Plan: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Raw", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Raw = append(m.Raw[:0], dAtA[iNdEx:postIndex]...) + if m.Raw == nil { + m.Raw = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) @@ -10375,6 +10752,41 @@ func (m *TailRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Plan", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var v github_com_grafana_loki_pkg_querier_plan.QueryPlan + m.Plan = &v + if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 35824a78e2..0aa49d6b5a 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -51,7 +51,7 @@ message StreamRate { } message QueryRequest { - string selector = 1; + string selector = 1 [deprecated = true]; uint32 limit = 2; google.protobuf.Timestamp start = 3 [ (gogoproto.stdtime) = true, @@ -65,10 +65,11 @@ message QueryRequest { reserved 6; repeated string shards = 7 [(gogoproto.jsontag) = "shards,omitempty"]; repeated Delete deletes = 8; + Plan plan = 9 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; } message SampleQueryRequest { - string selector = 1; + string selector = 1 [deprecated = true]; // mark as reserved once we've fully migrated to plan. google.protobuf.Timestamp start = 2 [ (gogoproto.stdtime) = true, (gogoproto.nullable) = false @@ -79,6 +80,11 @@ message SampleQueryRequest { ]; repeated string shards = 4 [(gogoproto.jsontag) = "shards,omitempty"]; repeated Delete deletes = 5; + Plan plan = 6 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; +} + +message Plan { + bytes raw = 1; } message Delete { @@ -148,7 +154,7 @@ message Series { } message TailRequest { - string query = 1; + string query = 1 [deprecated = true]; reserved 2; uint32 delayFor = 3; uint32 limit = 4; @@ -156,6 +162,7 @@ message TailRequest { (gogoproto.stdtime) = true, (gogoproto.nullable) = false ]; + bytes plan = 6 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; } message TailResponse { diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 2cd706c812..27cb3e849f 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "strings" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -155,6 +156,50 @@ func (c ConcatLogSelectorExpr) string(maxDepth int) string { return fmt.Sprintf("%s ++ %s", c.DownstreamLogSelectorExpr.String(), c.next.string(maxDepth-1)) } +// QuantileSketchEvalExpr evaluates a quantile sketch to the actual quantile. +type QuantileSketchEvalExpr struct { + syntax.SampleExpr + quantileMergeExpr *QuantileSketchMergeExpr + quantile *float64 +} + +func (e QuantileSketchEvalExpr) String() string { + return fmt.Sprintf("quantileSketchEval<%s>", e.quantileMergeExpr.String()) +} + +func (e *QuantileSketchEvalExpr) Walk(f syntax.WalkFn) { + f(e) + e.quantileMergeExpr.Walk(f) +} + +type QuantileSketchMergeExpr struct { + syntax.SampleExpr + downstreams []DownstreamSampleExpr +} + +func (e QuantileSketchMergeExpr) String() string { + var sb strings.Builder + for i, d := range e.downstreams { + if i >= defaultMaxDepth { + break + } + + if i > 0 { + sb.WriteString(" ++ ") + } + + sb.WriteString(d.String()) + } + return fmt.Sprintf("quantileSketchMerge<%s>", sb.String()) +} + +func (e *QuantileSketchMergeExpr) Walk(f syntax.WalkFn) { + f(e) + for _, d := range e.downstreams { + d.Walk(f) + } +} + type Shards []astmapper.ShardAnnotation func (xs Shards) Encode() (encoded []string) { @@ -308,6 +353,47 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( } return NewConcatStepEvaluator(xs), nil + case *QuantileSketchEvalExpr: + var queries []DownstreamQuery + if e.quantileMergeExpr != nil { + for _, d := range e.quantileMergeExpr.downstreams { + qry := DownstreamQuery{ + Params: ParamsWithExpressionOverride{ + Params: params, + ExpressionOverride: d.SampleExpr, + }, + } + if shard := d.shard; shard != nil { + qry.Params = ParamsWithShardsOverride{ + Params: qry.Params, + ShardsOverride: Shards{*shard}.Encode(), + } + } + queries = append(queries, qry) + } + } + + results, err := ev.Downstream(ctx, queries) + if err != nil { + return nil, fmt.Errorf("error running quantile sketch downstream query: %w", err) + } + + xs := make([]StepEvaluator, 0, len(queries)) + for _, res := range results { + if res.Data.Type() != QuantileSketchMatrixType { + return nil, fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), QuantileSketchMatrixType) + } + data, ok := res.Data.(ProbabilisticQuantileMatrix) + if !ok { + return nil, fmt.Errorf("unexpected matrix type: got (%T), want (ProbabilisticQuantileMatrix)", res.Data) + } + stepper := NewQuantileSketchMatrixStepEvaluator(data, params) + xs = append(xs, stepper) + } + + inner := NewQuantileSketchMergeStepEvaluator(xs) + + return NewQuantileSketchVectorStepEvaluator(inner, *e.quantile), nil default: return ev.defaultEvaluator.NewStepEvaluator(ctx, nextEvFactory, e, params) diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index 0f4d1cd099..218957f862 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -54,6 +54,7 @@ func TestMappingEquivalence(t *testing.T) { {`sum(rate({a=~".+"} |= "foo" != "foo"[1s]) or vector(1))`, false}, {`avg_over_time({a=~".+"} | logfmt | unwrap value [1s])`, false}, {`avg_over_time({a=~".+"} | logfmt | unwrap value [1s]) by (a)`, true}, + {`quantile_over_time(0.99, {a=~".+"} | logfmt | unwrap value [1s])`, true}, // topk prefers already-seen values in tiebreakers. Since the test data generates // the same log lines for each series & the resulting promql.Vectors aren't deterministically // sorted by labels, we don't expect this to pass. @@ -85,17 +86,17 @@ func TestMappingEquivalence(t *testing.T) { qry := regular.Query(params) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics) + mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{}) _, _, mapped, err := mapper.Parse(params.GetExpression()) - require.Nil(t, err) + require.NoError(t, err) shardedQry := sharded.Query(ctx, ParamsWithExpressionOverride{Params: params, ExpressionOverride: mapped}) res, err := qry.Exec(ctx) - require.Nil(t, err) + require.NoError(t, err) shardedRes, err := shardedQry.Exec(ctx) - require.Nil(t, err) + require.NoError(t, err) if tc.approximate { approximatelyEquals(t, res.Data.(promql.Matrix), shardedRes.Data.(promql.Matrix)) @@ -106,6 +107,70 @@ func TestMappingEquivalence(t *testing.T) { } } +func TestMappingEquivalenceSketches(t *testing.T) { + var ( + shards = 3 + nStreams = 10_000 + rounds = 20 + streams = randomStreams(nStreams, rounds+1, shards, []string{"a", "b", "c", "d"}, true) + start = time.Unix(0, 0) + end = time.Unix(0, int64(time.Second*time.Duration(rounds))) + step = time.Second + interval = time.Duration(0) + limit = 100 + ) + + for _, tc := range []struct { + query string + realtiveError float64 + }{ + {`quantile_over_time(0.70, {a=~".+"} | logfmt | unwrap value [1s]) by (a)`, 0.03}, + {`quantile_over_time(0.99, {a=~".+"} | logfmt | unwrap value [1s]) by (a)`, 0.02}, + } { + q := NewMockQuerier( + shards, + streams, + ) + + opts := EngineOpts{} + regular := NewEngine(opts, q, NoLimits, log.NewNopLogger()) + sharded := NewDownstreamEngine(opts, MockDownstreamer{regular}, NoLimits, log.NewNopLogger()) + + t.Run(tc.query, func(t *testing.T) { + params, err := NewLiteralParams( + tc.query, + start, + end, + step, + interval, + logproto.FORWARD, + uint32(limit), + nil, + ) + require.NoError(t, err) + qry := regular.Query(params) + ctx := user.InjectOrgID(context.Background(), "fake") + + mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) + _, _, mapped, err := mapper.Parse(params.GetExpression()) + require.NoError(t, err) + + shardedQry := sharded.Query(ctx, ParamsWithExpressionOverride{ + Params: params, + ExpressionOverride: mapped, + }) + + res, err := qry.Exec(ctx) + require.NoError(t, err) + + shardedRes, err := shardedQry.Exec(ctx) + require.NoError(t, err) + + relativeError(t, res.Data.(promql.Matrix), shardedRes.Data.(promql.Matrix), tc.realtiveError) + }) + } +} + func TestShardCounter(t *testing.T) { var ( shards = 3 @@ -151,7 +216,7 @@ func TestShardCounter(t *testing.T) { require.NoError(t, err) ctx := user.InjectOrgID(context.Background(), "fake") - mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics) + mapper := NewShardMapper(ConstantShards(shards), nilShardMetrics, []string{ShardQuantileOverTime}) noop, _, mapped, err := mapper.Parse(params.GetExpression()) require.NoError(t, err) @@ -412,13 +477,13 @@ func TestRangeMappingEquivalence(t *testing.T) { // Regular engine qry := regularEngine.Query(params) res, err := qry.Exec(ctx) - require.Nil(t, err) + require.NoError(t, err) // Downstream engine - split by range rangeMapper, err := NewRangeMapper(tc.splitByInterval, nilRangeMetrics, NewMapperStats()) - require.Nil(t, err) + require.NoError(t, err) noop, rangeExpr, err := rangeMapper.Parse(syntax.MustParseExpr(tc.query)) - require.Nil(t, err) + require.NoError(t, err) require.False(t, noop, "downstream engine cannot execute noop") @@ -451,3 +516,22 @@ func approximatelyEquals(t *testing.T, as, bs promql.Matrix) { require.Equalf(t, a, b, "metric %s differs from %s at %d", a.Metric, b.Metric, i) } } + +func relativeError(t *testing.T, expected, actual promql.Matrix, alpha float64) { + require.Len(t, actual, len(expected)) + + for i := 0; i < len(expected); i++ { + expectedSeries := expected[i] + actualSeries := actual[i] + require.Equal(t, expectedSeries.Metric, actualSeries.Metric) + require.Lenf(t, actualSeries.Floats, len(expectedSeries.Floats), "for series %s", expectedSeries.Metric) + + e := make([]float64, len(expectedSeries.Floats)) + a := make([]float64, len(expectedSeries.Floats)) + for j := 0; j < len(expectedSeries.Floats); j++ { + e[j] = expectedSeries.Floats[j].F + a[j] = actualSeries.Floats[j].F + } + require.InEpsilonSlice(t, e, a, alpha) + } +} diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index e04cf1dcff..89490d479e 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -2,6 +2,7 @@ package logql import ( "context" + "errors" "flag" "fmt" "math" @@ -83,7 +84,14 @@ func (s SelectLogParams) String() string { // LogSelector returns the LogSelectorExpr from the SelectParams. // The `LogSelectorExpr` can then returns all matchers and filters to use for that request. func (s SelectLogParams) LogSelector() (syntax.LogSelectorExpr, error) { - return syntax.ParseLogSelector(s.Selector, true) + if s.QueryRequest.Plan == nil { + return nil, errors.New("query plan is empty") + } + expr, ok := s.QueryRequest.Plan.AST.(syntax.LogSelectorExpr) + if !ok { + return nil, errors.New("only log selector is supported") + } + return expr, nil } type SelectSampleParams struct { @@ -93,13 +101,20 @@ type SelectSampleParams struct { // 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) { - return syntax.ParseSampleExpr(s.Selector) + if s.SampleQueryRequest.Plan == nil { + return nil, errors.New("query plan is empty") + } + expr, ok := s.SampleQueryRequest.Plan.AST.(syntax.SampleExpr) + if !ok { + return nil, errors.New("only sample expression supported") + } + return expr, nil } // LogSelector returns the LogSelectorExpr from the SelectParams. // The `LogSelectorExpr` can then returns all matchers and filters to use for that request. func (s SelectSampleParams) LogSelector() (syntax.LogSelectorExpr, error) { - expr, err := syntax.ParseSampleExpr(s.Selector) + expr, err := s.Expr() if err != nil { return nil, err } @@ -327,21 +342,37 @@ func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_ if err != nil { return nil, err } + stepEvaluator, err := q.evaluator.NewStepEvaluator(ctx, q.evaluator, expr, q.params) if err != nil { return nil, err } defer util.LogErrorWithContext(ctx, "closing SampleExpr", stepEvaluator.Close) - maxSeriesCapture := func(id string) int { return q.limits.MaxQuerySeries(ctx, id) } - maxSeries := validation.SmallestPositiveIntPerTenant(tenantIDs, maxSeriesCapture) - - seriesIndex := map[uint64]*promql.Series{} - next, ts, r := stepEvaluator.Next() if stepEvaluator.Error() != nil { return nil, stepEvaluator.Error() } + + if next && r != nil { + switch vec := r.(type) { + case SampleVector: + maxSeriesCapture := func(id string) int { return q.limits.MaxQuerySeries(ctx, id) } + maxSeries := validation.SmallestPositiveIntPerTenant(tenantIDs, maxSeriesCapture) + return q.JoinSampleVector(next, ts, vec, stepEvaluator, maxSeries) + case ProbabilisticQuantileVector: + return JoinQuantileSketchVector(next, vec, stepEvaluator) + default: + return nil, fmt.Errorf("unsupported result type: %T", r) + } + } + return nil, nil +} + +func (q *query) JoinSampleVector(next bool, ts int64, r StepResult, stepEvaluator StepEvaluator, maxSeries int) (promql_parser.Value, error) { + + seriesIndex := map[uint64]*promql.Series{} + vec := promql.Vector{} if next { vec = r.SampleVector() diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index e0b6ab3dff..2dce4ba57e 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -12,6 +12,7 @@ import ( "time" "github.com/grafana/loki/pkg/logqlmodel/metadata" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions" "github.com/go-kit/log" @@ -64,8 +65,15 @@ func TestEngine_LogsRateUnwrap(t *testing.T) { {newSeries(testSize, offset(46, constantValue(1)), `{app="foo"}`)}, }, []SelectSampleParams{ - {&logproto.SampleQueryRequest{Start: time.Unix(30, 0), End: time.Unix(60, 0), Selector: `rate({app="foo"} | unwrap foo[30s])`}}, - }, + {&logproto.SampleQueryRequest{ + Start: time.Unix(30, 0), + End: time.Unix(60, 0), + Selector: `rate({app="foo"} | unwrap foo[30s])`, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate({app="foo"} | unwrap foo[30s])`), + }, + }, + }}, // there are 15 samples (from 47 to 61) matched from the generated series // SUM(n=47, 61, 1) = 15 // 15 / 30 = 0.5 @@ -82,7 +90,14 @@ func TestEngine_LogsRateUnwrap(t *testing.T) { {newSeries(testSize, offset(46, incValue(1)), `{app="foo"}`)}, }, []SelectSampleParams{ - {&logproto.SampleQueryRequest{Start: time.Unix(30, 0), End: time.Unix(60, 0), Selector: `rate({app="foo"} | unwrap foo[30s])`}}, + {&logproto.SampleQueryRequest{ + Start: time.Unix(30, 0), + End: time.Unix(60, 0), + Selector: `rate({app="foo"} | unwrap foo[30s])`, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate({app="foo"} | unwrap foo[30s])`), + }, + }}, }, // there are 15 samples (from 47 to 61) matched from the generated series // SUM(n=47, 61, n) = (47+48+...+61) = 810 @@ -100,7 +115,14 @@ func TestEngine_LogsRateUnwrap(t *testing.T) { {newSeries(testSize, offset(46, constantValue(1)), `{app="foo"}`)}, }, []SelectSampleParams{ - {&logproto.SampleQueryRequest{Start: time.Unix(30, 0), End: time.Unix(60, 0), Selector: `rate_counter({app="foo"} | unwrap foo[30s])`}}, + {&logproto.SampleQueryRequest{ + Start: time.Unix(30, 0), + End: time.Unix(60, 0), + Selector: `rate_counter({app="foo"} | unwrap foo[30s])`, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate_counter({app="foo"} | unwrap foo[30s])`), + }, + }}, }, // there are 15 samples (from 47 to 61) matched from the generated series // (1 - 1) / 30 = 0 @@ -2669,6 +2691,9 @@ func newQuerierRecorder(t *testing.T, data interface{}, params interface{}) *que if streamsIn, ok := data.([][]logproto.Stream); ok { if paramsIn, ok2 := params.([]SelectLogParams); ok2 { for i, p := range paramsIn { + p.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(p.Selector), + } streams[paramsID(p)] = streamsIn[i] } } @@ -2678,6 +2703,9 @@ func newQuerierRecorder(t *testing.T, data interface{}, params interface{}) *que if seriesIn, ok := data.([][]logproto.Series); ok { if paramsIn, ok2 := params.([]SelectSampleParams); ok2 { for i, p := range paramsIn { + p.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(p.Selector), + } series[paramsID(p)] = seriesIn[i] } } diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index fdb9190956..2d6837ef6a 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -17,6 +17,7 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/util" ) @@ -210,6 +211,9 @@ func (ev *DefaultEvaluator) NewIterator(ctx context.Context, expr syntax.LogSele Direction: q.Direction(), Selector: expr.String(), Shards: q.Shards(), + Plan: &plan.QueryPlan{ + AST: expr, + }, }, } @@ -238,6 +242,9 @@ func (ev *DefaultEvaluator) NewStepEvaluator( End: q.End().Add(-rangExpr.Left.Offset), Selector: e.String(), // intentionally send the vector for reducing labels. Shards: q.Shards(), + Plan: &plan.QueryPlan{ + AST: expr, + }, }, }) if err != nil { @@ -254,6 +261,9 @@ func (ev *DefaultEvaluator) NewStepEvaluator( End: q.End().Add(-e.Left.Offset), Selector: expr.String(), Shards: q.Shards(), + Plan: &plan.QueryPlan{ + AST: expr, + }, }, }) if err != nil { @@ -515,17 +525,18 @@ func newRangeAggEvaluator( q Params, o time.Duration, ) (StepEvaluator, error) { + switch expr.Operation { + case syntax.OpRangeTypeAbsent: + iter, err := newRangeVectorIterator( + it, expr, + expr.Left.Interval.Nanoseconds(), + q.Step().Nanoseconds(), + q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), + ) + if err != nil { + return nil, err + } - iter, err := newRangeVectorIterator( - it, expr, - expr.Left.Interval.Nanoseconds(), - q.Step().Nanoseconds(), - q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), - ) - if err != nil { - return nil, err - } - if expr.Operation == syntax.OpRangeTypeAbsent { absentLabels, err := absentLabels(expr) if err != nil { return nil, err @@ -534,10 +545,32 @@ func newRangeAggEvaluator( iter: iter, lbs: absentLabels, }, nil + case syntax.OpRangeTypeQuantileSketch: + iter := newQuantileSketchIterator( + it, + expr.Left.Interval.Nanoseconds(), + q.Step().Nanoseconds(), + q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), + ) + + return &QuantileSketchStepEvaluator{ + iter: iter, + }, nil + default: + iter, err := newRangeVectorIterator( + it, expr, + expr.Left.Interval.Nanoseconds(), + q.Step().Nanoseconds(), + q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), + ) + if err != nil { + return nil, err + } + + return &RangeVectorEvaluator{ + iter: iter, + }, nil } - return &RangeVectorEvaluator{ - iter: iter, - }, nil } type RangeVectorEvaluator struct { diff --git a/pkg/logql/explain.go b/pkg/logql/explain.go index ef161b38c8..4890d150f0 100644 --- a/pkg/logql/explain.go +++ b/pkg/logql/explain.go @@ -1,5 +1,9 @@ package logql +// MaxChildrenDisplay defines the maximum number of children that should be +// shown by explain. +const MaxChildrenDisplay = 3 + func (e *LiteralStepEvaluator) Explain(parent Node) { b := parent.Child("Literal") e.nextEv.Explain(b) @@ -25,7 +29,7 @@ func (e *VectorStepEvaluator) Explain(parent Node) { func (e *ConcatStepEvaluator) Explain(parent Node) { b := parent.Child("Concat") - if len(e.evaluators) < 3 { + if len(e.evaluators) < MaxChildrenDisplay { for _, child := range e.evaluators { child.Explain(b) } diff --git a/pkg/logql/explain_test.go b/pkg/logql/explain_test.go index 5ae2f840e1..307aa10cfa 100644 --- a/pkg/logql/explain_test.go +++ b/pkg/logql/explain_test.go @@ -28,7 +28,7 @@ func TestExplain(t *testing.T) { defaultEv := NewDefaultEvaluator(querier, 30*time.Second) downEv := &DownstreamEvaluator{Downstreamer: MockDownstreamer{regular}, defaultEvaluator: defaultEv} - mapper := NewShardMapper(ConstantShards(4), nilShardMetrics) + mapper := NewShardMapper(ConstantShards(4), nilShardMetrics, []string{ShardQuantileOverTime}) _, _, expr, err := mapper.Parse(syntax.MustParseExpr(query)) require.NoError(t, err) diff --git a/pkg/logql/optimize.go b/pkg/logql/optimize.go index 1f00153e18..2f9c80a64f 100644 --- a/pkg/logql/optimize.go +++ b/pkg/logql/optimize.go @@ -8,7 +8,7 @@ func optimizeSampleExpr(expr syntax.SampleExpr) (syntax.SampleExpr, error) { // we skip sharding AST for now, it's not easy to clone them since they are not part of the language. expr.Walk(func(e syntax.Expr) { switch e.(type) { - case *ConcatSampleExpr, *DownstreamSampleExpr: + case *ConcatSampleExpr, *DownstreamSampleExpr, *QuantileSketchEvalExpr, *QuantileSketchMergeExpr: skip = true return } @@ -16,9 +16,7 @@ func optimizeSampleExpr(expr syntax.SampleExpr) (syntax.SampleExpr, error) { if skip { return expr, nil } - // clone the expr. - q := expr.String() - expr, err := syntax.ParseSampleExpr(q) + expr, err := syntax.Clone[syntax.SampleExpr](expr) if err != nil { return nil, err } diff --git a/pkg/logql/quantile_over_time_sketch.go b/pkg/logql/quantile_over_time_sketch.go new file mode 100644 index 0000000000..94aea83dcd --- /dev/null +++ b/pkg/logql/quantile_over_time_sketch.go @@ -0,0 +1,413 @@ +package logql + +import ( + "fmt" + "time" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" + promql_parser "github.com/prometheus/prometheus/promql/parser" + + "github.com/grafana/loki/pkg/iter" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/sketch" + "github.com/grafana/loki/pkg/logqlmodel" +) + +const ( + QuantileSketchMatrixType = "QuantileSketchMatrix" +) + +type ProbabilisticQuantileVector []ProbabilisticQuantileSample +type ProbabilisticQuantileMatrix []ProbabilisticQuantileVector + +func (q ProbabilisticQuantileVector) Merge(right ProbabilisticQuantileVector) (ProbabilisticQuantileVector, error) { + // labels hash to vector index map + groups := make(map[uint64]int) + for i, sample := range q { + groups[sample.Metric.Hash()] = i + } + + for _, sample := range right { + i, ok := groups[sample.Metric.Hash()] + if !ok { + q = append(q, sample) + continue + } + + _, err := q[i].F.Merge(sample.F) + if err != nil { + return q, err + } + } + + return q, nil +} + +func (ProbabilisticQuantileVector) SampleVector() promql.Vector { + return promql.Vector{} +} + +func (q ProbabilisticQuantileVector) QuantileSketchVec() ProbabilisticQuantileVector { + return q +} + +func (q ProbabilisticQuantileVector) ToProto() *logproto.QuantileSketchVector { + samples := make([]*logproto.QuantileSketchSample, len(q)) + for i, sample := range q { + samples[i] = sample.ToProto() + } + return &logproto.QuantileSketchVector{Samples: samples} +} + +func ProbabilisticQuantileVectorFromProto(proto *logproto.QuantileSketchVector) (ProbabilisticQuantileVector, error) { + out := make([]ProbabilisticQuantileSample, len(proto.Samples)) + var s ProbabilisticQuantileSample + var err error + for i, sample := range proto.Samples { + s, err = probabilisticQuantileSampleFromProto(sample) + if err != nil { + return ProbabilisticQuantileVector{}, err + } + out[i] = s + } + return out, nil +} + +func (ProbabilisticQuantileMatrix) String() string { + return "QuantileSketchMatrix()" +} + +func (ProbabilisticQuantileMatrix) Type() promql_parser.ValueType { return QuantileSketchMatrixType } + +func (m ProbabilisticQuantileMatrix) ToProto() *logproto.QuantileSketchMatrix { + values := make([]*logproto.QuantileSketchVector, len(m)) + for i, vec := range m { + values[i] = vec.ToProto() + } + return &logproto.QuantileSketchMatrix{Values: values} +} + +func ProbabilisticQuantileMatrixFromProto(proto *logproto.QuantileSketchMatrix) (ProbabilisticQuantileMatrix, error) { + out := make([]ProbabilisticQuantileVector, len(proto.Values)) + var s ProbabilisticQuantileVector + var err error + for i, v := range proto.Values { + s, err = ProbabilisticQuantileVectorFromProto(v) + if err != nil { + return ProbabilisticQuantileMatrix{}, err + } + out[i] = s + } + return out, nil +} + +type QuantileSketchStepEvaluator struct { + iter RangeVectorIterator + + err error +} + +func (e *QuantileSketchStepEvaluator) Next() (bool, int64, StepResult) { + next := e.iter.Next() + if !next { + return false, 0, ProbabilisticQuantileVector{} + } + ts, r := e.iter.At() + vec := r.QuantileSketchVec() + for _, s := range vec { + // Errors are not allowed in metrics unless they've been specifically requested. + if s.Metric.Has(logqlmodel.ErrorLabel) && s.Metric.Get(logqlmodel.PreserveErrorLabel) != "true" { + e.err = logqlmodel.NewPipelineErr(s.Metric) + return false, 0, ProbabilisticQuantileVector{} + } + } + return true, ts, vec +} + +func (e *QuantileSketchStepEvaluator) Close() error { return e.iter.Close() } + +func (e *QuantileSketchStepEvaluator) Error() error { + if e.err != nil { + return e.err + } + return e.iter.Error() +} + +func (e *QuantileSketchStepEvaluator) Explain(parent Node) { + parent.Child("QuantileSketch") +} + +func newQuantileSketchIterator( + it iter.PeekingSampleIterator, + selRange, step, start, end, offset int64) RangeVectorIterator { + inner := &batchRangeVectorIterator{ + iter: it, + step: step, + end: end, + selRange: selRange, + metrics: map[string]labels.Labels{}, + window: map[string]*promql.Series{}, + agg: nil, + current: start - step, // first loop iteration will set it to start + offset: offset, + } + return &quantileSketchBatchRangeVectorIterator{ + batchRangeVectorIterator: inner, + } +} + +//batch + +type ProbabilisticQuantileSample struct { + T int64 + F sketch.QuantileSketch + + Metric labels.Labels +} + +func (q ProbabilisticQuantileSample) ToProto() *logproto.QuantileSketchSample { + metric := make([]*logproto.LabelPair, len(q.Metric)) + for i, m := range q.Metric { + metric[i] = &logproto.LabelPair{Name: m.Name, Value: m.Value} + } + + sketch := q.F.ToProto() + + return &logproto.QuantileSketchSample{ + F: sketch, + TimestampMs: q.T, + Metric: metric, + } +} + +func probabilisticQuantileSampleFromProto(proto *logproto.QuantileSketchSample) (ProbabilisticQuantileSample, error) { + s, err := sketch.QuantileSketchFromProto(proto.F) + if err != nil { + return ProbabilisticQuantileSample{}, err + } + out := ProbabilisticQuantileSample{ + T: proto.TimestampMs, + F: s, + Metric: make(labels.Labels, len(proto.Metric)), + } + + for i, p := range proto.Metric { + out.Metric[i] = labels.Label{Name: p.Name, Value: p.Value} + } + + return out, nil +} + +type quantileSketchBatchRangeVectorIterator struct { + *batchRangeVectorIterator + at []ProbabilisticQuantileSample +} + +func (r *quantileSketchBatchRangeVectorIterator) At() (int64, StepResult) { + if r.at == nil { + r.at = make([]ProbabilisticQuantileSample, 0, len(r.window)) + } + r.at = r.at[:0] + // convert ts from nano to milli seconds as the iterator work with nanoseconds + ts := r.current/1e+6 + r.offset/1e+6 + for _, series := range r.window { + r.at = append(r.at, ProbabilisticQuantileSample{ + F: r.agg(series.Floats), + T: ts, + Metric: series.Metric, + }) + } + return ts, ProbabilisticQuantileVector(r.at) +} + +func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sketch.QuantileSketch { + s := sketch.NewDDSketch() + for _, v := range samples { + // The sketch from the underlying sketch package we are using + // cannot return an error when calling Add. + s.Add(v.F) //nolint:errcheck + } + return s +} + +// JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. +func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator) (promql_parser.Value, error) { + vec := r.QuantileSketchVec() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + result := make([]ProbabilisticQuantileVector, 0) + + for next { + result = append(result, vec) + + next, _, r = stepEvaluator.Next() + vec = r.QuantileSketchVec() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + } + + return ProbabilisticQuantileMatrix(result), stepEvaluator.Error() +} + +// QuantileSketchMatrixStepEvaluator steps through a matrix of quantile sketch +// vectors, ie t-digest or DDSketch structures per time step. +type QuantileSketchMatrixStepEvaluator struct { + start, end, ts time.Time + step time.Duration + m ProbabilisticQuantileMatrix +} + +func NewQuantileSketchMatrixStepEvaluator(m ProbabilisticQuantileMatrix, params Params) *QuantileSketchMatrixStepEvaluator { + var ( + start = params.Start() + end = params.End() + step = params.Step() + ) + return &QuantileSketchMatrixStepEvaluator{ + start: start, + end: end, + ts: start.Add(-step), // will be corrected on first Next() call + step: step, + m: m, + } +} + +func (m *QuantileSketchMatrixStepEvaluator) Next() (bool, int64, StepResult) { + m.ts = m.ts.Add(m.step) + if m.ts.After(m.end) { + return false, 0, nil + } + + ts := m.ts.UnixNano() / int64(time.Millisecond) + + if len(m.m) == 0 { + return false, 0, nil + } + + vec := m.m[0] + + // Reset for next step + m.m = m.m[1:] + + return true, ts, vec +} + +func (*QuantileSketchMatrixStepEvaluator) Close() error { return nil } + +func (*QuantileSketchMatrixStepEvaluator) Error() error { return nil } + +func (*QuantileSketchMatrixStepEvaluator) Explain(parent Node) { + parent.Child("QuantileSketchMatrix") +} + +// QuantileSketchMergeStepEvaluator merges multiple quantile sketches into one for each +// step. +type QuantileSketchMergeStepEvaluator struct { + evaluators []StepEvaluator + err error +} + +func NewQuantileSketchMergeStepEvaluator(evaluators []StepEvaluator) *QuantileSketchMergeStepEvaluator { + return &QuantileSketchMergeStepEvaluator{ + evaluators: evaluators, + err: nil, + } +} + +func (e *QuantileSketchMergeStepEvaluator) Next() (bool, int64, StepResult) { + ok, ts, r := e.evaluators[0].Next() + var cur ProbabilisticQuantileVector + if ok { + cur = r.QuantileSketchVec() + } + + if len(e.evaluators) == 1 { + return ok, ts, cur + } + + for _, eval := range e.evaluators[1:] { + ok, nextTs, vec := eval.Next() + if ok { + if cur == nil { + cur = vec.QuantileSketchVec() + } else { + if ts != nextTs { + e.err = fmt.Errorf("timestamps of sketches differ: %d!=%d", ts, nextTs) + return false, 0, nil + } + + _, e.err = cur.Merge(vec.QuantileSketchVec()) + if e.err != nil { + return false, 0, nil + } + } + } + } + + return ok, ts, cur +} + +func (*QuantileSketchMergeStepEvaluator) Close() error { return nil } + +func (e *QuantileSketchMergeStepEvaluator) Error() error { return e.err } + +func (e *QuantileSketchMergeStepEvaluator) Explain(parent Node) { + b := parent.Child("QuantileSketchMerge") + if len(e.evaluators) < MaxChildrenDisplay { + for _, child := range e.evaluators { + child.Explain(b) + } + } else { + e.evaluators[0].Explain(b) + b.Child("...") + e.evaluators[len(e.evaluators)-1].Explain(b) + } +} + +// QuantileSketchVectorStepEvaluator evaluates a quantile sketch into a +// promql.Vector. +type QuantileSketchVectorStepEvaluator struct { + inner StepEvaluator + quantile float64 +} + +var _ StepEvaluator = NewQuantileSketchVectorStepEvaluator(nil, 0) + +func NewQuantileSketchVectorStepEvaluator(inner StepEvaluator, quantile float64) *QuantileSketchVectorStepEvaluator { + return &QuantileSketchVectorStepEvaluator{ + inner: inner, + quantile: quantile, + } +} + +func (e *QuantileSketchVectorStepEvaluator) Next() (bool, int64, StepResult) { + ok, ts, r := e.inner.Next() + quantileSketchVec := r.QuantileSketchVec() + + vec := make(promql.Vector, len(quantileSketchVec)) + + for i, quantileSketch := range quantileSketchVec { + f, _ := quantileSketch.F.Quantile(e.quantile) + + vec[i] = promql.Sample{ + T: quantileSketch.T, + F: f, + Metric: quantileSketch.Metric, + } + } + + return ok, ts, SampleVector(vec) +} + +func (*QuantileSketchVectorStepEvaluator) Close() error { return nil } + +func (*QuantileSketchVectorStepEvaluator) Error() error { return nil } + +func (e *QuantileSketchVectorStepEvaluator) Explain(parent Node) { + b := parent.Child("QuantileSketchVector") + e.inner.Explain(b) +} diff --git a/pkg/logql/quantile_over_time_sketch_test.go b/pkg/logql/quantile_over_time_sketch_test.go new file mode 100644 index 0000000000..9a9ff1b603 --- /dev/null +++ b/pkg/logql/quantile_over_time_sketch_test.go @@ -0,0 +1,109 @@ +package logql + +import ( + "errors" + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/sketch" + "github.com/grafana/loki/pkg/logqlmodel" +) + +func TestProbabilisticMQuantileMatrixSerialization(t *testing.T) { + emptySketch := sketch.NewDDSketch() + ddsketchBytes := make([]byte, 0) + emptySketch.Encode(&ddsketchBytes, false) + + matrix := ProbabilisticQuantileMatrix([]ProbabilisticQuantileVector{ + []ProbabilisticQuantileSample{ + {T: 0, F: emptySketch, Metric: []labels.Label{{Name: "foo", Value: "bar"}}}, + }, + }) + + proto := &logproto.QuantileSketchMatrix{ + Values: []*logproto.QuantileSketchVector{ + { + Samples: []*logproto.QuantileSketchSample{ + { + TimestampMs: 0, + F: &logproto.QuantileSketch{Sketch: &logproto.QuantileSketch_Ddsketch{Ddsketch: ddsketchBytes}}, + Metric: []*logproto.LabelPair{{Name: "foo", Value: "bar"}}, + }, + }, + }, + }, + } + + actual := matrix.ToProto() + require.Equal(t, proto, actual) + + _, err := ProbabilisticQuantileMatrixFromProto(actual) + require.NoError(t, err) +} + +func TestQuantileSketchStepEvaluatorError(t *testing.T) { + iter := errorRangeVectorIterator{ + result: ProbabilisticQuantileVector([]ProbabilisticQuantileSample{ + {T: 43, F: nil, Metric: labels.Labels{{Name: logqlmodel.ErrorLabel, Value: "my error"}}}, + }), + } + ev := QuantileSketchStepEvaluator{ + iter: iter, + } + ok, _, _ := ev.Next() + require.False(t, ok) + + err := ev.Error() + require.ErrorContains(t, err, "my error") +} + +func TestJoinQuantileSketchVectorError(t *testing.T) { + result := ProbabilisticQuantileVector{} + ev := errorStepEvaluator{ + err: errors.New("could not evaluate"), + } + _, err := JoinQuantileSketchVector(true, result, ev) + require.ErrorContains(t, err, "could not evaluate") +} + +type errorRangeVectorIterator struct { + err error + result StepResult +} + +func (e errorRangeVectorIterator) Next() bool { + return e.result != nil +} + +func (e errorRangeVectorIterator) At() (int64, StepResult) { + return 0, e.result +} + +func (errorRangeVectorIterator) Close() error { + return nil +} + +func (e errorRangeVectorIterator) Error() error { + return e.err +} + +type errorStepEvaluator struct { + err error +} + +func (errorStepEvaluator) Next() (ok bool, ts int64, r StepResult) { + return false, 0, nil +} + +func (errorStepEvaluator) Close() error { + return nil +} + +func (e errorStepEvaluator) Error() error { + return e.err +} + +func (e errorStepEvaluator) Explain(Node) {} diff --git a/pkg/logql/range_vector_test.go b/pkg/logql/range_vector_test.go index 089bcff9e2..c7176bed2a 100644 --- a/pkg/logql/range_vector_test.go +++ b/pkg/logql/range_vector_test.go @@ -3,6 +3,8 @@ package logql import ( "context" "fmt" + "math/rand" + "sort" "testing" "time" @@ -13,7 +15,9 @@ import ( "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/sketch" "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/logql/vector" ) var samples = []logproto.Sample{ @@ -442,3 +446,69 @@ func value(value float64, negative bool) float64 { } return value } + +func TestQuantiles(t *testing.T) { + // v controls the distribution of values along the curve, a greater v + // value means there's a large distance between generated values + vs := []float64{1.0, 5.0, 10.0} + // s controls the exponential curve of the distribution + // the higher the s values the faster the drop off from max value to lesser values + // s must be > 1.0 + ss := []float64{1.01, 2.0, 3.0, 4.0} + + // T-Digest is too big for 1_000 samples. However, we did not optimize + // the format for size. + nSamples := []int{5_000, 10_000, 100_000, 1_000_000} + + factories := []struct { + newSketch sketch.QuantileSketchFactory + name string + relativeError float64 + }{ + {newSketch: func() sketch.QuantileSketch { return sketch.NewDDSketch() }, name: "DDSketch", relativeError: 0.02}, + {newSketch: sketch.NewTDigestSketch, name: "T-Digest", relativeError: 0.05}, + } + + for _, tc := range factories { + for _, samplesCount := range nSamples { + for _, s := range ss { + for _, v := range vs { + t.Run(fmt.Sprintf("sketch=%s, s=%.2f, v=%.2f, events=%d", tc.name, s, v, samplesCount), func(t *testing.T) { + sk := tc.newSketch() + + r := rand.New(rand.NewSource(42)) + z := rand.NewZipf(r, s, v, 1_000) + values := make(vector.HeapByMaxValue, 0) + for i := 0; i < samplesCount; i++ { + + value := float64(z.Uint64()) + values = append(values, promql.Sample{F: value}) + err := sk.Add(value) + require.NoError(t, err) + } + sort.Sort(values) + + // Size + var buf []byte + var err error + switch s := sk.(type) { + case *sketch.DDSketchQuantile: + buf, err = proto.Marshal(s.DDSketch.ToProto()) + require.NoError(t, err) + case *sketch.TDigestQuantile: + buf, err = proto.Marshal(s.ToProto()) + require.NoError(t, err) + } + require.Less(t, len(buf), samplesCount*8) + + // Accuracy + expected := Quantile(0.99, values) + actual, err := sk.Quantile(0.99) + require.NoError(t, err) + require.InEpsilonf(t, expected, actual, tc.relativeError, "expected quantile %f, actual quantile %f", expected, actual) + }) + } + } + } + } +} diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index f1ee7e4ba6..4bee2616bf 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -25,15 +25,27 @@ type ConstantShards int func (s ConstantShards) Shards(_ syntax.Expr) (int, uint64, error) { return int(s), 0, nil } func (s ConstantShards) GetStats(_ syntax.Expr) (stats.Stats, error) { return stats.Stats{}, nil } +const ( + ShardQuantileOverTime = "quantile_over_time" +) + type ShardMapper struct { - shards ShardResolver - metrics *MapperMetrics + shards ShardResolver + metrics *MapperMetrics + quantileOverTimeSharding bool } -func NewShardMapper(resolver ShardResolver, metrics *MapperMetrics) ShardMapper { +func NewShardMapper(resolver ShardResolver, metrics *MapperMetrics, shardAggregation []string) ShardMapper { + quantileOverTimeSharding := false + for _, a := range shardAggregation { + if a == ShardQuantileOverTime { + quantileOverTimeSharding = true + } + } return ShardMapper{ - shards: resolver, - metrics: metrics, + shards: resolver, + metrics: metrics, + quantileOverTimeSharding: quantileOverTimeSharding, } } @@ -158,11 +170,11 @@ func (m ShardMapper) mapSampleExpr(expr syntax.SampleExpr, r *downstreamRecorder }, }, bytesPerShard, nil } - for i := shards - 1; i >= 0; i-- { + for shard := shards - 1; shard >= 0; shard-- { head = &ConcatSampleExpr{ DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ - Shard: i, + Shard: shard, Of: shards, }, SampleExpr: expr, @@ -374,7 +386,7 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, return m.mapSampleExpr(expr, r) } - // avg_overtime() by (foo) -> sum by (foo) (sum_over_time()) / sum by (foo) (count_over_time()) + // avg_over_time() by (foo) -> sum by (foo) (sum_over_time()) / sum by (foo) (count_over_time()) lhs, lhsBytesPerShard, err := m.mapVectorAggregationExpr(&syntax.VectorAggregationExpr{ Left: &syntax.RangeAggregationExpr{ Left: expr.Left, @@ -414,6 +426,43 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, Op: syntax.OpTypeDiv, }, bytesPerShard, nil + case syntax.OpRangeTypeQuantile: + potentialConflict := syntax.ReducesLabels(expr) + if !potentialConflict && (expr.Grouping == nil || expr.Grouping.Noop()) { + return m.mapSampleExpr(expr, r) + } + + shards, bytesPerShard, err := m.shards.Shards(expr) + if err != nil { + return nil, 0, err + } + if shards == 0 || !m.quantileOverTimeSharding { + return m.mapSampleExpr(expr, r) + } + + // quantile_over_time() by (foo) -> + // quantile_sketch_eval(quantile_merge by (foo) + // (__quantile_sketch_over_time__() by (foo))) + + downstreams := make([]DownstreamSampleExpr, 0, shards) + expr.Operation = syntax.OpRangeTypeQuantileSketch + for shard := shards - 1; shard >= 0; shard-- { + downstreams = append(downstreams, DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: shard, + Of: shards, + }, + SampleExpr: expr, + }) + } + + return &QuantileSketchEvalExpr{ + quantileMergeExpr: &QuantileSketchMergeExpr{ + downstreams: downstreams, + }, + quantile: expr.Params, + }, bytesPerShard, nil + default: // don't shard if there's not an appropriate optimization exprStats, err := m.shards.GetStats(expr) diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 80b2e68751..4dc4aac0fb 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -51,7 +51,7 @@ func TestShardedStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics) + m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) for _, tc := range []struct { in syntax.SampleExpr @@ -113,7 +113,7 @@ func TestMapSampleExpr(t *testing.T) { } func TestMappingStrings(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics) + m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) for _, tc := range []struct { in string out string @@ -418,7 +418,7 @@ func TestMappingStrings(t *testing.T) { } func TestMapping(t *testing.T) { - m := NewShardMapper(ConstantShards(2), nilShardMetrics) + m := NewShardMapper(ConstantShards(2), nilShardMetrics, []string{ShardQuantileOverTime}) for _, tc := range []struct { in string @@ -1409,7 +1409,7 @@ func TestStringTrimming(t *testing.T) { }, } { t.Run(tc.expr.String(), func(t *testing.T) { - m := NewShardMapper(ConstantShards(tc.shards), nilShardMetrics) + m := NewShardMapper(ConstantShards(tc.shards), nilShardMetrics, []string{ShardQuantileOverTime}) _, _, mappedExpr, err := m.Parse(tc.expr) require.Nil(t, err) require.Equal(t, removeWhiteSpace(tc.expected), removeWhiteSpace(mappedExpr.String())) diff --git a/pkg/logql/sketch/quantile.go b/pkg/logql/sketch/quantile.go index 14b44e69f5..3a0526fcfc 100644 --- a/pkg/logql/sketch/quantile.go +++ b/pkg/logql/sketch/quantile.go @@ -6,107 +6,10 @@ import ( "github.com/DataDog/sketches-go/ddsketch" "github.com/influxdata/tdigest" - "github.com/prometheus/prometheus/model/labels" - promql_parser "github.com/prometheus/prometheus/promql/parser" "github.com/grafana/loki/pkg/logproto" ) -// QuantileSketchVector represents multiple qunatile sketches at the same point in -// time. -type QuantileSketchVector []quantileSketchSample - -// QuantileSketchMatrix contains multiples QuantileSketchVectors across many -// points in time. -type QuantileSketchMatrix []QuantileSketchVector - -// ToProto converts a quantile sketch vector to its protobuf definition. -func (q QuantileSketchVector) ToProto() *logproto.QuantileSketchVector { - samples := make([]*logproto.QuantileSketchSample, len(q)) - for i, sample := range q { - samples[i] = sample.ToProto() - } - return &logproto.QuantileSketchVector{Samples: samples} -} - -func QuantileSketchVectorFromProto(proto *logproto.QuantileSketchVector) (QuantileSketchVector, error) { - out := make([]quantileSketchSample, len(proto.Samples)) - var err error - for i, s := range proto.Samples { - out[i], err = quantileSketchSampleFromProto(s) - if err != nil { - return nil, err - } - } - return out, nil -} - -func (QuantileSketchMatrix) String() string { - return "QuantileSketchMatrix()" -} - -func (QuantileSketchMatrix) Type() promql_parser.ValueType { return "QuantileSketchMatrix" } - -func (m QuantileSketchMatrix) ToProto() *logproto.QuantileSketchMatrix { - values := make([]*logproto.QuantileSketchVector, len(m)) - for i, vec := range m { - values[i] = vec.ToProto() - } - return &logproto.QuantileSketchMatrix{Values: values} -} - -func QuantileSketchMatrixFromProto(proto *logproto.QuantileSketchMatrix) (QuantileSketchMatrix, error) { - out := make([]QuantileSketchVector, len(proto.Values)) - var err error - for i, v := range proto.Values { - out[i], err = QuantileSketchVectorFromProto(v) - if err != nil { - return nil, err - } - } - return out, nil -} - -type quantileSketchSample struct { - T int64 - F QuantileSketch - - Metric labels.Labels -} - -func (q quantileSketchSample) ToProto() *logproto.QuantileSketchSample { - metric := make([]*logproto.LabelPair, len(q.Metric)) - for i, m := range q.Metric { - metric[i] = &logproto.LabelPair{Name: m.Name, Value: m.Value} - } - - sketch := q.F.ToProto() - - return &logproto.QuantileSketchSample{ - F: sketch, - TimestampMs: q.T, - Metric: metric, - } -} - -func quantileSketchSampleFromProto(proto *logproto.QuantileSketchSample) (quantileSketchSample, error) { - sketch, err := QuantileSketchFromProto(proto.F) - if err != nil { - return quantileSketchSample{}, err - } - out := quantileSketchSample{ - T: proto.TimestampMs, - F: sketch, - Metric: make(labels.Labels, len(proto.Metric)), - } - - for i, p := range proto.Metric { - out.Metric[i] = labels.Label{Name: p.Name, Value: p.Value} - } - - return out, nil -} - // QuantileSketch estimates quantiles over time. type QuantileSketch interface { Add(float64) error diff --git a/pkg/logql/sketch/quantile_test.go b/pkg/logql/sketch/quantile_test.go deleted file mode 100644 index 3b2f34c0e8..0000000000 --- a/pkg/logql/sketch/quantile_test.go +++ /dev/null @@ -1,81 +0,0 @@ -package sketch - -import ( - "fmt" - "math/rand" - "sort" - "testing" - - "github.com/gogo/protobuf/proto" - "github.com/prometheus/prometheus/promql" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/logql" - "github.com/grafana/loki/pkg/logql/vector" -) - -func TestQuantiles(t *testing.T) { - // v controls the distribution of values along the curve, a greater v - // value means there's a large distance between generated values - vs := []float64{1.0, 5.0, 10.0} - // s controls the exponential curve of the distribution - // the higher the s values the faster the drop off from max value to lesser values - // s must be > 1.0 - ss := []float64{1.01, 2.0, 3.0, 4.0} - - // T-Digest is too big for 1_000 samples. However, we did not optimize - // the format for size. - nSamples := []int{5_000, 10_000, 100_000, 1_000_000} - - factories := []struct { - newSketch QuantileSketchFactory - name string - relativeError float64 - }{ - {newSketch: func() QuantileSketch { return NewDDSketch() }, name: "DDSketch", relativeError: 0.02}, - {newSketch: NewTDigestSketch, name: "T-Digest", relativeError: 0.05}, - } - - for _, tc := range factories { - for _, samplesCount := range nSamples { - for _, s := range ss { - for _, v := range vs { - t.Run(fmt.Sprintf("sketch=%s, s=%.2f, v=%.2f, events=%d", tc.name, s, v, samplesCount), func(t *testing.T) { - sketch := tc.newSketch() - - r := rand.New(rand.NewSource(42)) - z := rand.NewZipf(r, s, v, 1_000) - values := make(vector.HeapByMaxValue, 0) - for i := 0; i < samplesCount; i++ { - - value := float64(z.Uint64()) - values = append(values, promql.Sample{F: value}) - err := sketch.Add(value) - require.NoError(t, err) - } - sort.Sort(values) - - // Size - var buf []byte - var err error - switch s := sketch.(type) { - case *DDSketchQuantile: - buf, err = proto.Marshal(s.DDSketch.ToProto()) - require.NoError(t, err) - case *TDigestQuantile: - buf, err = proto.Marshal(s.ToProto()) - require.NoError(t, err) - } - require.Less(t, len(buf), samplesCount*8) - - // Accuracy - expected := logql.Quantile(0.99, values) - actual, err := sketch.Quantile(0.99) - require.NoError(t, err) - require.InEpsilonf(t, expected, actual, tc.relativeError, "expected quantile %f, actual quantile %f", expected, actual) - }) - } - } - } - } -} diff --git a/pkg/logql/step_evaluator.go b/pkg/logql/step_evaluator.go index 3831c8babd..955f9e2b97 100644 --- a/pkg/logql/step_evaluator.go +++ b/pkg/logql/step_evaluator.go @@ -6,6 +6,7 @@ import ( type StepResult interface { SampleVector() promql.Vector + QuantileSketchVec() ProbabilisticQuantileVector } type SampleVector promql.Vector @@ -16,6 +17,10 @@ func (p SampleVector) SampleVector() promql.Vector { return promql.Vector(p) } +func (p SampleVector) QuantileSketchVec() ProbabilisticQuantileVector { + return ProbabilisticQuantileVector{} +} + // StepEvaluator evaluate a single step of a query. type StepEvaluator interface { // while Next returns a promql.Value, the only acceptable types are Scalar and Vector. diff --git a/pkg/logql/syntax/ast.go b/pkg/logql/syntax/ast.go index 0bbb611e7e..e110b37236 100644 --- a/pkg/logql/syntax/ast.go +++ b/pkg/logql/syntax/ast.go @@ -37,17 +37,23 @@ type Expr interface { func Clone[T Expr](e T) (T, error) { var empty T - copied, err := ParseExpr(e.String()) - if err != nil { - return empty, err - } - cast, ok := copied.(T) + v := &cloneVisitor{} + e.Accept(v) + cast, ok := v.cloned.(T) if !ok { - return empty, fmt.Errorf("unpexpected type of cloned expression: want %T, got %T", empty, copied) + return empty, fmt.Errorf("unexpected type of cloned expression: want %T, got %T", empty, v.cloned) } return cast, nil } +func MustClone[T Expr](e T) T { + copied, err := Clone[T](e) + if err != nil { + panic(err) + } + return copied +} + // implicit holds default implementations type implicit struct{} @@ -1156,6 +1162,11 @@ const ( // parser flags OpStrict = "--strict" OpKeepEmpty = "--keep-empty" + + // internal expressions not represented in LogQL. These are used to + // evaluate expressions differently resulting in intermediate formats + // that are not consumable by LogQL clients but are used for sharding. + OpRangeTypeQuantileSketch = "__quantile_sketch_over_time__" ) func IsComparisonOperator(op string) bool { @@ -1204,7 +1215,7 @@ type RangeAggregationExpr struct { func newRangeAggregationExpr(left *LogRange, operation string, gr *Grouping, stringParams *string) SampleExpr { var params *float64 if stringParams != nil { - if operation != OpRangeTypeQuantile { + if operation != OpRangeTypeQuantile && operation != OpRangeTypeQuantileSketch { return &RangeAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter %s not supported for operation %s", *stringParams, operation), 0, 0)} } var err error @@ -1259,7 +1270,7 @@ func (e *RangeAggregationExpr) MatcherGroups() ([]MatcherRange, error) { func (e RangeAggregationExpr) validate() error { if e.Grouping != nil { switch e.Operation { - case OpRangeTypeAvg, OpRangeTypeStddev, OpRangeTypeStdvar, OpRangeTypeQuantile, OpRangeTypeMax, OpRangeTypeMin, OpRangeTypeFirst, OpRangeTypeLast: + case OpRangeTypeAvg, OpRangeTypeStddev, OpRangeTypeStdvar, OpRangeTypeQuantile, OpRangeTypeQuantileSketch, OpRangeTypeMax, OpRangeTypeMin, OpRangeTypeFirst, OpRangeTypeLast: default: return fmt.Errorf("grouping not allowed for %s aggregation", e.Operation) } @@ -1268,7 +1279,7 @@ func (e RangeAggregationExpr) validate() error { switch e.Operation { case OpRangeTypeAvg, OpRangeTypeSum, OpRangeTypeMax, OpRangeTypeMin, OpRangeTypeStddev, OpRangeTypeStdvar, OpRangeTypeQuantile, OpRangeTypeRate, OpRangeTypeRateCounter, - OpRangeTypeAbsent, OpRangeTypeFirst, OpRangeTypeLast: + OpRangeTypeAbsent, OpRangeTypeFirst, OpRangeTypeLast, OpRangeTypeQuantileSketch: return nil default: return fmt.Errorf("invalid aggregation %s with unwrap", e.Operation) @@ -2128,6 +2139,7 @@ var shardableOps = map[string]bool{ OpRangeTypeSum: true, OpRangeTypeMax: true, OpRangeTypeMin: true, + OpRangeTypeQuantile: true, // binops - arith OpTypeAdd: true, diff --git a/pkg/logql/syntax/clone.go b/pkg/logql/syntax/clone.go new file mode 100644 index 0000000000..07aeb141ea --- /dev/null +++ b/pkg/logql/syntax/clone.go @@ -0,0 +1,298 @@ +package syntax + +import ( + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/logql/log" +) + +type cloneVisitor struct { + cloned Expr +} + +var _ RootVisitor = &cloneVisitor{} + +func cloneGrouping(g *Grouping) *Grouping { + copied := &Grouping{ + Without: g.Without, + } + if g.Groups != nil { + copied.Groups = make([]string, len(g.Groups)) + copy(copied.Groups, g.Groups) + } + return copied +} + +func cloneVectorMatching(v *VectorMatching) *VectorMatching { + copied := *v + copy(copied.Include, v.Include) + copy(copied.MatchingLabels, v.MatchingLabels) + + return &copied +} + +func (v *cloneVisitor) VisitBinOp(e *BinOpExpr) { + lhs := MustClone[SampleExpr](e.SampleExpr) + rhs := MustClone[SampleExpr](e.RHS) + copied := &BinOpExpr{ + SampleExpr: lhs, + RHS: rhs, + Op: e.Op, + } + + if e.Opts != nil { + copied.Opts = &BinOpOptions{ + ReturnBool: e.Opts.ReturnBool, + VectorMatching: cloneVectorMatching(e.Opts.VectorMatching), + } + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitVectorAggregation(e *VectorAggregationExpr) { + copied := &VectorAggregationExpr{ + Left: MustClone[SampleExpr](e.Left), + Params: e.Params, + Operation: e.Operation, + } + + if e.Grouping != nil { + copied.Grouping = cloneGrouping(e.Grouping) + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitRangeAggregation(e *RangeAggregationExpr) { + copied := &RangeAggregationExpr{ + Left: MustClone[*LogRange](e.Left), + Operation: e.Operation, + } + + if e.Grouping != nil { + copied.Grouping = cloneGrouping(e.Grouping) + } + + if e.Params != nil { + tmp := *e.Params + copied.Params = &tmp + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitLabelReplace(e *LabelReplaceExpr) { + left := MustClone[SampleExpr](e.Left) + v.cloned = mustNewLabelReplaceExpr(left, e.Dst, e.Replacement, e.Src, e.Regex) +} + +func (v *cloneVisitor) VisitLiteral(e *LiteralExpr) { + v.cloned = &LiteralExpr{Val: e.Val} +} + +func (v *cloneVisitor) VisitVector(e *VectorExpr) { + v.cloned = &VectorExpr{Val: e.Val} +} + +func (v *cloneVisitor) VisitLogRange(e *LogRange) { + copied := &LogRange{ + Left: MustClone[LogSelectorExpr](e.Left), + Interval: e.Interval, + Offset: e.Offset, + } + if e.Unwrap != nil { + copied.Unwrap = &UnwrapExpr{ + Identifier: e.Unwrap.Identifier, + Operation: e.Unwrap.Operation, + } + if e.Unwrap.PostFilters != nil { + copied.Unwrap.PostFilters = make([]log.LabelFilterer, len(e.Unwrap.PostFilters)) + for i, f := range e.Unwrap.PostFilters { + copied.Unwrap.PostFilters[i] = cloneLabelFilterer(f) + } + } + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitMatchers(e *MatchersExpr) { + copied := &MatchersExpr{ + Mts: make([]*labels.Matcher, len(e.Mts)), + } + for i, m := range e.Mts { + copied.Mts[i] = labels.MustNewMatcher(m.Type, m.Name, m.Value) + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitPipeline(e *PipelineExpr) { + copied := &PipelineExpr{ + Left: MustClone[*MatchersExpr](e.Left), + MultiStages: make(MultiStageExpr, len(e.MultiStages)), + } + for i, s := range e.MultiStages { + copied.MultiStages[i] = MustClone[StageExpr](s) + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitDecolorize(*DecolorizeExpr) { + v.cloned = &DecolorizeExpr{} +} + +func (v *cloneVisitor) VisitDropLabels(e *DropLabelsExpr) { + copied := &DropLabelsExpr{ + dropLabels: make([]log.DropLabel, len(e.dropLabels)), + } + for i, l := range e.dropLabels { + var matcher *labels.Matcher + if l.Matcher != nil { + matcher = labels.MustNewMatcher(l.Matcher.Type, l.Matcher.Name, l.Matcher.Value) + } + copied.dropLabels[i] = log.NewDropLabel(matcher, l.Name) + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitJSONExpressionParser(e *JSONExpressionParser) { + copied := &JSONExpressionParser{ + Expressions: make([]log.LabelExtractionExpr, len(e.Expressions)), + } + copy(copied.Expressions, e.Expressions) + + v.cloned = copied +} + +func (v *cloneVisitor) VisitKeepLabel(e *KeepLabelsExpr) { + copied := &KeepLabelsExpr{ + keepLabels: make([]log.KeepLabel, len(e.keepLabels)), + } + for i, k := range e.keepLabels { + copied.keepLabels[i] = log.KeepLabel{ + Name: k.Name, + } + if k.Matcher != nil { + copied.keepLabels[i].Matcher = labels.MustNewMatcher(k.Matcher.Type, k.Matcher.Name, k.Matcher.Value) + } + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitLabelFilter(e *LabelFilterExpr) { + v.cloned = &LabelFilterExpr{ + LabelFilterer: cloneLabelFilterer(e.LabelFilterer), + } +} + +func cloneLabelFilterer(filter log.LabelFilterer) log.LabelFilterer { + switch concrete := filter.(type) { + case *log.BinaryLabelFilter: + return &log.BinaryLabelFilter{ + Left: cloneLabelFilterer(concrete.Left), + Right: cloneLabelFilterer(concrete.Right), + And: concrete.And, + } + case *log.NoopLabelFilter: + copied := &log.NoopLabelFilter{} + if concrete.Matcher != nil { + copied.Matcher = mustNewMatcher(concrete.Type, concrete.Name, concrete.Value) + } + + return copied + case *log.BytesLabelFilter: + return &log.BytesLabelFilter{ + Name: concrete.Name, + Value: concrete.Value, + Type: concrete.Type, + } + case *log.DurationLabelFilter: + return &log.DurationLabelFilter{ + Name: concrete.Name, + Value: concrete.Value, + Type: concrete.Type, + } + case *log.NumericLabelFilter: + return &log.NumericLabelFilter{ + Name: concrete.Name, + Value: concrete.Value, + Type: concrete.Type, + } + case *log.StringLabelFilter: + copied := &log.StringLabelFilter{} + if concrete.Matcher != nil { + copied.Matcher = mustNewMatcher(concrete.Type, concrete.Name, concrete.Value) + } + return copied + case *log.LineFilterLabelFilter: + copied := &log.LineFilterLabelFilter{} + if concrete.Matcher != nil { + copied.Matcher = mustNewMatcher(concrete.Type, concrete.Name, concrete.Value) + } + return copied + case *log.IPLabelFilter: + return log.NewIPLabelFilter(concrete.Pattern, concrete.Label, concrete.Ty) + } + return nil +} + +func (v *cloneVisitor) VisitLabelFmt(e *LabelFmtExpr) { + copied := &LabelFmtExpr{ + Formats: make([]log.LabelFmt, len(e.Formats)), + } + copy(copied.Formats, e.Formats) + v.cloned = copied +} + +func (v *cloneVisitor) VisitLabelParser(e *LabelParserExpr) { + v.cloned = &LabelParserExpr{ + Op: e.Op, + Param: e.Param, + } +} + +func (v *cloneVisitor) VisitLineFilter(e *LineFilterExpr) { + copied := &LineFilterExpr{ + Ty: e.Ty, + Match: e.Match, + Op: e.Op, + IsOrChild: e.IsOrChild, + } + + if e.Left != nil { + copied.Left = MustClone[*LineFilterExpr](e.Left) + } + + if e.Or != nil { + copied.Or = MustClone[*LineFilterExpr](e.Or) + } + + v.cloned = copied +} + +func (v *cloneVisitor) VisitLineFmt(e *LineFmtExpr) { + v.cloned = &LineFmtExpr{Value: e.Value} +} + +func (v *cloneVisitor) VisitLogfmtExpressionParser(e *LogfmtExpressionParser) { + copied := &LogfmtExpressionParser{ + Expressions: make([]log.LabelExtractionExpr, len(e.Expressions)), + Strict: e.Strict, + KeepEmpty: e.KeepEmpty, + } + copy(copied.Expressions, e.Expressions) + + v.cloned = copied +} + +func (v *cloneVisitor) VisitLogfmtParser(e *LogfmtParserExpr) { + v.cloned = &LogfmtParserExpr{ + Strict: e.Strict, + KeepEmpty: e.KeepEmpty, + } +} diff --git a/pkg/logql/syntax/clone_test.go b/pkg/logql/syntax/clone_test.go new file mode 100644 index 0000000000..01364919a6 --- /dev/null +++ b/pkg/logql/syntax/clone_test.go @@ -0,0 +1,114 @@ +package syntax + +import ( + "strings" + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/logql/log" +) + +func TestClone(t *testing.T) { + tests := map[string]struct { + query string + }{ + "simple matchers": { + query: `{env="prod", app=~"loki.*"}`, + }, + "simple aggregation": { + query: `count_over_time({env="prod", app=~"loki.*"}[5m])`, + }, + "simple aggregation with unwrap": { + query: `sum_over_time({env="prod", app=~"loki.*"} | unwrap bytes[5m])`, + }, + "bin op": { + query: `(count_over_time({env="prod", app=~"loki.*"}[5m]) >= 0)`, + }, + "label filter": { + query: `{app="foo"} |= "bar" | json | ( latency>=250ms or ( status_code<500 , status_code>200 ) )`, + }, + "line filter": { + query: `{app="foo"} |= "bar" | json |= "500" or "200"`, + }, + "drop label": { + query: `{app="foo"} |= "bar" | json | drop latency, status_code="200"`, + }, + "keep label": { + query: `{app="foo"} |= "bar" | json | keep latency, status_code="200"`, + }, + "regexp": { + query: `{env="prod", app=~"loki.*"} |~ ".*foo.*"`, + }, + "vector matching": { + query: `(sum by (cluster)(rate({foo="bar"}[5m])) / ignoring (cluster) count(rate({foo="bar"}[5m])))`, + }, + "sum over or vector": { + query: `(sum(count_over_time({foo="bar"}[5m])) or vector(1.000000))`, + }, + "label replace": { + query: `label_replace(vector(0.000000),"foo","bar","","")`, + }, + "filters with bytes": { + query: `{app="foo"} |= "bar" | json | ( status_code <500 or ( status_code>200 , size>=2.5KiB ) )`, + }, + "post filter": { + query: `quantile_over_time(0.99998,{app="foo"} |= "bar" | json | latency >= 250ms or ( status_code < 500 and status_code > 200) + | line_format "blip{{ .foo }}blop {{.status_code}}" | label_format foo=bar,status_code="buzz{{.bar}}" | unwrap foo + | __error__ !~".+"[5m]) by (namespace,instance)`, + }, + "multiple post filters": { + query: `rate({app="foo"} | json | unwrap foo | latency >= 250ms or bytes > 42B or ( status_code < 500 and status_code > 200) or source = ip("") and user = "me" [1m])`, + }, + "true filter": { + query: `{ foo = "bar" } | foo =~".*"`, + }, + } + + for name, test := range tests { + t.Run(name, func(t *testing.T) { + + expr, err := ParseExpr(test.query) + require.NoError(t, err) + + actual, err := Clone[Expr](expr) + require.NoError(t, err) + + require.Equal(t, expr.Pretty(0), actual.Pretty(0)) + }) + } +} + +func TestCloneStringLabelFilter(t *testing.T) { + expr := newPipelineExpr( + newMatcherExpr([]*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}), + MultiStageExpr{ + newLogfmtParserExpr(nil), + newLabelFilterExpr(&log.StringLabelFilter{Matcher: labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")}), + }, + ) + actual, err := Clone[Expr](expr) + require.NoError(t, err) + + require.Equal(t, expr.Pretty(0), actual.Pretty(0)) +} + +func TestCloneParseTestCases(t *testing.T) { + for _, tc := range ParseTestCases { + if tc.err == nil { + t.Run(tc.in, func(t *testing.T) { + ast, err := ParseExpr(tc.in) + require.NoError(t, err) + if strings.Contains(tc.in, "KiB") { + t.Skipf("Byte roundtrip conversion is broken. '%s' vs '%s'", tc.in, ast.String()) + } + + actual, err := Clone[Expr](ast) + require.NoError(t, err) + + require.Equal(t, ast.Pretty(0), actual.Pretty(0)) + }) + } + } +} diff --git a/pkg/logqlmodel/logqlmodel.go b/pkg/logqlmodel/logqlmodel.go index da9d7f083f..8ba0e198c4 100644 --- a/pkg/logqlmodel/logqlmodel.go +++ b/pkg/logqlmodel/logqlmodel.go @@ -5,8 +5,8 @@ import ( "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions" - "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/push" ) // ValueTypeStreams promql.ValueType for log streams @@ -23,7 +23,7 @@ type Result struct { } // Streams is promql.Value -type Streams []logproto.Stream +type Streams []push.Stream func (streams Streams) Len() int { return len(streams) } func (streams Streams) Swap(i, j int) { streams[i], streams[j] = streams[j], streams[i] } diff --git a/pkg/loki/config_compat.go b/pkg/loki/config_compat.go index cd15b05f2d..1e4f800c46 100644 --- a/pkg/loki/config_compat.go +++ b/pkg/loki/config_compat.go @@ -1,15 +1,18 @@ package loki import ( + "errors" "fmt" "github.com/grafana/loki/pkg/ingester/index" + frontend "github.com/grafana/loki/pkg/lokifrontend/frontend/v2" "github.com/grafana/loki/pkg/storage/config" ) func ValidateConfigCompatibility(c Config) error { for _, fn := range []func(Config) error{ ensureInvertedIndexShardingCompatibility, + ensureProtobufEncodingForAggregationSharding, } { if err := fn(c); err != nil { return err @@ -40,3 +43,10 @@ func ensureInvertedIndexShardingCompatibility(c Config) error { } return nil } + +func ensureProtobufEncodingForAggregationSharding(c Config) error { + if len(c.QueryRange.ShardAggregations) > 0 && c.Frontend.FrontendV2.Encoding != frontend.EncodingProtobuf { + return errors.New("shard_aggregation requires frontend.encoding=protobuf") + } + return nil +} diff --git a/pkg/lokifrontend/frontend/v2/frontend_test.go b/pkg/lokifrontend/frontend/v2/frontend_test.go index 3ab1028e96..9a87c5ff1c 100644 --- a/pkg/lokifrontend/frontend/v2/frontend_test.go +++ b/pkg/lokifrontend/frontend/v2/frontend_test.go @@ -19,7 +19,9 @@ import ( "go.uber.org/atomic" "google.golang.org/grpc" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/lokifrontend/frontend/v2/frontendv2pb" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/querier/queryrange" "github.com/grafana/loki/pkg/querier/stats" "github.com/grafana/loki/pkg/scheduler/schedulerpb" @@ -29,7 +31,7 @@ import ( const testFrontendWorkerConcurrency = 5 -func setupFrontend(t *testing.T, schedulerReplyFunc func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend) (*Frontend, *mockScheduler) { +func setupFrontend(t *testing.T, cfg Config, schedulerReplyFunc func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend) (*Frontend, *mockScheduler) { l, err := net.Listen("tcp", "") require.NoError(t, err) @@ -41,8 +43,6 @@ func setupFrontend(t *testing.T, schedulerReplyFunc func(f *Frontend, msg *sched grpcPort, err := strconv.Atoi(p) require.NoError(t, err) - cfg := Config{} - flagext.DefaultValues(&cfg) cfg.SchedulerAddress = l.Addr().String() cfg.WorkerConcurrency = testFrontendWorkerConcurrency cfg.Addr = h @@ -102,7 +102,9 @@ func TestFrontendBasicWorkflow(t *testing.T) { userID = "test" ) - f, _ := setupFrontend(t, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + cfg := Config{} + flagext.DefaultValues(&cfg) + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { // We cannot call QueryResult directly, as Frontend is not yet waiting for the response. // It first needs to be told that enqueuing has succeeded. go sendResponseWithDelay(f, 100*time.Millisecond, userID, msg.QueryID, &httpgrpc.HTTPResponse{ @@ -119,6 +121,41 @@ func TestFrontendBasicWorkflow(t *testing.T) { require.Equal(t, []byte(body), resp.Body) } +func TestFrontendBasicWorkflowProto(t *testing.T) { + const ( + userID = "test" + ) + + ctx := user.InjectOrgID(context.Background(), userID) + + req := &queryrange.LokiRequest{ + Query: `{foo="bar"} | json`, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"} | json`), + }, + } + + resp, err := queryrange.NewEmptyResponse(req) + require.NoError(t, err) + httpReq := &httpgrpc.HTTPRequest{Url: "/loki/api/v1/query_range"} + httpResp, err := queryrange.DefaultCodec.EncodeHTTPGrpcResponse(ctx, httpReq, resp) + require.NoError(t, err) + + cfg := Config{} + flagext.DefaultValues(&cfg) + cfg.Encoding = EncodingProtobuf + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + // We cannot call QueryResult directly, as Frontend is not yet waiting for the response. + // It first needs to be told that enqueuing has succeeded. + go sendResponseWithDelay(f, 100*time.Millisecond, userID, msg.QueryID, httpResp) + + return &schedulerpb.SchedulerToFrontend{Status: schedulerpb.OK} + }) + actualResp, err := f.Do(ctx, req) + require.NoError(t, err) + require.Equal(t, resp.(*queryrange.LokiResponse).Data, actualResp.(*queryrange.LokiResponse).Data) +} + func TestFrontendRetryEnqueue(t *testing.T) { // Frontend uses worker concurrency to compute number of retries. We use one less failure. failures := atomic.NewInt64(testFrontendWorkerConcurrency - 1) @@ -127,7 +164,9 @@ func TestFrontendRetryEnqueue(t *testing.T) { userID = "test" ) - f, _ := setupFrontend(t, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + cfg := Config{} + flagext.DefaultValues(&cfg) + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { fail := failures.Dec() if fail >= 0 { return &schedulerpb.SchedulerToFrontend{Status: schedulerpb.SHUTTING_DOWN} @@ -145,7 +184,9 @@ func TestFrontendRetryEnqueue(t *testing.T) { } func TestFrontendEnqueueFailure(t *testing.T) { - f, _ := setupFrontend(t, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + cfg := Config{} + flagext.DefaultValues(&cfg) + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { return &schedulerpb.SchedulerToFrontend{Status: schedulerpb.SHUTTING_DOWN} }) @@ -155,7 +196,9 @@ func TestFrontendEnqueueFailure(t *testing.T) { } func TestFrontendCancellation(t *testing.T) { - f, ms := setupFrontend(t, nil) + cfg := Config{} + flagext.DefaultValues(&cfg) + f, ms := setupFrontend(t, cfg, nil) ctx, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) defer cancel() @@ -184,7 +227,9 @@ func TestFrontendCancellation(t *testing.T) { // all the frontend workers thus not reaching the scheduler as well. // Issue: https://github.com/grafana/loki/issues/5132 func TestFrontendWorkerCancellation(t *testing.T) { - f, ms := setupFrontend(t, nil) + cfg := Config{} + flagext.DefaultValues(&cfg) + f, ms := setupFrontend(t, cfg, nil) ctx, cancel := context.WithTimeout(context.Background(), 200*time.Millisecond) defer cancel() @@ -219,7 +264,9 @@ func TestFrontendWorkerCancellation(t *testing.T) { } func TestFrontendFailedCancellation(t *testing.T) { - f, ms := setupFrontend(t, nil) + cfg := Config{} + flagext.DefaultValues(&cfg) + f, ms := setupFrontend(t, cfg, nil) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -258,7 +305,9 @@ func TestFrontendFailedCancellation(t *testing.T) { func TestFrontendStoppingWaitsForEmptyInflightRequests(t *testing.T) { delayResponse := 10 * time.Millisecond - f, _ := setupFrontend(t, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + cfg := Config{} + flagext.DefaultValues(&cfg) + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { // We cannot call QueryResult directly, as Frontend is not yet waiting for the response. // It first needs to be told that enqueuing has succeeded. go sendResponseWithDelay(f, 2*delayResponse, "test", msg.QueryID, &httpgrpc.HTTPResponse{ @@ -296,7 +345,9 @@ func TestFrontendStoppingWaitsForEmptyInflightRequests(t *testing.T) { func TestFrontendShuttingDownLetsSubRequestsPass(t *testing.T) { delayResponse := 100 * time.Millisecond - f, _ := setupFrontend(t, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { + cfg := Config{} + flagext.DefaultValues(&cfg) + f, _ := setupFrontend(t, cfg, func(f *Frontend, msg *schedulerpb.FrontendToScheduler) *schedulerpb.SchedulerToFrontend { // We cannot call QueryResult directly, as Frontend is not yet waiting for the response. // It first needs to be told that enqueuing has succeeded. go sendResponseWithDelay(f, delayResponse, "test", msg.QueryID, &httpgrpc.HTTPResponse{ diff --git a/pkg/querier/http.go b/pkg/querier/http.go index 1ecde15626..dc29c2f61e 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -69,7 +69,7 @@ func NewQuerierAPI(cfg Config, querier Querier, limits Limits, logger log.Logger // RangeQueryHandler is a http.HandlerFunc for range queries and legacy log queries func (q *QuerierAPI) RangeQueryHandler(ctx context.Context, req *queryrange.LokiRequest) (logqlmodel.Result, error) { - if err := q.validateMaxEntriesLimits(ctx, req.Query, req.Limit); err != nil { + if err := q.validateMaxEntriesLimits(ctx, req.Plan.AST, req.Limit); err != nil { return logqlmodel.Result{}, err } @@ -84,7 +84,7 @@ func (q *QuerierAPI) RangeQueryHandler(ctx context.Context, req *queryrange.Loki // InstantQueryHandler is a http.HandlerFunc for instant queries. func (q *QuerierAPI) InstantQueryHandler(ctx context.Context, req *queryrange.LokiInstantRequest) (logqlmodel.Result, error) { - if err := q.validateMaxEntriesLimits(ctx, req.Query, req.Limit); err != nil { + if err := q.validateMaxEntriesLimits(ctx, req.Plan.AST, req.Limit); err != nil { return logqlmodel.Result{}, err } @@ -343,17 +343,12 @@ func (q *QuerierAPI) VolumeHandler(ctx context.Context, req *logproto.VolumeRequ return resp, nil } -func (q *QuerierAPI) validateMaxEntriesLimits(ctx context.Context, query string, limit uint32) error { +func (q *QuerierAPI) validateMaxEntriesLimits(ctx context.Context, expr syntax.Expr, limit uint32) error { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { return httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - expr, err := syntax.ParseExpr(query) - if err != nil { - return err - } - // entry limit does not apply to metric queries. if _, ok := expr.(syntax.SampleExpr); ok { return nil diff --git a/pkg/querier/http_test.go b/pkg/querier/http_test.go index 5b121ad891..3e7eb5494a 100644 --- a/pkg/querier/http_test.go +++ b/pkg/querier/http_test.go @@ -31,7 +31,14 @@ func TestTailHandler(t *testing.T) { api := NewQuerierAPI(mockQuerierConfig(), nil, limits, log.NewNopLogger()) - req, err := http.NewRequest("GET", "/", nil) + req, err := http.NewRequest("GET", `/`, nil) + require.NoError(t, err) + q := req.URL.Query() + q.Add("query", `{app="loki"}`) + req.URL.RawQuery = q.Encode() + err = req.ParseForm() + require.NoError(t, err) + ctx := user.InjectOrgID(req.Context(), "1|2") req = req.WithContext(ctx) require.NoError(t, err) diff --git a/pkg/querier/multi_tenant_querier.go b/pkg/querier/multi_tenant_querier.go index f4881df48a..1727b4d7d2 100644 --- a/pkg/querier/multi_tenant_querier.go +++ b/pkg/querier/multi_tenant_querier.go @@ -2,7 +2,9 @@ package querier import ( "context" + "fmt" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/storage/stores/index/seriesvolume" "github.com/go-kit/log" @@ -53,6 +55,14 @@ func (q *MultiTenantQuerier) SelectLogs(ctx context.Context, params logql.Select matchedTenants, filteredMatchers := filterValuesByMatchers(defaultTenantLabel, tenantIDs, selector.Matchers()...) params.Selector = replaceMatchers(selector, filteredMatchers).String() + parsed, err := syntax.ParseLogSelector(params.Selector, true) + if err != nil { + return nil, fmt.Errorf("log selector is invalid after matcher update: %w", err) + } + params.Plan = &plan.QueryPlan{ + AST: parsed, + } + iters := make([]iter.EntryIterator, len(matchedTenants)) i := 0 for id := range matchedTenants { diff --git a/pkg/querier/multi_tenant_querier_test.go b/pkg/querier/multi_tenant_querier_test.go index 0a74fe9576..b503f59e31 100644 --- a/pkg/querier/multi_tenant_querier_test.go +++ b/pkg/querier/multi_tenant_querier_test.go @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/plan" ) func TestMultiTenantQuerier_SelectLogs(t *testing.T) { @@ -90,6 +91,9 @@ func TestMultiTenantQuerier_SelectLogs(t *testing.T) { Shards: nil, Start: time.Unix(0, 1), End: time.Unix(0, time.Now().UnixNano()), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.selector), + }, }} iter, err := multiTenantQuerier.SelectLogs(ctx, params) require.NoError(t, err) @@ -161,6 +165,9 @@ func TestMultiTenantQuerier_SelectSamples(t *testing.T) { ctx := user.InjectOrgID(context.Background(), tc.orgID) params := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ Selector: tc.selector, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.selector), + }, }} iter, err := multiTenantQuerier.SelectSamples(ctx, params) require.NoError(t, err) @@ -191,6 +198,9 @@ func TestMultiTenantQuerier_TenantFilter(t *testing.T) { t.Run(tc.selector, func(t *testing.T) { params := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ Selector: tc.selector, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.selector), + }, }} _, updatedSelector, err := removeTenantSelector(params, []string{}) require.NoError(t, err) diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 964d92d58c..003354d408 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -29,6 +29,7 @@ import ( "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/syntax" querier_limits "github.com/grafana/loki/pkg/querier/limits" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/storage" "github.com/grafana/loki/pkg/storage/stores/index/stats" listutil "github.com/grafana/loki/pkg/util" @@ -443,6 +444,16 @@ func (q *SingleTenantQuerier) Tail(ctx context.Context, req *logproto.TailReques return nil, err } + if req.Plan == nil { + parsed, err := syntax.ParseExpr(req.Query) + if err != nil { + return nil, err + } + req.Plan = &plan.QueryPlan{ + AST: parsed, + } + } + deletes, err := q.deletesForUser(ctx, req.Start, time.Now()) if err != nil { level.Error(spanlogger.FromContext(ctx)).Log("msg", "failed loading deletes for user", "err", err) @@ -456,6 +467,7 @@ func (q *SingleTenantQuerier) Tail(ctx context.Context, req *logproto.TailReques Limit: req.Limit, Direction: logproto.BACKWARD, Deletes: deletes, + Plan: req.Plan, }, } @@ -629,6 +641,15 @@ func (q *SingleTenantQuerier) seriesForMatchers( // seriesForMatcher fetches series from the store for a given matcher func (q *SingleTenantQuerier) seriesForMatcher(ctx context.Context, from, through time.Time, matcher string, shards []string) ([]logproto.SeriesIdentifier, error) { + var parsed syntax.Expr + var err error + if matcher != "" { + parsed, err = syntax.ParseExpr(matcher) + if err != nil { + return nil, err + } + } + ids, err := q.store.SelectSeries(ctx, logql.SelectLogParams{ QueryRequest: &logproto.QueryRequest{ Selector: matcher, @@ -637,6 +658,9 @@ func (q *SingleTenantQuerier) seriesForMatcher(ctx context.Context, from, throug End: through, Direction: logproto.FORWARD, Shards: shards, + Plan: &plan.QueryPlan{ + AST: parsed, + }, }, }) if err != nil { diff --git a/pkg/querier/querier_test.go b/pkg/querier/querier_test.go index 4c8ee491cd..a7dd3cb792 100644 --- a/pkg/querier/querier_test.go +++ b/pkg/querier/querier_test.go @@ -23,6 +23,8 @@ import ( "github.com/grafana/loki/pkg/ingester/client" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/storage" "github.com/grafana/loki/pkg/util/constants" "github.com/grafana/loki/pkg/validation" @@ -84,10 +86,13 @@ func TestQuerier_Label_QueryTimeoutConfigFlag(t *testing.T) { func TestQuerier_Tail_QueryTimeoutConfigFlag(t *testing.T) { request := logproto.TailRequest{ - Query: "{type=\"test\"}", + Query: `{type="test"}`, DelayFor: 0, Limit: 10, Start: time.Now(), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{type="test"}`), + }, } store := newStoreMock() @@ -168,11 +173,14 @@ func defaultLimitsTestConfig() validation.Limits { func TestQuerier_validateQueryRequest(t *testing.T) { request := logproto.QueryRequest{ - Selector: "{type=\"test\", fail=\"yes\"} |= \"foo\"", + Selector: `{type="test", fail="yes"} |= "foo"`, Limit: 10, Start: time.Now().Add(-1 * time.Minute), End: time.Now(), Direction: logproto.FORWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{type="test", fail="yes"} |= "foo"`), + }, } store := newStoreMock() @@ -205,7 +213,10 @@ func TestQuerier_validateQueryRequest(t *testing.T) { _, err = q.SelectLogs(ctx, logql.SelectLogParams{QueryRequest: &request}) require.Equal(t, httpgrpc.Errorf(http.StatusBadRequest, "max streams matchers per query exceeded, matchers-count > limit (2 > 1)"), err) - request.Selector = "{type=\"test\"}" + request.Selector = `{type="test"}` + request.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{type="test"}`), + } _, err = q.SelectLogs(ctx, logql.SelectLogParams{QueryRequest: &request}) require.NoError(t, err) @@ -395,6 +406,9 @@ func TestQuerier_IngesterMaxQueryLookback(t *testing.T) { Start: tc.end.Add(-6 * time.Hour), End: tc.end, Direction: logproto.FORWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"}`), + }, } queryClient := newQueryClientMock() @@ -442,6 +456,9 @@ func TestQuerier_concurrentTailLimits(t *testing.T) { DelayFor: 0, Limit: 10, Start: time.Now(), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr("{type=\"test\"}"), + }, } t.Parallel() @@ -879,11 +896,14 @@ func TestQuerier_RequestingIngesters(t *testing.T) { do: func(querier *SingleTenantQuerier, start, end time.Time) error { _, err := querier.SelectLogs(ctx, logql.SelectLogParams{ QueryRequest: &logproto.QueryRequest{ - Selector: "{type=\"test\", fail=\"yes\"} |= \"foo\"", + Selector: `{type="test", fail="yes"} |= "foo"`, Limit: 10, Start: start, End: end, Direction: logproto.FORWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{type="test", fail="yes"} |= "foo"`), + }, }, }) @@ -895,9 +915,12 @@ func TestQuerier_RequestingIngesters(t *testing.T) { do: func(querier *SingleTenantQuerier, start, end time.Time) error { _, err := querier.SelectSamples(ctx, logql.SelectSampleParams{ SampleQueryRequest: &logproto.SampleQueryRequest{ - Selector: "count_over_time({foo=\"bar\"}[5m])", + Selector: `count_over_time({foo="bar"}[5m])`, Start: start, End: end, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`count_over_time({foo="bar"}[5m])`), + }, }, }) return err @@ -1204,6 +1227,9 @@ func TestQuerier_SelectLogWithDeletes(t *testing.T) { Start: time.Unix(0, 300000000), End: time.Unix(0, 600000000), Direction: logproto.FORWARD, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{type="test"} |= "foo"`), + }, } _, err = q.SelectLogs(ctx, logql.SelectLogParams{QueryRequest: &request}) @@ -1220,6 +1246,9 @@ func TestQuerier_SelectLogWithDeletes(t *testing.T) { {Selector: "2", Start: 400000000, End: 500000000}, {Selector: "3", Start: 500000000, End: 700000000}, }, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(request.Selector), + }, } require.Contains(t, store.Calls[0].Arguments, logql.SelectLogParams{QueryRequest: expectedRequest}) @@ -1264,6 +1293,9 @@ func TestQuerier_SelectSamplesWithDeletes(t *testing.T) { Selector: `count_over_time({foo="bar"}[5m])`, Start: time.Unix(0, 300000000), End: time.Unix(0, 600000000), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`count_over_time({foo="bar"}[5m])`), + }, } _, err = q.SelectSamples(ctx, logql.SelectSampleParams{SampleQueryRequest: &request}) @@ -1279,6 +1311,9 @@ func TestQuerier_SelectSamplesWithDeletes(t *testing.T) { {Selector: "2", Start: 400000000, End: 500000000}, {Selector: "3", Start: 500000000, End: 700000000}, }, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(request.Selector), + }, }, } diff --git a/pkg/querier/queryrange/limits.go b/pkg/querier/queryrange/limits.go index a544dd3d24..673c995a60 100644 --- a/pkg/querier/queryrange/limits.go +++ b/pkg/querier/queryrange/limits.go @@ -304,7 +304,7 @@ func (q *querySizeLimiter) getBytesReadForRequest(ctx context.Context, r queryra } func (q *querySizeLimiter) getSchemaCfg(r queryrangebase.Request) (config.PeriodConfig, error) { - maxRVDuration, maxOffset, err := maxRangeVectorAndOffsetDuration(r.GetQuery()) + maxRVDuration, maxOffset, err := maxRangeVectorAndOffsetDurationFromQueryString(r.GetQuery()) if err != nil { return config.PeriodConfig{}, errors.New("failed to get range-vector and offset duration: " + err.Error()) } diff --git a/pkg/querier/queryrange/limits_test.go b/pkg/querier/queryrange/limits_test.go index b4bff9f96d..efc9b030f7 100644 --- a/pkg/querier/queryrange/limits_test.go +++ b/pkg/querier/queryrange/limits_test.go @@ -17,7 +17,9 @@ import ( "gopkg.in/yaml.v2" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" + "github.com/grafana/loki/pkg/querier/plan" base "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/util/constants" @@ -72,6 +74,9 @@ func Test_seriesLimiter(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate({app="foo"} |= "foo"[1m])`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -241,6 +246,9 @@ func Test_MaxQueryLookBack(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"} |= "foo"`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -589,6 +597,9 @@ func Test_MaxQuerySize(t *testing.T) { EndTs: tc.queryEnd, Direction: logproto.FORWARD, Path: "/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.query), + }, } ctx := user.InjectOrgID(context.Background(), "foo") diff --git a/pkg/querier/queryrange/marshal.go b/pkg/querier/queryrange/marshal.go index ef782d977b..5b227a2efa 100644 --- a/pkg/querier/queryrange/marshal.go +++ b/pkg/querier/queryrange/marshal.go @@ -119,7 +119,7 @@ func ResultToResponse(result logqlmodel.Result, params logql.Params) (queryrange case sketch.TopKMatrix: sk, err := data.ToProto() return &TopKSketchesResponse{Response: sk}, err - case sketch.QuantileSketchMatrix: + case logql.ProbabilisticQuantileMatrix: return &QuantileSketchResponse{Response: data.ToProto()}, nil } @@ -172,7 +172,7 @@ func ResponseToResult(resp queryrangebase.Response) (logqlmodel.Result, error) { Headers: resp.GetHeaders(), }, nil case *QuantileSketchResponse: - matrix, err := sketch.QuantileSketchMatrixFromProto(r.Response) + matrix, err := logql.ProbabilisticQuantileMatrixFromProto(r.Response) if err != nil { return logqlmodel.Result{}, fmt.Errorf("cannot decode quantile sketch: %w", err) } diff --git a/pkg/querier/queryrange/marshal_test.go b/pkg/querier/queryrange/marshal_test.go index e83ea607af..00dcaa2c6b 100644 --- a/pkg/querier/queryrange/marshal_test.go +++ b/pkg/querier/queryrange/marshal_test.go @@ -7,6 +7,8 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/loghttp" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" ) @@ -32,6 +34,18 @@ func TestResultToResponse(t *testing.T) { }, }, }, + { + name: "empty probabilistic quantile matrix", + result: logqlmodel.Result{ + Data: logql.ProbabilisticQuantileMatrix([]logql.ProbabilisticQuantileVector{}), + }, + response: &QuantileSketchResponse{ + Response: &logproto.QuantileSketchMatrix{ + Values: []*logproto.QuantileSketchVector{}, + }, + Headers: []queryrangebase.PrometheusResponseHeader(nil), + }, + }, } for _, tt := range tests { diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index c2cce1dc51..cbc541a044 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -4,6 +4,7 @@ package queryrange import ( + bytes "bytes" fmt "fmt" rpc "github.com/gogo/googleapis/google/rpc" _ "github.com/gogo/protobuf/gogoproto" @@ -232,6 +233,49 @@ func (m *LokiInstantRequest) GetShards() []string { return nil } +type Plan struct { + Raw []byte `protobuf:"bytes,1,opt,name=raw,proto3" json:"raw,omitempty"` +} + +func (m *Plan) Reset() { *m = Plan{} } +func (*Plan) ProtoMessage() {} +func (*Plan) Descriptor() ([]byte, []int) { + return fileDescriptor_51b9d53b40d11902, []int{2} +} +func (m *Plan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Plan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Plan.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Plan) XXX_Merge(src proto.Message) { + xxx_messageInfo_Plan.Merge(m, src) +} +func (m *Plan) XXX_Size() int { + return m.Size() +} +func (m *Plan) XXX_DiscardUnknown() { + xxx_messageInfo_Plan.DiscardUnknown(m) +} + +var xxx_messageInfo_Plan proto.InternalMessageInfo + +func (m *Plan) GetRaw() []byte { + if m != nil { + return m.Raw + } + return nil +} + type LokiResponse struct { Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` Data LokiData `protobuf:"bytes,2,opt,name=Data,proto3" json:"data,omitempty"` @@ -247,7 +291,7 @@ type LokiResponse struct { func (m *LokiResponse) Reset() { *m = LokiResponse{} } func (*LokiResponse) ProtoMessage() {} func (*LokiResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{2} + return fileDescriptor_51b9d53b40d11902, []int{3} } func (m *LokiResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -343,7 +387,7 @@ type LokiSeriesRequest struct { func (m *LokiSeriesRequest) Reset() { *m = LokiSeriesRequest{} } func (*LokiSeriesRequest) ProtoMessage() {} func (*LokiSeriesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{3} + return fileDescriptor_51b9d53b40d11902, []int{4} } func (m *LokiSeriesRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -418,7 +462,7 @@ type LokiSeriesResponse struct { func (m *LokiSeriesResponse) Reset() { *m = LokiSeriesResponse{} } func (*LokiSeriesResponse) ProtoMessage() {} func (*LokiSeriesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{4} + return fileDescriptor_51b9d53b40d11902, []int{5} } func (m *LokiSeriesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -486,7 +530,7 @@ type LokiLabelNamesResponse struct { func (m *LokiLabelNamesResponse) Reset() { *m = LokiLabelNamesResponse{} } func (*LokiLabelNamesResponse) ProtoMessage() {} func (*LokiLabelNamesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{5} + return fileDescriptor_51b9d53b40d11902, []int{6} } func (m *LokiLabelNamesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -551,7 +595,7 @@ type LokiData struct { func (m *LokiData) Reset() { *m = LokiData{} } func (*LokiData) ProtoMessage() {} func (*LokiData) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{6} + return fileDescriptor_51b9d53b40d11902, []int{7} } func (m *LokiData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -596,7 +640,7 @@ type LokiPromResponse struct { func (m *LokiPromResponse) Reset() { *m = LokiPromResponse{} } func (*LokiPromResponse) ProtoMessage() {} func (*LokiPromResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{7} + return fileDescriptor_51b9d53b40d11902, []int{8} } func (m *LokiPromResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -647,7 +691,7 @@ type IndexStatsResponse struct { func (m *IndexStatsResponse) Reset() { *m = IndexStatsResponse{} } func (*IndexStatsResponse) ProtoMessage() {} func (*IndexStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{8} + return fileDescriptor_51b9d53b40d11902, []int{9} } func (m *IndexStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -684,7 +728,7 @@ type VolumeResponse struct { func (m *VolumeResponse) Reset() { *m = VolumeResponse{} } func (*VolumeResponse) ProtoMessage() {} func (*VolumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{9} + return fileDescriptor_51b9d53b40d11902, []int{10} } func (m *VolumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -721,7 +765,7 @@ type TopKSketchesResponse struct { func (m *TopKSketchesResponse) Reset() { *m = TopKSketchesResponse{} } func (*TopKSketchesResponse) ProtoMessage() {} func (*TopKSketchesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{10} + return fileDescriptor_51b9d53b40d11902, []int{11} } func (m *TopKSketchesResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -758,7 +802,7 @@ type QuantileSketchResponse struct { func (m *QuantileSketchResponse) Reset() { *m = QuantileSketchResponse{} } func (*QuantileSketchResponse) ProtoMessage() {} func (*QuantileSketchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{11} + return fileDescriptor_51b9d53b40d11902, []int{12} } func (m *QuantileSketchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -804,7 +848,7 @@ type QueryResponse struct { func (m *QueryResponse) Reset() { *m = QueryResponse{} } func (*QueryResponse) ProtoMessage() {} func (*QueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{12} + return fileDescriptor_51b9d53b40d11902, []int{13} } func (m *QueryResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -973,7 +1017,7 @@ type QueryRequest struct { func (m *QueryRequest) Reset() { *m = QueryRequest{} } func (*QueryRequest) ProtoMessage() {} func (*QueryRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_51b9d53b40d11902, []int{13} + return fileDescriptor_51b9d53b40d11902, []int{14} } func (m *QueryRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1106,6 +1150,7 @@ func (*QueryRequest) XXX_OneofWrappers() []interface{} { func init() { proto.RegisterType((*LokiRequest)(nil), "queryrange.LokiRequest") proto.RegisterType((*LokiInstantRequest)(nil), "queryrange.LokiInstantRequest") + proto.RegisterType((*Plan)(nil), "queryrange.Plan") proto.RegisterType((*LokiResponse)(nil), "queryrange.LokiResponse") proto.RegisterType((*LokiSeriesRequest)(nil), "queryrange.LokiSeriesRequest") proto.RegisterType((*LokiSeriesResponse)(nil), "queryrange.LokiSeriesResponse") @@ -1126,101 +1171,102 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 1498 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x4b, 0x6f, 0xdb, 0xc6, - 0x1a, 0x15, 0xf5, 0xb4, 0xc6, 0x8f, 0x9b, 0x3b, 0x36, 0x1c, 0x5e, 0x27, 0x97, 0x14, 0x04, 0xdc, - 0x44, 0xb7, 0x68, 0xa9, 0xc6, 0x4e, 0xf3, 0x6c, 0x8b, 0x86, 0x4d, 0x02, 0xa7, 0x4d, 0x8a, 0x84, - 0x36, 0xba, 0xe8, 0x6e, 0x2c, 0x8d, 0x25, 0x56, 0x7c, 0x99, 0x33, 0x32, 0xe2, 0x5d, 0x7f, 0x40, - 0x0b, 0xe4, 0x07, 0x74, 0x5d, 0x14, 0x68, 0x50, 0xa0, 0x8b, 0x6e, 0xba, 0xec, 0x2a, 0xcb, 0x2c, - 0x03, 0x01, 0x65, 0x1b, 0xa5, 0x8b, 0xc2, 0xab, 0xfc, 0x84, 0x62, 0x1e, 0xa4, 0x48, 0x49, 0x49, - 0xe4, 0xb4, 0x8b, 0x04, 0xe8, 0x46, 0x9a, 0x19, 0x7e, 0x87, 0x1c, 0x9e, 0x73, 0xbe, 0x6f, 0x66, - 0x08, 0x4e, 0x07, 0xbd, 0x4e, 0x73, 0xaf, 0x8f, 0x43, 0x1b, 0x87, 0xfc, 0xff, 0x20, 0x44, 0x5e, - 0x07, 0xa7, 0x9a, 0x46, 0x10, 0xfa, 0xd4, 0x87, 0x60, 0x34, 0xb2, 0xb6, 0xde, 0xb1, 0x69, 0xb7, - 0xbf, 0x63, 0xb4, 0x7c, 0xb7, 0xd9, 0xf1, 0x3b, 0x7e, 0xb3, 0xe3, 0xfb, 0x1d, 0x07, 0xa3, 0xc0, - 0x26, 0xb2, 0xd9, 0x0c, 0x83, 0x56, 0x93, 0x50, 0x44, 0xfb, 0x44, 0xe0, 0xd7, 0x56, 0x58, 0x20, - 0x6f, 0x72, 0x88, 0x1c, 0xd5, 0x65, 0x38, 0xef, 0xed, 0xf4, 0x77, 0x9b, 0xd4, 0x76, 0x31, 0xa1, - 0xc8, 0x0d, 0x64, 0xc0, 0x09, 0x36, 0x3f, 0xc7, 0xef, 0x08, 0x64, 0xdc, 0x90, 0x17, 0xff, 0x93, - 0xb9, 0x48, 0x7a, 0x98, 0xb6, 0xba, 0xf2, 0x52, 0x4d, 0x5e, 0xda, 0x73, 0x5c, 0xbf, 0x8d, 0x1d, - 0x3e, 0x17, 0x22, 0x7e, 0x65, 0xc4, 0x32, 0x8b, 0x08, 0xfa, 0xa4, 0xcb, 0x7f, 0xe4, 0xe0, 0x87, - 0x2f, 0xa4, 0x63, 0x07, 0x11, 0xdc, 0x6c, 0xe3, 0x5d, 0xdb, 0xb3, 0xa9, 0xed, 0x7b, 0x24, 0xdd, - 0x96, 0x37, 0x39, 0x37, 0xdb, 0x4d, 0xc6, 0x29, 0xae, 0x7f, 0x5d, 0x00, 0xf3, 0x37, 0xfd, 0x9e, - 0x6d, 0xe1, 0xbd, 0x3e, 0x26, 0x14, 0xae, 0x80, 0x12, 0x8f, 0x51, 0x95, 0x9a, 0xd2, 0xa8, 0x5a, - 0xa2, 0xc3, 0x46, 0x1d, 0xdb, 0xb5, 0xa9, 0x9a, 0xaf, 0x29, 0x8d, 0x45, 0x4b, 0x74, 0x20, 0x04, - 0x45, 0x42, 0x71, 0xa0, 0x16, 0x6a, 0x4a, 0xa3, 0x60, 0xf1, 0x36, 0x5c, 0x03, 0x73, 0xb6, 0x47, - 0x71, 0xb8, 0x8f, 0x1c, 0xb5, 0xca, 0xc7, 0x93, 0x3e, 0x7c, 0x1f, 0x54, 0x08, 0x45, 0x21, 0xdd, - 0x26, 0x6a, 0xb1, 0xa6, 0x34, 0xe6, 0xd7, 0xd7, 0x0c, 0x21, 0x85, 0x11, 0x4b, 0x61, 0x6c, 0xc7, - 0x52, 0x98, 0x73, 0x0f, 0x22, 0x3d, 0x77, 0xef, 0x57, 0x5d, 0xb1, 0x62, 0x10, 0xbc, 0x04, 0x4a, - 0xd8, 0x6b, 0x6f, 0x13, 0xb5, 0x74, 0x04, 0xb4, 0x80, 0xc0, 0x33, 0xa0, 0xda, 0xb6, 0x43, 0xdc, - 0x62, 0x9c, 0xa9, 0xe5, 0x9a, 0xd2, 0x58, 0x5a, 0x5f, 0x36, 0x12, 0x69, 0xaf, 0xc6, 0x97, 0xac, - 0x51, 0x14, 0x7b, 0xbd, 0x00, 0xd1, 0xae, 0x5a, 0xe1, 0x4c, 0xf0, 0x36, 0xac, 0x83, 0x32, 0xe9, - 0xa2, 0xb0, 0x4d, 0xd4, 0xb9, 0x5a, 0xa1, 0x51, 0x35, 0xc1, 0x61, 0xa4, 0xcb, 0x11, 0x4b, 0xfe, - 0xc3, 0x8f, 0x40, 0x31, 0x70, 0x90, 0xa7, 0x82, 0x9a, 0xd2, 0x58, 0x30, 0xcf, 0x0d, 0x22, 0x3d, - 0xe3, 0xdd, 0x10, 0xed, 0x22, 0x0f, 0x35, 0x1d, 0xbf, 0x67, 0x37, 0xd3, 0xa2, 0x31, 0x8c, 0x71, - 0x87, 0xd1, 0x7d, 0xdb, 0x41, 0x9e, 0xc5, 0xef, 0x51, 0xff, 0x31, 0x0f, 0x20, 0x93, 0xe7, 0x86, - 0x47, 0x28, 0xf2, 0xe8, 0xcb, 0xa8, 0xf4, 0x2e, 0x28, 0x33, 0x83, 0x6f, 0x13, 0xae, 0xd3, 0xac, - 0xb4, 0x49, 0x4c, 0x96, 0xb7, 0xe2, 0x91, 0x78, 0x2b, 0x4d, 0xe5, 0xad, 0xfc, 0x42, 0xde, 0x2a, - 0x7f, 0x03, 0x6f, 0xdf, 0x15, 0xc1, 0x82, 0xb0, 0x35, 0x09, 0x7c, 0x8f, 0x60, 0x36, 0x81, 0x2d, - 0x5e, 0x1a, 0x04, 0x65, 0x72, 0x02, 0x7c, 0xc4, 0x92, 0x57, 0xe0, 0x07, 0xa0, 0x78, 0x15, 0x51, - 0xc4, 0xe9, 0x9b, 0x5f, 0x5f, 0x31, 0x52, 0xc9, 0xc2, 0xee, 0xc5, 0xae, 0x99, 0xab, 0x8c, 0xa1, - 0xc3, 0x48, 0x5f, 0x6a, 0x23, 0x8a, 0xde, 0xf4, 0x5d, 0x9b, 0x62, 0x37, 0xa0, 0x07, 0x16, 0x47, - 0xc2, 0x77, 0x40, 0xf5, 0x5a, 0x18, 0xfa, 0xe1, 0xf6, 0x41, 0x80, 0x39, 0xdd, 0x55, 0xf3, 0xf8, - 0x61, 0xa4, 0x2f, 0xe3, 0x78, 0x30, 0x85, 0x18, 0x45, 0xc2, 0xff, 0x83, 0x12, 0xef, 0x70, 0x82, - 0xab, 0xe6, 0xf2, 0x61, 0xa4, 0xff, 0x8b, 0x43, 0x52, 0xe1, 0x22, 0x22, 0xab, 0x47, 0x69, 0x26, - 0x3d, 0x12, 0x5b, 0x94, 0xd3, 0xb6, 0x50, 0x41, 0x65, 0x1f, 0x87, 0x84, 0xdd, 0xa6, 0xc2, 0xc7, - 0xe3, 0x2e, 0xbc, 0x02, 0x00, 0x23, 0xc6, 0x26, 0xd4, 0x6e, 0x31, 0x9f, 0x33, 0x32, 0x16, 0x0d, - 0x51, 0xc6, 0x2c, 0x4c, 0xfa, 0x0e, 0x35, 0xa1, 0x64, 0x21, 0x15, 0x68, 0xa5, 0xda, 0xf0, 0xbe, - 0x02, 0x2a, 0x9b, 0x18, 0xb5, 0x71, 0x48, 0xd4, 0x6a, 0xad, 0xd0, 0x98, 0x5f, 0xff, 0x9f, 0x91, - 0xae, 0x59, 0xb7, 0x43, 0xdf, 0xc5, 0xb4, 0x8b, 0xfb, 0x24, 0x16, 0x48, 0x44, 0x9b, 0xbd, 0x41, - 0xa4, 0xef, 0xcc, 0xa2, 0xfa, 0x4c, 0x75, 0xf2, 0x99, 0xcf, 0x39, 0x8c, 0x74, 0xe5, 0x2d, 0x2b, - 0x9e, 0x62, 0xfd, 0x17, 0x05, 0xfc, 0x9b, 0x29, 0xbc, 0xc5, 0xee, 0x4d, 0x52, 0x49, 0xe6, 0x22, - 0xda, 0xea, 0xaa, 0x0a, 0xb3, 0xac, 0x25, 0x3a, 0xe9, 0x22, 0x96, 0xff, 0x4b, 0x45, 0xac, 0x70, - 0xf4, 0x22, 0x16, 0x67, 0x56, 0x71, 0x6a, 0x66, 0x95, 0x9e, 0x95, 0x59, 0xf5, 0x2f, 0x0b, 0xa2, - 0x8a, 0xc4, 0xef, 0x77, 0x84, 0x9c, 0xb8, 0x9e, 0xe4, 0x44, 0x81, 0xcf, 0x36, 0xb1, 0x9a, 0xb8, - 0xd7, 0x8d, 0x36, 0xf6, 0xa8, 0xbd, 0x6b, 0xe3, 0xf0, 0x05, 0x99, 0x91, 0xb2, 0x5b, 0x21, 0x6b, - 0xb7, 0xb4, 0x57, 0x8a, 0xaf, 0xbc, 0x57, 0xc6, 0xb2, 0xa3, 0xf4, 0x12, 0xd9, 0x51, 0x7f, 0x9a, - 0x07, 0xab, 0x4c, 0x8e, 0x9b, 0x68, 0x07, 0x3b, 0x9f, 0x20, 0xf7, 0x88, 0x92, 0x9c, 0x4a, 0x49, - 0x52, 0x35, 0xe1, 0x3f, 0x94, 0xcf, 0x40, 0xf9, 0x37, 0x0a, 0x98, 0x8b, 0x6b, 0x38, 0x34, 0x00, - 0x10, 0x30, 0x5e, 0xa6, 0x05, 0xd1, 0x4b, 0x0c, 0x1c, 0x26, 0xa3, 0x56, 0x2a, 0x02, 0x7e, 0x0e, - 0xca, 0xa2, 0x27, 0xb3, 0xe0, 0x78, 0x2a, 0x0b, 0x68, 0x88, 0x91, 0x7b, 0xa5, 0x8d, 0x02, 0x8a, - 0x43, 0xf3, 0x22, 0x9b, 0xc5, 0x20, 0xd2, 0x4f, 0x3f, 0x8f, 0x22, 0xbe, 0xf3, 0x13, 0x38, 0x26, - 0xae, 0x78, 0xa6, 0x25, 0x9f, 0x50, 0xff, 0x4a, 0x01, 0xc7, 0xd8, 0x44, 0x19, 0x35, 0x89, 0x2b, - 0xae, 0x82, 0xb9, 0x50, 0xb6, 0xf9, 0x74, 0xe7, 0xd7, 0xeb, 0x46, 0x96, 0xd6, 0x29, 0x54, 0x9a, - 0xc5, 0x07, 0x91, 0xae, 0x58, 0x09, 0x12, 0x6e, 0x64, 0x68, 0xcc, 0x4f, 0xa3, 0x91, 0x41, 0x72, - 0x19, 0xe2, 0x7e, 0xca, 0x03, 0x78, 0xc3, 0x6b, 0xe3, 0xbb, 0xcc, 0x7c, 0x23, 0x9f, 0xf6, 0x27, - 0x66, 0x74, 0x72, 0x44, 0xca, 0x64, 0xbc, 0x79, 0x79, 0x10, 0xe9, 0xe7, 0x9f, 0xc7, 0xca, 0x73, - 0xc0, 0xa9, 0x57, 0x48, 0x1b, 0x37, 0xff, 0xea, 0xaf, 0x2b, 0xdf, 0xe7, 0xc1, 0xd2, 0xa7, 0xbe, - 0xd3, 0x77, 0x71, 0x42, 0x9c, 0x3b, 0x41, 0x9c, 0x3a, 0x22, 0x2e, 0x1b, 0x6b, 0x9e, 0x1f, 0x44, - 0xfa, 0xc6, 0x4c, 0xa4, 0x65, 0x81, 0xaf, 0x2f, 0x61, 0xf7, 0xf3, 0x60, 0x65, 0xdb, 0x0f, 0x3e, - 0xde, 0xe2, 0xc7, 0xaa, 0x54, 0x5d, 0xc4, 0x13, 0xb4, 0xad, 0x8c, 0x68, 0x63, 0x88, 0x5b, 0x88, - 0x86, 0xf6, 0x5d, 0x73, 0x63, 0x10, 0xe9, 0xcd, 0x99, 0x28, 0x1b, 0x81, 0x5e, 0x5f, 0xba, 0x7e, - 0xce, 0x83, 0xd5, 0x3b, 0x7d, 0xe4, 0x51, 0xdb, 0xc1, 0x82, 0xb2, 0x84, 0xb0, 0x83, 0x09, 0xc2, - 0xb4, 0x11, 0x61, 0x59, 0x8c, 0xa4, 0xee, 0xbd, 0x41, 0xa4, 0x5f, 0x9c, 0x89, 0xba, 0x69, 0xf0, - 0xd7, 0x97, 0xc4, 0x1f, 0x8a, 0x60, 0x91, 0x1f, 0x1f, 0x12, 0xee, 0xde, 0x00, 0x72, 0xc9, 0x95, - 0xcc, 0xc1, 0x78, 0x8f, 0x16, 0x06, 0x2d, 0x63, 0x4b, 0x2e, 0xc6, 0x22, 0x02, 0x5e, 0x00, 0x65, - 0xc2, 0x77, 0x42, 0xb2, 0xa0, 0x6a, 0xe3, 0xa7, 0x86, 0xec, 0x9e, 0x6b, 0x33, 0x67, 0xc9, 0x78, - 0x76, 0x2e, 0x73, 0xd8, 0x06, 0x20, 0xde, 0x09, 0xd6, 0xc7, 0x91, 0x93, 0xdb, 0x03, 0x86, 0x16, - 0x18, 0x78, 0x0e, 0x94, 0x78, 0xe5, 0x96, 0x27, 0xe9, 0xcc, 0x63, 0x27, 0x4b, 0xe8, 0x66, 0xce, - 0x12, 0xe1, 0x70, 0x1d, 0x14, 0x83, 0xd0, 0x77, 0xe5, 0x2a, 0x7a, 0x72, 0xfc, 0x99, 0xe9, 0x65, - 0x67, 0x33, 0x67, 0xf1, 0x58, 0x78, 0x96, 0x6d, 0x79, 0xd9, 0x7a, 0x45, 0xf8, 0x11, 0x82, 0x95, - 0xac, 0x31, 0x58, 0x0a, 0x12, 0x87, 0xc2, 0xb3, 0xa0, 0xbc, 0xcf, 0xcb, 0x12, 0x3f, 0x5f, 0xb0, - 0xbd, 0x63, 0x0a, 0x94, 0x2d, 0x58, 0xec, 0xbd, 0x44, 0x2c, 0xbc, 0x0e, 0x16, 0xa8, 0x1f, 0xf4, - 0xe2, 0x02, 0x20, 0x8f, 0x1f, 0xb5, 0x34, 0x76, 0x5a, 0x81, 0xd8, 0xcc, 0x59, 0x19, 0x1c, 0xbc, - 0x0d, 0x8e, 0xed, 0x65, 0x6c, 0x8a, 0x09, 0xff, 0x1e, 0x31, 0xc6, 0xf3, 0xf4, 0xec, 0xd9, 0xcc, - 0x59, 0x13, 0x68, 0x13, 0x8c, 0x32, 0xaa, 0xfe, 0x7b, 0x01, 0x2c, 0x48, 0xcf, 0x88, 0xb3, 0xc2, - 0xf9, 0xc4, 0x06, 0xc2, 0x32, 0xff, 0x7d, 0x96, 0x0d, 0x78, 0x78, 0xca, 0x05, 0x6f, 0x27, 0x2e, - 0x10, 0xfe, 0x59, 0x1d, 0x65, 0x29, 0xd7, 0x3f, 0x85, 0x90, 0xca, 0x6f, 0xc4, 0xca, 0x0b, 0xdb, - 0x9c, 0x98, 0xbe, 0xee, 0xc6, 0x28, 0x29, 0xfb, 0x25, 0x50, 0xb1, 0xc5, 0x27, 0x84, 0x69, 0x86, - 0x99, 0xfc, 0xc2, 0xc0, 0x84, 0x94, 0x00, 0xb8, 0x31, 0x92, 0x5f, 0xb8, 0xe6, 0xf8, 0xa4, 0xfc, - 0x09, 0x28, 0x56, 0xff, 0x4c, 0xa2, 0x7e, 0x59, 0x62, 0x26, 0x16, 0xab, 0xe4, 0xc5, 0xa4, 0xf4, - 0x9b, 0x60, 0xce, 0xc5, 0x14, 0xb1, 0xbd, 0xac, 0x5a, 0xe1, 0x75, 0xe3, 0x54, 0x56, 0xaa, 0x11, - 0xdf, 0xc6, 0x2d, 0x19, 0x78, 0xcd, 0xa3, 0xe1, 0x81, 0xdc, 0xb6, 0x24, 0xe8, 0xb5, 0xcb, 0x60, - 0x31, 0x13, 0x00, 0x8f, 0x81, 0x42, 0x0f, 0xc7, 0x5f, 0x4b, 0x58, 0x93, 0x1d, 0xee, 0xf6, 0x91, - 0xd3, 0xc7, 0x9c, 0xf6, 0xaa, 0x25, 0x3a, 0x97, 0xf2, 0x17, 0x14, 0xb3, 0x0a, 0x2a, 0xa1, 0x78, - 0x8a, 0xd9, 0x7e, 0xf8, 0x58, 0xcb, 0x3d, 0x7a, 0xac, 0xe5, 0x9e, 0x3e, 0xd6, 0x94, 0x2f, 0x86, - 0x9a, 0xf2, 0xed, 0x50, 0x53, 0x1e, 0x0c, 0x35, 0xe5, 0xe1, 0x50, 0x53, 0x7e, 0x1b, 0x6a, 0xca, - 0x1f, 0x43, 0x2d, 0xf7, 0x74, 0xa8, 0x29, 0xf7, 0x9e, 0x68, 0xb9, 0x87, 0x4f, 0xb4, 0xdc, 0xa3, - 0x27, 0x5a, 0xee, 0x33, 0xe3, 0x68, 0x25, 0x6c, 0xa7, 0xcc, 0x69, 0xd9, 0xf8, 0x33, 0x00, 0x00, - 0xff, 0xff, 0x33, 0xb4, 0xee, 0x07, 0x17, 0x15, 0x00, 0x00, + // 1514 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x5b, 0x6f, 0x1b, 0x45, + 0x1b, 0xf6, 0xfa, 0x18, 0x4f, 0x0e, 0x5f, 0xbe, 0x49, 0x94, 0xee, 0x97, 0xf6, 0xdb, 0xb5, 0x2c, + 0xd1, 0x06, 0x04, 0x6b, 0x9a, 0x94, 0x1e, 0x01, 0xd1, 0xa5, 0xad, 0x52, 0xd1, 0xa2, 0x76, 0x13, + 0x71, 0x81, 0xb8, 0x99, 0xd8, 0x13, 0x7b, 0xf1, 0x9e, 0xb2, 0x33, 0x0e, 0xcd, 0x1d, 0x3f, 0x00, + 0xa4, 0xfe, 0x0a, 0x84, 0x44, 0x55, 0x89, 0x5b, 0x2e, 0xb9, 0xa1, 0x97, 0xbd, 0xac, 0x2c, 0xb1, + 0x50, 0x97, 0x0b, 0x94, 0xab, 0xfe, 0x04, 0x34, 0x87, 0x5d, 0xef, 0xda, 0x6e, 0xeb, 0x14, 0x21, + 0xb5, 0x12, 0x37, 0xf6, 0x1c, 0xde, 0x67, 0xf6, 0xdd, 0xe7, 0x79, 0xdf, 0x77, 0x66, 0x16, 0x9c, + 0x0a, 0xba, 0xed, 0xc6, 0x5e, 0x0f, 0x87, 0x36, 0x0e, 0xf9, 0xff, 0x41, 0x88, 0xbc, 0x36, 0x4e, + 0x35, 0x8d, 0x20, 0xf4, 0xa9, 0x0f, 0xc1, 0x70, 0x64, 0x75, 0xbd, 0x6d, 0xd3, 0x4e, 0x6f, 0xc7, + 0x68, 0xfa, 0x6e, 0xa3, 0xed, 0xb7, 0xfd, 0x46, 0xdb, 0xf7, 0xdb, 0x0e, 0x46, 0x81, 0x4d, 0x64, + 0xb3, 0x11, 0x06, 0xcd, 0x06, 0xa1, 0x88, 0xf6, 0x88, 0xc0, 0xaf, 0x2e, 0x33, 0x43, 0xde, 0xe4, + 0x10, 0x39, 0xaa, 0x4b, 0x73, 0xde, 0xdb, 0xe9, 0xed, 0x36, 0xa8, 0xed, 0x62, 0x42, 0x91, 0x1b, + 0x48, 0x83, 0xe3, 0xcc, 0x3f, 0xc7, 0x6f, 0x0b, 0x64, 0xdc, 0x90, 0x93, 0xff, 0xcb, 0x4c, 0x92, + 0x2e, 0xa6, 0xcd, 0x8e, 0x9c, 0xaa, 0xc9, 0xa9, 0x3d, 0xc7, 0xf5, 0x5b, 0xd8, 0xe1, 0xbe, 0x10, + 0xf1, 0x2b, 0x2d, 0x96, 0x98, 0x45, 0xd0, 0x23, 0x1d, 0xfe, 0x23, 0x07, 0x3f, 0x7e, 0x21, 0x1d, + 0x3b, 0x88, 0xe0, 0x46, 0x0b, 0xef, 0xda, 0x9e, 0x4d, 0x6d, 0xdf, 0x23, 0xe9, 0xb6, 0x5c, 0xe4, + 0xec, 0x74, 0x8b, 0x8c, 0x52, 0x5c, 0xbf, 0x5f, 0x00, 0xb3, 0x37, 0xfc, 0xae, 0x6d, 0xe1, 0xbd, + 0x1e, 0x26, 0x14, 0x2e, 0x83, 0x12, 0xb7, 0x51, 0x95, 0x9a, 0xb2, 0x56, 0xb5, 0x44, 0x87, 0x8d, + 0x3a, 0xb6, 0x6b, 0x53, 0x35, 0x5f, 0x53, 0xd6, 0xe6, 0x2d, 0xd1, 0x81, 0x10, 0x14, 0x09, 0xc5, + 0x81, 0x5a, 0xa8, 0x29, 0x6b, 0x05, 0x8b, 0xb7, 0xe1, 0x2a, 0x98, 0xb1, 0x3d, 0x8a, 0xc3, 0x7d, + 0xe4, 0xa8, 0x55, 0x3e, 0x9e, 0xf4, 0xe1, 0x87, 0xa0, 0x42, 0x28, 0x0a, 0xe9, 0x36, 0x51, 0x8b, + 0x35, 0x65, 0x6d, 0x76, 0x7d, 0xd5, 0x10, 0x52, 0x18, 0xb1, 0x14, 0xc6, 0x76, 0x2c, 0x85, 0x39, + 0xf3, 0x20, 0xd2, 0x73, 0x77, 0x7f, 0xd3, 0x15, 0x2b, 0x06, 0xc1, 0x8b, 0xa0, 0x84, 0xbd, 0xd6, + 0x36, 0x51, 0x4b, 0x47, 0x40, 0x0b, 0x08, 0x3c, 0x0d, 0xaa, 0x2d, 0x3b, 0xc4, 0x4d, 0xc6, 0x99, + 0x5a, 0xae, 0x29, 0x6b, 0x0b, 0xeb, 0x4b, 0x46, 0x22, 0xed, 0x95, 0x78, 0xca, 0x1a, 0x5a, 0xb1, + 0xd7, 0x0b, 0x10, 0xed, 0xa8, 0x15, 0xce, 0x04, 0x6f, 0xc3, 0x3a, 0x28, 0x93, 0x0e, 0x0a, 0x5b, + 0x44, 0x9d, 0xa9, 0x15, 0xd6, 0xaa, 0x26, 0x38, 0x8c, 0x74, 0x39, 0x62, 0xc9, 0x7f, 0xf8, 0x05, + 0x28, 0x06, 0x0e, 0xf2, 0x54, 0xc0, 0xbd, 0x5c, 0x34, 0x52, 0x9c, 0xdf, 0x72, 0x90, 0x67, 0x9e, + 0xed, 0x47, 0x7a, 0x26, 0x9a, 0x43, 0xb4, 0x8b, 0x3c, 0xd4, 0x70, 0xfc, 0xae, 0xdd, 0x48, 0xcb, + 0xc8, 0x56, 0x31, 0x6e, 0x33, 0x34, 0xc3, 0x59, 0x7c, 0xd5, 0xfa, 0x2f, 0x79, 0x00, 0x99, 0x60, + 0xd7, 0x3d, 0x42, 0x91, 0x47, 0x5f, 0x46, 0xb7, 0xf7, 0x41, 0x99, 0x85, 0xfc, 0x36, 0xe1, 0xca, + 0x4d, 0x4b, 0xa4, 0xc4, 0x64, 0x99, 0x2c, 0x1e, 0x89, 0xc9, 0xd2, 0x44, 0x26, 0xcb, 0x2f, 0x64, + 0xb2, 0xf2, 0x8f, 0x30, 0xa9, 0x82, 0x22, 0xeb, 0xc1, 0x45, 0x50, 0x08, 0xd1, 0x57, 0x9c, 0xb8, + 0x39, 0x8b, 0x35, 0xeb, 0x3f, 0x14, 0xc1, 0x9c, 0x48, 0x0a, 0x12, 0xf8, 0x1e, 0xc1, 0xcc, 0xd9, + 0x2d, 0x5e, 0x58, 0x04, 0xbd, 0xd2, 0x59, 0x3e, 0x62, 0xc9, 0x19, 0xf8, 0x11, 0x28, 0x5e, 0x41, + 0x14, 0x71, 0xaa, 0x67, 0xd7, 0x97, 0xd3, 0xce, 0xb2, 0xb5, 0xd8, 0x9c, 0xb9, 0xc2, 0xd8, 0x3c, + 0x8c, 0xf4, 0x85, 0x16, 0xa2, 0xe8, 0x6d, 0xdf, 0xb5, 0x29, 0x76, 0x03, 0x7a, 0x60, 0x71, 0x24, + 0x7c, 0x0f, 0x54, 0xaf, 0x86, 0xa1, 0x1f, 0x6e, 0x1f, 0x04, 0x98, 0x4b, 0x53, 0x35, 0x8f, 0x1d, + 0x46, 0xfa, 0x12, 0x8e, 0x07, 0x53, 0x88, 0xa1, 0x25, 0x7c, 0x13, 0x94, 0x78, 0x87, 0x8b, 0x51, + 0x35, 0x97, 0x0e, 0x23, 0xfd, 0x3f, 0x1c, 0x92, 0x32, 0x17, 0x16, 0x59, 0xed, 0x4a, 0x53, 0x69, + 0x97, 0x84, 0x50, 0x39, 0x1d, 0x42, 0x2a, 0xa8, 0xec, 0xe3, 0x90, 0xb0, 0x65, 0x2a, 0x7c, 0x3c, + 0xee, 0xc2, 0xcb, 0x00, 0x30, 0x62, 0x6c, 0x42, 0xed, 0x26, 0xcb, 0x12, 0x46, 0xc6, 0xbc, 0x21, + 0x8a, 0xa0, 0x85, 0x49, 0xcf, 0xa1, 0x26, 0x94, 0x2c, 0xa4, 0x0c, 0xad, 0x54, 0x1b, 0xde, 0x53, + 0x40, 0x65, 0x13, 0xa3, 0x16, 0x0e, 0x89, 0x5a, 0xad, 0x15, 0xd6, 0x66, 0xd7, 0xdf, 0x30, 0xd2, + 0x15, 0xef, 0x56, 0xe8, 0xbb, 0x98, 0x76, 0x70, 0x8f, 0xc4, 0x02, 0x09, 0x6b, 0xb3, 0xdb, 0x8f, + 0xf4, 0x9d, 0x69, 0xe2, 0x61, 0xaa, 0x2a, 0xfb, 0xcc, 0xe7, 0x1c, 0x46, 0xba, 0xf2, 0x8e, 0x15, + 0xbb, 0x58, 0xff, 0x55, 0x01, 0xff, 0x65, 0x0a, 0x6f, 0xb1, 0xb5, 0x49, 0x2a, 0x21, 0x5d, 0x44, + 0x9b, 0x1d, 0x55, 0x61, 0xe1, 0x6d, 0x89, 0x4e, 0xba, 0x04, 0xe6, 0xff, 0x56, 0x09, 0x2c, 0x1c, + 0xbd, 0x04, 0xc6, 0x59, 0x58, 0x9c, 0x98, 0x85, 0xa5, 0x67, 0x65, 0x61, 0xfd, 0x9b, 0x82, 0xa8, + 0x38, 0xf1, 0xfb, 0x1d, 0x21, 0x27, 0xae, 0x25, 0x39, 0x51, 0xe0, 0xde, 0x26, 0xa1, 0x26, 0xd6, + 0xba, 0xde, 0xc2, 0x1e, 0xb5, 0x77, 0x6d, 0x1c, 0xbe, 0x20, 0x33, 0x52, 0xe1, 0x56, 0xc8, 0x86, + 0x5b, 0x3a, 0x56, 0x8a, 0xaf, 0x7c, 0xac, 0x8c, 0x64, 0x47, 0xe9, 0x25, 0xb2, 0xa3, 0xfe, 0x34, + 0x0f, 0x56, 0x98, 0x1c, 0x37, 0xd0, 0x0e, 0x76, 0x3e, 0x45, 0xee, 0x11, 0x25, 0x39, 0x99, 0x92, + 0xa4, 0x6a, 0xc2, 0x7f, 0x29, 0x9f, 0x82, 0xf2, 0xef, 0x14, 0x30, 0x13, 0xd7, 0x70, 0x68, 0x00, + 0x20, 0x60, 0xbc, 0x4c, 0x0b, 0xa2, 0x17, 0x18, 0x38, 0x4c, 0x46, 0xad, 0x94, 0x05, 0xfc, 0x12, + 0x94, 0x45, 0x4f, 0x66, 0xc1, 0xb1, 0x54, 0x16, 0xd0, 0x10, 0x23, 0xf7, 0x72, 0x0b, 0x05, 0x14, + 0x87, 0xe6, 0x05, 0xe6, 0x45, 0x3f, 0xd2, 0x4f, 0x3d, 0x8f, 0x22, 0x7e, 0x6e, 0x14, 0x38, 0x26, + 0xae, 0x78, 0xa6, 0x25, 0x9f, 0x50, 0xff, 0x56, 0x01, 0x8b, 0xcc, 0x51, 0x46, 0x4d, 0x12, 0x15, + 0x57, 0xc0, 0x4c, 0x28, 0xdb, 0xdc, 0xdd, 0xd9, 0xf5, 0xba, 0x91, 0xa5, 0x75, 0x02, 0x95, 0x66, + 0xf1, 0x41, 0xa4, 0x2b, 0x56, 0x82, 0x84, 0x1b, 0x19, 0x1a, 0xf3, 0x93, 0x68, 0x64, 0x90, 0x5c, + 0x86, 0xb8, 0x9f, 0xf2, 0x00, 0x5e, 0xf7, 0x5a, 0xf8, 0x0e, 0x0b, 0xbe, 0x61, 0x9c, 0xf6, 0xc6, + 0x3c, 0x3a, 0x31, 0x24, 0x65, 0xdc, 0xde, 0xbc, 0xd4, 0x8f, 0xf4, 0x73, 0xcf, 0x63, 0xe5, 0x39, + 0xe0, 0xd4, 0x2b, 0xa4, 0x03, 0x37, 0xff, 0xea, 0xef, 0x2b, 0xf7, 0xf3, 0x60, 0xe1, 0x33, 0xdf, + 0xe9, 0xb9, 0x38, 0x21, 0xce, 0x1d, 0x23, 0x4e, 0x1d, 0x12, 0x97, 0xb5, 0x35, 0xcf, 0xf5, 0x23, + 0x7d, 0x63, 0x2a, 0xd2, 0xb2, 0xc0, 0xd7, 0x97, 0xb0, 0x7b, 0x79, 0xb0, 0xbc, 0xed, 0x07, 0x9f, + 0x6c, 0xf1, 0x4b, 0x59, 0xaa, 0x2e, 0xe2, 0x31, 0xda, 0x96, 0x87, 0xb4, 0x31, 0xc4, 0x4d, 0x44, + 0x43, 0xfb, 0x8e, 0xb9, 0xd1, 0x8f, 0xf4, 0xc6, 0x54, 0x94, 0x0d, 0x41, 0xaf, 0x2f, 0x5d, 0x3f, + 0xe7, 0xc1, 0xca, 0xed, 0x1e, 0xf2, 0xa8, 0xed, 0x60, 0x41, 0x59, 0x42, 0xd8, 0xc1, 0x18, 0x61, + 0xda, 0x90, 0xb0, 0x2c, 0x46, 0x52, 0xf7, 0x41, 0x3f, 0xd2, 0x2f, 0x4c, 0x45, 0xdd, 0x24, 0xf8, + 0xeb, 0x4b, 0xe2, 0x8f, 0x45, 0x30, 0xcf, 0x2f, 0x16, 0x09, 0x77, 0x6f, 0x01, 0xb9, 0xe5, 0x4a, + 0xe6, 0x60, 0x7c, 0x46, 0x0b, 0x83, 0xa6, 0xb1, 0x25, 0x37, 0x63, 0x61, 0x01, 0xcf, 0x83, 0x32, + 0xe1, 0x27, 0x21, 0x59, 0x50, 0xb5, 0xd1, 0x5b, 0x43, 0xf6, 0xcc, 0xb5, 0x99, 0xb3, 0xa4, 0x3d, + 0xbb, 0xc3, 0x39, 0xec, 0x00, 0x10, 0x9f, 0x04, 0xeb, 0xa3, 0xc8, 0xf1, 0xe3, 0x01, 0x43, 0x0b, + 0x0c, 0x3c, 0x0b, 0x4a, 0xbc, 0x72, 0xcb, 0x7b, 0x78, 0xe6, 0xb1, 0xe3, 0x25, 0x74, 0x33, 0x67, + 0x09, 0x73, 0xb8, 0x0e, 0x8a, 0x41, 0xe8, 0xbb, 0x72, 0x17, 0x3d, 0x31, 0xfa, 0xcc, 0xf4, 0xb6, + 0xb3, 0x99, 0xb3, 0xb8, 0x2d, 0x3c, 0xc3, 0x8e, 0xbc, 0x6c, 0xbf, 0x22, 0xfc, 0x0a, 0xc1, 0x4a, + 0xd6, 0x08, 0x2c, 0x05, 0x89, 0x4d, 0xe1, 0x19, 0x50, 0xde, 0xe7, 0x65, 0x49, 0x5e, 0xfe, 0x56, + 0xd3, 0xa0, 0x6c, 0xc1, 0x62, 0xef, 0x25, 0x6c, 0xe1, 0x35, 0x30, 0x47, 0xfd, 0xa0, 0x1b, 0x17, + 0x00, 0x79, 0xfd, 0xa8, 0xa5, 0xb1, 0x93, 0x0a, 0xc4, 0x66, 0xce, 0xca, 0xe0, 0xe0, 0x2d, 0xb0, + 0xb8, 0x97, 0x09, 0x53, 0x4c, 0xf8, 0xd7, 0x8c, 0x11, 0x9e, 0x27, 0x67, 0xcf, 0x66, 0xce, 0x1a, + 0x43, 0x9b, 0x60, 0x98, 0x51, 0xf5, 0x3f, 0x0a, 0x60, 0x4e, 0xc6, 0x8c, 0xb8, 0x2b, 0x9c, 0x4b, + 0xc2, 0x40, 0x84, 0xcc, 0xff, 0x9f, 0x15, 0x06, 0xdc, 0x3c, 0x15, 0x05, 0xef, 0x26, 0x51, 0x20, + 0xe2, 0x67, 0x65, 0x98, 0xa5, 0x5c, 0xff, 0x14, 0x42, 0x2a, 0xbf, 0x11, 0x2b, 0x2f, 0xc2, 0xe6, + 0xf8, 0xe4, 0x7d, 0x37, 0x46, 0x49, 0xd9, 0x2f, 0x82, 0x8a, 0x2d, 0x3e, 0x37, 0x4c, 0x0a, 0x98, + 0xf1, 0xaf, 0x11, 0x4c, 0x48, 0x09, 0x80, 0x1b, 0x43, 0xf9, 0x45, 0xd4, 0x1c, 0x1b, 0x97, 0x3f, + 0x01, 0xc5, 0xea, 0x9f, 0x4e, 0xd4, 0x2f, 0x4b, 0xcc, 0xd8, 0x66, 0x95, 0xbc, 0x98, 0x94, 0x7e, + 0x13, 0xcc, 0xb8, 0x98, 0x22, 0x76, 0x96, 0x55, 0x2b, 0xbc, 0x6e, 0x9c, 0xcc, 0x4a, 0x35, 0xe4, + 0xdb, 0xb8, 0x29, 0x0d, 0xaf, 0x7a, 0x34, 0x3c, 0x90, 0xc7, 0x96, 0x04, 0xbd, 0x7a, 0x09, 0xcc, + 0x67, 0x0c, 0xe0, 0x22, 0x28, 0x74, 0x71, 0xfc, 0x65, 0x85, 0x35, 0xd9, 0xe5, 0x6e, 0x1f, 0x39, + 0x3d, 0xcc, 0x69, 0xaf, 0x5a, 0xa2, 0x73, 0x31, 0x7f, 0x5e, 0x31, 0xab, 0xa0, 0x12, 0x8a, 0xa7, + 0x98, 0xad, 0x87, 0x8f, 0xb5, 0xdc, 0xa3, 0xc7, 0x5a, 0xee, 0xe9, 0x63, 0x4d, 0xf9, 0x7a, 0xa0, + 0x29, 0xdf, 0x0f, 0x34, 0xe5, 0xc1, 0x40, 0x53, 0x1e, 0x0e, 0x34, 0xe5, 0xf7, 0x81, 0xa6, 0xfc, + 0x39, 0xd0, 0x72, 0x4f, 0x07, 0x9a, 0x72, 0xf7, 0x89, 0x96, 0x7b, 0xf8, 0x44, 0xcb, 0x3d, 0x7a, + 0xa2, 0xe5, 0x3e, 0x37, 0x8e, 0x56, 0xc2, 0x76, 0xca, 0x9c, 0x96, 0x8d, 0xbf, 0x02, 0x00, 0x00, + 0xff, 0xff, 0xe6, 0x4a, 0x9a, 0x06, 0x55, 0x15, 0x00, 0x00, } func (this *LokiRequest) Equal(that interface{}) bool { @@ -1334,6 +1380,30 @@ func (this *LokiInstantRequest) Equal(that interface{}) bool { } return true } +func (this *Plan) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Plan) + if !ok { + that2, ok := that.(Plan) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !bytes.Equal(this.Raw, that1.Raw) { + return false + } + return true +} func (this *LokiResponse) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2170,6 +2240,16 @@ func (this *LokiInstantRequest) GoString() string { s = append(s, "}") return strings.Join(s, "") } +func (this *Plan) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&queryrange.Plan{") + s = append(s, "Raw: "+fmt.Sprintf("%#v", this.Raw)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} func (this *LokiResponse) GoString() string { if this == nil { return "nil" @@ -2522,21 +2602,21 @@ func (m *LokiRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x30 } - n1, err1 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) - if err1 != nil { - return 0, err1 - } - i -= n1 - i = encodeVarintQueryrange(dAtA, i, uint64(n1)) - i-- - dAtA[i] = 0x2a - n2, err2 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) + 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):]) + if err3 != nil { + return 0, err3 + } + i -= n3 + i = encodeVarintQueryrange(dAtA, i, uint64(n3)) + i-- dAtA[i] = 0x22 if m.Step != 0 { i = encodeVarintQueryrange(dAtA, i, uint64(m.Step)) @@ -2611,12 +2691,12 @@ func (m *LokiInstantRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x20 } - n3, err3 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.TimeTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.TimeTs):]) - if err3 != nil { - return 0, err3 + 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 } - i -= n3 - i = encodeVarintQueryrange(dAtA, i, uint64(n3)) + i -= n5 + i = encodeVarintQueryrange(dAtA, i, uint64(n5)) i-- dAtA[i] = 0x1a if m.Limit != 0 { @@ -2634,6 +2714,36 @@ func (m *LokiInstantRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *Plan) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Plan) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Plan) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Raw) > 0 { + i -= len(m.Raw) + copy(dAtA[i:], m.Raw) + i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Raw))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *LokiResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2763,20 +2873,20 @@ func (m *LokiSeriesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x22 } - n6, err6 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.EndTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.EndTs):]) - if err6 != nil { - return 0, err6 + 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 } - i -= n6 - i = encodeVarintQueryrange(dAtA, i, uint64(n6)) + i -= n8 + i = encodeVarintQueryrange(dAtA, i, uint64(n8)) i-- dAtA[i] = 0x1a - n7, err7 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTs, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTs):]) - if err7 != nil { - return 0, err7 + 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 } - i -= n7 - i = encodeVarintQueryrange(dAtA, i, uint64(n7)) + i -= n9 + i = encodeVarintQueryrange(dAtA, i, uint64(n9)) i-- dAtA[i] = 0x12 if len(m.Match) > 0 { @@ -3681,6 +3791,19 @@ func (m *LokiInstantRequest) Size() (n int) { return n } +func (m *Plan) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Raw) + if l > 0 { + n += 1 + l + sovQueryrange(uint64(l)) + } + return n +} + func (m *LokiResponse) Size() (n int) { if m == nil { return 0 @@ -4166,6 +4289,16 @@ func (this *LokiInstantRequest) String() string { }, "") return s } +func (this *Plan) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Plan{`, + `Raw:` + fmt.Sprintf("%v", this.Raw) + `,`, + `}`, + }, "") + return s +} func (this *LokiResponse) String() string { if this == nil { return "nil" @@ -4748,7 +4881,7 @@ func (m *LokiRequest) Unmarshal(dAtA []byte) error { if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Plan", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQueryrange @@ -4758,23 +4891,24 @@ func (m *LokiRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthQueryrange } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthQueryrange } if postIndex > l { return io.ErrUnexpectedEOF } - var v github_com_grafana_loki_pkg_querier_plan.QueryPlan - m.Plan = &v + if m.Plan == nil { + m.Plan = &github_com_grafana_loki_pkg_querier_plan.QueryPlan{} + } if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5003,6 +5137,95 @@ func (m *LokiInstantRequest) Unmarshal(dAtA []byte) error { if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Plan", 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.Plan == nil { + m.Plan = &github_com_grafana_loki_pkg_querier_plan.QueryPlan{} + } + if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQueryrange(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQueryrange + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Plan) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Plan: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Plan: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Raw", wireType) + } var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -5028,10 +5251,9 @@ func (m *LokiInstantRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - var v github_com_grafana_loki_pkg_querier_plan.QueryPlan - m.Plan = &v - if err := m.Plan.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Raw = append(m.Raw[:0], dAtA[iNdEx:postIndex]...) + if m.Raw == nil { + m.Raw = []byte{} } iNdEx = postIndex default: diff --git a/pkg/querier/queryrange/queryrange.proto b/pkg/querier/queryrange/queryrange.proto index 8eb43e34ca..f673464acf 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -18,7 +18,7 @@ option (gogoproto.sizer_all) = true; option (gogoproto.unmarshaler_all) = true; message LokiRequest { - string query = 1; + string query = 1; // mark as reserved once we've fully migrated to plan. uint32 limit = 2; int64 step = 3; int64 interval = 9; @@ -33,7 +33,7 @@ message LokiRequest { logproto.Direction direction = 6; string path = 7; repeated string shards = 8 [(gogoproto.jsontag) = "shards"]; - bytes plan = 10 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; + Plan plan = 10 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; } message LokiInstantRequest { @@ -46,7 +46,11 @@ message LokiInstantRequest { logproto.Direction direction = 4; string path = 5; repeated string shards = 6 [(gogoproto.jsontag) = "shards"]; - bytes plan = 7 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; + Plan plan = 7 [(gogoproto.customtype) = "github.com/grafana/loki/pkg/querier/plan.QueryPlan"]; +} + +message Plan { + bytes raw = 1; } message LokiResponse { diff --git a/pkg/querier/queryrange/queryrangebase/roundtrip.go b/pkg/querier/queryrange/queryrangebase/roundtrip.go index 847d311323..cfc7a4382d 100644 --- a/pkg/querier/queryrange/queryrangebase/roundtrip.go +++ b/pkg/querier/queryrange/queryrangebase/roundtrip.go @@ -22,6 +22,8 @@ import ( "time" "github.com/pkg/errors" + + "github.com/grafana/dskit/flagext" ) const day = 24 * time.Hour @@ -33,11 +35,12 @@ var PassthroughMiddleware = MiddlewareFunc(func(next Handler) Handler { // Config for query_range middleware chain. type Config struct { - AlignQueriesWithStep bool `yaml:"align_queries_with_step"` - ResultsCacheConfig ResultsCacheConfig `yaml:"results_cache"` - CacheResults bool `yaml:"cache_results"` - MaxRetries int `yaml:"max_retries"` - ShardedQueries bool `yaml:"parallelise_shardable_queries"` + AlignQueriesWithStep bool `yaml:"align_queries_with_step"` + ResultsCacheConfig ResultsCacheConfig `yaml:"results_cache"` + CacheResults bool `yaml:"cache_results"` + MaxRetries int `yaml:"max_retries"` + ShardedQueries bool `yaml:"parallelise_shardable_queries"` + ShardAggregations flagext.StringSliceCSV `yaml:"shard_aggregations"` } // RegisterFlags adds the flags required to config this to the given FlagSet. @@ -47,6 +50,10 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.CacheResults, "querier.cache-results", false, "Cache query results.") f.BoolVar(&cfg.ShardedQueries, "querier.parallelise-shardable-queries", true, "Perform query parallelisations based on storage sharding configuration and query ASTs. This feature is supported only by the chunks storage engine.") + cfg.ShardAggregations = []string{} + f.Var(&cfg.ShardAggregations, "querier.shard-aggregation", + "A comma-separated list of LogQL vector and range aggregations that should be sharded") + cfg.ResultsCacheConfig.RegisterFlags(f) } @@ -57,6 +64,11 @@ func (cfg *Config) Validate() error { return errors.Wrap(err, "invalid results_cache config") } } + + if len(cfg.ShardAggregations) > 0 && !cfg.ShardedQueries { + return errors.New("shard_aggregation requires parallelise_shardable_queries=true") + } + return nil } diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 1df7bb4616..a6c32b1525 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -41,6 +41,7 @@ func NewQueryShardMiddleware( limits Limits, maxShards int, statsHandler queryrangebase.Handler, + shardAggregation []string, ) queryrangebase.Middleware { noshards := !hasShards(confs) @@ -54,7 +55,7 @@ func NewQueryShardMiddleware( } mapperware := queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler { - return newASTMapperware(confs, engineOpts, next, statsHandler, logger, shardingMetrics, limits, maxShards) + return newASTMapperware(confs, engineOpts, next, statsHandler, logger, shardingMetrics, limits, maxShards, shardAggregation) }) return queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler { @@ -79,16 +80,18 @@ func newASTMapperware( metrics *logql.MapperMetrics, limits Limits, maxShards int, + shardAggregation []string, ) *astMapperware { ast := &astMapperware{ - confs: confs, - logger: log.With(logger, "middleware", "QueryShard.astMapperware"), - limits: limits, - next: next, - statsHandler: next, - ng: logql.NewDownstreamEngine(engineOpts, DownstreamHandler{next: next, limits: limits}, limits, logger), - metrics: metrics, - maxShards: maxShards, + confs: confs, + logger: log.With(logger, "middleware", "QueryShard.astMapperware"), + limits: limits, + next: next, + statsHandler: next, + ng: logql.NewDownstreamEngine(engineOpts, DownstreamHandler{next: next, limits: limits}, limits, logger), + metrics: metrics, + maxShards: maxShards, + shardAggregation: shardAggregation, } if statsHandler != nil { @@ -107,6 +110,10 @@ type astMapperware struct { ng *logql.DownstreamEngine metrics *logql.MapperMetrics maxShards int + + // Feature flag for sharding range and vector aggregations such as + // quantile_ver_time with probabilistic data structures. + shardAggregation []string } func (ast *astMapperware) checkQuerySizeLimit(ctx context.Context, bytesPerShard uint64, notShardable bool) error { @@ -143,7 +150,12 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que util_log.WithContext(ctx, ast.logger), ) - maxRVDuration, maxOffset, err := maxRangeVectorAndOffsetDuration(r.GetQuery()) + params, err := ParamsFromRequest(r) + if err != nil { + return nil, err + } + + maxRVDuration, maxOffset, err := maxRangeVectorAndOffsetDuration(params.GetExpression()) if err != nil { level.Warn(logger).Log("err", err.Error(), "msg", "failed to get range-vector and offset duration so skipped AST mapper for request") return ast.next.Do(ctx, r) @@ -183,12 +195,7 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrangebase.Request) (que return ast.next.Do(ctx, r) } - mapper := logql.NewShardMapper(resolver, ast.metrics) - - params, err := ParamsFromRequest(r) - if err != nil { - return nil, err - } + mapper := logql.NewShardMapper(resolver, ast.metrics, ast.shardAggregation) noop, bytesPerShard, parsed, err := mapper.Parse(params.GetExpression()) if err != nil { diff --git a/pkg/querier/queryrange/querysharding_test.go b/pkg/querier/queryrange/querysharding_test.go index a9ffdd4fbc..95d809776b 100644 --- a/pkg/querier/queryrange/querysharding_test.go +++ b/pkg/querier/queryrange/querysharding_test.go @@ -172,6 +172,7 @@ func Test_astMapper(t *testing.T) { nilShardingMetrics, fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1, queryTimeout: time.Second}, 0, + []string{}, ) req := defaultReq() @@ -316,6 +317,7 @@ func Test_astMapper_QuerySizeLimits(t *testing.T) { maxQuerierBytesRead: tc.maxQuerierBytesSize, }, 0, + []string{}, ) req := defaultReq() @@ -354,6 +356,7 @@ func Test_ShardingByPass(t *testing.T) { nilShardingMetrics, fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1}, 0, + []string{}, ) req := defaultReq() @@ -434,7 +437,9 @@ func Test_InstantSharding(t *testing.T) { queryTimeout: time.Second, }, 0, - nil) + nil, + []string{}, + ) response, err := sharding.Wrap(queryrangebase.HandlerFunc(func(c context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { lock.Lock() defer lock.Unlock() @@ -722,6 +727,7 @@ func TestShardingAcrossConfigs_ASTMapper(t *testing.T) { nilShardingMetrics, fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1, queryTimeout: time.Second}, 0, + []string{}, ) // currently all the tests call `defaultReq()` which creates an instance of the type LokiRequest @@ -856,6 +862,7 @@ func Test_ASTMapper_MaxLookBackPeriod(t *testing.T) { nilShardingMetrics, fakeLimits{maxSeries: math.MaxInt32, tsdbMaxQueryParallelism: 1, queryTimeout: time.Second}, 0, + []string{}, ) q := `{cluster="dev-us-central-0"}` diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index f5d5147d32..c03d459ba9 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -242,11 +242,6 @@ func (r roundTripper) Do(ctx context.Context, req base.Request) (base.Response, switch op := req.(type) { case *LokiRequest: - expr, err := syntax.ParseExpr(op.Query) - if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - queryHash := util.HashedQuery(op.Query) level.Info(logger).Log( "msg", "executing query", @@ -261,7 +256,11 @@ func (r roundTripper) Do(ctx context.Context, req base.Request) (base.Response, "query_hash", queryHash, ) - switch e := expr.(type) { + if op.Plan == nil { + return nil, errors.New("query plan is empty") + } + + switch e := op.Plan.AST.(type) { case syntax.SampleExpr: // The error will be handled later. groups, err := e.MatcherGroups() @@ -302,15 +301,10 @@ func (r roundTripper) Do(ctx context.Context, req base.Request) (base.Response, return r.labels.Do(ctx, req) case *LokiInstantRequest: - expr, err := syntax.ParseExpr(op.Query) - if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) - } - queryHash := util.HashedQuery(op.Query) level.Info(logger).Log("msg", "executing query", "type", "instant", "query", op.Query, "query_hash", queryHash) - switch expr.(type) { + switch op.Plan.AST.(type) { case syntax.SampleExpr: return r.instantMetric.Do(ctx, req) default: @@ -440,6 +434,7 @@ func NewLogFilterTripperware( limits, 0, // 0 is unlimited shards statsHandler, + cfg.ShardAggregations, ), ) } else { @@ -664,6 +659,7 @@ func NewMetricTripperware( limits, 0, // 0 is unlimited shards statsHandler, + cfg.ShardAggregations, ), ) } else { @@ -728,6 +724,7 @@ func NewInstantMetricTripperware( limits, 0, // 0 is unlimited shards statsHandler, + cfg.ShardAggregations, ), ) } diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index eed2a6cfb7..f6379ce728 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -199,6 +199,9 @@ func TestMetricsTripperware(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`rate({app="foo"} |= "foo"[1m])`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -282,6 +285,9 @@ func TestLogFilterTripperware(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"} |= "foo"`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -806,6 +812,9 @@ func TestLogNoFilter(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"}`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -817,7 +826,12 @@ func TestLogNoFilter(t *testing.T) { } func TestPostQueries(t *testing.T) { - lreq := &LokiRequest{Query: `{app="foo"} |~ "foo"`} + lreq := &LokiRequest{ + Query: `{app="foo"} |~ "foo"`, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"} |~ "foo"`), + }, + } ctx := user.InjectOrgID(context.Background(), "1") handler := base.HandlerFunc(func(context.Context, base.Request) (base.Response, error) { t.Error("unexpected default roundtripper called") @@ -855,6 +869,9 @@ func TestTripperware_EntriesLimit(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{app="foo"}`), + }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -902,6 +919,9 @@ func TestTripperware_RequiredLabels(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(test.qs), + }, } // See loghttp.step step := time.Duration(int(math.Max(math.Floor(lreq.EndTs.Sub(lreq.StartTs).Seconds()/250), 1))) * time.Second @@ -1007,6 +1027,9 @@ func TestTripperware_RequiredNumberLabels(t *testing.T) { EndTs: testTime, Direction: logproto.FORWARD, Path: "/loki/api/v1/query_range", + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.query), + }, } // See loghttp.step step := time.Duration(int(math.Max(math.Floor(lreq.EndTs.Sub(lreq.StartTs).Seconds()/250), 1))) * time.Second diff --git a/pkg/querier/queryrange/split_by_interval.go b/pkg/querier/queryrange/split_by_interval.go index 84b3a519f1..d568fe65dd 100644 --- a/pkg/querier/queryrange/split_by_interval.go +++ b/pkg/querier/queryrange/split_by_interval.go @@ -311,13 +311,17 @@ func splitByTime(req queryrangebase.Request, interval time.Duration) ([]queryran return reqs, nil } -// maxRangeVectorAndOffsetDuration returns the maximum range vector and offset duration within a LogQL query. -func maxRangeVectorAndOffsetDuration(q string) (time.Duration, time.Duration, error) { - expr, err := syntax.ParseExpr(q) +// maxRangeVectorAndOffsetDurationFromQueryString +func maxRangeVectorAndOffsetDurationFromQueryString(q string) (time.Duration, time.Duration, error) { + parsed, err := syntax.ParseExpr(q) if err != nil { return 0, 0, err } + return maxRangeVectorAndOffsetDuration(parsed) +} +// maxRangeVectorAndOffsetDuration returns the maximum range vector and offset duration within a LogQL query. +func maxRangeVectorAndOffsetDuration(expr syntax.Expr) (time.Duration, time.Duration, error) { if _, ok := expr.(syntax.SampleExpr); !ok { return 0, 0, nil } @@ -338,8 +342,8 @@ func maxRangeVectorAndOffsetDuration(q string) (time.Duration, time.Duration, er // reduceSplitIntervalForRangeVector reduces the split interval for a range query based on the duration of the range vector. // Large range vector durations will not be split into smaller intervals because it can cause the queries to be slow by over-processing data. -func reduceSplitIntervalForRangeVector(r queryrangebase.Request, interval time.Duration) (time.Duration, error) { - maxRange, _, err := maxRangeVectorAndOffsetDuration(r.GetQuery()) +func reduceSplitIntervalForRangeVector(r *LokiRequest, interval time.Duration) (time.Duration, error) { + maxRange, _, err := maxRangeVectorAndOffsetDuration(r.Plan.AST) if err != nil { return 0, err } @@ -352,13 +356,13 @@ func reduceSplitIntervalForRangeVector(r queryrangebase.Request, interval time.D func splitMetricByTime(r queryrangebase.Request, interval time.Duration) ([]queryrangebase.Request, error) { var reqs []queryrangebase.Request - interval, err := reduceSplitIntervalForRangeVector(r, interval) + lokiReq := r.(*LokiRequest) + + interval, err := reduceSplitIntervalForRangeVector(lokiReq, interval) if err != nil { return nil, err } - lokiReq := r.(*LokiRequest) - // step align start and end time of the query. Start time is rounded down and end time is rounded up. stepNs := r.GetStep() * 1e6 startNs := lokiReq.StartTs.UnixNano() diff --git a/pkg/querier/queryrange/split_by_interval_test.go b/pkg/querier/queryrange/split_by_interval_test.go index 78d74b111a..2730cf49d9 100644 --- a/pkg/querier/queryrange/split_by_interval_test.go +++ b/pkg/querier/queryrange/split_by_interval_test.go @@ -228,7 +228,7 @@ func Test_splitMetricQuery(t *testing.T) { const seconds = 1e3 // 1e3 milliseconds per second. for i, tc := range []struct { - input queryrangebase.Request + input *LokiRequest expected []queryrangebase.Request interval time.Duration }{ @@ -600,6 +600,17 @@ func Test_splitMetricQuery(t *testing.T) { interval: 15 * time.Minute, }, } { + // Set query plans + tc.input.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(tc.input.Query), + } + + for _, e := range tc.expected { + e.(*LokiRequest).Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(e.GetQuery()), + } + } + t.Run(strconv.Itoa(i), func(t *testing.T) { splits, err := splitMetricByTime(tc.input, tc.interval) require.NoError(t, err) diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 9311de2090..fe0f1245be 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -25,7 +25,9 @@ import ( "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/client/local" "github.com/grafana/loki/pkg/storage/config" @@ -494,6 +496,10 @@ func Test_store_SelectLogs(t *testing.T) { chunkMetrics: NilMetrics, } + tt.req.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(tt.req.Selector), + } + ctx = user.InjectOrgID(context.Background(), "test-user") it, err := s.SelectLogs(ctx, logql.SelectLogParams{QueryRequest: tt.req}) if err != nil { @@ -818,6 +824,10 @@ func Test_store_SelectSample(t *testing.T) { chunkMetrics: NilMetrics, } + tt.req.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(tt.req.Selector), + } + ctx = user.InjectOrgID(context.Background(), "test-user") it, err := s.SelectSamples(ctx, logql.SelectSampleParams{SampleQueryRequest: tt.req}) if err != nil { @@ -1385,6 +1395,9 @@ func Test_OverlappingChunks(t *testing.T) { Direction: logproto.BACKWARD, Start: time.Unix(0, 0), End: time.Unix(0, 10), + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(`{foo="bar"}`), + }, }}) if err != nil { t.Errorf("store.SelectLogs() error = %v", err) @@ -1497,6 +1510,15 @@ func Test_GetSeries(t *testing.T) { } { tt := tt t.Run(tt.name, func(t *testing.T) { + if tt.req.Selector != "" { + tt.req.Plan = &plan.QueryPlan{ + AST: syntax.MustParseExpr(tt.req.Selector), + } + } else { + tt.req.Plan = &plan.QueryPlan{ + AST: nil, + } + } series, err := store.SelectSeries(ctx, tt.req) require.NoError(t, err) require.Equal(t, tt.expectedSeries, series) diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 6dff5146af..b59b729b0d 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -17,6 +17,7 @@ import ( "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel/stats" "github.com/grafana/loki/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/querier/plan" "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/cache" chunkclient "github.com/grafana/loki/pkg/storage/chunk/client" @@ -135,6 +136,9 @@ func newQuery(query string, start, end time.Time, shards []astmapper.ShardAnnota End: end, Direction: logproto.FORWARD, Deletes: deletes, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(query), + }, } for _, shard := range shards { req.Shards = append(req.Shards, shard.String()) @@ -148,6 +152,9 @@ func newSampleQuery(query string, start, end time.Time, deletes []*logproto.Dele Start: start, End: end, Deletes: deletes, + Plan: &plan.QueryPlan{ + AST: syntax.MustParseExpr(query), + }, } return req } diff --git a/tools/dev/loki-boltdb-storage-s3/compose-up.sh b/tools/dev/loki-boltdb-storage-s3/compose-up.sh index 1841f312ca..2d26a83123 100755 --- a/tools/dev/loki-boltdb-storage-s3/compose-up.sh +++ b/tools/dev/loki-boltdb-storage-s3/compose-up.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash set -e diff --git a/tools/dev/loki-boltdb-storage-s3/config/loki.yaml b/tools/dev/loki-boltdb-storage-s3/config/loki.yaml index de0dbd713d..83149885fe 100644 --- a/tools/dev/loki-boltdb-storage-s3/config/loki.yaml +++ b/tools/dev/loki-boltdb-storage-s3/config/loki.yaml @@ -108,6 +108,7 @@ schema_config: object_store: s3 schema: v11 store: boltdb-shipper + row_shards: 4 server: graceful_shutdown_timeout: 5s grpc_server_max_concurrent_streams: 1000