diff --git a/pkg/querier/tripperware/custom.go b/pkg/querier/tripperware/custom.go new file mode 100644 index 0000000000..283262af7d --- /dev/null +++ b/pkg/querier/tripperware/custom.go @@ -0,0 +1,35 @@ +package tripperware + +import ( + "fmt" + + "github.com/gogo/protobuf/types" +) + +func (e *Extent) ToResponse() (Response, error) { + msg, err := types.EmptyAny(e.Response) + if err != nil { + return nil, err + } + + if err := types.UnmarshalAny(e.Response, msg); err != nil { + return nil, err + } + + resp, ok := msg.(Response) + if !ok { + return nil, fmt.Errorf("bad cached type") + } + return resp, nil +} + +func (m *Sample) GetTimestampMs() int64 { + if m != nil { + if m.Sample != nil { + return m.Sample.TimestampMs + } else if m.Histogram != nil { + return m.Histogram.TimestampMs + } + } + return 0 +} diff --git a/pkg/querier/tripperware/instantquery/custom.go b/pkg/querier/tripperware/instantquery/custom.go deleted file mode 100644 index c88f9284e1..0000000000 --- a/pkg/querier/tripperware/instantquery/custom.go +++ /dev/null @@ -1,12 +0,0 @@ -package instantquery - -func (m *Sample) GetTimestampMs() int64 { - if m != nil { - if m.Sample != nil { - return m.Sample.TimestampMs - } else if m.Histogram != nil { - return m.Histogram.TimestampMs - } - } - return 0 -} diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 135670ec37..5a8f875845 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -7,25 +7,16 @@ import ( "io" "net/http" "net/url" - "sort" "strings" "time" - "unsafe" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/timestamp" - promqlparser "github.com/prometheus/prometheus/promql/parser" - "github.com/thanos-io/thanos/pkg/strutil" "github.com/weaveworks/common/httpgrpc" "google.golang.org/grpc/status" - "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/spanlogger" ) @@ -40,72 +31,6 @@ var ( }.Froze() ) -type PrometheusRequest struct { - tripperware.Request - Time int64 - Stats string - Query string - Path string - Headers http.Header -} - -// GetTime returns time in milliseconds. -func (r *PrometheusRequest) GetTime() int64 { - return r.Time -} - -// GetStart returns always 0 for instant query. -func (r *PrometheusRequest) GetStart() int64 { - return 0 -} - -// GetEnd returns always 0 for instant query. -func (r *PrometheusRequest) GetEnd() int64 { - return 0 -} - -// GetStep returns always 0 for instant query. -func (r *PrometheusRequest) GetStep() int64 { - return 0 -} - -// GetQuery returns the query of the request. -func (r *PrometheusRequest) GetQuery() string { - return r.Query -} - -// WithStartEnd clone the current request with different start and end timestamp. -func (r *PrometheusRequest) WithStartEnd(int64, int64) tripperware.Request { - return r -} - -// WithQuery clone the current request with a different query. -func (r *PrometheusRequest) WithQuery(query string) tripperware.Request { - q := *r - q.Query = query - return &q -} - -// LogToSpan writes information about this request to an OpenTracing span -func (r *PrometheusRequest) LogToSpan(sp opentracing.Span) { - sp.LogFields( - otlog.String("query", r.GetQuery()), - otlog.String("time", timestamp.Time(r.GetTime()).String()), - ) -} - -// GetStats returns the stats of the request. -func (r *PrometheusRequest) GetStats() string { - return r.Stats -} - -// WithStats clones the current `PrometheusRequest` with a new stats. -func (r *PrometheusRequest) WithStats(stats string) tripperware.Request { - q := *r - q.Stats = stats - return &q -} - type instantQueryCodec struct { tripperware.Codec now func() time.Time @@ -115,22 +40,8 @@ func newInstantQueryCodec() instantQueryCodec { return instantQueryCodec{now: time.Now} } -func (resp *PrometheusInstantQueryResponse) HTTPHeaders() map[string][]string { - if resp != nil && resp.GetHeaders() != nil { - r := map[string][]string{} - for _, header := range resp.GetHeaders() { - if header != nil { - r[header.Name] = header.Values - } - } - - return r - } - return nil -} - func (c instantQueryCodec) DecodeRequest(_ context.Context, r *http.Request, forwardHeaders []string) (tripperware.Request, error) { - result := PrometheusRequest{Headers: map[string][]string{}} + result := tripperware.PrometheusRequest{Headers: map[string][]string{}} var err error result.Time, err = util.ParseTimeParam(r, "time", c.now().Unix()) if err != nil { @@ -155,7 +66,7 @@ func (c instantQueryCodec) DecodeRequest(_ context.Context, r *http.Request, for } func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ tripperware.Request) (tripperware.Response, error) { - log, ctx := spanlogger.New(ctx, "PrometheusInstantQueryResponse") //nolint:ineffassign,staticcheck + log, ctx := spanlogger.New(ctx, "tripperware.PrometheusResponse") //nolint:ineffassign,staticcheck defer log.Finish() if err := ctx.Err(); err != nil { @@ -171,7 +82,7 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ return nil, httpgrpc.Errorf(r.StatusCode, string(buf)) } - var resp PrometheusInstantQueryResponse + var resp tripperware.PrometheusResponse if err := json.Unmarshal(buf, &resp); err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } @@ -183,7 +94,7 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ } func (instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { - promReq, ok := r.(*PrometheusRequest) + promReq, ok := r.(*tripperware.PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") } @@ -227,7 +138,7 @@ func (instantQueryCodec) EncodeResponse(ctx context.Context, res tripperware.Res sp, _ := opentracing.StartSpanFromContext(ctx, "APIResponse.ToHTTPResponse") defer sp.Finish() - a, ok := res.(*PrometheusInstantQueryResponse) + a, ok := res.(*tripperware.PrometheusResponse) if !ok { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "invalid response format") } @@ -251,293 +162,15 @@ func (instantQueryCodec) EncodeResponse(ctx context.Context, res tripperware.Res } func (instantQueryCodec) MergeResponse(ctx context.Context, req tripperware.Request, responses ...tripperware.Response) (tripperware.Response, error) { - sp, _ := opentracing.StartSpanFromContext(ctx, "PrometheusInstantQueryResponse.MergeResponse") + sp, _ := opentracing.StartSpanFromContext(ctx, "InstantQueryResponse.MergeResponse") sp.SetTag("response_count", len(responses)) defer sp.Finish() if len(responses) == 0 { - return NewEmptyPrometheusInstantQueryResponse(), nil - } else if len(responses) == 1 { - return responses[0], nil + return tripperware.NewEmptyPrometheusResponse(true), nil } - promResponses := make([]*PrometheusInstantQueryResponse, 0, len(responses)) - warnings := make([][]string, 0, len(responses)) - for _, resp := range responses { - promResponses = append(promResponses, resp.(*PrometheusInstantQueryResponse)) - if w := resp.(*PrometheusInstantQueryResponse).Warnings; w != nil { - warnings = append(warnings, w) - } - } - - var data PrometheusInstantQueryData - // For now, we only shard queries that returns a vector. - switch promResponses[0].Data.ResultType { - case model.ValVector.String(): - v, err := vectorMerge(ctx, req, promResponses) - if err != nil { - return nil, err - } - data = PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: v, - }, - }, - Stats: statsMerge(promResponses), - } - case model.ValMatrix.String(): - sampleStreams, err := matrixMerge(ctx, promResponses) - if err != nil { - return nil, err - } - - data = PrometheusInstantQueryData{ - ResultType: model.ValMatrix.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Matrix{ - Matrix: &Matrix{ - SampleStreams: sampleStreams, - }, - }, - }, - Stats: statsMerge(promResponses), - } - default: - return nil, fmt.Errorf("unexpected result type on instant query: %s", promResponses[0].Data.ResultType) - } - - res := &PrometheusInstantQueryResponse{ - Status: queryrange.StatusSuccess, - Data: data, - Warnings: strutil.MergeUnsortedSlices(warnings...), - } - return res, nil -} - -func vectorMerge(ctx context.Context, req tripperware.Request, resps []*PrometheusInstantQueryResponse) (*Vector, error) { - output := map[string]*Sample{} - metrics := []string{} // Used to preserve the order for topk and bottomk. - sortPlan, err := sortPlanForQuery(req.GetQuery()) - if err != nil { - return nil, err - } - buf := make([]byte, 0, 1024) - for _, resp := range resps { - if err = ctx.Err(); err != nil { - return nil, err - } - if resp == nil { - continue - } - // Merge vector result samples only. Skip other types such as - // string, scalar as those are not sharable. - if resp.Data.Result.GetVector() == nil { - continue - } - for _, sample := range resp.Data.Result.GetVector().Samples { - s := sample - if s == nil { - continue - } - metric := string(cortexpb.FromLabelAdaptersToLabels(sample.Labels).Bytes(buf)) - if existingSample, ok := output[metric]; !ok { - output[metric] = s - metrics = append(metrics, metric) // Preserve the order of metric. - } else if existingSample.GetTimestampMs() < s.GetTimestampMs() { - // Choose the latest sample if we see overlap. - output[metric] = s - } - } - } - - result := &Vector{ - Samples: make([]*Sample, 0, len(output)), - } - - if len(output) == 0 { - return result, nil - } - - if sortPlan == mergeOnly { - for _, k := range metrics { - result.Samples = append(result.Samples, output[k]) - } - return result, nil - } - - samples := make([]*pair, 0, len(output)) - for k, v := range output { - samples = append(samples, &pair{ - metric: k, - s: v, - }) - } - - // TODO: What if we have mixed float and histogram samples in the response? - // Then the sorting behavior is undefined. Prometheus doesn't handle it. - sort.Slice(samples, func(i, j int) bool { - // Order is determined by vector. - switch sortPlan { - case sortByValuesAsc: - return getSortValueFromPair(samples, i) < getSortValueFromPair(samples, j) - case sortByValuesDesc: - return getSortValueFromPair(samples, i) > getSortValueFromPair(samples, j) - } - return samples[i].metric < samples[j].metric - }) - - for _, p := range samples { - result.Samples = append(result.Samples, p.s) - } - return result, nil -} - -type sortPlan int - -const ( - mergeOnly sortPlan = 0 - sortByValuesAsc sortPlan = 1 - sortByValuesDesc sortPlan = 2 - sortByLabels sortPlan = 3 -) - -type pair struct { - metric string - s *Sample -} - -// getSortValueFromPair gets the float value used for sorting from samples. -// If float sample, use sample value. If histogram sample, use histogram sum. -// This is the same behavior as Prometheus https://github.com/prometheus/prometheus/blob/v2.53.0/promql/functions.go#L1595. -func getSortValueFromPair(samples []*pair, i int) float64 { - if samples[i].s.Histogram != nil { - return samples[i].s.Histogram.Histogram.Sum - } - // Impossible to have both histogram and sample nil. - return samples[i].s.Sample.Value -} - -func sortPlanForQuery(q string) (sortPlan, error) { - expr, err := promqlparser.ParseExpr(q) - if err != nil { - return 0, err - } - // Check if the root expression is topk or bottomk - if aggr, ok := expr.(*promqlparser.AggregateExpr); ok { - if aggr.Op == promqlparser.TOPK || aggr.Op == promqlparser.BOTTOMK { - return mergeOnly, nil - } - } - checkForSort := func(expr promqlparser.Expr) (sortAsc, sortDesc bool) { - if n, ok := expr.(*promqlparser.Call); ok { - if n.Func != nil { - if n.Func.Name == "sort" { - sortAsc = true - } - if n.Func.Name == "sort_desc" { - sortDesc = true - } - } - } - return sortAsc, sortDesc - } - // Check the root expression for sort - if sortAsc, sortDesc := checkForSort(expr); sortAsc || sortDesc { - if sortAsc { - return sortByValuesAsc, nil - } - return sortByValuesDesc, nil - } - - // If the root expression is a binary expression, check the LHS and RHS for sort - if bin, ok := expr.(*promqlparser.BinaryExpr); ok { - if sortAsc, sortDesc := checkForSort(bin.LHS); sortAsc || sortDesc { - if sortAsc { - return sortByValuesAsc, nil - } - return sortByValuesDesc, nil - } - if sortAsc, sortDesc := checkForSort(bin.RHS); sortAsc || sortDesc { - if sortAsc { - return sortByValuesAsc, nil - } - return sortByValuesDesc, nil - } - } - return sortByLabels, nil -} - -func matrixMerge(ctx context.Context, resps []*PrometheusInstantQueryResponse) ([]tripperware.SampleStream, error) { - output := make(map[string]tripperware.SampleStream) - for _, resp := range resps { - if err := ctx.Err(); err != nil { - return nil, err - } - if resp == nil { - continue - } - if resp.Data.Result.GetMatrix() == nil { - continue - } - tripperware.MergeSampleStreams(output, resp.Data.Result.GetMatrix().GetSampleStreams()) - } - - keys := make([]string, 0, len(output)) - for key := range output { - keys = append(keys, key) - } - sort.Strings(keys) - - result := make([]tripperware.SampleStream, 0, len(output)) - for _, key := range keys { - result = append(result, output[key]) - } - - return result, nil -} - -// NewEmptyPrometheusInstantQueryResponse returns an empty successful Prometheus query range response. -func NewEmptyPrometheusInstantQueryResponse() *PrometheusInstantQueryResponse { - return &PrometheusInstantQueryResponse{ - Status: queryrange.StatusSuccess, - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{}, - }, - }, - } -} - -func statsMerge(resps []*PrometheusInstantQueryResponse) *tripperware.PrometheusResponseStats { - output := map[int64]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{} - hasStats := false - for _, resp := range resps { - if resp.Data.Stats == nil { - continue - } - - hasStats = true - if resp.Data.Stats.Samples == nil { - continue - } - - for _, s := range resp.Data.Stats.Samples.TotalQueryableSamplesPerStep { - if stats, ok := output[s.GetTimestampMs()]; ok { - stats.Value += s.Value - } else { - output[s.GetTimestampMs()] = s - } - } - } - - if !hasStats { - return nil - } - - return tripperware.StatsMerge(output) + return tripperware.MergeResponse(ctx, true, req, responses...) } func decorateWithParamName(err error, field string) error { @@ -547,139 +180,3 @@ func decorateWithParamName(err error, field string) error { } return fmt.Errorf(errTmpl, field, err) } - -func init() { - jsoniter.RegisterTypeEncoderFunc("instantquery.Sample", encodeSample, marshalJSONIsEmpty) - jsoniter.RegisterTypeDecoderFunc("instantquery.Sample", decodeSample) -} - -func marshalJSONIsEmpty(ptr unsafe.Pointer) bool { - return false -} - -func decodeSample(ptr unsafe.Pointer, iter *jsoniter.Iterator) { - ss := (*Sample)(ptr) - for field := iter.ReadObject(); field != ""; field = iter.ReadObject() { - switch field { - case "metric": - metricString := iter.ReadAny().ToString() - lbls := labels.Labels{} - if err := json.UnmarshalFromString(metricString, &lbls); err != nil { - iter.ReportError("unmarshal Sample", err.Error()) - return - } - ss.Labels = cortexpb.FromLabelsToLabelAdapters(lbls) - case "value": - ss.Sample = &cortexpb.Sample{} - cortexpb.SampleJsoniterDecode(unsafe.Pointer(ss.Sample), iter) - case "histogram": - ss.Histogram = &tripperware.SampleHistogramPair{} - tripperware.UnmarshalSampleHistogramPairJSON(unsafe.Pointer(ss.Histogram), iter) - default: - iter.ReportError("unmarshal Sample", fmt.Sprint("unexpected key:", field)) - return - } - } -} - -func encodeSample(ptr unsafe.Pointer, stream *jsoniter.Stream) { - ss := (*Sample)(ptr) - stream.WriteObjectStart() - - stream.WriteObjectField(`metric`) - lbls, err := cortexpb.FromLabelAdaptersToLabels(ss.Labels).MarshalJSON() - if err != nil { - stream.Error = err - return - } - stream.SetBuffer(append(stream.Buffer(), lbls...)) - - if ss.Sample != nil { - stream.WriteMore() - stream.WriteObjectField(`value`) - cortexpb.SampleJsoniterEncode(unsafe.Pointer(ss.Sample), stream) - } - - if ss.Histogram != nil { - stream.WriteMore() - stream.WriteObjectField(`histogram`) - tripperware.MarshalSampleHistogramPairJSON(unsafe.Pointer(ss.Histogram), stream) - } - - stream.WriteObjectEnd() -} - -// UnmarshalJSON implements json.Unmarshaler. -func (s *PrometheusInstantQueryData) UnmarshalJSON(data []byte) error { - var queryData struct { - ResultType string `json:"resultType"` - Stats *tripperware.PrometheusResponseStats `json:"stats,omitempty"` - } - - if err := json.Unmarshal(data, &queryData); err != nil { - return err - } - s.ResultType = queryData.ResultType - s.Stats = queryData.Stats - switch s.ResultType { - case model.ValVector.String(): - var result struct { - Samples []*Sample `json:"result"` - } - if err := json.Unmarshal(data, &result); err != nil { - return err - } - s.Result = PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{Vector: &Vector{ - Samples: result.Samples, - }}, - } - case model.ValMatrix.String(): - var result struct { - SampleStreams []tripperware.SampleStream `json:"result"` - } - if err := json.Unmarshal(data, &result); err != nil { - return err - } - s.Result = PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Matrix{Matrix: &Matrix{ - SampleStreams: result.SampleStreams, - }}, - } - default: - s.Result = PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_RawBytes{data}, - } - } - return nil -} - -// MarshalJSON implements json.Marshaler. -func (s *PrometheusInstantQueryData) MarshalJSON() ([]byte, error) { - switch s.ResultType { - case model.ValVector.String(): - res := struct { - ResultType string `json:"resultType"` - Data []*Sample `json:"result"` - Stats *tripperware.PrometheusResponseStats `json:"stats,omitempty"` - }{ - ResultType: s.ResultType, - Data: s.Result.GetVector().Samples, - Stats: s.Stats, - } - return json.Marshal(res) - case model.ValMatrix.String(): - res := struct { - ResultType string `json:"resultType"` - Data []tripperware.SampleStream `json:"result"` - Stats *tripperware.PrometheusResponseStats `json:"stats,omitempty"` - }{ - ResultType: s.ResultType, - Data: s.Result.GetMatrix().SampleStreams, - Stats: s.Stats, - } - return json.Marshal(res) - default: - return s.Result.GetRawBytes(), nil - } -} diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 3a19641b97..f1581899a5 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -4,6 +4,7 @@ import ( "bytes" "compress/gzip" "context" + "errors" "fmt" "io" "net/http" @@ -30,13 +31,13 @@ func TestRequest(t *testing.T) { for _, tc := range []struct { url string expectedURL string - expected *PrometheusRequest + expected *tripperware.PrometheusRequest expectedErr error }{ { url: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29&stats=all&time=1536673680", expectedURL: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29&stats=all&time=1536673680", - expected: &PrometheusRequest{ + expected: &tripperware.PrometheusRequest{ Path: "/api/v1/query", Time: 1536673680 * 1e3, Query: "sum(container_memory_rss) by (namespace)", @@ -49,7 +50,7 @@ func TestRequest(t *testing.T) { { url: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29&time=1536673680", expectedURL: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29&time=1536673680", - expected: &PrometheusRequest{ + expected: &tripperware.PrometheusRequest{ Path: "/api/v1/query", Time: 1536673680 * 1e3, Query: "sum(container_memory_rss) by (namespace)", @@ -62,7 +63,7 @@ func TestRequest(t *testing.T) { { url: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29", expectedURL: "/api/v1/query?query=sum%28container_memory_rss%29+by+%28namespace%29&time=", - expected: &PrometheusRequest{ + expected: &tripperware.PrometheusRequest{ Path: "/api/v1/query", Time: 0, Query: "sum(container_memory_rss) by (namespace)", @@ -219,7 +220,7 @@ func TestResponse(t *testing.T) { func TestMergeResponse(t *testing.T) { t.Parallel() - defaultReq := &PrometheusRequest{ + defaultReq := &tripperware.PrometheusRequest{ Query: "sum(up)", } for _, tc := range []struct { @@ -300,7 +301,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two responses with sort", - req: &PrometheusRequest{Query: "sort(sum by (job) (up))"}, + req: &tripperware.PrometheusRequest{Query: "sort(sum by (job) (up))"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, @@ -309,7 +310,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two histogram responses with sort", - req: &PrometheusRequest{Query: "sort(sum by (job) (up))"}, + req: &tripperware.PrometheusRequest{Query: "sort(sum by (job) (up))"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, @@ -318,7 +319,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two responses with sort_desc", - req: &PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, + req: &tripperware.PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, @@ -327,7 +328,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two histogram responses with sort_desc", - req: &PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, + req: &tripperware.PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, @@ -336,7 +337,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two responses with topk", - req: &PrometheusRequest{Query: "topk(10, up) by(job)"}, + req: &tripperware.PrometheusRequest{Query: "topk(10, up) by(job)"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, @@ -345,7 +346,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge two histogram responses with topk", - req: &PrometheusRequest{Query: "topk(10, up) by(job)"}, + req: &tripperware.PrometheusRequest{Query: "topk(10, up) by(job)"}, resps: []string{ `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, @@ -354,7 +355,7 @@ func TestMergeResponse(t *testing.T) { }, { name: "merge with warnings.", - req: &PrometheusRequest{Query: "topk(10, up) by(job)"}, + req: &tripperware.PrometheusRequest{Query: "topk(10, up) by(job)"}, resps: []string{ `{"status":"success","warnings":["warning1","warning2"],"data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, `{"status":"success","warnings":["warning1","warning3"],"data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, @@ -377,7 +378,7 @@ func TestMergeResponse(t *testing.T) { `{"status":"success","data":{"resultType":"string","result":[1662682521.409,"foo"]}}`, `{"status":"success","data":{"resultType":"string","result":[1662682521.409,"foo"]}}`, }, - expectedErr: fmt.Errorf("unexpected result type on instant query: %s", "string"), + expectedErr: errors.New("unexpected result type: string"), }, { name: "single matrix response", @@ -473,78 +474,12 @@ func TestMergeResponse(t *testing.T) { assert.Equal(t, tc.expectedErr, err) contents, err := io.ReadAll(dr.Body) assert.Equal(t, tc.expectedErr, err) - assert.Equal(t, string(contents), tc.expectedResp) + assert.Equal(t, tc.expectedResp, string(contents)) cancelCtx() }) } } -func Test_sortPlanForQuery(t *testing.T) { - tc := []struct { - query string - expectedPlan sortPlan - err bool - }{ - { - query: "invalid(10, up)", - expectedPlan: mergeOnly, - err: true, - }, - { - query: "topk(10, up)", - expectedPlan: mergeOnly, - err: false, - }, - { - query: "bottomk(10, up)", - expectedPlan: mergeOnly, - err: false, - }, - { - query: "1 + topk(10, up)", - expectedPlan: sortByLabels, - err: false, - }, - { - query: "1 + sort_desc(sum by (job) (up) )", - expectedPlan: sortByValuesDesc, - err: false, - }, - { - query: "sort(topk by (job) (10, up))", - expectedPlan: sortByValuesAsc, - err: false, - }, - { - query: "topk(5, up) by (job) + sort_desc(up)", - expectedPlan: sortByValuesDesc, - err: false, - }, - { - query: "sort(up) + topk(5, up) by (job)", - expectedPlan: sortByValuesAsc, - err: false, - }, - { - query: "sum(up) by (job)", - expectedPlan: sortByLabels, - err: false, - }, - } - - for _, tc := range tc { - t.Run(tc.query, func(t *testing.T) { - p, err := sortPlanForQuery(tc.query) - if tc.err { - assert.Error(t, err) - } else { - assert.NoError(t, err) - assert.Equal(t, tc.expectedPlan, p) - } - }) - } -} - func Benchmark_Decode(b *testing.B) { maxSamplesCount := 1000000 samples := make([]tripperware.SampleStream, maxSamplesCount) @@ -576,12 +511,12 @@ func Benchmark_Decode(b *testing.B) { }, } { b.Run(name, func(b *testing.B) { - r := PrometheusInstantQueryResponse{ - Data: PrometheusInstantQueryData{ + r := tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: model.ValMatrix.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Matrix{ - Matrix: &Matrix{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ SampleStreams: tc.sampleStream, }, }, diff --git a/pkg/querier/tripperware/instantquery/instantquery.pb.go b/pkg/querier/tripperware/instantquery/instantquery.pb.go deleted file mode 100644 index 6324739eb3..0000000000 --- a/pkg/querier/tripperware/instantquery/instantquery.pb.go +++ /dev/null @@ -1,2508 +0,0 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: instantquery.proto - -package instantquery - -import ( - bytes "bytes" - fmt "fmt" - cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" - github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" - tripperware "github.com/cortexproject/cortex/pkg/querier/tripperware" - _ "github.com/gogo/protobuf/gogoproto" - proto "github.com/gogo/protobuf/proto" - io "io" - math "math" - math_bits "math/bits" - reflect "reflect" - strings "strings" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package - -type PrometheusInstantQueryResponse struct { - Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` - Data PrometheusInstantQueryData `protobuf:"bytes,2,opt,name=Data,proto3" json:"data,omitempty"` - ErrorType string `protobuf:"bytes,3,opt,name=ErrorType,proto3" json:"errorType,omitempty"` - Error string `protobuf:"bytes,4,opt,name=Error,proto3" json:"error,omitempty"` - Headers []*tripperware.PrometheusResponseHeader `protobuf:"bytes,5,rep,name=Headers,proto3" json:"-"` - Warnings []string `protobuf:"bytes,6,rep,name=Warnings,proto3" json:"warnings,omitempty"` -} - -func (m *PrometheusInstantQueryResponse) Reset() { *m = PrometheusInstantQueryResponse{} } -func (*PrometheusInstantQueryResponse) ProtoMessage() {} -func (*PrometheusInstantQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{0} -} -func (m *PrometheusInstantQueryResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusInstantQueryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusInstantQueryResponse.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 *PrometheusInstantQueryResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusInstantQueryResponse.Merge(m, src) -} -func (m *PrometheusInstantQueryResponse) XXX_Size() int { - return m.Size() -} -func (m *PrometheusInstantQueryResponse) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusInstantQueryResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusInstantQueryResponse proto.InternalMessageInfo - -func (m *PrometheusInstantQueryResponse) GetStatus() string { - if m != nil { - return m.Status - } - return "" -} - -func (m *PrometheusInstantQueryResponse) GetData() PrometheusInstantQueryData { - if m != nil { - return m.Data - } - return PrometheusInstantQueryData{} -} - -func (m *PrometheusInstantQueryResponse) GetErrorType() string { - if m != nil { - return m.ErrorType - } - return "" -} - -func (m *PrometheusInstantQueryResponse) GetError() string { - if m != nil { - return m.Error - } - return "" -} - -func (m *PrometheusInstantQueryResponse) GetHeaders() []*tripperware.PrometheusResponseHeader { - if m != nil { - return m.Headers - } - return nil -} - -func (m *PrometheusInstantQueryResponse) GetWarnings() []string { - if m != nil { - return m.Warnings - } - return nil -} - -type PrometheusInstantQueryData struct { - ResultType string `protobuf:"bytes,1,opt,name=ResultType,proto3" json:"resultType"` - Result PrometheusInstantQueryResult `protobuf:"bytes,2,opt,name=Result,proto3" json:"result"` - Stats *tripperware.PrometheusResponseStats `protobuf:"bytes,3,opt,name=stats,proto3" json:"stats,omitempty"` -} - -func (m *PrometheusInstantQueryData) Reset() { *m = PrometheusInstantQueryData{} } -func (*PrometheusInstantQueryData) ProtoMessage() {} -func (*PrometheusInstantQueryData) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{1} -} -func (m *PrometheusInstantQueryData) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusInstantQueryData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusInstantQueryData.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 *PrometheusInstantQueryData) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusInstantQueryData.Merge(m, src) -} -func (m *PrometheusInstantQueryData) XXX_Size() int { - return m.Size() -} -func (m *PrometheusInstantQueryData) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusInstantQueryData.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusInstantQueryData proto.InternalMessageInfo - -func (m *PrometheusInstantQueryData) GetResultType() string { - if m != nil { - return m.ResultType - } - return "" -} - -func (m *PrometheusInstantQueryData) GetResult() PrometheusInstantQueryResult { - if m != nil { - return m.Result - } - return PrometheusInstantQueryResult{} -} - -func (m *PrometheusInstantQueryData) GetStats() *tripperware.PrometheusResponseStats { - if m != nil { - return m.Stats - } - return nil -} - -type PrometheusInstantQueryResult struct { - // Types that are valid to be assigned to Result: - // *PrometheusInstantQueryResult_Vector - // *PrometheusInstantQueryResult_RawBytes - // *PrometheusInstantQueryResult_Matrix - Result isPrometheusInstantQueryResult_Result `protobuf_oneof:"result"` -} - -func (m *PrometheusInstantQueryResult) Reset() { *m = PrometheusInstantQueryResult{} } -func (*PrometheusInstantQueryResult) ProtoMessage() {} -func (*PrometheusInstantQueryResult) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{2} -} -func (m *PrometheusInstantQueryResult) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusInstantQueryResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusInstantQueryResult.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 *PrometheusInstantQueryResult) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusInstantQueryResult.Merge(m, src) -} -func (m *PrometheusInstantQueryResult) XXX_Size() int { - return m.Size() -} -func (m *PrometheusInstantQueryResult) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusInstantQueryResult.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusInstantQueryResult proto.InternalMessageInfo - -type isPrometheusInstantQueryResult_Result interface { - isPrometheusInstantQueryResult_Result() - Equal(interface{}) bool - MarshalTo([]byte) (int, error) - Size() int -} - -type PrometheusInstantQueryResult_Vector struct { - Vector *Vector `protobuf:"bytes,1,opt,name=vector,proto3,oneof"` -} -type PrometheusInstantQueryResult_RawBytes struct { - RawBytes []byte `protobuf:"bytes,2,opt,name=rawBytes,proto3,oneof"` -} -type PrometheusInstantQueryResult_Matrix struct { - Matrix *Matrix `protobuf:"bytes,3,opt,name=matrix,proto3,oneof"` -} - -func (*PrometheusInstantQueryResult_Vector) isPrometheusInstantQueryResult_Result() {} -func (*PrometheusInstantQueryResult_RawBytes) isPrometheusInstantQueryResult_Result() {} -func (*PrometheusInstantQueryResult_Matrix) isPrometheusInstantQueryResult_Result() {} - -func (m *PrometheusInstantQueryResult) GetResult() isPrometheusInstantQueryResult_Result { - if m != nil { - return m.Result - } - return nil -} - -func (m *PrometheusInstantQueryResult) GetVector() *Vector { - if x, ok := m.GetResult().(*PrometheusInstantQueryResult_Vector); ok { - return x.Vector - } - return nil -} - -func (m *PrometheusInstantQueryResult) GetRawBytes() []byte { - if x, ok := m.GetResult().(*PrometheusInstantQueryResult_RawBytes); ok { - return x.RawBytes - } - return nil -} - -func (m *PrometheusInstantQueryResult) GetMatrix() *Matrix { - if x, ok := m.GetResult().(*PrometheusInstantQueryResult_Matrix); ok { - return x.Matrix - } - return nil -} - -// XXX_OneofWrappers is for the internal use of the proto package. -func (*PrometheusInstantQueryResult) XXX_OneofWrappers() []interface{} { - return []interface{}{ - (*PrometheusInstantQueryResult_Vector)(nil), - (*PrometheusInstantQueryResult_RawBytes)(nil), - (*PrometheusInstantQueryResult_Matrix)(nil), - } -} - -type Vector struct { - Samples []*Sample `protobuf:"bytes,1,rep,name=samples,proto3" json:"samples,omitempty"` -} - -func (m *Vector) Reset() { *m = Vector{} } -func (*Vector) ProtoMessage() {} -func (*Vector) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{3} -} -func (m *Vector) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Vector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_Vector.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 *Vector) XXX_Merge(src proto.Message) { - xxx_messageInfo_Vector.Merge(m, src) -} -func (m *Vector) XXX_Size() int { - return m.Size() -} -func (m *Vector) XXX_DiscardUnknown() { - xxx_messageInfo_Vector.DiscardUnknown(m) -} - -var xxx_messageInfo_Vector proto.InternalMessageInfo - -func (m *Vector) GetSamples() []*Sample { - if m != nil { - return m.Samples - } - return nil -} - -type Sample struct { - Labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter" json:"metric"` - Sample *cortexpb.Sample `protobuf:"bytes,2,opt,name=sample,proto3" json:"value"` - Histogram *tripperware.SampleHistogramPair `protobuf:"bytes,3,opt,name=histogram,proto3" json:"histogram"` -} - -func (m *Sample) Reset() { *m = Sample{} } -func (*Sample) ProtoMessage() {} -func (*Sample) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{4} -} -func (m *Sample) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Sample) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_Sample.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 *Sample) XXX_Merge(src proto.Message) { - xxx_messageInfo_Sample.Merge(m, src) -} -func (m *Sample) XXX_Size() int { - return m.Size() -} -func (m *Sample) XXX_DiscardUnknown() { - xxx_messageInfo_Sample.DiscardUnknown(m) -} - -var xxx_messageInfo_Sample proto.InternalMessageInfo - -func (m *Sample) GetSample() *cortexpb.Sample { - if m != nil { - return m.Sample - } - return nil -} - -func (m *Sample) GetHistogram() *tripperware.SampleHistogramPair { - if m != nil { - return m.Histogram - } - return nil -} - -type Matrix struct { - SampleStreams []tripperware.SampleStream `protobuf:"bytes,1,rep,name=sampleStreams,proto3" json:"sampleStreams"` -} - -func (m *Matrix) Reset() { *m = Matrix{} } -func (*Matrix) ProtoMessage() {} -func (*Matrix) Descriptor() ([]byte, []int) { - return fileDescriptor_d2ce36475a368033, []int{5} -} -func (m *Matrix) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Matrix) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_Matrix.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 *Matrix) XXX_Merge(src proto.Message) { - xxx_messageInfo_Matrix.Merge(m, src) -} -func (m *Matrix) XXX_Size() int { - return m.Size() -} -func (m *Matrix) XXX_DiscardUnknown() { - xxx_messageInfo_Matrix.DiscardUnknown(m) -} - -var xxx_messageInfo_Matrix proto.InternalMessageInfo - -func (m *Matrix) GetSampleStreams() []tripperware.SampleStream { - if m != nil { - return m.SampleStreams - } - return nil -} - -func init() { - proto.RegisterType((*PrometheusInstantQueryResponse)(nil), "instantquery.PrometheusInstantQueryResponse") - proto.RegisterType((*PrometheusInstantQueryData)(nil), "instantquery.PrometheusInstantQueryData") - proto.RegisterType((*PrometheusInstantQueryResult)(nil), "instantquery.PrometheusInstantQueryResult") - proto.RegisterType((*Vector)(nil), "instantquery.Vector") - proto.RegisterType((*Sample)(nil), "instantquery.Sample") - proto.RegisterType((*Matrix)(nil), "instantquery.Matrix") -} - -func init() { proto.RegisterFile("instantquery.proto", fileDescriptor_d2ce36475a368033) } - -var fileDescriptor_d2ce36475a368033 = []byte{ - // 713 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xc1, 0x6e, 0xd3, 0x4a, - 0x14, 0xf5, 0x34, 0x8d, 0xdb, 0x4c, 0xda, 0xbe, 0xf7, 0xa6, 0x55, 0x5f, 0x5e, 0x55, 0x8d, 0xa3, - 0xe8, 0x21, 0x05, 0x04, 0x8e, 0x14, 0x84, 0x80, 0x25, 0x86, 0xa2, 0x20, 0x40, 0x6d, 0xa7, 0x15, - 0x48, 0xec, 0x26, 0xe9, 0x28, 0x35, 0xc4, 0xb1, 0x99, 0x99, 0xb4, 0xcd, 0x8e, 0x2f, 0x40, 0x7c, - 0x05, 0xe2, 0x53, 0xba, 0xec, 0xb2, 0x62, 0x61, 0xd1, 0x74, 0x83, 0xbc, 0x2a, 0x3b, 0x96, 0xc8, - 0x33, 0xe3, 0xc6, 0x81, 0x02, 0x65, 0xe7, 0xb9, 0xf7, 0x9c, 0x73, 0xef, 0x3d, 0x77, 0x3c, 0x10, - 0xf9, 0x7d, 0x21, 0x69, 0x5f, 0xbe, 0x1e, 0x30, 0x3e, 0x74, 0x23, 0x1e, 0xca, 0x10, 0xcd, 0xe5, - 0x63, 0x2b, 0x4b, 0xdd, 0xb0, 0x1b, 0xaa, 0x44, 0x23, 0xfd, 0xd2, 0x98, 0x95, 0xbb, 0x5d, 0x5f, - 0xee, 0x0e, 0xda, 0x6e, 0x27, 0x0c, 0x1a, 0x9d, 0x90, 0x4b, 0x76, 0x10, 0xf1, 0xf0, 0x25, 0xeb, - 0x48, 0x73, 0x6a, 0x44, 0xaf, 0xba, 0x59, 0xa2, 0x6d, 0x3e, 0x0c, 0xf5, 0xfe, 0x65, 0xa8, 0x69, - 0x6d, 0x9f, 0xf1, 0x86, 0xe4, 0x7e, 0x14, 0x31, 0xbe, 0x4f, 0x39, 0x6b, 0xe4, 0x7a, 0xac, 0x7d, - 0x9d, 0x82, 0x78, 0x83, 0x87, 0x01, 0x93, 0xbb, 0x6c, 0x20, 0x1e, 0xe9, 0x86, 0x37, 0x53, 0x00, - 0x61, 0x22, 0x0a, 0xfb, 0x82, 0xa1, 0x1a, 0xb4, 0xb7, 0x24, 0x95, 0x03, 0x51, 0x01, 0x55, 0x50, - 0x2f, 0x79, 0x30, 0x89, 0x1d, 0x5b, 0xa8, 0x08, 0x31, 0x19, 0xb4, 0x0d, 0xa7, 0x1f, 0x50, 0x49, - 0x2b, 0x53, 0x55, 0x50, 0x2f, 0x37, 0xeb, 0xee, 0x84, 0x1b, 0x17, 0xeb, 0xa7, 0x78, 0x6f, 0xf9, - 0x30, 0x76, 0xac, 0x24, 0x76, 0x16, 0x76, 0xa8, 0xa4, 0xd7, 0xc3, 0xc0, 0x97, 0x2c, 0x88, 0xe4, - 0x90, 0x28, 0x35, 0x74, 0x0b, 0x96, 0xd6, 0x38, 0x0f, 0xf9, 0xf6, 0x30, 0x62, 0x95, 0x82, 0x2a, - 0xfe, 0x6f, 0x12, 0x3b, 0x8b, 0x2c, 0x0b, 0xe6, 0x18, 0x63, 0x24, 0xba, 0x0a, 0x8b, 0xea, 0x50, - 0x99, 0x56, 0x94, 0xc5, 0x24, 0x76, 0xfe, 0x52, 0x94, 0x1c, 0x5c, 0x23, 0xd0, 0x43, 0x38, 0xd3, - 0x62, 0x74, 0x87, 0x71, 0x51, 0x29, 0x56, 0x0b, 0xf5, 0x72, 0xf3, 0x8a, 0x9b, 0x73, 0x2a, 0xd7, - 0x79, 0xe6, 0x86, 0x46, 0x7b, 0xc5, 0x24, 0x76, 0xc0, 0x0d, 0x92, 0x91, 0x51, 0x13, 0xce, 0x3e, - 0xa7, 0xbc, 0xef, 0xf7, 0xbb, 0xa2, 0x62, 0x57, 0x0b, 0xf5, 0x92, 0xb7, 0x9c, 0xc4, 0x0e, 0xda, - 0x37, 0xb1, 0x5c, 0xe1, 0x73, 0x5c, 0xed, 0x0b, 0x80, 0x2b, 0x3f, 0xb7, 0x06, 0xb9, 0x10, 0x12, - 0x26, 0x06, 0x3d, 0xa9, 0xa6, 0xd7, 0xd6, 0x2f, 0x24, 0xb1, 0x03, 0xf9, 0x79, 0x94, 0xe4, 0x10, - 0x88, 0x40, 0x5b, 0x9f, 0xcc, 0x12, 0xae, 0x5d, 0x66, 0x09, 0x9a, 0xe1, 0x2d, 0x98, 0x35, 0xd8, - 0x5a, 0x9b, 0x18, 0x25, 0xb4, 0x0e, 0x8b, 0xe9, 0xa2, 0x85, 0x32, 0xbf, 0xdc, 0xfc, 0xff, 0x37, - 0xe6, 0xa4, 0x97, 0x41, 0x68, 0xbf, 0x15, 0x2d, 0xef, 0xb7, 0x0a, 0xd4, 0xde, 0x03, 0xb8, 0xfa, - 0xab, 0x4e, 0x90, 0x0b, 0xed, 0x3d, 0xd6, 0x91, 0x21, 0x57, 0x13, 0x97, 0x9b, 0x4b, 0x93, 0x53, - 0x3c, 0x53, 0xb9, 0x96, 0x45, 0x0c, 0x0a, 0xad, 0xc2, 0x59, 0x4e, 0xf7, 0xbd, 0xa1, 0x64, 0x42, - 0xcd, 0x3d, 0xd7, 0xb2, 0xc8, 0x79, 0x24, 0x55, 0x0b, 0xa8, 0xe4, 0xfe, 0x81, 0x19, 0xe0, 0x3b, - 0xb5, 0xa7, 0x2a, 0x97, 0xaa, 0x69, 0x94, 0x37, 0x0b, 0x8d, 0x03, 0xb5, 0x3b, 0xd0, 0xd6, 0xb5, - 0x90, 0x0b, 0x67, 0x04, 0x0d, 0xa2, 0x1e, 0x4b, 0xef, 0x7f, 0xe1, 0x47, 0x91, 0x2d, 0x95, 0x24, - 0x19, 0xa8, 0xf6, 0x76, 0x0a, 0xda, 0x3a, 0x86, 0x0e, 0xa0, 0xdd, 0xa3, 0x6d, 0xd6, 0xcb, 0x98, - 0x8b, 0x6e, 0xf6, 0x27, 0xbb, 0x4f, 0xd2, 0xf8, 0x06, 0xf5, 0xb9, 0xf7, 0x38, 0xf5, 0xfe, 0x63, - 0xec, 0xfc, 0xd1, 0x4b, 0xa0, 0xf9, 0xf7, 0x76, 0x68, 0x24, 0x19, 0x4f, 0x17, 0x17, 0x30, 0xc9, - 0xfd, 0x0e, 0x31, 0xf5, 0xd0, 0x6d, 0x68, 0xeb, 0x7e, 0xcc, 0x65, 0xf8, 0x7b, 0x5c, 0x59, 0xf7, - 0xe6, 0xcd, 0x1f, 0xc6, 0x0e, 0x48, 0x62, 0xa7, 0xb8, 0x47, 0x7b, 0x03, 0x46, 0x0c, 0x1c, 0x6d, - 0xc2, 0xd2, 0xae, 0x2f, 0x64, 0xd8, 0xe5, 0x34, 0x30, 0xa6, 0x55, 0x27, 0xb6, 0xae, 0xe9, 0xad, - 0x0c, 0xa3, 0x46, 0xf8, 0xc7, 0x68, 0x8d, 0xa9, 0x64, 0xfc, 0x59, 0x5b, 0x87, 0xb6, 0x36, 0x1a, - 0xad, 0xc1, 0x79, 0x5d, 0x66, 0x4b, 0x72, 0x46, 0x83, 0xcc, 0x96, 0xff, 0x2e, 0x28, 0xa0, 0x11, - 0xde, 0x74, 0x6a, 0x0e, 0x99, 0x64, 0x79, 0xde, 0xd1, 0x09, 0xb6, 0x8e, 0x4f, 0xb0, 0x75, 0x76, - 0x82, 0xc1, 0x9b, 0x11, 0x06, 0x1f, 0x46, 0x18, 0x1c, 0x8e, 0x30, 0x38, 0x1a, 0x61, 0xf0, 0x69, - 0x84, 0xc1, 0xe7, 0x11, 0xb6, 0xce, 0x46, 0x18, 0xbc, 0x3b, 0xc5, 0xd6, 0xd1, 0x29, 0xb6, 0x8e, - 0x4f, 0xb1, 0xf5, 0x62, 0xe2, 0x35, 0x6e, 0xdb, 0xea, 0xf9, 0xbb, 0xf9, 0x2d, 0x00, 0x00, 0xff, - 0xff, 0x3f, 0x4a, 0x4a, 0x48, 0xb8, 0x05, 0x00, 0x00, -} - -func (this *PrometheusInstantQueryResponse) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryResponse) - if !ok { - that2, ok := that.(PrometheusInstantQueryResponse) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Status != that1.Status { - return false - } - if !this.Data.Equal(&that1.Data) { - return false - } - if this.ErrorType != that1.ErrorType { - return false - } - if this.Error != that1.Error { - return false - } - if len(this.Headers) != len(that1.Headers) { - return false - } - for i := range this.Headers { - if !this.Headers[i].Equal(that1.Headers[i]) { - return false - } - } - if len(this.Warnings) != len(that1.Warnings) { - return false - } - for i := range this.Warnings { - if this.Warnings[i] != that1.Warnings[i] { - return false - } - } - return true -} -func (this *PrometheusInstantQueryData) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryData) - if !ok { - that2, ok := that.(PrometheusInstantQueryData) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.ResultType != that1.ResultType { - return false - } - if !this.Result.Equal(&that1.Result) { - return false - } - if !this.Stats.Equal(that1.Stats) { - return false - } - return true -} -func (this *PrometheusInstantQueryResult) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryResult) - if !ok { - that2, ok := that.(PrometheusInstantQueryResult) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if that1.Result == nil { - if this.Result != nil { - return false - } - } else if this.Result == nil { - return false - } else if !this.Result.Equal(that1.Result) { - return false - } - return true -} -func (this *PrometheusInstantQueryResult_Vector) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryResult_Vector) - if !ok { - that2, ok := that.(PrometheusInstantQueryResult_Vector) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if !this.Vector.Equal(that1.Vector) { - return false - } - return true -} -func (this *PrometheusInstantQueryResult_RawBytes) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryResult_RawBytes) - if !ok { - that2, ok := that.(PrometheusInstantQueryResult_RawBytes) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if !bytes.Equal(this.RawBytes, that1.RawBytes) { - return false - } - return true -} -func (this *PrometheusInstantQueryResult_Matrix) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusInstantQueryResult_Matrix) - if !ok { - that2, ok := that.(PrometheusInstantQueryResult_Matrix) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if !this.Matrix.Equal(that1.Matrix) { - return false - } - return true -} -func (this *Vector) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*Vector) - if !ok { - that2, ok := that.(Vector) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if len(this.Samples) != len(that1.Samples) { - return false - } - for i := range this.Samples { - if !this.Samples[i].Equal(that1.Samples[i]) { - return false - } - } - return true -} -func (this *Sample) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*Sample) - if !ok { - that2, ok := that.(Sample) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if len(this.Labels) != len(that1.Labels) { - return false - } - for i := range this.Labels { - if !this.Labels[i].Equal(that1.Labels[i]) { - return false - } - } - if !this.Sample.Equal(that1.Sample) { - return false - } - if !this.Histogram.Equal(that1.Histogram) { - return false - } - return true -} -func (this *Matrix) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*Matrix) - if !ok { - that2, ok := that.(Matrix) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if len(this.SampleStreams) != len(that1.SampleStreams) { - return false - } - for i := range this.SampleStreams { - if !this.SampleStreams[i].Equal(&that1.SampleStreams[i]) { - return false - } - } - return true -} -func (this *PrometheusInstantQueryResponse) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 10) - s = append(s, "&instantquery.PrometheusInstantQueryResponse{") - s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") - s = append(s, "Data: "+strings.Replace(this.Data.GoString(), `&`, ``, 1)+",\n") - s = append(s, "ErrorType: "+fmt.Sprintf("%#v", this.ErrorType)+",\n") - s = append(s, "Error: "+fmt.Sprintf("%#v", this.Error)+",\n") - if this.Headers != nil { - s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") - } - s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") - s = append(s, "}") - return strings.Join(s, "") -} -func (this *PrometheusInstantQueryData) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 7) - s = append(s, "&instantquery.PrometheusInstantQueryData{") - s = append(s, "ResultType: "+fmt.Sprintf("%#v", this.ResultType)+",\n") - s = append(s, "Result: "+strings.Replace(this.Result.GoString(), `&`, ``, 1)+",\n") - if this.Stats != nil { - s = append(s, "Stats: "+fmt.Sprintf("%#v", this.Stats)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *PrometheusInstantQueryResult) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 7) - s = append(s, "&instantquery.PrometheusInstantQueryResult{") - if this.Result != nil { - s = append(s, "Result: "+fmt.Sprintf("%#v", this.Result)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *PrometheusInstantQueryResult_Vector) GoString() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&instantquery.PrometheusInstantQueryResult_Vector{` + - `Vector:` + fmt.Sprintf("%#v", this.Vector) + `}`}, ", ") - return s -} -func (this *PrometheusInstantQueryResult_RawBytes) GoString() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&instantquery.PrometheusInstantQueryResult_RawBytes{` + - `RawBytes:` + fmt.Sprintf("%#v", this.RawBytes) + `}`}, ", ") - return s -} -func (this *PrometheusInstantQueryResult_Matrix) GoString() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&instantquery.PrometheusInstantQueryResult_Matrix{` + - `Matrix:` + fmt.Sprintf("%#v", this.Matrix) + `}`}, ", ") - return s -} -func (this *Vector) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 5) - s = append(s, "&instantquery.Vector{") - if this.Samples != nil { - s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *Sample) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 7) - s = append(s, "&instantquery.Sample{") - s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") - if this.Sample != nil { - s = append(s, "Sample: "+fmt.Sprintf("%#v", this.Sample)+",\n") - } - if this.Histogram != nil { - s = append(s, "Histogram: "+fmt.Sprintf("%#v", this.Histogram)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *Matrix) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 5) - s = append(s, "&instantquery.Matrix{") - if this.SampleStreams != nil { - vs := make([]*tripperware.SampleStream, len(this.SampleStreams)) - for i := range vs { - vs[i] = &this.SampleStreams[i] - } - s = append(s, "SampleStreams: "+fmt.Sprintf("%#v", vs)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func valueToGoStringInstantquery(v interface{}, typ string) string { - rv := reflect.ValueOf(v) - if rv.IsNil() { - return "nil" - } - pv := reflect.Indirect(rv).Interface() - return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) -} -func (m *PrometheusInstantQueryResponse) 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 *PrometheusInstantQueryResponse) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusInstantQueryResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Warnings) > 0 { - for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Warnings[iNdEx]) - copy(dAtA[i:], m.Warnings[iNdEx]) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.Warnings[iNdEx]))) - i-- - dAtA[i] = 0x32 - } - } - if len(m.Headers) > 0 { - for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Headers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x2a - } - } - if len(m.Error) > 0 { - i -= len(m.Error) - copy(dAtA[i:], m.Error) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.Error))) - i-- - dAtA[i] = 0x22 - } - if len(m.ErrorType) > 0 { - i -= len(m.ErrorType) - copy(dAtA[i:], m.ErrorType) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.ErrorType))) - i-- - dAtA[i] = 0x1a - } - { - size, err := m.Data.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - if len(m.Status) > 0 { - i -= len(m.Status) - copy(dAtA[i:], m.Status) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.Status))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *PrometheusInstantQueryData) 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 *PrometheusInstantQueryData) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusInstantQueryData) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Stats != nil { - { - size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - { - size, err := m.Result.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - if len(m.ResultType) > 0 { - i -= len(m.ResultType) - copy(dAtA[i:], m.ResultType) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.ResultType))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *PrometheusInstantQueryResult) 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 *PrometheusInstantQueryResult) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusInstantQueryResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Result != nil { - { - size := m.Result.Size() - i -= size - if _, err := m.Result.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } - } - return len(dAtA) - i, nil -} - -func (m *PrometheusInstantQueryResult_Vector) MarshalTo(dAtA []byte) (int, error) { - return m.MarshalToSizedBuffer(dAtA[:m.Size()]) -} - -func (m *PrometheusInstantQueryResult_Vector) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.Vector != nil { - { - size, err := m.Vector.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} -func (m *PrometheusInstantQueryResult_RawBytes) MarshalTo(dAtA []byte) (int, error) { - return m.MarshalToSizedBuffer(dAtA[:m.Size()]) -} - -func (m *PrometheusInstantQueryResult_RawBytes) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.RawBytes != nil { - i -= len(m.RawBytes) - copy(dAtA[i:], m.RawBytes) - i = encodeVarintInstantquery(dAtA, i, uint64(len(m.RawBytes))) - i-- - dAtA[i] = 0x12 - } - return len(dAtA) - i, nil -} -func (m *PrometheusInstantQueryResult_Matrix) MarshalTo(dAtA []byte) (int, error) { - return m.MarshalToSizedBuffer(dAtA[:m.Size()]) -} - -func (m *PrometheusInstantQueryResult_Matrix) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.Matrix != nil { - { - size, err := m.Matrix.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - return len(dAtA) - i, nil -} -func (m *Vector) 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 *Vector) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Vector) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Samples) > 0 { - for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - } - return len(dAtA) - i, nil -} - -func (m *Sample) 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 *Sample) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Histogram != nil { - { - size, err := m.Histogram.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if m.Sample != nil { - { - size, err := m.Sample.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - if len(m.Labels) > 0 { - for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { - { - size := m.Labels[iNdEx].Size() - i -= size - if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - } - return len(dAtA) - i, nil -} - -func (m *Matrix) 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 *Matrix) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Matrix) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.SampleStreams) > 0 { - for iNdEx := len(m.SampleStreams) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.SampleStreams[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintInstantquery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - } - return len(dAtA) - i, nil -} - -func encodeVarintInstantquery(dAtA []byte, offset int, v uint64) int { - offset -= sovInstantquery(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return base -} -func (m *PrometheusInstantQueryResponse) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Status) - if l > 0 { - n += 1 + l + sovInstantquery(uint64(l)) - } - l = m.Data.Size() - n += 1 + l + sovInstantquery(uint64(l)) - l = len(m.ErrorType) - if l > 0 { - n += 1 + l + sovInstantquery(uint64(l)) - } - l = len(m.Error) - if l > 0 { - n += 1 + l + sovInstantquery(uint64(l)) - } - if len(m.Headers) > 0 { - for _, e := range m.Headers { - l = e.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - } - if len(m.Warnings) > 0 { - for _, s := range m.Warnings { - l = len(s) - n += 1 + l + sovInstantquery(uint64(l)) - } - } - return n -} - -func (m *PrometheusInstantQueryData) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.ResultType) - if l > 0 { - n += 1 + l + sovInstantquery(uint64(l)) - } - l = m.Result.Size() - n += 1 + l + sovInstantquery(uint64(l)) - if m.Stats != nil { - l = m.Stats.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - return n -} - -func (m *PrometheusInstantQueryResult) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Result != nil { - n += m.Result.Size() - } - return n -} - -func (m *PrometheusInstantQueryResult_Vector) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Vector != nil { - l = m.Vector.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - return n -} -func (m *PrometheusInstantQueryResult_RawBytes) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.RawBytes != nil { - l = len(m.RawBytes) - n += 1 + l + sovInstantquery(uint64(l)) - } - return n -} -func (m *PrometheusInstantQueryResult_Matrix) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Matrix != nil { - l = m.Matrix.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - return n -} -func (m *Vector) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Samples) > 0 { - for _, e := range m.Samples { - l = e.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - } - return n -} - -func (m *Sample) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Labels) > 0 { - for _, e := range m.Labels { - l = e.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - } - if m.Sample != nil { - l = m.Sample.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - if m.Histogram != nil { - l = m.Histogram.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - return n -} - -func (m *Matrix) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.SampleStreams) > 0 { - for _, e := range m.SampleStreams { - l = e.Size() - n += 1 + l + sovInstantquery(uint64(l)) - } - } - return n -} - -func sovInstantquery(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozInstantquery(x uint64) (n int) { - return sovInstantquery(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (this *PrometheusInstantQueryResponse) String() string { - if this == nil { - return "nil" - } - repeatedStringForHeaders := "[]*PrometheusResponseHeader{" - for _, f := range this.Headers { - repeatedStringForHeaders += strings.Replace(fmt.Sprintf("%v", f), "PrometheusResponseHeader", "tripperware.PrometheusResponseHeader", 1) + "," - } - repeatedStringForHeaders += "}" - s := strings.Join([]string{`&PrometheusInstantQueryResponse{`, - `Status:` + fmt.Sprintf("%v", this.Status) + `,`, - `Data:` + strings.Replace(strings.Replace(this.Data.String(), "PrometheusInstantQueryData", "PrometheusInstantQueryData", 1), `&`, ``, 1) + `,`, - `ErrorType:` + fmt.Sprintf("%v", this.ErrorType) + `,`, - `Error:` + fmt.Sprintf("%v", this.Error) + `,`, - `Headers:` + repeatedStringForHeaders + `,`, - `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusInstantQueryData) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&PrometheusInstantQueryData{`, - `ResultType:` + fmt.Sprintf("%v", this.ResultType) + `,`, - `Result:` + strings.Replace(strings.Replace(this.Result.String(), "PrometheusInstantQueryResult", "PrometheusInstantQueryResult", 1), `&`, ``, 1) + `,`, - `Stats:` + strings.Replace(fmt.Sprintf("%v", this.Stats), "PrometheusResponseStats", "tripperware.PrometheusResponseStats", 1) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusInstantQueryResult) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&PrometheusInstantQueryResult{`, - `Result:` + fmt.Sprintf("%v", this.Result) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusInstantQueryResult_Vector) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&PrometheusInstantQueryResult_Vector{`, - `Vector:` + strings.Replace(fmt.Sprintf("%v", this.Vector), "Vector", "Vector", 1) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusInstantQueryResult_RawBytes) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&PrometheusInstantQueryResult_RawBytes{`, - `RawBytes:` + fmt.Sprintf("%v", this.RawBytes) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusInstantQueryResult_Matrix) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&PrometheusInstantQueryResult_Matrix{`, - `Matrix:` + strings.Replace(fmt.Sprintf("%v", this.Matrix), "Matrix", "Matrix", 1) + `,`, - `}`, - }, "") - return s -} -func (this *Vector) String() string { - if this == nil { - return "nil" - } - repeatedStringForSamples := "[]*Sample{" - for _, f := range this.Samples { - repeatedStringForSamples += strings.Replace(f.String(), "Sample", "Sample", 1) + "," - } - repeatedStringForSamples += "}" - s := strings.Join([]string{`&Vector{`, - `Samples:` + repeatedStringForSamples + `,`, - `}`, - }, "") - return s -} -func (this *Sample) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&Sample{`, - `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, - `Sample:` + strings.Replace(fmt.Sprintf("%v", this.Sample), "Sample", "cortexpb.Sample", 1) + `,`, - `Histogram:` + strings.Replace(fmt.Sprintf("%v", this.Histogram), "SampleHistogramPair", "tripperware.SampleHistogramPair", 1) + `,`, - `}`, - }, "") - return s -} -func (this *Matrix) String() string { - if this == nil { - return "nil" - } - repeatedStringForSampleStreams := "[]SampleStream{" - for _, f := range this.SampleStreams { - repeatedStringForSampleStreams += fmt.Sprintf("%v", f) + "," - } - repeatedStringForSampleStreams += "}" - s := strings.Join([]string{`&Matrix{`, - `SampleStreams:` + repeatedStringForSampleStreams + `,`, - `}`, - }, "") - return s -} -func valueToStringInstantquery(v interface{}) string { - rv := reflect.ValueOf(v) - if rv.IsNil() { - return "nil" - } - pv := reflect.Indirect(rv).Interface() - return fmt.Sprintf("*%v", pv) -} -func (m *PrometheusInstantQueryResponse) 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 ErrIntOverflowInstantquery - } - 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: PrometheusInstantQueryResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusInstantQueryResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Status = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.Data.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ErrorType", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ErrorType = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Error = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Headers = append(m.Headers, &tripperware.PrometheusResponseHeader{}) - if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *PrometheusInstantQueryData) 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 ErrIntOverflowInstantquery - } - 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: PrometheusInstantQueryData: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusInstantQueryData: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultType", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ResultType = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.Result.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Stats == nil { - m.Stats = &tripperware.PrometheusResponseStats{} - } - if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *PrometheusInstantQueryResult) 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 ErrIntOverflowInstantquery - } - 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: PrometheusInstantQueryResult: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusInstantQueryResult: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Vector", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := &Vector{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - m.Result = &PrometheusInstantQueryResult_Vector{v} - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RawBytes", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := make([]byte, postIndex-iNdEx) - copy(v, dAtA[iNdEx:postIndex]) - m.Result = &PrometheusInstantQueryResult_RawBytes{v} - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Matrix", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := &Matrix{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - m.Result = &PrometheusInstantQueryResult_Matrix{v} - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Vector) 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 ErrIntOverflowInstantquery - } - 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: Vector: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Vector: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Samples = append(m.Samples, &Sample{}) - if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Sample) 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 ErrIntOverflowInstantquery - } - 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: Sample: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Sample: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Labels = append(m.Labels, github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{}) - if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Sample", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Sample == nil { - m.Sample = &cortexpb.Sample{} - } - if err := m.Sample.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Histogram", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Histogram == nil { - m.Histogram = &tripperware.SampleHistogramPair{} - } - if err := m.Histogram.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Matrix) 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 ErrIntOverflowInstantquery - } - 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: Matrix: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Matrix: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SampleStreams", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowInstantquery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthInstantquery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthInstantquery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.SampleStreams = append(m.SampleStreams, tripperware.SampleStream{}) - if err := m.SampleStreams[len(m.SampleStreams)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipInstantquery(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthInstantquery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func skipInstantquery(dAtA []byte) (n int, err error) { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowInstantquery - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowInstantquery - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - iNdEx++ - if dAtA[iNdEx-1] < 0x80 { - break - } - } - return iNdEx, nil - case 1: - iNdEx += 8 - return iNdEx, nil - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowInstantquery - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if length < 0 { - return 0, ErrInvalidLengthInstantquery - } - iNdEx += length - if iNdEx < 0 { - return 0, ErrInvalidLengthInstantquery - } - return iNdEx, nil - case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowInstantquery - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipInstantquery(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - if iNdEx < 0 { - return 0, ErrInvalidLengthInstantquery - } - } - return iNdEx, nil - case 4: - return iNdEx, nil - case 5: - iNdEx += 4 - return iNdEx, nil - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - } - panic("unreachable") -} - -var ( - ErrInvalidLengthInstantquery = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowInstantquery = fmt.Errorf("proto: integer overflow") -) diff --git a/pkg/querier/tripperware/instantquery/instantquery.proto b/pkg/querier/tripperware/instantquery/instantquery.proto deleted file mode 100644 index d67b1ad061..0000000000 --- a/pkg/querier/tripperware/instantquery/instantquery.proto +++ /dev/null @@ -1,50 +0,0 @@ -syntax = "proto3"; - -package instantquery; - -option go_package = "instantquery"; - -import "gogoproto/gogo.proto"; -import "github.com/cortexproject/cortex/pkg/cortexpb/cortex.proto"; -import "github.com/cortexproject/cortex/pkg/querier/tripperware/query.proto"; - -option (gogoproto.marshaler_all) = true; -option (gogoproto.unmarshaler_all) = true; - - -message PrometheusInstantQueryResponse { - string Status = 1 [(gogoproto.jsontag) = "status"]; - PrometheusInstantQueryData Data = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "data,omitempty"]; - string ErrorType = 3 [(gogoproto.jsontag) = "errorType,omitempty"]; - string Error = 4 [(gogoproto.jsontag) = "error,omitempty"]; - repeated tripperware.PrometheusResponseHeader Headers = 5 [(gogoproto.jsontag) = "-"]; - repeated string Warnings = 6 [(gogoproto.jsontag) = "warnings,omitempty"]; -} - -message PrometheusInstantQueryData { - string ResultType = 1 [(gogoproto.jsontag) = "resultType"]; - PrometheusInstantQueryResult Result = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "result"]; - tripperware.PrometheusResponseStats stats = 3 [(gogoproto.jsontag) = "stats,omitempty"]; -} - -message PrometheusInstantQueryResult { - oneof result { - Vector vector = 1; - bytes rawBytes = 2; - Matrix matrix = 3; - } -} - -message Vector { - repeated Sample samples = 1; -} - -message Sample { - repeated cortexpb.LabelPair labels = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "metric", (gogoproto.customtype) = "github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter"]; - cortexpb.Sample sample = 2 [(gogoproto.nullable) = true, (gogoproto.jsontag) = "value"]; - tripperware.SampleHistogramPair histogram = 3 [(gogoproto.nullable) = true, (gogoproto.jsontag) = "histogram"]; -} - -message Matrix { - repeated tripperware.SampleStream sampleStreams = 1 [(gogoproto.nullable) = false]; -} diff --git a/pkg/querier/tripperware/instantquery/limits_test.go b/pkg/querier/tripperware/instantquery/limits_test.go index 4bf1cfac3c..e209d366f3 100644 --- a/pkg/querier/tripperware/instantquery/limits_test.go +++ b/pkg/querier/tripperware/instantquery/limits_test.go @@ -71,12 +71,12 @@ func TestLimitsMiddleware_MaxQueryLength(t *testing.T) { testData := testData t.Run(testName, func(t *testing.T) { t.Parallel() - req := &PrometheusRequest{Query: testData.query} + req := &tripperware.PrometheusRequest{Query: testData.query} limits := &mockLimits{maxQueryLength: testData.maxQueryLength} middleware := NewLimitsMiddleware(limits, 5*time.Minute) - innerRes := NewEmptyPrometheusInstantQueryResponse() + innerRes := tripperware.NewEmptyPrometheusResponse(true) inner := &mockHandler{} inner.On("Do", mock.Anything, mock.Anything).Return(innerRes, nil) diff --git a/pkg/querier/tripperware/merge.go b/pkg/querier/tripperware/merge.go index 2ae5a9793c..6266946b4a 100644 --- a/pkg/querier/tripperware/merge.go +++ b/pkg/querier/tripperware/merge.go @@ -1,13 +1,335 @@ package tripperware import ( + "context" + "fmt" "sort" + "github.com/prometheus/common/model" + promqlparser "github.com/prometheus/prometheus/promql/parser" + "github.com/thanos-io/thanos/pkg/strutil" + "github.com/cortexproject/cortex/pkg/cortexpb" ) -// MergeSampleStreams deduplicates sample streams using a map. -func MergeSampleStreams(output map[string]SampleStream, sampleStreams []SampleStream) { +const StatusSuccess = "success" + +type byFirstTime []*PrometheusResponse + +func (a byFirstTime) Len() int { return len(a) } +func (a byFirstTime) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a byFirstTime) Less(i, j int) bool { return a[i].minTime() < a[j].minTime() } + +func (resp *PrometheusResponse) minTime() int64 { + data := resp.GetData() + res := data.GetResult() + // minTime should only be called when the response is fron range query. + matrix := res.GetMatrix() + sampleStreams := matrix.GetSampleStreams() + if len(sampleStreams) == 0 { + return -1 + } + if len(sampleStreams[0].Samples) == 0 { + return -1 + } + return sampleStreams[0].Samples[0].TimestampMs +} + +// MergeResponse merges multiple Response into one. +func MergeResponse(ctx context.Context, sumStats bool, req Request, responses ...Response) (Response, error) { + if len(responses) == 1 { + return responses[0], nil + } + promResponses := make([]*PrometheusResponse, 0, len(responses)) + warnings := make([][]string, 0, len(responses)) + for _, resp := range responses { + promResponses = append(promResponses, resp.(*PrometheusResponse)) + if w := resp.(*PrometheusResponse).Warnings; w != nil { + warnings = append(warnings, w) + } + } + + // Check if it is a range query. Range query passed req as nil since + // we only use request when result type is a vector. + if req == nil { + sort.Sort(byFirstTime(promResponses)) + } + var data PrometheusData + // For now, we only shard queries that returns a vector. + switch promResponses[0].Data.ResultType { + case model.ValVector.String(): + v, err := vectorMerge(ctx, req, promResponses) + if err != nil { + return nil, err + } + data = PrometheusData{ + ResultType: model.ValVector.String(), + Result: PrometheusQueryResult{ + Result: &PrometheusQueryResult_Vector{ + Vector: v, + }, + }, + Stats: statsMerge(sumStats, promResponses), + } + case model.ValMatrix.String(): + sampleStreams, err := matrixMerge(ctx, promResponses) + if err != nil { + return nil, err + } + + data = PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusQueryResult{ + Result: &PrometheusQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: sampleStreams, + }, + }, + }, + Stats: statsMerge(sumStats, promResponses), + } + default: + return nil, fmt.Errorf("unexpected result type: %s", promResponses[0].Data.ResultType) + } + + res := &PrometheusResponse{ + Status: StatusSuccess, + Data: data, + Warnings: strutil.MergeUnsortedSlices(warnings...), + } + return res, nil +} + +func matrixMerge(ctx context.Context, resps []*PrometheusResponse) ([]SampleStream, error) { + output := make(map[string]SampleStream) + for _, resp := range resps { + if err := ctx.Err(); err != nil { + return nil, err + } + if resp == nil { + continue + } + if resp.Data.Result.GetMatrix() == nil { + continue + } + mergeSampleStreams(output, resp.Data.Result.GetMatrix().GetSampleStreams()) + } + + keys := make([]string, 0, len(output)) + for key := range output { + keys = append(keys, key) + } + sort.Strings(keys) + + result := make([]SampleStream, 0, len(output)) + for _, key := range keys { + result = append(result, output[key]) + } + + return result, nil +} + +func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) (*Vector, error) { + output := map[string]*Sample{} + metrics := []string{} // Used to preserve the order for topk and bottomk. + sortPlan, err := sortPlanForQuery(req.GetQuery()) + if err != nil { + return nil, err + } + buf := make([]byte, 0, 1024) + for _, resp := range resps { + if err = ctx.Err(); err != nil { + return nil, err + } + if resp == nil { + continue + } + // Merge vector result samples only. Skip other types such as + // string, scalar as those are not sharable. + if resp.Data.Result.GetVector() == nil { + continue + } + for _, sample := range resp.Data.Result.GetVector().Samples { + s := sample + if s == nil { + continue + } + metric := string(cortexpb.FromLabelAdaptersToLabels(sample.Labels).Bytes(buf)) + if existingSample, ok := output[metric]; !ok { + output[metric] = s + metrics = append(metrics, metric) // Preserve the order of metric. + } else if existingSample.GetTimestampMs() < s.GetTimestampMs() { + // Choose the latest sample if we see overlap. + output[metric] = s + } + } + } + + result := &Vector{ + Samples: make([]*Sample, 0, len(output)), + } + + if len(output) == 0 { + return result, nil + } + + if sortPlan == mergeOnly { + for _, k := range metrics { + result.Samples = append(result.Samples, output[k]) + } + return result, nil + } + + samples := make([]*pair, 0, len(output)) + for k, v := range output { + samples = append(samples, &pair{ + metric: k, + s: v, + }) + } + + // TODO: What if we have mixed float and histogram samples in the response? + // Then the sorting behavior is undefined. Prometheus doesn't handle it. + sort.Slice(samples, func(i, j int) bool { + // Order is determined by vector. + switch sortPlan { + case sortByValuesAsc: + return getSortValueFromPair(samples, i) < getSortValueFromPair(samples, j) + case sortByValuesDesc: + return getSortValueFromPair(samples, i) > getSortValueFromPair(samples, j) + } + return samples[i].metric < samples[j].metric + }) + + for _, p := range samples { + result.Samples = append(result.Samples, p.s) + } + return result, nil +} + +// statsMerge merge the stats from 2 responses this function is similar to matrixMerge +func statsMerge(shouldSumStats bool, resps []*PrometheusResponse) *PrometheusResponseStats { + output := map[int64]*PrometheusResponseQueryableSamplesStatsPerStep{} + hasStats := false + for _, resp := range resps { + if resp.Data.Stats == nil { + continue + } + + hasStats = true + if resp.Data.Stats.Samples == nil { + continue + } + + for _, s := range resp.Data.Stats.Samples.TotalQueryableSamplesPerStep { + if shouldSumStats { + if stats, ok := output[s.GetTimestampMs()]; ok { + stats.Value += s.Value + } else { + output[s.GetTimestampMs()] = s + } + } else { + output[s.GetTimestampMs()] = s + } + } + } + + if !hasStats { + return nil + } + keys := make([]int64, 0, len(output)) + for key := range output { + keys = append(keys, key) + } + + sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) + + result := &PrometheusResponseStats{Samples: &PrometheusResponseSamplesStats{}} + for _, key := range keys { + result.Samples.TotalQueryableSamplesPerStep = append(result.Samples.TotalQueryableSamplesPerStep, output[key]) + result.Samples.TotalQueryableSamples += output[key].Value + } + + return result +} + +type sortPlan int + +const ( + mergeOnly sortPlan = 0 + sortByValuesAsc sortPlan = 1 + sortByValuesDesc sortPlan = 2 + sortByLabels sortPlan = 3 +) + +type pair struct { + metric string + s *Sample +} + +// getSortValueFromPair gets the float value used for sorting from samples. +// If float sample, use sample value. If histogram sample, use histogram sum. +// This is the same behavior as Prometheus https://github.com/prometheus/prometheus/blob/v2.53.0/promql/functions.go#L1595. +func getSortValueFromPair(samples []*pair, i int) float64 { + if samples[i].s.Histogram != nil { + return samples[i].s.Histogram.Histogram.Sum + } + // Impossible to have both histogram and sample nil. + return samples[i].s.Sample.Value +} + +func sortPlanForQuery(q string) (sortPlan, error) { + expr, err := promqlparser.ParseExpr(q) + if err != nil { + return 0, err + } + // Check if the root expression is topk or bottomk + if aggr, ok := expr.(*promqlparser.AggregateExpr); ok { + if aggr.Op == promqlparser.TOPK || aggr.Op == promqlparser.BOTTOMK { + return mergeOnly, nil + } + } + checkForSort := func(expr promqlparser.Expr) (sortAsc, sortDesc bool) { + if n, ok := expr.(*promqlparser.Call); ok { + if n.Func != nil { + if n.Func.Name == "sort" { + sortAsc = true + } + if n.Func.Name == "sort_desc" { + sortDesc = true + } + } + } + return sortAsc, sortDesc + } + // Check the root expression for sort + if sortAsc, sortDesc := checkForSort(expr); sortAsc || sortDesc { + if sortAsc { + return sortByValuesAsc, nil + } + return sortByValuesDesc, nil + } + + // If the root expression is a binary expression, check the LHS and RHS for sort + if bin, ok := expr.(*promqlparser.BinaryExpr); ok { + if sortAsc, sortDesc := checkForSort(bin.LHS); sortAsc || sortDesc { + if sortAsc { + return sortByValuesAsc, nil + } + return sortByValuesDesc, nil + } + if sortAsc, sortDesc := checkForSort(bin.RHS); sortAsc || sortDesc { + if sortAsc { + return sortByValuesAsc, nil + } + return sortByValuesDesc, nil + } + } + return sortByLabels, nil +} + +// mergeSampleStreams deduplicates sample streams using a map. +func mergeSampleStreams(output map[string]SampleStream, sampleStreams []SampleStream) { buf := make([]byte, 0, 1024) for _, stream := range sampleStreams { metric := string(cortexpb.FromLabelAdaptersToLabels(stream.Labels).Bytes(buf)) diff --git a/pkg/querier/tripperware/merge_test.go b/pkg/querier/tripperware/merge_test.go index 124cfd08b5..6a51ca7072 100644 --- a/pkg/querier/tripperware/merge_test.go +++ b/pkg/querier/tripperware/merge_test.go @@ -363,7 +363,7 @@ func TestMergeSampleStreams(t *testing.T) { t.Run(tc.name, func(t *testing.T) { t.Parallel() output := make(map[string]SampleStream) - MergeSampleStreams(output, tc.sampleStreams) + mergeSampleStreams(output, tc.sampleStreams) assert.Equal(t, tc.expectedOutput, output) }) } @@ -596,3 +596,69 @@ func TestSliceHistograms(t *testing.T) { }) } } + +func Test_sortPlanForQuery(t *testing.T) { + tc := []struct { + query string + expectedPlan sortPlan + err bool + }{ + { + query: "invalid(10, up)", + expectedPlan: mergeOnly, + err: true, + }, + { + query: "topk(10, up)", + expectedPlan: mergeOnly, + err: false, + }, + { + query: "bottomk(10, up)", + expectedPlan: mergeOnly, + err: false, + }, + { + query: "1 + topk(10, up)", + expectedPlan: sortByLabels, + err: false, + }, + { + query: "1 + sort_desc(sum by (job) (up) )", + expectedPlan: sortByValuesDesc, + err: false, + }, + { + query: "sort(topk by (job) (10, up))", + expectedPlan: sortByValuesAsc, + err: false, + }, + { + query: "topk(5, up) by (job) + sort_desc(up)", + expectedPlan: sortByValuesDesc, + err: false, + }, + { + query: "sort(up) + topk(5, up) by (job)", + expectedPlan: sortByValuesAsc, + err: false, + }, + { + query: "sum(up) by (job)", + expectedPlan: sortByLabels, + err: false, + }, + } + + for _, tc := range tc { + t.Run(tc.query, func(t *testing.T) { + p, err := sortPlanForQuery(tc.query) + if tc.err { + assert.Error(t, err) + } else { + assert.NoError(t, err) + assert.Equal(t, tc.expectedPlan, p) + } + }) + } +} diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index 0e8237c74e..903ea9679e 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -7,7 +7,6 @@ import ( "fmt" "io" "net/http" - "sort" "strconv" "strings" "time" @@ -17,9 +16,11 @@ import ( "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" + otlog "github.com/opentracing/opentracing-go/log" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/util/jsonutil" "github.com/weaveworks/common/httpgrpc" @@ -117,6 +118,164 @@ func decodeSampleStream(ptr unsafe.Pointer, iter *jsoniter.Iterator) { } } +type CachingOptions struct { + Disabled bool +} + +type PrometheusRequest struct { + Request + Time int64 + Start int64 + End int64 + Step int64 + Timeout time.Duration + Query string + Path string + Headers http.Header + Stats string + CachingOptions CachingOptions +} + +func (m *PrometheusRequest) GetPath() string { + if m != nil { + return m.Path + } + return "" +} + +func (m *PrometheusRequest) GetStart() int64 { + if m != nil { + return m.Start + } + return 0 +} + +func (m *PrometheusRequest) GetEnd() int64 { + if m != nil { + return m.End + } + return 0 +} + +func (m *PrometheusRequest) GetStep() int64 { + if m != nil { + return m.Step + } + return 0 +} + +func (m *PrometheusRequest) GetTimeout() time.Duration { + if m != nil { + return m.Timeout + } + return 0 +} + +func (m *PrometheusRequest) GetQuery() string { + if m != nil { + return m.Query + } + return "" +} + +func (m *PrometheusRequest) GetCachingOptions() CachingOptions { + if m != nil { + return m.CachingOptions + } + return CachingOptions{} +} + +func (m *PrometheusRequest) GetHeaders() http.Header { + if m != nil { + return m.Headers + } + return nil +} + +func (m *PrometheusRequest) GetStats() string { + if m != nil { + return m.Stats + } + return "" +} + +// WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. +func (m *PrometheusRequest) WithStartEnd(start int64, end int64) Request { + new := *m + new.Start = start + new.End = end + return &new +} + +// WithQuery clones the current `PrometheusRequest` with a new query. +func (m *PrometheusRequest) WithQuery(query string) Request { + new := *m + new.Query = query + return &new +} + +// WithStats clones the current `PrometheusRequest` with a new stats. +func (m *PrometheusRequest) WithStats(stats string) Request { + new := *m + new.Stats = stats + return &new +} + +// LogToSpan logs the current `PrometheusRequest` parameters to the specified span. +func (m *PrometheusRequest) LogToSpan(sp opentracing.Span) { + if m.GetStep() > 0 { + sp.LogFields( + otlog.String("query", m.GetQuery()), + otlog.String("start", timestamp.Time(m.GetStart()).String()), + otlog.String("end", timestamp.Time(m.GetEnd()).String()), + otlog.Int64("step (ms)", m.GetStep()), + ) + } else if m != nil { + sp.LogFields( + otlog.String("query", m.GetQuery()), + otlog.String("time", timestamp.Time(m.Time).String()), + ) + } +} + +func (resp *PrometheusResponse) HTTPHeaders() map[string][]string { + if resp != nil && resp.GetHeaders() != nil { + r := map[string][]string{} + for _, header := range resp.GetHeaders() { + if header != nil { + r[header.Name] = header.Values + } + } + + return r + } + return nil +} + +// NewEmptyPrometheusResponse returns an empty successful Prometheus query range response. +func NewEmptyPrometheusResponse(instant bool) *PrometheusResponse { + if instant { + return &PrometheusResponse{ + Status: StatusSuccess, + Data: PrometheusData{ + ResultType: model.ValVector.String(), + Result: PrometheusQueryResult{ + Result: &PrometheusQueryResult_Vector{}, + }, + }, + } + } + return &PrometheusResponse{ + Status: StatusSuccess, + Data: PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusQueryResult{ + Result: &PrometheusQueryResult_Matrix{}, + }, + }, + } +} + func encodeSampleStream(ptr unsafe.Pointer, stream *jsoniter.Stream) { ss := (*SampleStream)(ptr) stream.WriteObjectStart() @@ -158,6 +317,58 @@ func encodeSampleStream(ptr unsafe.Pointer, stream *jsoniter.Stream) { stream.WriteObjectEnd() } +func decodeSample(ptr unsafe.Pointer, iter *jsoniter.Iterator) { + ss := (*Sample)(ptr) + for field := iter.ReadObject(); field != ""; field = iter.ReadObject() { + switch field { + case "metric": + metricString := iter.ReadAny().ToString() + lbls := labels.Labels{} + if err := json.UnmarshalFromString(metricString, &lbls); err != nil { + iter.ReportError("unmarshal Sample", err.Error()) + return + } + ss.Labels = cortexpb.FromLabelsToLabelAdapters(lbls) + case "value": + ss.Sample = &cortexpb.Sample{} + cortexpb.SampleJsoniterDecode(unsafe.Pointer(ss.Sample), iter) + case "histogram": + ss.Histogram = &SampleHistogramPair{} + UnmarshalSampleHistogramPairJSON(unsafe.Pointer(ss.Histogram), iter) + default: + iter.ReportError("unmarshal Sample", fmt.Sprint("unexpected key:", field)) + return + } + } +} + +func encodeSample(ptr unsafe.Pointer, stream *jsoniter.Stream) { + ss := (*Sample)(ptr) + stream.WriteObjectStart() + + stream.WriteObjectField(`metric`) + lbls, err := cortexpb.FromLabelAdaptersToLabels(ss.Labels).MarshalJSON() + if err != nil { + stream.Error = err + return + } + stream.SetBuffer(append(stream.Buffer(), lbls...)) + + if ss.Sample != nil { + stream.WriteMore() + stream.WriteObjectField(`value`) + cortexpb.SampleJsoniterEncode(unsafe.Pointer(ss.Sample), stream) + } + + if ss.Histogram != nil { + stream.WriteMore() + stream.WriteObjectField(`histogram`) + MarshalSampleHistogramPairJSON(unsafe.Pointer(ss.Histogram), stream) + } + + stream.WriteObjectEnd() +} + func PrometheusResponseQueryableSamplesStatsPerStepJsoniterDecode(ptr unsafe.Pointer, iter *jsoniter.Iterator) { if !iter.ReadArray() { iter.ReportError("tripperware.PrometheusResponseQueryableSamplesStatsPerStep", "expected [") @@ -196,6 +407,8 @@ func init() { jsoniter.RegisterTypeDecoderFunc("tripperware.PrometheusResponseQueryableSamplesStatsPerStep", PrometheusResponseQueryableSamplesStatsPerStepJsoniterDecode) jsoniter.RegisterTypeEncoderFunc("tripperware.SampleStream", encodeSampleStream, marshalJSONIsEmpty) jsoniter.RegisterTypeDecoderFunc("tripperware.SampleStream", decodeSampleStream) + jsoniter.RegisterTypeEncoderFunc("tripperware.Sample", encodeSample, marshalJSONIsEmpty) + jsoniter.RegisterTypeDecoderFunc("tripperware.Sample", decodeSample) jsoniter.RegisterTypeEncoderFunc("tripperware.SampleHistogramPair", MarshalSampleHistogramPairJSON, marshalJSONIsEmpty) jsoniter.RegisterTypeDecoderFunc("tripperware.SampleHistogramPair", UnmarshalSampleHistogramPairJSON) } @@ -266,21 +479,79 @@ func BodyBufferFromHTTPGRPCResponse(res *httpgrpc.HTTPResponse, logger log.Logge return res.Body, nil } -func StatsMerge(stats map[int64]*PrometheusResponseQueryableSamplesStatsPerStep) *PrometheusResponseStats { - keys := make([]int64, 0, len(stats)) - for key := range stats { - keys = append(keys, key) +// UnmarshalJSON implements json.Unmarshaler. +func (s *PrometheusData) UnmarshalJSON(data []byte) error { + var queryData struct { + ResultType string `json:"resultType"` + Stats *PrometheusResponseStats `json:"stats,omitempty"` } - sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) - - result := &PrometheusResponseStats{Samples: &PrometheusResponseSamplesStats{}} - for _, key := range keys { - result.Samples.TotalQueryableSamplesPerStep = append(result.Samples.TotalQueryableSamplesPerStep, stats[key]) - result.Samples.TotalQueryableSamples += stats[key].Value + if err := json.Unmarshal(data, &queryData); err != nil { + return err } + s.ResultType = queryData.ResultType + s.Stats = queryData.Stats + switch s.ResultType { + case model.ValVector.String(): + var result struct { + Samples []*Sample `json:"result"` + } + if err := json.Unmarshal(data, &result); err != nil { + return err + } + s.Result = PrometheusQueryResult{ + Result: &PrometheusQueryResult_Vector{Vector: &Vector{ + Samples: result.Samples, + }}, + } + case model.ValMatrix.String(): + var result struct { + SampleStreams []SampleStream `json:"result"` + } + if err := json.Unmarshal(data, &result); err != nil { + return err + } + s.Result = PrometheusQueryResult{ + Result: &PrometheusQueryResult_Matrix{Matrix: &Matrix{ + SampleStreams: result.SampleStreams, + }}, + } + default: + s.Result = PrometheusQueryResult{ + Result: &PrometheusQueryResult_RawBytes{data}, + } + } + return nil +} - return result +// MarshalJSON implements json.Marshaler. +func (s *PrometheusData) MarshalJSON() ([]byte, error) { + switch s.ResultType { + case model.ValVector.String(): + res := struct { + ResultType string `json:"resultType"` + Data []*Sample `json:"result"` + Stats *PrometheusResponseStats `json:"stats,omitempty"` + }{ + ResultType: s.ResultType, + Data: s.Result.GetVector().Samples, + Stats: s.Stats, + } + return json.Marshal(res) + case model.ValMatrix.String(): + res := struct { + ResultType string `json:"resultType"` + Data []SampleStream `json:"result"` + Stats *PrometheusResponseStats `json:"stats,omitempty"` + }{ + ResultType: s.ResultType, + Data: s.Result.GetMatrix().SampleStreams, + Stats: s.Stats, + } + return json.Marshal(res) + default: + return s.Result.GetRawBytes(), nil + } } // Adapted from https://github.com/prometheus/client_golang/blob/4b158abea9470f75b6f07460cdc2189b91914562/api/prometheus/v1/api.go#L84. diff --git a/pkg/querier/tripperware/query.pb.go b/pkg/querier/tripperware/query.pb.go index 5c46214404..84c1c406bd 100644 --- a/pkg/querier/tripperware/query.pb.go +++ b/pkg/querier/tripperware/query.pb.go @@ -4,12 +4,15 @@ package tripperware import ( + bytes "bytes" encoding_binary "encoding/binary" fmt "fmt" cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" _ "github.com/gogo/protobuf/gogoproto" proto "github.com/gogo/protobuf/proto" + types "github.com/gogo/protobuf/types" + _ "github.com/golang/protobuf/ptypes/duration" io "io" math "math" math_bits "math/bits" @@ -28,6 +31,267 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type PrometheusResponse struct { + Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` + Data PrometheusData `protobuf:"bytes,2,opt,name=Data,proto3" json:"data,omitempty"` + ErrorType string `protobuf:"bytes,3,opt,name=ErrorType,proto3" json:"errorType,omitempty"` + Error string `protobuf:"bytes,4,opt,name=Error,proto3" json:"error,omitempty"` + Headers []*PrometheusResponseHeader `protobuf:"bytes,5,rep,name=Headers,proto3" json:"-"` + Warnings []string `protobuf:"bytes,6,rep,name=Warnings,proto3" json:"warnings,omitempty"` +} + +func (m *PrometheusResponse) Reset() { *m = PrometheusResponse{} } +func (*PrometheusResponse) ProtoMessage() {} +func (*PrometheusResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{0} +} +func (m *PrometheusResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PrometheusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_PrometheusResponse.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 *PrometheusResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PrometheusResponse.Merge(m, src) +} +func (m *PrometheusResponse) XXX_Size() int { + return m.Size() +} +func (m *PrometheusResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PrometheusResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PrometheusResponse proto.InternalMessageInfo + +func (m *PrometheusResponse) GetStatus() string { + if m != nil { + return m.Status + } + return "" +} + +func (m *PrometheusResponse) GetData() PrometheusData { + if m != nil { + return m.Data + } + return PrometheusData{} +} + +func (m *PrometheusResponse) GetErrorType() string { + if m != nil { + return m.ErrorType + } + return "" +} + +func (m *PrometheusResponse) GetError() string { + if m != nil { + return m.Error + } + return "" +} + +func (m *PrometheusResponse) GetHeaders() []*PrometheusResponseHeader { + if m != nil { + return m.Headers + } + return nil +} + +func (m *PrometheusResponse) GetWarnings() []string { + if m != nil { + return m.Warnings + } + return nil +} + +type PrometheusData struct { + ResultType string `protobuf:"bytes,1,opt,name=ResultType,proto3" json:"resultType"` + Result PrometheusQueryResult `protobuf:"bytes,2,opt,name=Result,proto3" json:"result"` + Stats *PrometheusResponseStats `protobuf:"bytes,3,opt,name=stats,proto3" json:"stats,omitempty"` +} + +func (m *PrometheusData) Reset() { *m = PrometheusData{} } +func (*PrometheusData) ProtoMessage() {} +func (*PrometheusData) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{1} +} +func (m *PrometheusData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PrometheusData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_PrometheusData.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 *PrometheusData) XXX_Merge(src proto.Message) { + xxx_messageInfo_PrometheusData.Merge(m, src) +} +func (m *PrometheusData) XXX_Size() int { + return m.Size() +} +func (m *PrometheusData) XXX_DiscardUnknown() { + xxx_messageInfo_PrometheusData.DiscardUnknown(m) +} + +var xxx_messageInfo_PrometheusData proto.InternalMessageInfo + +func (m *PrometheusData) GetResultType() string { + if m != nil { + return m.ResultType + } + return "" +} + +func (m *PrometheusData) GetResult() PrometheusQueryResult { + if m != nil { + return m.Result + } + return PrometheusQueryResult{} +} + +func (m *PrometheusData) GetStats() *PrometheusResponseStats { + if m != nil { + return m.Stats + } + return nil +} + +type CachedResponse struct { + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key"` + // List of cached responses; non-overlapping and in order. + Extents []Extent `protobuf:"bytes,2,rep,name=extents,proto3" json:"extents"` +} + +func (m *CachedResponse) Reset() { *m = CachedResponse{} } +func (*CachedResponse) ProtoMessage() {} +func (*CachedResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{2} +} +func (m *CachedResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CachedResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_CachedResponse.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 *CachedResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CachedResponse.Merge(m, src) +} +func (m *CachedResponse) XXX_Size() int { + return m.Size() +} +func (m *CachedResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CachedResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CachedResponse proto.InternalMessageInfo + +func (m *CachedResponse) GetKey() string { + if m != nil { + return m.Key + } + return "" +} + +func (m *CachedResponse) GetExtents() []Extent { + if m != nil { + return m.Extents + } + return nil +} + +type Extent struct { + Start int64 `protobuf:"varint,1,opt,name=start,proto3" json:"start"` + End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end"` + TraceId string `protobuf:"bytes,4,opt,name=trace_id,json=traceId,proto3" json:"-"` + Response *types.Any `protobuf:"bytes,5,opt,name=response,proto3" json:"response"` +} + +func (m *Extent) Reset() { *m = Extent{} } +func (*Extent) ProtoMessage() {} +func (*Extent) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{3} +} +func (m *Extent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Extent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Extent.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 *Extent) XXX_Merge(src proto.Message) { + xxx_messageInfo_Extent.Merge(m, src) +} +func (m *Extent) XXX_Size() int { + return m.Size() +} +func (m *Extent) XXX_DiscardUnknown() { + xxx_messageInfo_Extent.DiscardUnknown(m) +} + +var xxx_messageInfo_Extent proto.InternalMessageInfo + +func (m *Extent) GetStart() int64 { + if m != nil { + return m.Start + } + return 0 +} + +func (m *Extent) GetEnd() int64 { + if m != nil { + return m.End + } + return 0 +} + +func (m *Extent) GetTraceId() string { + if m != nil { + return m.TraceId + } + return "" +} + +func (m *Extent) GetResponse() *types.Any { + if m != nil { + return m.Response + } + return nil +} + type SampleStream struct { Labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter" json:"metric"` Samples []cortexpb.Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"values"` @@ -37,7 +301,7 @@ type SampleStream struct { func (m *SampleStream) Reset() { *m = SampleStream{} } func (*SampleStream) ProtoMessage() {} func (*SampleStream) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{0} + return fileDescriptor_5c6ac9b241082464, []int{4} } func (m *SampleStream) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -88,7 +352,7 @@ type SampleHistogramPair struct { func (m *SampleHistogramPair) Reset() { *m = SampleHistogramPair{} } func (*SampleHistogramPair) ProtoMessage() {} func (*SampleHistogramPair) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{1} + return fileDescriptor_5c6ac9b241082464, []int{5} } func (m *SampleHistogramPair) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -140,7 +404,7 @@ type SampleHistogram struct { func (m *SampleHistogram) Reset() { *m = SampleHistogram{} } func (*SampleHistogram) ProtoMessage() {} func (*SampleHistogram) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{2} + return fileDescriptor_5c6ac9b241082464, []int{6} } func (m *SampleHistogram) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -200,7 +464,7 @@ type HistogramBucket struct { func (m *HistogramBucket) Reset() { *m = HistogramBucket{} } func (*HistogramBucket) ProtoMessage() {} func (*HistogramBucket) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{3} + return fileDescriptor_5c6ac9b241082464, []int{7} } func (m *HistogramBucket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -264,7 +528,7 @@ type PrometheusResponseStats struct { func (m *PrometheusResponseStats) Reset() { *m = PrometheusResponseStats{} } func (*PrometheusResponseStats) ProtoMessage() {} func (*PrometheusResponseStats) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{4} + return fileDescriptor_5c6ac9b241082464, []int{8} } func (m *PrometheusResponseStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,7 +572,7 @@ type PrometheusResponseSamplesStats struct { func (m *PrometheusResponseSamplesStats) Reset() { *m = PrometheusResponseSamplesStats{} } func (*PrometheusResponseSamplesStats) ProtoMessage() {} func (*PrometheusResponseSamplesStats) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{5} + return fileDescriptor_5c6ac9b241082464, []int{9} } func (m *PrometheusResponseSamplesStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -361,7 +625,7 @@ func (m *PrometheusResponseQueryableSamplesStatsPerStep) Reset() { } func (*PrometheusResponseQueryableSamplesStatsPerStep) ProtoMessage() {} func (*PrometheusResponseQueryableSamplesStatsPerStep) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{6} + return fileDescriptor_5c6ac9b241082464, []int{10} } func (m *PrometheusResponseQueryableSamplesStatsPerStep) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,7 +676,7 @@ type PrometheusResponseHeader struct { func (m *PrometheusResponseHeader) Reset() { *m = PrometheusResponseHeader{} } func (*PrometheusResponseHeader) ProtoMessage() {} func (*PrometheusResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{7} + return fileDescriptor_5c6ac9b241082464, []int{11} } func (m *PrometheusResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -455,22 +719,26 @@ func (m *PrometheusResponseHeader) GetValues() []string { return nil } -type PrometheusRequestHeader struct { - Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"-"` - Values []string `protobuf:"bytes,2,rep,name=Values,proto3" json:"-"` +type PrometheusQueryResult struct { + // Types that are valid to be assigned to Result: + // + // *PrometheusQueryResult_Vector + // *PrometheusQueryResult_RawBytes + // *PrometheusQueryResult_Matrix + Result isPrometheusQueryResult_Result `protobuf_oneof:"result"` } -func (m *PrometheusRequestHeader) Reset() { *m = PrometheusRequestHeader{} } -func (*PrometheusRequestHeader) ProtoMessage() {} -func (*PrometheusRequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_5c6ac9b241082464, []int{8} +func (m *PrometheusQueryResult) Reset() { *m = PrometheusQueryResult{} } +func (*PrometheusQueryResult) ProtoMessage() {} +func (*PrometheusQueryResult) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{12} } -func (m *PrometheusRequestHeader) XXX_Unmarshal(b []byte) error { +func (m *PrometheusQueryResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *PrometheusRequestHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *PrometheusQueryResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_PrometheusRequestHeader.Marshal(b, m, deterministic) + return xxx_messageInfo_PrometheusQueryResult.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -480,125 +748,491 @@ func (m *PrometheusRequestHeader) XXX_Marshal(b []byte, deterministic bool) ([]b return b[:n], nil } } -func (m *PrometheusRequestHeader) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusRequestHeader.Merge(m, src) +func (m *PrometheusQueryResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_PrometheusQueryResult.Merge(m, src) } -func (m *PrometheusRequestHeader) XXX_Size() int { +func (m *PrometheusQueryResult) XXX_Size() int { return m.Size() } -func (m *PrometheusRequestHeader) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusRequestHeader.DiscardUnknown(m) +func (m *PrometheusQueryResult) XXX_DiscardUnknown() { + xxx_messageInfo_PrometheusQueryResult.DiscardUnknown(m) } -var xxx_messageInfo_PrometheusRequestHeader proto.InternalMessageInfo +var xxx_messageInfo_PrometheusQueryResult proto.InternalMessageInfo -func (m *PrometheusRequestHeader) GetName() string { - if m != nil { - return m.Name - } - return "" +type isPrometheusQueryResult_Result interface { + isPrometheusQueryResult_Result() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type PrometheusQueryResult_Vector struct { + Vector *Vector `protobuf:"bytes,1,opt,name=vector,proto3,oneof"` +} +type PrometheusQueryResult_RawBytes struct { + RawBytes []byte `protobuf:"bytes,2,opt,name=rawBytes,proto3,oneof"` } +type PrometheusQueryResult_Matrix struct { + Matrix *Matrix `protobuf:"bytes,3,opt,name=matrix,proto3,oneof"` +} + +func (*PrometheusQueryResult_Vector) isPrometheusQueryResult_Result() {} +func (*PrometheusQueryResult_RawBytes) isPrometheusQueryResult_Result() {} +func (*PrometheusQueryResult_Matrix) isPrometheusQueryResult_Result() {} -func (m *PrometheusRequestHeader) GetValues() []string { +func (m *PrometheusQueryResult) GetResult() isPrometheusQueryResult_Result { if m != nil { - return m.Values + return m.Result } return nil } -func init() { - proto.RegisterType((*SampleStream)(nil), "tripperware.SampleStream") - proto.RegisterType((*SampleHistogramPair)(nil), "tripperware.SampleHistogramPair") - proto.RegisterType((*SampleHistogram)(nil), "tripperware.SampleHistogram") - proto.RegisterType((*HistogramBucket)(nil), "tripperware.HistogramBucket") - proto.RegisterType((*PrometheusResponseStats)(nil), "tripperware.PrometheusResponseStats") - proto.RegisterType((*PrometheusResponseSamplesStats)(nil), "tripperware.PrometheusResponseSamplesStats") - proto.RegisterType((*PrometheusResponseQueryableSamplesStatsPerStep)(nil), "tripperware.PrometheusResponseQueryableSamplesStatsPerStep") - proto.RegisterType((*PrometheusResponseHeader)(nil), "tripperware.PrometheusResponseHeader") - proto.RegisterType((*PrometheusRequestHeader)(nil), "tripperware.PrometheusRequestHeader") +func (m *PrometheusQueryResult) GetVector() *Vector { + if x, ok := m.GetResult().(*PrometheusQueryResult_Vector); ok { + return x.Vector + } + return nil } -func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } - -var fileDescriptor_5c6ac9b241082464 = []byte{ - // 646 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0x4f, 0x4f, 0x13, 0x41, - 0x14, 0xdf, 0x69, 0xa1, 0x84, 0x29, 0x11, 0x32, 0x60, 0x2c, 0x04, 0x67, 0xeb, 0x9e, 0x48, 0x8c, - 0x25, 0xc1, 0xc4, 0x44, 0xbd, 0xc8, 0x9e, 0x48, 0xfc, 0x87, 0x53, 0xe2, 0xc1, 0x8b, 0x99, 0x2d, - 0x93, 0xb2, 0xb2, 0xcb, 0x2c, 0x33, 0xb3, 0x82, 0x9e, 0xfc, 0x08, 0x5e, 0xbc, 0x78, 0xf3, 0xe6, - 0x47, 0xe1, 0xc8, 0x91, 0x78, 0xd8, 0xc8, 0x72, 0x31, 0x3d, 0xf1, 0x11, 0xcc, 0xcc, 0xec, 0xb6, - 0x85, 0x36, 0x35, 0xc4, 0xdb, 0xbc, 0xdf, 0x7b, 0xbf, 0xf7, 0x7b, 0xef, 0xed, 0x7b, 0x0b, 0xeb, - 0x87, 0x29, 0x13, 0x9f, 0x5a, 0x89, 0xe0, 0x8a, 0xa3, 0xba, 0x12, 0x61, 0x92, 0x30, 0x71, 0x44, - 0x05, 0x5b, 0x59, 0xea, 0xf2, 0x2e, 0x37, 0xf8, 0xba, 0x7e, 0xd9, 0x90, 0x95, 0xc7, 0xdd, 0x50, - 0xed, 0xa5, 0x41, 0xab, 0xc3, 0xe3, 0xf5, 0x0e, 0x17, 0x8a, 0x1d, 0x27, 0x82, 0x7f, 0x60, 0x1d, - 0x55, 0x58, 0xeb, 0xc9, 0x7e, 0xb7, 0x74, 0x04, 0xc5, 0xc3, 0x52, 0xbd, 0xef, 0x15, 0x38, 0xd7, - 0xa6, 0x71, 0x12, 0xb1, 0xb6, 0x12, 0x8c, 0xc6, 0xe8, 0x18, 0xd6, 0x22, 0x1a, 0xb0, 0x48, 0x36, - 0x40, 0xb3, 0xba, 0x56, 0xdf, 0x58, 0x6c, 0x95, 0xc4, 0xd6, 0x0b, 0x8d, 0x6f, 0xd3, 0x50, 0xf8, - 0xcf, 0x4f, 0x32, 0xd7, 0xf9, 0x95, 0xb9, 0x37, 0x12, 0xb6, 0xfc, 0xcd, 0x5d, 0x9a, 0x28, 0x26, - 0x7a, 0x99, 0x5b, 0x8b, 0x99, 0x12, 0x61, 0x87, 0x14, 0x7a, 0xe8, 0x09, 0x9c, 0x91, 0xa6, 0x12, - 0xd9, 0xa8, 0x18, 0xe9, 0x85, 0x81, 0xb4, 0x2d, 0xd1, 0xbf, 0xa5, 0x75, 0x35, 0xf5, 0x23, 0x8d, - 0x52, 0x26, 0x49, 0x49, 0x40, 0x3b, 0x10, 0xee, 0x85, 0x52, 0xf1, 0xae, 0xa0, 0xb1, 0x6c, 0x54, - 0x0d, 0xbd, 0xd9, 0x1a, 0x9a, 0x5c, 0x91, 0x61, 0xab, 0x0c, 0x32, 0x6d, 0xa0, 0x22, 0xdd, 0x10, - 0x97, 0x0c, 0xbd, 0xbd, 0xcf, 0x70, 0x71, 0x0c, 0x0d, 0xdd, 0x83, 0x73, 0x2a, 0x8c, 0x99, 0x54, - 0x34, 0x4e, 0xde, 0xc7, 0x7a, 0x50, 0x60, 0xad, 0x4a, 0xea, 0x7d, 0xec, 0xa5, 0x44, 0xcf, 0xe0, - 0x6c, 0x3f, 0x4f, 0xa3, 0xd2, 0x04, 0x6b, 0xf5, 0x8d, 0xd5, 0x49, 0xe5, 0xf8, 0x53, 0xba, 0x14, - 0x32, 0x20, 0x79, 0x87, 0x70, 0xfe, 0x5a, 0x0c, 0x5a, 0x82, 0xd3, 0x1d, 0x9e, 0x1e, 0x28, 0x23, - 0x08, 0x88, 0x35, 0xd0, 0x02, 0xac, 0xca, 0xd4, 0x8a, 0x00, 0xa2, 0x9f, 0xe8, 0x11, 0x9c, 0x09, - 0xd2, 0xce, 0x3e, 0x53, 0xe5, 0x24, 0xae, 0x4a, 0x0f, 0x44, 0x4d, 0x10, 0x29, 0x83, 0x3d, 0x09, - 0xe7, 0xaf, 0xf9, 0x10, 0x86, 0x30, 0xe0, 0xe9, 0xc1, 0x2e, 0x15, 0x21, 0xb3, 0x8d, 0x4e, 0x93, - 0x21, 0x44, 0x97, 0x14, 0xf1, 0x23, 0x26, 0x0a, 0x79, 0x6b, 0x68, 0x34, 0xd5, 0x72, 0x8d, 0xaa, - 0x45, 0x8d, 0x31, 0x28, 0x7f, 0x6a, 0xa8, 0x7c, 0x2f, 0x86, 0x77, 0xb6, 0x05, 0x8f, 0x99, 0xda, - 0x63, 0xa9, 0x24, 0x4c, 0x26, 0xfc, 0x40, 0xb2, 0xb6, 0xa2, 0x4a, 0x22, 0x32, 0x58, 0x08, 0x60, - 0x46, 0x78, 0xff, 0x4a, 0x1f, 0x63, 0x68, 0x36, 0xda, 0xb0, 0xfd, 0x7a, 0x2f, 0x73, 0x4b, 0x7e, - 0x7f, 0x51, 0xbc, 0x6f, 0x15, 0x88, 0x27, 0x13, 0xd1, 0x6b, 0x78, 0x5b, 0x71, 0x45, 0xa3, 0x37, - 0xfa, 0x08, 0x69, 0x10, 0x95, 0x5e, 0xfb, 0x9d, 0xfd, 0xe5, 0x5e, 0xe6, 0x8e, 0x0f, 0x20, 0xe3, - 0x61, 0xf4, 0x03, 0xc0, 0xd5, 0xb1, 0x9e, 0x6d, 0x26, 0xda, 0x8a, 0x25, 0xc5, 0xba, 0x3f, 0xfd, - 0x47, 0x77, 0xd7, 0xd9, 0xa6, 0xda, 0x22, 0x85, 0xdf, 0xec, 0x65, 0xee, 0x44, 0x11, 0x32, 0xd1, - 0xeb, 0x85, 0xf0, 0x86, 0x8a, 0xfa, 0x73, 0x9a, 0x2b, 0x2c, 0xd6, 0xdf, 0x1a, 0x23, 0xb7, 0x51, - 0x19, 0xb9, 0x0d, 0x6f, 0x07, 0x36, 0x46, 0xa5, 0xb6, 0x18, 0xdd, 0x65, 0x02, 0x2d, 0xc3, 0xa9, - 0x57, 0x34, 0xb6, 0x39, 0x67, 0xfd, 0xe9, 0x5e, 0xe6, 0x82, 0x07, 0xc4, 0x40, 0xe8, 0x2e, 0xac, - 0xbd, 0x35, 0x57, 0x6f, 0xc6, 0xd5, 0x77, 0x16, 0xa0, 0xd7, 0xbe, 0xba, 0x47, 0x87, 0x29, 0x93, - 0xea, 0x7f, 0x93, 0xfa, 0x9b, 0xa7, 0xe7, 0xd8, 0x39, 0x3b, 0xc7, 0xce, 0xe5, 0x39, 0x06, 0x5f, - 0x72, 0x0c, 0x7e, 0xe6, 0x18, 0x9c, 0xe4, 0x18, 0x9c, 0xe6, 0x18, 0xfc, 0xce, 0x31, 0xf8, 0x93, - 0x63, 0xe7, 0x32, 0xc7, 0xe0, 0xeb, 0x05, 0x76, 0x4e, 0x2f, 0xb0, 0x73, 0x76, 0x81, 0x9d, 0x77, - 0xc3, 0x7f, 0xec, 0xa0, 0x66, 0xfe, 0xb3, 0x0f, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x33, 0x14, - 0x72, 0x0f, 0xd4, 0x05, 0x00, 0x00, +func (m *PrometheusQueryResult) GetRawBytes() []byte { + if x, ok := m.GetResult().(*PrometheusQueryResult_RawBytes); ok { + return x.RawBytes + } + return nil } -func (this *SampleStream) Equal(that interface{}) bool { - if that == nil { - return this == nil +func (m *PrometheusQueryResult) GetMatrix() *Matrix { + if x, ok := m.GetResult().(*PrometheusQueryResult_Matrix); ok { + return x.Matrix } + return nil +} - that1, ok := that.(*SampleStream) - if !ok { - that2, ok := that.(SampleStream) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if len(this.Labels) != len(that1.Labels) { - return false +// XXX_OneofWrappers is for the internal use of the proto package. +func (*PrometheusQueryResult) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*PrometheusQueryResult_Vector)(nil), + (*PrometheusQueryResult_RawBytes)(nil), + (*PrometheusQueryResult_Matrix)(nil), } - for i := range this.Labels { - if !this.Labels[i].Equal(that1.Labels[i]) { - return false +} + +type Vector struct { + Samples []*Sample `protobuf:"bytes,1,rep,name=samples,proto3" json:"samples,omitempty"` +} + +func (m *Vector) Reset() { *m = Vector{} } +func (*Vector) ProtoMessage() {} +func (*Vector) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{13} +} +func (m *Vector) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Vector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Vector.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err } + return b[:n], nil } - if len(this.Samples) != len(that1.Samples) { - return false +} +func (m *Vector) XXX_Merge(src proto.Message) { + xxx_messageInfo_Vector.Merge(m, src) +} +func (m *Vector) XXX_Size() int { + return m.Size() +} +func (m *Vector) XXX_DiscardUnknown() { + xxx_messageInfo_Vector.DiscardUnknown(m) +} + +var xxx_messageInfo_Vector proto.InternalMessageInfo + +func (m *Vector) GetSamples() []*Sample { + if m != nil { + return m.Samples } - for i := range this.Samples { - if !this.Samples[i].Equal(&that1.Samples[i]) { - return false - } + return nil +} + +type Sample struct { + Labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter `protobuf:"bytes,1,rep,name=labels,proto3,customtype=github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter" json:"metric"` + Sample *cortexpb.Sample `protobuf:"bytes,2,opt,name=sample,proto3" json:"value"` + Histogram *SampleHistogramPair `protobuf:"bytes,3,opt,name=histogram,proto3" json:"histogram"` +} + +func (m *Sample) Reset() { *m = Sample{} } +func (*Sample) ProtoMessage() {} +func (*Sample) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{14} +} +func (m *Sample) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Sample) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Sample.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 *Sample) XXX_Merge(src proto.Message) { + xxx_messageInfo_Sample.Merge(m, src) +} +func (m *Sample) XXX_Size() int { + return m.Size() +} +func (m *Sample) XXX_DiscardUnknown() { + xxx_messageInfo_Sample.DiscardUnknown(m) +} + +var xxx_messageInfo_Sample proto.InternalMessageInfo + +func (m *Sample) GetSample() *cortexpb.Sample { + if m != nil { + return m.Sample + } + return nil +} + +func (m *Sample) GetHistogram() *SampleHistogramPair { + if m != nil { + return m.Histogram + } + return nil +} + +type Matrix struct { + SampleStreams []SampleStream `protobuf:"bytes,1,rep,name=sampleStreams,proto3" json:"sampleStreams"` +} + +func (m *Matrix) Reset() { *m = Matrix{} } +func (*Matrix) ProtoMessage() {} +func (*Matrix) Descriptor() ([]byte, []int) { + return fileDescriptor_5c6ac9b241082464, []int{15} +} +func (m *Matrix) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Matrix) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Matrix.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 *Matrix) XXX_Merge(src proto.Message) { + xxx_messageInfo_Matrix.Merge(m, src) +} +func (m *Matrix) XXX_Size() int { + return m.Size() +} +func (m *Matrix) XXX_DiscardUnknown() { + xxx_messageInfo_Matrix.DiscardUnknown(m) +} + +var xxx_messageInfo_Matrix proto.InternalMessageInfo + +func (m *Matrix) GetSampleStreams() []SampleStream { + if m != nil { + return m.SampleStreams + } + return nil +} + +func init() { + proto.RegisterType((*PrometheusResponse)(nil), "tripperware.PrometheusResponse") + proto.RegisterType((*PrometheusData)(nil), "tripperware.PrometheusData") + proto.RegisterType((*CachedResponse)(nil), "tripperware.CachedResponse") + proto.RegisterType((*Extent)(nil), "tripperware.Extent") + proto.RegisterType((*SampleStream)(nil), "tripperware.SampleStream") + proto.RegisterType((*SampleHistogramPair)(nil), "tripperware.SampleHistogramPair") + proto.RegisterType((*SampleHistogram)(nil), "tripperware.SampleHistogram") + proto.RegisterType((*HistogramBucket)(nil), "tripperware.HistogramBucket") + proto.RegisterType((*PrometheusResponseStats)(nil), "tripperware.PrometheusResponseStats") + proto.RegisterType((*PrometheusResponseSamplesStats)(nil), "tripperware.PrometheusResponseSamplesStats") + proto.RegisterType((*PrometheusResponseQueryableSamplesStatsPerStep)(nil), "tripperware.PrometheusResponseQueryableSamplesStatsPerStep") + proto.RegisterType((*PrometheusResponseHeader)(nil), "tripperware.PrometheusResponseHeader") + proto.RegisterType((*PrometheusQueryResult)(nil), "tripperware.PrometheusQueryResult") + proto.RegisterType((*Vector)(nil), "tripperware.Vector") + proto.RegisterType((*Sample)(nil), "tripperware.Sample") + proto.RegisterType((*Matrix)(nil), "tripperware.Matrix") +} + +func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } + +var fileDescriptor_5c6ac9b241082464 = []byte{ + // 1173 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x4b, 0x6f, 0x1b, 0xd5, + 0x17, 0xf7, 0xf8, 0x31, 0x71, 0x8e, 0xd3, 0xa4, 0xff, 0x9b, 0x3e, 0x9c, 0xfe, 0xcb, 0x8c, 0x19, + 0x81, 0x14, 0x04, 0x71, 0xa4, 0x20, 0xa8, 0x00, 0xa9, 0x22, 0x03, 0x81, 0x50, 0x28, 0x49, 0x6f, + 0xa2, 0x22, 0xb1, 0xa9, 0xae, 0xed, 0x8b, 0x33, 0xc4, 0xe3, 0x99, 0xde, 0xb9, 0xd3, 0xc4, 0xac, + 0xf8, 0x04, 0x88, 0x0d, 0x1b, 0x24, 0x16, 0xec, 0x58, 0xf0, 0x41, 0xb2, 0xcc, 0xb2, 0x42, 0x62, + 0x20, 0xce, 0x06, 0xcd, 0xaa, 0x1f, 0x01, 0xdd, 0xc7, 0xd8, 0xe3, 0xc4, 0x49, 0xd4, 0x15, 0x1b, + 0x67, 0xee, 0x39, 0xbf, 0xf3, 0xfa, 0xdd, 0x73, 0xce, 0x0d, 0xd4, 0x9e, 0xc6, 0x94, 0x0d, 0x9a, + 0x21, 0x0b, 0x78, 0x80, 0x6a, 0x9c, 0x79, 0x61, 0x48, 0xd9, 0x01, 0x61, 0xf4, 0xce, 0x8d, 0x6e, + 0xd0, 0x0d, 0xa4, 0x7c, 0x55, 0x7c, 0x29, 0xc8, 0x1d, 0xab, 0x1b, 0x04, 0xdd, 0x1e, 0x5d, 0x95, + 0xa7, 0x56, 0xfc, 0xcd, 0x6a, 0x27, 0x66, 0x84, 0x7b, 0x41, 0x5f, 0xeb, 0x97, 0xce, 0xea, 0x49, + 0x5f, 0x7b, 0xbf, 0xf3, 0x5e, 0xd7, 0xe3, 0x7b, 0x71, 0xab, 0xd9, 0x0e, 0xfc, 0xd5, 0x76, 0xc0, + 0x38, 0x3d, 0x0c, 0x59, 0xf0, 0x2d, 0x6d, 0x73, 0x7d, 0x5a, 0x0d, 0xf7, 0xbb, 0x99, 0xa2, 0xa5, + 0x3f, 0x94, 0xa9, 0xf3, 0x57, 0x11, 0xd0, 0x36, 0x0b, 0x7c, 0xca, 0xf7, 0x68, 0x1c, 0x61, 0x1a, + 0x85, 0x41, 0x3f, 0xa2, 0xc8, 0x01, 0x73, 0x87, 0x13, 0x1e, 0x47, 0x75, 0xa3, 0x61, 0x2c, 0xcf, + 0xba, 0x90, 0x26, 0xb6, 0x19, 0x49, 0x09, 0xd6, 0x1a, 0xf4, 0x29, 0x94, 0x3f, 0x26, 0x9c, 0xd4, + 0x8b, 0x0d, 0x63, 0xb9, 0xb6, 0xf6, 0xff, 0x66, 0xae, 0xc4, 0xe6, 0xd8, 0xa5, 0x80, 0xb8, 0xb7, + 0x8e, 0x12, 0xbb, 0x90, 0x26, 0xf6, 0x7c, 0x87, 0x70, 0xf2, 0x56, 0xe0, 0x7b, 0x9c, 0xfa, 0x21, + 0x1f, 0x60, 0xe9, 0x00, 0xbd, 0x03, 0xb3, 0x1b, 0x8c, 0x05, 0x6c, 0x77, 0x10, 0xd2, 0x7a, 0x49, + 0xc6, 0xbb, 0x9d, 0x26, 0xf6, 0x22, 0xcd, 0x84, 0x39, 0x8b, 0x31, 0x12, 0xbd, 0x01, 0x15, 0x79, + 0xa8, 0x97, 0xa5, 0xc9, 0x62, 0x9a, 0xd8, 0x0b, 0xd2, 0x24, 0x07, 0x57, 0x08, 0xf4, 0x09, 0xcc, + 0x6c, 0x52, 0xd2, 0xa1, 0x2c, 0xaa, 0x57, 0x1a, 0xa5, 0xe5, 0xda, 0xda, 0xeb, 0x17, 0x64, 0x9b, + 0x11, 0xa0, 0xd0, 0x6e, 0x25, 0x4d, 0x6c, 0x63, 0x05, 0x67, 0xc6, 0x68, 0x0d, 0xaa, 0x5f, 0x11, + 0xd6, 0xf7, 0xfa, 0xdd, 0xa8, 0x6e, 0x36, 0x4a, 0xcb, 0xb3, 0xee, 0xad, 0x34, 0xb1, 0xd1, 0x81, + 0x96, 0xe5, 0x02, 0x8f, 0x70, 0xce, 0x9f, 0x06, 0xcc, 0x4f, 0xd2, 0x81, 0x9a, 0x00, 0x98, 0x46, + 0x71, 0x8f, 0xcb, 0x8a, 0x15, 0xc3, 0xf3, 0x69, 0x62, 0x03, 0x1b, 0x49, 0x71, 0x0e, 0x81, 0x1e, + 0x80, 0xa9, 0x4e, 0x9a, 0x6b, 0xe7, 0x82, 0xec, 0x1f, 0x89, 0x8e, 0x53, 0x48, 0x77, 0x5e, 0x53, + 0x6e, 0x2a, 0x9f, 0x58, 0x7b, 0x40, 0x5b, 0x50, 0x11, 0xf7, 0x18, 0x49, 0xa2, 0x6b, 0x6b, 0xaf, + 0x5d, 0x41, 0x84, 0xb8, 0xeb, 0x48, 0x71, 0x2b, 0xcd, 0xf2, 0xdc, 0x4a, 0x81, 0xb3, 0x0f, 0xf3, + 0x1f, 0x91, 0xf6, 0x1e, 0xed, 0x8c, 0x9a, 0x67, 0x09, 0x4a, 0xfb, 0x74, 0xa0, 0xeb, 0x9a, 0x49, + 0x13, 0x5b, 0x1c, 0xb1, 0xf8, 0x41, 0xf7, 0x61, 0x86, 0x1e, 0x72, 0xda, 0xe7, 0x51, 0xbd, 0x28, + 0x2f, 0x62, 0x71, 0x22, 0xfe, 0x86, 0xd4, 0xb9, 0x0b, 0x3a, 0xf7, 0x0c, 0x8b, 0xb3, 0x0f, 0xe7, + 0x77, 0x03, 0x4c, 0x05, 0x42, 0xb6, 0x2c, 0x84, 0x71, 0x19, 0xa7, 0xe4, 0xce, 0xa6, 0x89, 0xad, + 0x04, 0x58, 0xfd, 0x11, 0x69, 0xd0, 0x7e, 0x47, 0x52, 0x56, 0x52, 0x69, 0xd0, 0x7e, 0x07, 0x8b, + 0x1f, 0xd4, 0x80, 0x2a, 0x67, 0xa4, 0x4d, 0x9f, 0x78, 0x1d, 0xdd, 0x3d, 0xd9, 0x4d, 0x4b, 0xf1, + 0x67, 0x1d, 0x74, 0x1f, 0xaa, 0x4c, 0xd7, 0x53, 0xaf, 0x48, 0xa6, 0x6e, 0x34, 0xd5, 0x00, 0x36, + 0xb3, 0x01, 0x6c, 0xae, 0xf7, 0x07, 0xee, 0x5c, 0x9a, 0xd8, 0x23, 0x24, 0x1e, 0x7d, 0x3d, 0x28, + 0x57, 0x4b, 0xd7, 0xcb, 0xce, 0xcf, 0x45, 0x98, 0xdb, 0x21, 0x7e, 0xd8, 0xa3, 0x3b, 0x9c, 0x51, + 0xe2, 0xa3, 0x43, 0x30, 0x7b, 0xa4, 0x45, 0x7b, 0x62, 0xae, 0x54, 0xf9, 0xd9, 0x58, 0x36, 0xbf, + 0x10, 0xf2, 0x6d, 0xe2, 0x31, 0xf7, 0x73, 0x51, 0xfe, 0x1f, 0x89, 0xfd, 0x52, 0x63, 0xad, 0xec, + 0xd7, 0x3b, 0x24, 0xe4, 0x94, 0x89, 0x7b, 0xf7, 0x29, 0x67, 0x5e, 0x1b, 0xeb, 0x78, 0xe8, 0x7d, + 0x98, 0x89, 0x64, 0x26, 0x19, 0xf3, 0xd7, 0xc7, 0xa1, 0x55, 0x8a, 0xe3, 0x96, 0x79, 0x46, 0x7a, + 0x31, 0x8d, 0x70, 0x66, 0x80, 0x76, 0x01, 0xf6, 0xbc, 0x88, 0x07, 0x5d, 0x46, 0x7c, 0xd1, 0x38, + 0xc2, 0xbc, 0x31, 0x71, 0x71, 0xca, 0xc3, 0x66, 0x06, 0x92, 0x65, 0x20, 0xed, 0x2e, 0x67, 0x8b, + 0x73, 0xdf, 0xce, 0x77, 0xb0, 0x38, 0xc5, 0x0c, 0xbd, 0x0a, 0x73, 0xdc, 0xf3, 0x69, 0xc4, 0x89, + 0x1f, 0x3e, 0xf1, 0xd5, 0x02, 0x2a, 0xe1, 0xda, 0x48, 0xf6, 0x30, 0x42, 0x1f, 0xc2, 0xec, 0xc8, + 0x8f, 0x1e, 0x89, 0xbb, 0x97, 0xa5, 0xe3, 0x96, 0x45, 0x2a, 0x78, 0x6c, 0xe4, 0x3c, 0x85, 0x85, + 0x33, 0x18, 0x74, 0x03, 0x2a, 0xed, 0x20, 0xee, 0xab, 0x7e, 0x32, 0xb0, 0x3a, 0xa0, 0xeb, 0x50, + 0x8a, 0x62, 0x15, 0xc4, 0xc0, 0xe2, 0x13, 0xbd, 0x0b, 0x33, 0xad, 0xb8, 0xbd, 0x4f, 0x79, 0xc6, + 0xc4, 0x64, 0xe8, 0x71, 0x50, 0x09, 0xc2, 0x19, 0xd8, 0x89, 0x60, 0xe1, 0x8c, 0x0e, 0x59, 0x00, + 0xad, 0x20, 0xee, 0x77, 0x08, 0xf3, 0xa8, 0x2a, 0xb4, 0x82, 0x73, 0x12, 0x91, 0x52, 0x2f, 0x38, + 0xa0, 0x4c, 0x87, 0x57, 0x07, 0x21, 0x8d, 0x45, 0x38, 0x39, 0xc1, 0x06, 0x56, 0x87, 0x71, 0xfa, + 0xe5, 0x5c, 0xfa, 0x8e, 0x0f, 0xb7, 0x2f, 0x98, 0x69, 0x84, 0xc7, 0x0d, 0x61, 0x48, 0x0a, 0xdf, + 0xbc, 0x6a, 0x15, 0x28, 0xb4, 0xda, 0x08, 0x35, 0x31, 0x9e, 0xda, 0x7e, 0xd4, 0x28, 0xce, 0x4f, + 0x45, 0xb0, 0x2e, 0x37, 0x44, 0x5b, 0x70, 0x93, 0x07, 0x9c, 0xf4, 0xe4, 0xae, 0x22, 0xad, 0x5e, + 0xa6, 0xd5, 0x63, 0xbc, 0x94, 0x26, 0xf6, 0x74, 0x00, 0x9e, 0x2e, 0x46, 0xbf, 0x1a, 0x70, 0x77, + 0xaa, 0x66, 0x9b, 0xb2, 0x1d, 0x4e, 0x43, 0xdd, 0xee, 0x1f, 0x5c, 0x51, 0xdd, 0x59, 0x6b, 0x99, + 0xad, 0x76, 0xe1, 0x36, 0xd2, 0xc4, 0xbe, 0x34, 0x08, 0xbe, 0x54, 0xeb, 0x78, 0xf0, 0x92, 0x11, + 0xc5, 0x75, 0xca, 0x29, 0xd4, 0xed, 0xaf, 0x0e, 0xe7, 0x66, 0xa3, 0x78, 0x6e, 0x36, 0x9c, 0x5d, + 0xa8, 0x5f, 0xf4, 0x9c, 0xa1, 0x25, 0x28, 0x7f, 0x49, 0xfc, 0xec, 0xc5, 0xd1, 0x2b, 0x4f, 0x8a, + 0xd0, 0x2b, 0x60, 0x3e, 0x96, 0x53, 0x2f, 0xe9, 0x1a, 0x29, 0xb5, 0xd0, 0xf9, 0xc5, 0x80, 0x9b, + 0x53, 0xdf, 0x19, 0xb4, 0x02, 0xe6, 0x33, 0xda, 0xe6, 0x01, 0xd3, 0x5d, 0x34, 0xb9, 0xd0, 0x1f, + 0x4b, 0xd5, 0x66, 0x01, 0x6b, 0x10, 0xba, 0x0b, 0x55, 0x46, 0x0e, 0xdc, 0x01, 0xa7, 0x2a, 0xfb, + 0xb9, 0xcd, 0x02, 0x1e, 0x49, 0x84, 0x33, 0x9f, 0x70, 0xe6, 0x1d, 0xea, 0xd7, 0x69, 0xd2, 0xd9, + 0x43, 0xa9, 0x12, 0xce, 0x14, 0xc8, 0xad, 0x82, 0x7e, 0xdd, 0x9c, 0x7b, 0x60, 0xaa, 0x50, 0x68, + 0x25, 0xdf, 0xd6, 0xe7, 0x5f, 0x18, 0xc5, 0xf5, 0xb8, 0x63, 0x7f, 0x28, 0x82, 0xa9, 0x64, 0xff, + 0xe1, 0x6e, 0xbe, 0x07, 0xa6, 0xca, 0x47, 0x2f, 0xb3, 0xf3, 0xab, 0xf9, 0xda, 0x51, 0x62, 0x1b, + 0xe2, 0x85, 0x93, 0x7d, 0x80, 0x35, 0x1c, 0x3d, 0xca, 0x2f, 0x42, 0x45, 0xd9, 0xd5, 0x7b, 0xf9, + 0x7f, 0xda, 0xd7, 0xd8, 0x34, 0xbf, 0x19, 0xb7, 0xc0, 0x54, 0x3c, 0xa3, 0x0d, 0xb8, 0x16, 0xe5, + 0xde, 0xae, 0x8c, 0x96, 0xa5, 0x29, 0x01, 0x14, 0x42, 0xaf, 0xd9, 0x49, 0x2b, 0x77, 0xfd, 0xf8, + 0xc4, 0x2a, 0x3c, 0x3f, 0xb1, 0x0a, 0x2f, 0x4e, 0x2c, 0xe3, 0xfb, 0xa1, 0x65, 0xfc, 0x36, 0xb4, + 0x8c, 0xa3, 0xa1, 0x65, 0x1c, 0x0f, 0x2d, 0xe3, 0xef, 0xa1, 0x65, 0xfc, 0x33, 0xb4, 0x0a, 0x2f, + 0x86, 0x96, 0xf1, 0xe3, 0xa9, 0x55, 0x38, 0x3e, 0xb5, 0x0a, 0xcf, 0x4f, 0xad, 0xc2, 0xd7, 0xf9, + 0x7f, 0x98, 0x5b, 0xa6, 0x7c, 0x72, 0xdf, 0xfe, 0x37, 0x00, 0x00, 0xff, 0xff, 0xaa, 0xc5, 0x91, + 0xf0, 0x53, 0x0b, 0x00, 0x00, +} + +func (this *PrometheusResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PrometheusResponse) + if !ok { + that2, ok := that.(PrometheusResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Status != that1.Status { + return false + } + if !this.Data.Equal(&that1.Data) { + return false + } + if this.ErrorType != that1.ErrorType { + return false + } + if this.Error != that1.Error { + return false + } + if len(this.Headers) != len(that1.Headers) { + return false + } + for i := range this.Headers { + if !this.Headers[i].Equal(that1.Headers[i]) { + return false + } + } + if len(this.Warnings) != len(that1.Warnings) { + return false + } + for i := range this.Warnings { + if this.Warnings[i] != that1.Warnings[i] { + return false + } + } + return true +} +func (this *PrometheusData) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PrometheusData) + if !ok { + that2, ok := that.(PrometheusData) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.ResultType != that1.ResultType { + return false + } + if !this.Result.Equal(&that1.Result) { + return false + } + if !this.Stats.Equal(that1.Stats) { + return false + } + return true +} +func (this *CachedResponse) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*CachedResponse) + if !ok { + that2, ok := that.(CachedResponse) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Key != that1.Key { + return false + } + if len(this.Extents) != len(that1.Extents) { + return false + } + for i := range this.Extents { + if !this.Extents[i].Equal(&that1.Extents[i]) { + return false + } + } + return true +} +func (this *Extent) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Extent) + if !ok { + that2, ok := that.(Extent) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Start != that1.Start { + return false + } + if this.End != that1.End { + return false + } + if this.TraceId != that1.TraceId { + return false + } + if !this.Response.Equal(that1.Response) { + return false + } + return true +} +func (this *SampleStream) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*SampleStream) + if !ok { + that2, ok := that.(SampleStream) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if !this.Labels[i].Equal(that1.Labels[i]) { + return false + } + } + if len(this.Samples) != len(that1.Samples) { + return false + } + for i := range this.Samples { + if !this.Samples[i].Equal(&that1.Samples[i]) { + return false + } } if len(this.Histograms) != len(that1.Histograms) { return false @@ -820,14 +1454,14 @@ func (this *PrometheusResponseHeader) Equal(that interface{}) bool { } return true } -func (this *PrometheusRequestHeader) Equal(that interface{}) bool { +func (this *PrometheusQueryResult) Equal(that interface{}) bool { if that == nil { return this == nil } - that1, ok := that.(*PrometheusRequestHeader) + that1, ok := that.(*PrometheusQueryResult) if !ok { - that2, ok := that.(PrometheusRequestHeader) + that2, ok := that.(PrometheusQueryResult) if ok { that1 = &that2 } else { @@ -839,28 +1473,254 @@ func (this *PrometheusRequestHeader) Equal(that interface{}) bool { } else if this == nil { return false } - if this.Name != that1.Name { + if that1.Result == nil { + if this.Result != nil { + return false + } + } else if this.Result == nil { return false - } - if len(this.Values) != len(that1.Values) { + } else if !this.Result.Equal(that1.Result) { return false } - for i := range this.Values { - if this.Values[i] != that1.Values[i] { + return true +} +func (this *PrometheusQueryResult_Vector) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PrometheusQueryResult_Vector) + if !ok { + that2, ok := that.(PrometheusQueryResult_Vector) + if ok { + that1 = &that2 + } else { return false } } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Vector.Equal(that1.Vector) { + return false + } return true } -func (this *SampleStream) GoString() string { - if this == nil { - return "nil" +func (this *PrometheusQueryResult_RawBytes) Equal(that interface{}) bool { + if that == nil { + return this == nil } - s := make([]string, 0, 7) - s = append(s, "&tripperware.SampleStream{") - s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") - if this.Samples != nil { - vs := make([]*cortexpb.Sample, len(this.Samples)) + + that1, ok := that.(*PrometheusQueryResult_RawBytes) + if !ok { + that2, ok := that.(PrometheusQueryResult_RawBytes) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !bytes.Equal(this.RawBytes, that1.RawBytes) { + return false + } + return true +} +func (this *PrometheusQueryResult_Matrix) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PrometheusQueryResult_Matrix) + if !ok { + that2, ok := that.(PrometheusQueryResult_Matrix) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Matrix.Equal(that1.Matrix) { + return false + } + return true +} +func (this *Vector) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Vector) + if !ok { + that2, ok := that.(Vector) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Samples) != len(that1.Samples) { + return false + } + for i := range this.Samples { + if !this.Samples[i].Equal(that1.Samples[i]) { + return false + } + } + return true +} +func (this *Sample) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Sample) + if !ok { + that2, ok := that.(Sample) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.Labels) != len(that1.Labels) { + return false + } + for i := range this.Labels { + if !this.Labels[i].Equal(that1.Labels[i]) { + return false + } + } + if !this.Sample.Equal(that1.Sample) { + return false + } + if !this.Histogram.Equal(that1.Histogram) { + return false + } + return true +} +func (this *Matrix) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Matrix) + if !ok { + that2, ok := that.(Matrix) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if len(this.SampleStreams) != len(that1.SampleStreams) { + return false + } + for i := range this.SampleStreams { + if !this.SampleStreams[i].Equal(&that1.SampleStreams[i]) { + return false + } + } + return true +} +func (this *PrometheusResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 10) + s = append(s, "&tripperware.PrometheusResponse{") + s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") + s = append(s, "Data: "+strings.Replace(this.Data.GoString(), `&`, ``, 1)+",\n") + s = append(s, "ErrorType: "+fmt.Sprintf("%#v", this.ErrorType)+",\n") + s = append(s, "Error: "+fmt.Sprintf("%#v", this.Error)+",\n") + if this.Headers != nil { + s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") + } + s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") + s = append(s, "}") + return strings.Join(s, "") +} +func (this *PrometheusData) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&tripperware.PrometheusData{") + s = append(s, "ResultType: "+fmt.Sprintf("%#v", this.ResultType)+",\n") + s = append(s, "Result: "+strings.Replace(this.Result.GoString(), `&`, ``, 1)+",\n") + if this.Stats != nil { + s = append(s, "Stats: "+fmt.Sprintf("%#v", this.Stats)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *CachedResponse) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 6) + s = append(s, "&tripperware.CachedResponse{") + s = append(s, "Key: "+fmt.Sprintf("%#v", this.Key)+",\n") + if this.Extents != nil { + vs := make([]*Extent, len(this.Extents)) + for i := range vs { + vs[i] = &this.Extents[i] + } + s = append(s, "Extents: "+fmt.Sprintf("%#v", vs)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Extent) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 8) + s = append(s, "&tripperware.Extent{") + s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") + s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") + s = append(s, "TraceId: "+fmt.Sprintf("%#v", this.TraceId)+",\n") + if this.Response != nil { + s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *SampleStream) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&tripperware.SampleStream{") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + if this.Samples != nil { + vs := make([]*cortexpb.Sample, len(this.Samples)) for i := range vs { vs[i] = &this.Samples[i] } @@ -961,14 +1821,83 @@ func (this *PrometheusResponseHeader) GoString() string { s = append(s, "}") return strings.Join(s, "") } -func (this *PrometheusRequestHeader) GoString() string { +func (this *PrometheusQueryResult) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) - s = append(s, "&tripperware.PrometheusRequestHeader{") - s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") - s = append(s, "Values: "+fmt.Sprintf("%#v", this.Values)+",\n") + s := make([]string, 0, 7) + s = append(s, "&tripperware.PrometheusQueryResult{") + if this.Result != nil { + s = append(s, "Result: "+fmt.Sprintf("%#v", this.Result)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *PrometheusQueryResult_Vector) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&tripperware.PrometheusQueryResult_Vector{` + + `Vector:` + fmt.Sprintf("%#v", this.Vector) + `}`}, ", ") + return s +} +func (this *PrometheusQueryResult_RawBytes) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&tripperware.PrometheusQueryResult_RawBytes{` + + `RawBytes:` + fmt.Sprintf("%#v", this.RawBytes) + `}`}, ", ") + return s +} +func (this *PrometheusQueryResult_Matrix) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&tripperware.PrometheusQueryResult_Matrix{` + + `Matrix:` + fmt.Sprintf("%#v", this.Matrix) + `}`}, ", ") + return s +} +func (this *Vector) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&tripperware.Vector{") + if this.Samples != nil { + s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Sample) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 7) + s = append(s, "&tripperware.Sample{") + s = append(s, "Labels: "+fmt.Sprintf("%#v", this.Labels)+",\n") + if this.Sample != nil { + s = append(s, "Sample: "+fmt.Sprintf("%#v", this.Sample)+",\n") + } + if this.Histogram != nil { + s = append(s, "Histogram: "+fmt.Sprintf("%#v", this.Histogram)+",\n") + } + s = append(s, "}") + return strings.Join(s, "") +} +func (this *Matrix) GoString() string { + if this == nil { + return "nil" + } + s := make([]string, 0, 5) + s = append(s, "&tripperware.Matrix{") + if this.SampleStreams != nil { + vs := make([]*SampleStream, len(this.SampleStreams)) + for i := range vs { + vs[i] = &this.SampleStreams[i] + } + s = append(s, "SampleStreams: "+fmt.Sprintf("%#v", vs)+",\n") + } s = append(s, "}") return strings.Join(s, "") } @@ -980,7 +1909,7 @@ func valueToGoStringQuery(v interface{}, typ string) string { pv := reflect.Indirect(rv).Interface() return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) } -func (m *SampleStream) Marshal() (dAtA []byte, err error) { +func (m *PrometheusResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -990,34 +1919,29 @@ func (m *SampleStream) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SampleStream) MarshalTo(dAtA []byte) (int, error) { +func (m *PrometheusResponse) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *SampleStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *PrometheusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Histograms) > 0 { - for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQuery(dAtA, i, uint64(size)) - } + if len(m.Warnings) > 0 { + for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Warnings[iNdEx]) + copy(dAtA[i:], m.Warnings[iNdEx]) + i = encodeVarintQuery(dAtA, i, uint64(len(m.Warnings[iNdEx]))) i-- - dAtA[i] = 0x1a + dAtA[i] = 0x32 } } - if len(m.Samples) > 0 { - for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Headers) > 0 { + for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Headers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1025,27 +1949,44 @@ func (m *SampleStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintQuery(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x12 + dAtA[i] = 0x2a } } - if len(m.Labels) > 0 { - for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { - { - size := m.Labels[iNdEx].Size() - i -= size - if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - i = encodeVarintQuery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa + if len(m.Error) > 0 { + i -= len(m.Error) + copy(dAtA[i:], m.Error) + i = encodeVarintQuery(dAtA, i, uint64(len(m.Error))) + i-- + dAtA[i] = 0x22 + } + if len(m.ErrorType) > 0 { + i -= len(m.ErrorType) + copy(dAtA[i:], m.ErrorType) + i = encodeVarintQuery(dAtA, i, uint64(len(m.ErrorType))) + i-- + dAtA[i] = 0x1a + } + { + size, err := m.Data.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + if len(m.Status) > 0 { + i -= len(m.Status) + copy(dAtA[i:], m.Status) + i = encodeVarintQuery(dAtA, i, uint64(len(m.Status))) + i-- + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *SampleHistogramPair) Marshal() (dAtA []byte, err error) { +func (m *PrometheusData) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1055,18 +1996,231 @@ func (m *SampleHistogramPair) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SampleHistogramPair) MarshalTo(dAtA []byte) (int, error) { +func (m *PrometheusData) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *SampleHistogramPair) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *PrometheusData) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - { - size, err := m.Histogram.MarshalToSizedBuffer(dAtA[:i]) + if m.Stats != nil { + { + size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + { + size, err := m.Result.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + if len(m.ResultType) > 0 { + i -= len(m.ResultType) + copy(dAtA[i:], m.ResultType) + i = encodeVarintQuery(dAtA, i, uint64(len(m.ResultType))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *CachedResponse) 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 *CachedResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CachedResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Extents) > 0 { + for iNdEx := len(m.Extents) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Extents[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintQuery(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Extent) 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 *Extent) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Extent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Response != nil { + { + size, err := m.Response.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if len(m.TraceId) > 0 { + i -= len(m.TraceId) + copy(dAtA[i:], m.TraceId) + i = encodeVarintQuery(dAtA, i, uint64(len(m.TraceId))) + i-- + dAtA[i] = 0x22 + } + if m.End != 0 { + i = encodeVarintQuery(dAtA, i, uint64(m.End)) + i-- + dAtA[i] = 0x10 + } + if m.Start != 0 { + i = encodeVarintQuery(dAtA, i, uint64(m.Start)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *SampleStream) 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 *SampleStream) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SampleStream) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Histograms) > 0 { + for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *SampleHistogramPair) 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 *SampleHistogramPair) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SampleHistogramPair) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Histogram.MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -1327,7 +2481,7 @@ func (m *PrometheusResponseHeader) MarshalToSizedBuffer(dAtA []byte) (int, error return len(dAtA) - i, nil } -func (m *PrometheusRequestHeader) Marshal() (dAtA []byte, err error) { +func (m *PrometheusQueryResult) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -1337,65 +2491,344 @@ func (m *PrometheusRequestHeader) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *PrometheusRequestHeader) MarshalTo(dAtA []byte) (int, error) { +func (m *PrometheusQueryResult) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *PrometheusRequestHeader) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *PrometheusQueryResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int _ = l - if len(m.Values) > 0 { - for iNdEx := len(m.Values) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Values[iNdEx]) - copy(dAtA[i:], m.Values[iNdEx]) - i = encodeVarintQuery(dAtA, i, uint64(len(m.Values[iNdEx]))) - i-- - dAtA[i] = 0x12 + if m.Result != nil { + { + size := m.Result.Size() + i -= size + if _, err := m.Result.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } } } - if len(m.Name) > 0 { - i -= len(m.Name) - copy(dAtA[i:], m.Name) - i = encodeVarintQuery(dAtA, i, uint64(len(m.Name))) + return len(dAtA) - i, nil +} + +func (m *PrometheusQueryResult_Vector) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PrometheusQueryResult_Vector) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Vector != nil { + { + size, err := m.Vector.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } +func (m *PrometheusQueryResult_RawBytes) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} -func encodeVarintQuery(dAtA []byte, offset int, v uint64) int { - offset -= sovQuery(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *PrometheusQueryResult_RawBytes) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.RawBytes != nil { + i -= len(m.RawBytes) + copy(dAtA[i:], m.RawBytes) + i = encodeVarintQuery(dAtA, i, uint64(len(m.RawBytes))) + i-- + dAtA[i] = 0x12 } - dAtA[offset] = uint8(v) - return base + return len(dAtA) - i, nil } -func (m *SampleStream) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if len(m.Labels) > 0 { - for _, e := range m.Labels { - l = e.Size() - n += 1 + l + sovQuery(uint64(l)) +func (m *PrometheusQueryResult_Matrix) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PrometheusQueryResult_Matrix) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Matrix != nil { + { + size, err := m.Matrix.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x1a } - if len(m.Samples) > 0 { - for _, e := range m.Samples { - l = e.Size() - n += 1 + l + sovQuery(uint64(l)) - } + return len(dAtA) - i, nil +} +func (m *Vector) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err } - if len(m.Histograms) > 0 { + return dAtA[:n], nil +} + +func (m *Vector) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Vector) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Sample) 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 *Sample) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Histogram != nil { + { + size, err := m.Histogram.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Sample != nil { + { + size, err := m.Sample.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.Labels) > 0 { + for iNdEx := len(m.Labels) - 1; iNdEx >= 0; iNdEx-- { + { + size := m.Labels[iNdEx].Size() + i -= size + if _, err := m.Labels[iNdEx].MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *Matrix) 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 *Matrix) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Matrix) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.SampleStreams) > 0 { + for iNdEx := len(m.SampleStreams) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.SampleStreams[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func encodeVarintQuery(dAtA []byte, offset int, v uint64) int { + offset -= sovQuery(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *PrometheusResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Status) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + l = m.Data.Size() + n += 1 + l + sovQuery(uint64(l)) + l = len(m.ErrorType) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + if len(m.Headers) > 0 { + for _, e := range m.Headers { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + if len(m.Warnings) > 0 { + for _, s := range m.Warnings { + l = len(s) + n += 1 + l + sovQuery(uint64(l)) + } + } + return n +} + +func (m *PrometheusData) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ResultType) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + l = m.Result.Size() + n += 1 + l + sovQuery(uint64(l)) + if m.Stats != nil { + l = m.Stats.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} + +func (m *CachedResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + if len(m.Extents) > 0 { + for _, e := range m.Extents { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + return n +} + +func (m *Extent) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Start != 0 { + n += 1 + sovQuery(uint64(m.Start)) + } + if m.End != 0 { + n += 1 + sovQuery(uint64(m.End)) + } + l = len(m.TraceId) + if l > 0 { + n += 1 + l + sovQuery(uint64(l)) + } + if m.Response != nil { + l = m.Response.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} + +func (m *SampleStream) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + if len(m.Samples) > 0 { + for _, e := range m.Samples { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + if len(m.Histograms) > 0 { for _, e := range m.Histograms { l = e.Size() n += 1 + l + sovQuery(uint64(l)) @@ -1525,56 +2958,199 @@ func (m *PrometheusResponseHeader) Size() (n int) { return n } -func (m *PrometheusRequestHeader) Size() (n int) { +func (m *PrometheusQueryResult) Size() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sovQuery(uint64(l)) - } - if len(m.Values) > 0 { - for _, s := range m.Values { - l = len(s) - n += 1 + l + sovQuery(uint64(l)) - } + if m.Result != nil { + n += m.Result.Size() } return n } -func sovQuery(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozQuery(x uint64) (n int) { - return sovQuery(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (this *SampleStream) String() string { - if this == nil { - return "nil" - } - repeatedStringForSamples := "[]Sample{" - for _, f := range this.Samples { - repeatedStringForSamples += fmt.Sprintf("%v", f) + "," +func (m *PrometheusQueryResult_Vector) Size() (n int) { + if m == nil { + return 0 } - repeatedStringForSamples += "}" - repeatedStringForHistograms := "[]SampleHistogramPair{" - for _, f := range this.Histograms { - repeatedStringForHistograms += strings.Replace(strings.Replace(f.String(), "SampleHistogramPair", "SampleHistogramPair", 1), `&`, ``, 1) + "," + var l int + _ = l + if m.Vector != nil { + l = m.Vector.Size() + n += 1 + l + sovQuery(uint64(l)) } - repeatedStringForHistograms += "}" - s := strings.Join([]string{`&SampleStream{`, - `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, - `Samples:` + repeatedStringForSamples + `,`, - `Histograms:` + repeatedStringForHistograms + `,`, - `}`, - }, "") - return s + return n } -func (this *SampleHistogramPair) String() string { - if this == nil { - return "nil" +func (m *PrometheusQueryResult_RawBytes) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RawBytes != nil { + l = len(m.RawBytes) + n += 1 + l + sovQuery(uint64(l)) + } + return n +} +func (m *PrometheusQueryResult_Matrix) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Matrix != nil { + l = m.Matrix.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} +func (m *Vector) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Samples) > 0 { + for _, e := range m.Samples { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + return n +} + +func (m *Sample) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Labels) > 0 { + for _, e := range m.Labels { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + if m.Sample != nil { + l = m.Sample.Size() + n += 1 + l + sovQuery(uint64(l)) + } + if m.Histogram != nil { + l = m.Histogram.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} + +func (m *Matrix) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.SampleStreams) > 0 { + for _, e := range m.SampleStreams { + l = e.Size() + n += 1 + l + sovQuery(uint64(l)) + } + } + return n +} + +func sovQuery(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozQuery(x uint64) (n int) { + return sovQuery(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (this *PrometheusResponse) String() string { + if this == nil { + return "nil" + } + repeatedStringForHeaders := "[]*PrometheusResponseHeader{" + for _, f := range this.Headers { + repeatedStringForHeaders += strings.Replace(f.String(), "PrometheusResponseHeader", "PrometheusResponseHeader", 1) + "," + } + repeatedStringForHeaders += "}" + s := strings.Join([]string{`&PrometheusResponse{`, + `Status:` + fmt.Sprintf("%v", this.Status) + `,`, + `Data:` + strings.Replace(strings.Replace(this.Data.String(), "PrometheusData", "PrometheusData", 1), `&`, ``, 1) + `,`, + `ErrorType:` + fmt.Sprintf("%v", this.ErrorType) + `,`, + `Error:` + fmt.Sprintf("%v", this.Error) + `,`, + `Headers:` + repeatedStringForHeaders + `,`, + `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, + `}`, + }, "") + return s +} +func (this *PrometheusData) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PrometheusData{`, + `ResultType:` + fmt.Sprintf("%v", this.ResultType) + `,`, + `Result:` + strings.Replace(strings.Replace(this.Result.String(), "PrometheusQueryResult", "PrometheusQueryResult", 1), `&`, ``, 1) + `,`, + `Stats:` + strings.Replace(this.Stats.String(), "PrometheusResponseStats", "PrometheusResponseStats", 1) + `,`, + `}`, + }, "") + return s +} +func (this *CachedResponse) String() string { + if this == nil { + return "nil" + } + repeatedStringForExtents := "[]Extent{" + for _, f := range this.Extents { + repeatedStringForExtents += strings.Replace(strings.Replace(f.String(), "Extent", "Extent", 1), `&`, ``, 1) + "," + } + repeatedStringForExtents += "}" + s := strings.Join([]string{`&CachedResponse{`, + `Key:` + fmt.Sprintf("%v", this.Key) + `,`, + `Extents:` + repeatedStringForExtents + `,`, + `}`, + }, "") + return s +} +func (this *Extent) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Extent{`, + `Start:` + fmt.Sprintf("%v", this.Start) + `,`, + `End:` + fmt.Sprintf("%v", this.End) + `,`, + `TraceId:` + fmt.Sprintf("%v", this.TraceId) + `,`, + `Response:` + strings.Replace(fmt.Sprintf("%v", this.Response), "Any", "types.Any", 1) + `,`, + `}`, + }, "") + return s +} +func (this *SampleStream) String() string { + if this == nil { + return "nil" + } + repeatedStringForSamples := "[]Sample{" + for _, f := range this.Samples { + repeatedStringForSamples += fmt.Sprintf("%v", f) + "," + } + repeatedStringForSamples += "}" + repeatedStringForHistograms := "[]SampleHistogramPair{" + for _, f := range this.Histograms { + repeatedStringForHistograms += strings.Replace(strings.Replace(f.String(), "SampleHistogramPair", "SampleHistogramPair", 1), `&`, ``, 1) + "," + } + repeatedStringForHistograms += "}" + s := strings.Join([]string{`&SampleStream{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `Samples:` + repeatedStringForSamples + `,`, + `Histograms:` + repeatedStringForHistograms + `,`, + `}`, + }, "") + return s +} +func (this *SampleHistogramPair) String() string { + if this == nil { + return "nil" } s := strings.Join([]string{`&SampleHistogramPair{`, `TimestampMs:` + fmt.Sprintf("%v", this.TimestampMs) + `,`, @@ -1661,13 +3237,84 @@ func (this *PrometheusResponseHeader) String() string { }, "") return s } -func (this *PrometheusRequestHeader) String() string { +func (this *PrometheusQueryResult) String() string { if this == nil { return "nil" } - s := strings.Join([]string{`&PrometheusRequestHeader{`, - `Name:` + fmt.Sprintf("%v", this.Name) + `,`, - `Values:` + fmt.Sprintf("%v", this.Values) + `,`, + s := strings.Join([]string{`&PrometheusQueryResult{`, + `Result:` + fmt.Sprintf("%v", this.Result) + `,`, + `}`, + }, "") + return s +} +func (this *PrometheusQueryResult_Vector) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PrometheusQueryResult_Vector{`, + `Vector:` + strings.Replace(fmt.Sprintf("%v", this.Vector), "Vector", "Vector", 1) + `,`, + `}`, + }, "") + return s +} +func (this *PrometheusQueryResult_RawBytes) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PrometheusQueryResult_RawBytes{`, + `RawBytes:` + fmt.Sprintf("%v", this.RawBytes) + `,`, + `}`, + }, "") + return s +} +func (this *PrometheusQueryResult_Matrix) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PrometheusQueryResult_Matrix{`, + `Matrix:` + strings.Replace(fmt.Sprintf("%v", this.Matrix), "Matrix", "Matrix", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Vector) String() string { + if this == nil { + return "nil" + } + repeatedStringForSamples := "[]*Sample{" + for _, f := range this.Samples { + repeatedStringForSamples += strings.Replace(f.String(), "Sample", "Sample", 1) + "," + } + repeatedStringForSamples += "}" + s := strings.Join([]string{`&Vector{`, + `Samples:` + repeatedStringForSamples + `,`, + `}`, + }, "") + return s +} +func (this *Sample) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&Sample{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `Sample:` + strings.Replace(fmt.Sprintf("%v", this.Sample), "Sample", "cortexpb.Sample", 1) + `,`, + `Histogram:` + strings.Replace(this.Histogram.String(), "SampleHistogramPair", "SampleHistogramPair", 1) + `,`, + `}`, + }, "") + return s +} +func (this *Matrix) String() string { + if this == nil { + return "nil" + } + repeatedStringForSampleStreams := "[]SampleStream{" + for _, f := range this.SampleStreams { + repeatedStringForSampleStreams += strings.Replace(strings.Replace(f.String(), "SampleStream", "SampleStream", 1), `&`, ``, 1) + "," + } + repeatedStringForSampleStreams += "}" + s := strings.Join([]string{`&Matrix{`, + `SampleStreams:` + repeatedStringForSampleStreams + `,`, `}`, }, "") return s @@ -1680,7 +3327,7 @@ func valueToStringQuery(v interface{}) string { pv := reflect.Indirect(rv).Interface() return fmt.Sprintf("*%v", pv) } -func (m *SampleStream) Unmarshal(dAtA []byte) error { +func (m *PrometheusResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1703,15 +3350,47 @@ func (m *SampleStream) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SampleStream: wiretype end group for non-group") + return fmt.Errorf("proto: PrometheusResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SampleStream: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrometheusResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Status = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1738,16 +3417,15 @@ func (m *SampleStream) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Labels = append(m.Labels, github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{}) - if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Data.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ErrorType", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -1757,29 +3435,892 @@ func (m *SampleStream) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ErrorType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthQuery + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Error = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Headers = append(m.Headers, &PrometheusResponseHeader{}) + if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PrometheusData) 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 ErrIntOverflowQuery + } + 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: PrometheusData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PrometheusData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResultType", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ResultType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Result.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Stats == nil { + m.Stats = &PrometheusResponseStats{} + } + if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CachedResponse) 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 ErrIntOverflowQuery + } + 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: CachedResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CachedResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Extents", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Extents = append(m.Extents, Extent{}) + if err := m.Extents[len(m.Extents)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Extent) 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 ErrIntOverflowQuery + } + 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: Extent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Extent: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) + } + m.Start = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Start |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) + } + m.End = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.End |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TraceId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TraceId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Response == nil { + m.Response = &types.Any{} + } + if err := m.Response.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SampleStream) 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 ErrIntOverflowQuery + } + 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: SampleStream: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SampleStream: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Labels = append(m.Labels, github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Samples = append(m.Samples, cortexpb.Sample{}) + if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histograms", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Histograms = append(m.Histograms, SampleHistogramPair{}) + if err := m.Histograms[len(m.Histograms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SampleHistogramPair) 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 ErrIntOverflowQuery + } + 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: SampleHistogramPair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SampleHistogramPair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + } + m.TimestampMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimestampMs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histogram", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Histogram.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SampleHistogram) 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 ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQuery + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SampleHistogram: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SampleHistogram: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) } - if postIndex > l { + var v uint64 + if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.Samples = append(m.Samples, cortexpb.Sample{}) - if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = float64(math.Float64frombits(v)) + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) } - iNdEx = postIndex + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Sum = float64(math.Float64frombits(v)) case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Histograms", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Buckets", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1806,8 +4347,8 @@ func (m *SampleStream) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Histograms = append(m.Histograms, SampleHistogramPair{}) - if err := m.Histograms[len(m.Histograms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Buckets = append(m.Buckets, &HistogramBucket{}) + if err := m.Buckets[len(m.Buckets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1835,7 +4376,7 @@ func (m *SampleStream) Unmarshal(dAtA []byte) error { } return nil } -func (m *SampleHistogramPair) Unmarshal(dAtA []byte) error { +func (m *HistogramBucket) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1858,17 +4399,17 @@ func (m *SampleHistogramPair) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SampleHistogramPair: wiretype end group for non-group") + return fmt.Errorf("proto: HistogramBucket: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SampleHistogramPair: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: HistogramBucket: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Boundaries", wireType) } - m.TimestampMs = 0 + m.Boundaries = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -1878,14 +4419,100 @@ func (m *SampleHistogramPair) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TimestampMs |= int64(b&0x7F) << shift + m.Boundaries |= int32(b&0x7F) << shift if b < 0x80 { break } } case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Lower", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Lower = float64(math.Float64frombits(v)) + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Upper", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Upper = float64(math.Float64frombits(v)) + case 4: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = float64(math.Float64frombits(v)) + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *PrometheusResponseStats) 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 ErrIntOverflowQuery + } + 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: PrometheusResponseStats: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PrometheusResponseStats: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Histogram", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1912,7 +4539,10 @@ func (m *SampleHistogramPair) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := m.Histogram.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Samples == nil { + m.Samples = &PrometheusResponseSamplesStats{} + } + if err := m.Samples.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1940,7 +4570,7 @@ func (m *SampleHistogramPair) Unmarshal(dAtA []byte) error { } return nil } -func (m *SampleHistogram) Unmarshal(dAtA []byte) error { +func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1963,37 +4593,34 @@ func (m *SampleHistogram) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SampleHistogram: wiretype end group for non-group") + return fmt.Errorf("proto: PrometheusResponseSamplesStats: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SampleHistogram: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrometheusResponseSamplesStats: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TotalQueryableSamples", wireType) } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF + m.TotalQueryableSamples = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TotalQueryableSamples |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Count = float64(math.Float64frombits(v)) case 2: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) - } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Sum = float64(math.Float64frombits(v)) - case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Buckets", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TotalQueryableSamplesPerStep", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -2020,8 +4647,8 @@ func (m *SampleHistogram) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Buckets = append(m.Buckets, &HistogramBucket{}) - if err := m.Buckets[len(m.Buckets)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.TotalQueryableSamplesPerStep = append(m.TotalQueryableSamplesPerStep, &PrometheusResponseQueryableSamplesStatsPerStep{}) + if err := m.TotalQueryableSamplesPerStep[len(m.TotalQueryableSamplesPerStep)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -2049,7 +4676,7 @@ func (m *SampleHistogram) Unmarshal(dAtA []byte) error { } return nil } -func (m *HistogramBucket) Unmarshal(dAtA []byte) error { +func (m *PrometheusResponseQueryableSamplesStatsPerStep) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2072,17 +4699,17 @@ func (m *HistogramBucket) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: HistogramBucket: wiretype end group for non-group") + return fmt.Errorf("proto: PrometheusResponseQueryableSamplesStatsPerStep: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: HistogramBucket: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrometheusResponseQueryableSamplesStatsPerStep: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Boundaries", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) } - m.Boundaries = 0 + m.Value = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2092,44 +4719,30 @@ func (m *HistogramBucket) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Boundaries |= int32(b&0x7F) << shift + m.Value |= int64(b&0x7F) << shift if b < 0x80 { break } } case 2: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Lower", wireType) - } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Lower = float64(math.Float64frombits(v)) - case 3: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Upper", wireType) - } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Upper = float64(math.Float64frombits(v)) - case 4: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field Count", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) } - var v uint64 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF + m.TimestampMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimestampMs |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) - iNdEx += 8 - m.Count = float64(math.Float64frombits(v)) default: iNdEx = preIndex skippy, err := skipQuery(dAtA[iNdEx:]) @@ -2154,7 +4767,7 @@ func (m *HistogramBucket) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrometheusResponseStats) Unmarshal(dAtA []byte) error { +func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2177,17 +4790,17 @@ func (m *PrometheusResponseStats) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrometheusResponseStats: wiretype end group for non-group") + return fmt.Errorf("proto: PrometheusResponseHeader: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusResponseStats: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrometheusResponseHeader: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2197,27 +4810,55 @@ func (m *PrometheusResponseStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthQuery } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLengthQuery } if postIndex > l { return io.ErrUnexpectedEOF } - if m.Samples == nil { - m.Samples = &PrometheusResponseSamplesStats{} + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) } - if err := m.Samples.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF } + m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex @@ -2243,7 +4884,7 @@ func (m *PrometheusResponseStats) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { +func (m *PrometheusQueryResult) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2266,17 +4907,17 @@ func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrometheusResponseSamplesStats: wiretype end group for non-group") + return fmt.Errorf("proto: PrometheusQueryResult: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusResponseSamplesStats: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrometheusQueryResult: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TotalQueryableSamples", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Vector", wireType) } - m.TotalQueryableSamples = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2286,14 +4927,63 @@ func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TotalQueryableSamples |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &Vector{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Result = &PrometheusQueryResult_Vector{v} + iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TotalQueryableSamplesPerStep", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RawBytes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := make([]byte, postIndex-iNdEx) + copy(v, dAtA[iNdEx:postIndex]) + m.Result = &PrometheusQueryResult_RawBytes{v} + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Matrix", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -2320,10 +5010,11 @@ func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TotalQueryableSamplesPerStep = append(m.TotalQueryableSamplesPerStep, &PrometheusResponseQueryableSamplesStatsPerStep{}) - if err := m.TotalQueryableSamplesPerStep[len(m.TotalQueryableSamplesPerStep)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + v := &Matrix{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } + m.Result = &PrometheusQueryResult_Matrix{v} iNdEx = postIndex default: iNdEx = preIndex @@ -2349,7 +5040,7 @@ func (m *PrometheusResponseSamplesStats) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrometheusResponseQueryableSamplesStatsPerStep) Unmarshal(dAtA []byte) error { +func (m *Vector) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2372,17 +5063,17 @@ func (m *PrometheusResponseQueryableSamplesStatsPerStep) Unmarshal(dAtA []byte) fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrometheusResponseQueryableSamplesStatsPerStep: wiretype end group for non-group") + return fmt.Errorf("proto: Vector: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusResponseQueryableSamplesStatsPerStep: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Vector: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) } - m.Value = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2392,30 +5083,26 @@ func (m *PrometheusResponseQueryableSamplesStatsPerStep) Unmarshal(dAtA []byte) } b := dAtA[iNdEx] iNdEx++ - m.Value |= int64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimestampMs", wireType) + if msglen < 0 { + return ErrInvalidLengthQuery } - m.TimestampMs = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQuery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TimestampMs |= int64(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Samples = append(m.Samples, &Sample{}) + if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQuery(dAtA[iNdEx:]) @@ -2440,7 +5127,7 @@ func (m *PrometheusResponseQueryableSamplesStatsPerStep) Unmarshal(dAtA []byte) } return nil } -func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { +func (m *Sample) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2463,17 +5150,17 @@ func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrometheusResponseHeader: wiretype end group for non-group") + return fmt.Errorf("proto: Sample: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusResponseHeader: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Sample: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2483,29 +5170,31 @@ func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthQuery } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthQuery } if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.Labels = append(m.Labels, github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{}) + if err := m.Labels[len(m.Labels)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Sample", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2515,23 +5204,63 @@ func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthQuery } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthQuery } if postIndex > l { return io.ErrUnexpectedEOF } - m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) + if m.Sample == nil { + m.Sample = &cortexpb.Sample{} + } + if err := m.Sample.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histogram", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Histogram == nil { + m.Histogram = &SampleHistogramPair{} + } + if err := m.Histogram.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -2557,7 +5286,7 @@ func (m *PrometheusResponseHeader) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrometheusRequestHeader) Unmarshal(dAtA []byte) error { +func (m *Matrix) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -2580,17 +5309,17 @@ func (m *PrometheusRequestHeader) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrometheusRequestHeader: wiretype end group for non-group") + return fmt.Errorf("proto: Matrix: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusRequestHeader: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Matrix: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SampleStreams", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowQuery @@ -2600,55 +5329,25 @@ func (m *PrometheusRequestHeader) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthQuery } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthQuery } if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Values", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQuery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQuery - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQuery - } - if postIndex > l { - return io.ErrUnexpectedEOF + m.SampleStreams = append(m.SampleStreams, SampleStream{}) + if err := m.SampleStreams[len(m.SampleStreams)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.Values = append(m.Values, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex default: iNdEx = preIndex diff --git a/pkg/querier/tripperware/query.proto b/pkg/querier/tripperware/query.proto index 9664fcf528..c2eb585cb3 100644 --- a/pkg/querier/tripperware/query.proto +++ b/pkg/querier/tripperware/query.proto @@ -5,11 +5,44 @@ package tripperware; option go_package = "tripperware"; import "gogoproto/gogo.proto"; +import "google/protobuf/duration.proto"; +import "google/protobuf/any.proto"; import "github.com/cortexproject/cortex/pkg/cortexpb/cortex.proto"; option (gogoproto.marshaler_all) = true; option (gogoproto.unmarshaler_all) = true; +message PrometheusResponse { + string Status = 1 [(gogoproto.jsontag) = "status"]; + PrometheusData Data = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "data,omitempty"]; + string ErrorType = 3 [(gogoproto.jsontag) = "errorType,omitempty"]; + string Error = 4 [(gogoproto.jsontag) = "error,omitempty"]; + repeated tripperware.PrometheusResponseHeader Headers = 5 [(gogoproto.jsontag) = "-"]; + repeated string Warnings = 6 [(gogoproto.jsontag) = "warnings,omitempty"]; +} + +message PrometheusData { + string ResultType = 1 [(gogoproto.jsontag) = "resultType"]; + tripperware.PrometheusQueryResult Result = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "result"]; + tripperware.PrometheusResponseStats stats = 3 [(gogoproto.jsontag) = "stats,omitempty"]; +} + +message CachedResponse { + string key = 1 [(gogoproto.jsontag) = "key"]; + + // List of cached responses; non-overlapping and in order. + repeated Extent extents = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "extents"]; +} + +message Extent { + int64 start = 1 [(gogoproto.jsontag) = "start"]; + int64 end = 2 [(gogoproto.jsontag) = "end"]; + // reserved the previous key to ensure cache transition + reserved 3; + string trace_id = 4 [(gogoproto.jsontag) = "-"]; + google.protobuf.Any response = 5 [(gogoproto.jsontag) = "response"]; +} + message SampleStream { repeated cortexpb.LabelPair labels = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "metric", (gogoproto.customtype) = "github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter"]; repeated cortexpb.Sample samples = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "values"]; @@ -53,7 +86,24 @@ message PrometheusResponseHeader { repeated string Values = 2 [(gogoproto.jsontag) = "-"]; } -message PrometheusRequestHeader { - string Name = 1 [(gogoproto.jsontag) = "-"]; - repeated string Values = 2 [(gogoproto.jsontag) = "-"]; -} \ No newline at end of file +message PrometheusQueryResult { + oneof result { + Vector vector = 1; + bytes rawBytes = 2; + Matrix matrix = 3; + } +} + +message Vector { + repeated Sample samples = 1; +} + +message Sample { + repeated cortexpb.LabelPair labels = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "metric", (gogoproto.customtype) = "github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter"]; + cortexpb.Sample sample = 2 [(gogoproto.nullable) = true, (gogoproto.jsontag) = "value"]; + SampleHistogramPair histogram = 3 [(gogoproto.nullable) = true, (gogoproto.jsontag) = "histogram"]; +} + +message Matrix { + repeated SampleStream sampleStreams = 1 [(gogoproto.nullable) = false]; +} diff --git a/pkg/querier/tripperware/queryrange/limits.go b/pkg/querier/tripperware/queryrange/limits.go index a4d991aba8..4aa9318595 100644 --- a/pkg/querier/tripperware/queryrange/limits.go +++ b/pkg/querier/tripperware/queryrange/limits.go @@ -60,7 +60,7 @@ func (l limitsMiddleware) Do(ctx context.Context, r tripperware.Request) (trippe "redEnd", util.FormatTimeMillis(r.GetEnd()), "maxQueryLookback", maxQueryLookback) - return NewEmptyPrometheusResponse(), nil + return tripperware.NewEmptyPrometheusResponse(false), nil } if r.GetStart() < minStartTime { diff --git a/pkg/querier/tripperware/queryrange/limits_test.go b/pkg/querier/tripperware/queryrange/limits_test.go index 5ee78fbf0b..6c3e257986 100644 --- a/pkg/querier/tripperware/queryrange/limits_test.go +++ b/pkg/querier/tripperware/queryrange/limits_test.go @@ -74,7 +74,7 @@ func TestLimitsMiddleware_MaxQueryLookback(t *testing.T) { testData := testData t.Run(testName, func(t *testing.T) { t.Parallel() - req := &PrometheusRequest{ + req := &tripperware.PrometheusRequest{ Start: util.TimeToMillis(testData.reqStartTime), End: util.TimeToMillis(testData.reqEndTime), } @@ -82,7 +82,7 @@ func TestLimitsMiddleware_MaxQueryLookback(t *testing.T) { limits := mockLimits{maxQueryLookback: testData.maxQueryLookback} middleware := NewLimitsMiddleware(limits, 5*time.Minute) - innerRes := NewEmptyPrometheusResponse() + innerRes := tripperware.NewEmptyPrometheusResponse(false) inner := &mockHandler{} inner.On("Do", mock.Anything, mock.Anything).Return(innerRes, nil) @@ -193,7 +193,7 @@ func TestLimitsMiddleware_MaxQueryLength(t *testing.T) { testData := testData t.Run(testName, func(t *testing.T) { t.Parallel() - req := &PrometheusRequest{ + req := &tripperware.PrometheusRequest{ Query: testData.query, Start: util.TimeToMillis(testData.reqStartTime), End: util.TimeToMillis(testData.reqEndTime), @@ -205,7 +205,7 @@ func TestLimitsMiddleware_MaxQueryLength(t *testing.T) { limits := mockLimits{maxQueryLength: testData.maxQueryLength} middleware := NewLimitsMiddleware(limits, 5*time.Minute) - innerRes := NewEmptyPrometheusResponse() + innerRes := tripperware.NewEmptyPrometheusResponse(false) inner := &mockHandler{} inner.On("Do", mock.Anything, mock.Anything).Return(innerRes, nil) diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index 47ac83d9c5..ce84635dc1 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -3,7 +3,7 @@ package queryrange import ( "bytes" "context" - io "io" + "io" "math/rand" "net/http" "testing" @@ -57,7 +57,7 @@ func BenchmarkPrometheusCodec_EncodeResponse(b *testing.B) { } } -func mockPrometheusResponse(numSeries, numSamplesPerSeries int) *PrometheusResponse { +func mockPrometheusResponse(numSeries, numSamplesPerSeries int) *tripperware.PrometheusResponse { stream := make([]tripperware.SampleStream, numSeries) for s := 0; s < numSeries; s++ { // Generate random samples. @@ -82,11 +82,17 @@ func mockPrometheusResponse(numSeries, numSamplesPerSeries int) *PrometheusRespo } } - return &PrometheusResponse{ + return &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: "vector", - Result: stream, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: stream, + }, + }, + }, }, } } diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 82eb0c12fc..fc0b27a4f1 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -7,7 +7,6 @@ import ( "io" "net/http" "net/url" - "sort" "strconv" "strings" "time" @@ -17,8 +16,6 @@ import ( "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/timestamp" - "github.com/thanos-io/thanos/pkg/strutil" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/querier/tripperware" @@ -52,119 +49,19 @@ func NewPrometheusCodec(sharded bool) *prometheusCodec { //nolint:revive return &prometheusCodec{sharded: sharded} } -// WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. -func (q *PrometheusRequest) WithStartEnd(start int64, end int64) tripperware.Request { - new := *q - new.Start = start - new.End = end - return &new -} - -// WithQuery clones the current `PrometheusRequest` with a new query. -func (q *PrometheusRequest) WithQuery(query string) tripperware.Request { - new := *q - new.Query = query - return &new -} - -// WithStats clones the current `PrometheusRequest` with a new stats. -func (q *PrometheusRequest) WithStats(stats string) tripperware.Request { - new := *q - new.Stats = stats - return &new -} - -// LogToSpan logs the current `PrometheusRequest` parameters to the specified span. -func (q *PrometheusRequest) LogToSpan(sp opentracing.Span) { - sp.LogFields( - otlog.String("query", q.GetQuery()), - otlog.String("start", timestamp.Time(q.GetStart()).String()), - otlog.String("end", timestamp.Time(q.GetEnd()).String()), - otlog.Int64("step (ms)", q.GetStep()), - ) -} - -type byFirstTime []*PrometheusResponse - -func (a byFirstTime) Len() int { return len(a) } -func (a byFirstTime) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a byFirstTime) Less(i, j int) bool { return a[i].minTime() < a[j].minTime() } - -func (resp *PrometheusResponse) minTime() int64 { - result := resp.Data.Result - if len(result) == 0 { - return -1 - } - if len(result[0].Samples) == 0 { - return -1 - } - return result[0].Samples[0].TimestampMs -} - -func (resp *PrometheusResponse) HTTPHeaders() map[string][]string { - if resp != nil && resp.GetHeaders() != nil { - r := map[string][]string{} - for _, header := range resp.GetHeaders() { - if header != nil { - r[header.Name] = header.Values - } - } - - return r - } - return nil -} - -// NewEmptyPrometheusResponse returns an empty successful Prometheus query range response. -func NewEmptyPrometheusResponse() *PrometheusResponse { - return &PrometheusResponse{ - Status: StatusSuccess, - Data: PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{}, - }, - } -} - -func (c prometheusCodec) MergeResponse(ctx context.Context, _ tripperware.Request, responses ...tripperware.Response) (tripperware.Response, error) { +func (c prometheusCodec) MergeResponse(ctx context.Context, req tripperware.Request, responses ...tripperware.Response) (tripperware.Response, error) { sp, _ := opentracing.StartSpanFromContext(ctx, "QueryRangeResponse.MergeResponse") sp.SetTag("response_count", len(responses)) defer sp.Finish() if len(responses) == 0 { - return NewEmptyPrometheusResponse(), nil - } - - promResponses := make([]*PrometheusResponse, 0, len(responses)) - warnings := make([][]string, 0, len(responses)) - for _, res := range responses { - promResponses = append(promResponses, res.(*PrometheusResponse)) - if w := res.(*PrometheusResponse).Warnings; w != nil { - warnings = append(warnings, w) - } - } - - // Merge the responses. - sort.Sort(byFirstTime(promResponses)) - sampleStreams, err := matrixMerge(ctx, promResponses) - if err != nil { - return nil, err + return tripperware.NewEmptyPrometheusResponse(false), nil } - response := PrometheusResponse{ - Status: StatusSuccess, - Data: PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: sampleStreams, - Stats: statsMerge(c.sharded, promResponses), - }, - Warnings: strutil.MergeUnsortedSlices(warnings...), - } - - return &response, nil + return tripperware.MergeResponse(ctx, c.sharded, nil, responses...) } func (c prometheusCodec) DecodeRequest(_ context.Context, r *http.Request, forwardHeaders []string) (tripperware.Request, error) { - var result PrometheusRequest + result := tripperware.PrometheusRequest{Headers: map[string][]string{}} var err error result.Start, err = util.ParseTime(r.FormValue("start")) if err != nil { @@ -203,7 +100,7 @@ func (c prometheusCodec) DecodeRequest(_ context.Context, r *http.Request, forwa for _, header := range forwardHeaders { for h, hv := range r.Header { if strings.EqualFold(h, header) { - result.Headers = append(result.Headers, &tripperware.PrometheusRequestHeader{Name: h, Values: hv}) + result.Headers[h] = hv break } } @@ -220,7 +117,7 @@ func (c prometheusCodec) DecodeRequest(_ context.Context, r *http.Request, forwa } func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { - promReq, ok := r.(*PrometheusRequest) + promReq, ok := r.(*tripperware.PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") } @@ -237,9 +134,9 @@ func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) } var h = http.Header{} - for _, hv := range promReq.Headers { - for _, v := range hv.Values { - h.Add(hv.Name, v) + for n, hv := range promReq.Headers { + for _, v := range hv { + h.Add(n, v) } } @@ -275,7 +172,7 @@ func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ t } log.LogFields(otlog.Int("bytes", len(buf))) - var resp PrometheusResponse + var resp tripperware.PrometheusResponse if err := json.Unmarshal(buf, &resp); err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } @@ -290,12 +187,15 @@ func (prometheusCodec) EncodeResponse(ctx context.Context, res tripperware.Respo sp, _ := opentracing.StartSpanFromContext(ctx, "APIResponse.ToHTTPResponse") defer sp.Finish() - a, ok := res.(*PrometheusResponse) + a, ok := res.(*tripperware.PrometheusResponse) if !ok { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "invalid response format") } - sp.LogFields(otlog.Int("series", len(a.Data.Result))) + if a != nil { + m := a.Data.Result.GetMatrix() + sp.LogFields(otlog.Int("series", len(m.GetSampleStreams()))) + } b, err := json.Marshal(a) if err != nil { @@ -315,66 +215,6 @@ func (prometheusCodec) EncodeResponse(ctx context.Context, res tripperware.Respo return &resp, nil } -// statsMerge merge the stats from 2 responses -// this function is similar to matrixMerge -func statsMerge(shouldSumStats bool, resps []*PrometheusResponse) *tripperware.PrometheusResponseStats { - output := map[int64]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{} - hasStats := false - for _, resp := range resps { - if resp.Data.Stats == nil { - continue - } - - hasStats = true - if resp.Data.Stats.Samples == nil { - continue - } - - for _, s := range resp.Data.Stats.Samples.TotalQueryableSamplesPerStep { - if shouldSumStats { - if stats, ok := output[s.GetTimestampMs()]; ok { - stats.Value += s.Value - } else { - output[s.GetTimestampMs()] = s - } - } else { - output[s.GetTimestampMs()] = s - } - } - } - - if !hasStats { - return nil - } - return tripperware.StatsMerge(output) -} - -func matrixMerge(ctx context.Context, resps []*PrometheusResponse) ([]tripperware.SampleStream, error) { - output := make(map[string]tripperware.SampleStream) - for _, resp := range resps { - if err := ctx.Err(); err != nil { - return nil, err - } - if resp == nil { - continue - } - tripperware.MergeSampleStreams(output, resp.Data.GetResult()) - } - - keys := make([]string, 0, len(output)) - for key := range output { - keys = append(keys, key) - } - sort.Strings(keys) - - result := make([]tripperware.SampleStream, 0, len(output)) - for _, key := range keys { - result = append(result, output[key]) - } - - return result, nil -} - func encodeDurationMs(d int64) string { return strconv.FormatFloat(float64(d)/float64(time.Second/time.Millisecond), 'f', -1, 64) } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index b84d0f83ed..3e02ced959 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -96,7 +96,7 @@ func Middlewares( var c cache.Cache if cfg.CacheResults { shouldCache := func(r tripperware.Request) bool { - if v, ok := r.(*PrometheusRequest); ok { + if v, ok := r.(*tripperware.PrometheusRequest); ok { return !v.CachingOptions.Disabled } return false diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 0da8489188..32db902c0f 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -96,7 +96,7 @@ func TestResponse(t *testing.T) { rWithWarnings.Headers = respHeaders for i, tc := range []struct { body string - expected *PrometheusResponse + expected *tripperware.PrometheusResponse expectedDecodeErr error cancelCtxBeforeDecode bool }{ @@ -153,22 +153,28 @@ func TestResponseWithStats(t *testing.T) { t.Parallel() for i, tc := range []struct { body string - expected *PrometheusResponse + expected *tripperware.PrometheusResponse }{ { body: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "foo", Value: "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 137, TimestampMs: 1536673680000}, - {Value: 137, TimestampMs: 1536673780000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "foo", Value: "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 137, TimestampMs: 1536673680000}, + {Value: 137, TimestampMs: 1536673780000}, + }, + }, + }, }, }, }, @@ -224,100 +230,135 @@ func TestMergeAPIResponses(t *testing.T) { { name: "No responses shouldn't panic and return a non-null result and result type.", input: []tripperware.Response{}, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, }, }, }, { name: "A single empty response shouldn't panic.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Status: StatusSuccess, + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, }, }, }, { name: "Multiple empty responses shouldn't panic.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, }, }, - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{}, + }, + }, + }, }, }, }, { name: "Basic merging of two responses.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1}, + }, + }, + }, }, }, }, }, }, - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2}, - {Value: 3, TimestampMs: 3}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2}, + {Value: 3, TimestampMs: 3}, + }, + }, + }, }, }, }, }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1}, - {Value: 2, TimestampMs: 2}, - {Value: 3, TimestampMs: 3}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1}, + {Value: 2, TimestampMs: 2}, + {Value: 3, TimestampMs: 3}, + }, + }, + }, }, }, }, @@ -330,18 +371,24 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[0,"0"],[1,"1"]]}]}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"]]}]}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, }, }, }, @@ -354,17 +401,23 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"],[2,"2"]]}]}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"]]}]}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, }, }, }, @@ -377,19 +430,25 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"],[4,"4"],[5,"5"]]}]}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - {Value: 4, TimestampMs: 4000}, - {Value: 5, TimestampMs: 5000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + {Value: 5, TimestampMs: 5000}, + }, + }, + }, }, }, }, @@ -402,16 +461,22 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","warnings":["warning1","warning2"],"data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"]]}]}}`), mustParse(t, `{"status":"success","warnings":["warning1","warning3"],"data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"]]}]}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, Warnings: []string{"warning1", "warning2", "warning3"}, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + }, + }, + }, }, }, }, @@ -424,18 +489,24 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[2,"2"],[3,"3"]]}]}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"],[4,"4"],[5,"5"]]}]}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - {Value: 4, TimestampMs: 4000}, - {Value: 5, TimestampMs: 5000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + {Value: 5, TimestampMs: 5000}, + }, + }, + }, }, }, }, @@ -445,29 +516,41 @@ func TestMergeAPIResponses(t *testing.T) { { name: "Context cancel should cancel merge", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1}, + }, + }, + }, }, }, }, }, }, - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2}, - {Value: 3, TimestampMs: 3}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2}, + {Value: 3, TimestampMs: 3}, + }, + }, + }, }, }, }, @@ -480,62 +563,91 @@ func TestMergeAPIResponses(t *testing.T) { { name: "[stats] A single empty response shouldn't panic.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Status: StatusSuccess, + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{}, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, }, }, }, { name: "[stats] Multiple empty responses shouldn't panic.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, }, }, - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{}, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{}}, }, }, }, { name: "[stats] Basic merging of two responses.", input: []tripperware.Response{ - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1}, + }, + }, + }, }, }, }, @@ -548,15 +660,21 @@ func TestMergeAPIResponses(t *testing.T) { }}, }, }, - &PrometheusResponse{ - Data: PrometheusData{ + &tripperware.PrometheusResponse{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2}, - {Value: 3, TimestampMs: 3}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2}, + {Value: 3, TimestampMs: 3}, + }, + }, + }, }, }, }, @@ -570,18 +688,24 @@ func TestMergeAPIResponses(t *testing.T) { }, }, }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: []cortexpb.Sample{ - {Value: 0, TimestampMs: 0}, - {Value: 1, TimestampMs: 1}, - {Value: 2, TimestampMs: 2}, - {Value: 3, TimestampMs: 3}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: []cortexpb.Sample{ + {Value: 0, TimestampMs: 0}, + {Value: 1, TimestampMs: 1}, + {Value: 2, TimestampMs: 2}, + {Value: 3, TimestampMs: 3}, + }, + }, + }, }, }, }, @@ -603,17 +727,23 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"],[2,"2"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,5],[2,5]]}}}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[2,5],[3,15]]}}}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, }, }, }, @@ -635,21 +765,27 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"],[2,"2"],[3,"3"],[4,"4"]]}],"stats":{"samples":{"totalQueryableSamples":6,"totalQueryableSamplesPerStep":[[1,1],[2,2],[3,3],[4,4]]}}}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[5,"5"],[6,"6"],[7,"7"]]}],"stats":{"samples":{"totalQueryableSamples":18,"totalQueryableSamplesPerStep":[[5,5],[6,6],[7,7]]}}}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - {Value: 4, TimestampMs: 4000}, - {Value: 5, TimestampMs: 5000}, - {Value: 6, TimestampMs: 6000}, - {Value: 7, TimestampMs: 7000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + {Value: 5, TimestampMs: 5000}, + {Value: 6, TimestampMs: 6000}, + {Value: 7, TimestampMs: 7000}, + }, + }, + }, }, }, }, @@ -674,19 +810,25 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":6,"totalQueryableSamplesPerStep":[[1,1],[2,2],[3,3]]}}}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"],[4,"4"],[5,"5"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[2,2],[3,3],[4,4],[5,5]]}}}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - {Value: 4, TimestampMs: 4000}, - {Value: 5, TimestampMs: 5000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + {Value: 5, TimestampMs: 5000}, + }, + }, + }, }, }, }, @@ -709,18 +851,24 @@ func TestMergeAPIResponses(t *testing.T) { mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"a":"b","c":"d"},"values":[[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[2,2],[3,3]]}}}}`), mustParse(t, `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"c":"d","a":"b"},"values":[[2,"2"],[3,"3"],[4,"4"],[5,"5"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[2,2],[3,3],[4,4],[5,5]]}}}}`), }, - expected: &PrometheusResponse{ + expected: &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - {Value: 4, TimestampMs: 4000}, - {Value: 5, TimestampMs: 5000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{{Name: "a", Value: "b"}, {Name: "c", Value: "d"}}, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + {Value: 5, TimestampMs: 5000}, + }, + }, + }, }, }, }, @@ -815,7 +963,7 @@ func TestGzippedResponse(t *testing.T) { } func mustParse(t *testing.T, response string) tripperware.Response { - var resp PrometheusResponse + var resp tripperware.PrometheusResponse // Needed as goimports automatically add a json import otherwise. json := jsoniter.ConfigCompatibleWithStandardLibrary require.NoError(t, json.Unmarshal([]byte(response), &resp)) diff --git a/pkg/querier/tripperware/queryrange/queryrange.pb.go b/pkg/querier/tripperware/queryrange/queryrange.pb.go deleted file mode 100644 index edf1f6cbe8..0000000000 --- a/pkg/querier/tripperware/queryrange/queryrange.pb.go +++ /dev/null @@ -1,2641 +0,0 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: queryrange.proto - -package queryrange - -import ( - fmt "fmt" - tripperware "github.com/cortexproject/cortex/pkg/querier/tripperware" - _ "github.com/gogo/protobuf/gogoproto" - proto "github.com/gogo/protobuf/proto" - github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" - types "github.com/gogo/protobuf/types" - _ "github.com/golang/protobuf/ptypes/duration" - io "io" - math "math" - math_bits "math/bits" - reflect "reflect" - strings "strings" - time "time" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf -var _ = time.Kitchen - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package - -type PrometheusRequest struct { - Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` - Start int64 `protobuf:"varint,2,opt,name=start,proto3" json:"start,omitempty"` - End int64 `protobuf:"varint,3,opt,name=end,proto3" json:"end,omitempty"` - Step int64 `protobuf:"varint,4,opt,name=step,proto3" json:"step,omitempty"` - Timeout time.Duration `protobuf:"bytes,5,opt,name=timeout,proto3,stdduration" json:"timeout"` - Query string `protobuf:"bytes,6,opt,name=query,proto3" json:"query,omitempty"` - CachingOptions CachingOptions `protobuf:"bytes,7,opt,name=cachingOptions,proto3" json:"cachingOptions"` - Headers []*tripperware.PrometheusRequestHeader `protobuf:"bytes,8,rep,name=Headers,proto3" json:"-"` - Stats string `protobuf:"bytes,9,opt,name=stats,proto3" json:"stats,omitempty"` -} - -func (m *PrometheusRequest) Reset() { *m = PrometheusRequest{} } -func (*PrometheusRequest) ProtoMessage() {} -func (*PrometheusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{0} -} -func (m *PrometheusRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusRequest.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 *PrometheusRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusRequest.Merge(m, src) -} -func (m *PrometheusRequest) XXX_Size() int { - return m.Size() -} -func (m *PrometheusRequest) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusRequest proto.InternalMessageInfo - -func (m *PrometheusRequest) GetPath() string { - if m != nil { - return m.Path - } - return "" -} - -func (m *PrometheusRequest) GetStart() int64 { - if m != nil { - return m.Start - } - return 0 -} - -func (m *PrometheusRequest) GetEnd() int64 { - if m != nil { - return m.End - } - return 0 -} - -func (m *PrometheusRequest) GetStep() int64 { - if m != nil { - return m.Step - } - return 0 -} - -func (m *PrometheusRequest) GetTimeout() time.Duration { - if m != nil { - return m.Timeout - } - return 0 -} - -func (m *PrometheusRequest) GetQuery() string { - if m != nil { - return m.Query - } - return "" -} - -func (m *PrometheusRequest) GetCachingOptions() CachingOptions { - if m != nil { - return m.CachingOptions - } - return CachingOptions{} -} - -func (m *PrometheusRequest) GetHeaders() []*tripperware.PrometheusRequestHeader { - if m != nil { - return m.Headers - } - return nil -} - -func (m *PrometheusRequest) GetStats() string { - if m != nil { - return m.Stats - } - return "" -} - -type PrometheusResponse struct { - Status string `protobuf:"bytes,1,opt,name=Status,proto3" json:"status"` - Data PrometheusData `protobuf:"bytes,2,opt,name=Data,proto3" json:"data,omitempty"` - ErrorType string `protobuf:"bytes,3,opt,name=ErrorType,proto3" json:"errorType,omitempty"` - Error string `protobuf:"bytes,4,opt,name=Error,proto3" json:"error,omitempty"` - Headers []*tripperware.PrometheusResponseHeader `protobuf:"bytes,5,rep,name=Headers,proto3" json:"-"` - Warnings []string `protobuf:"bytes,6,rep,name=Warnings,proto3" json:"warnings,omitempty"` -} - -func (m *PrometheusResponse) Reset() { *m = PrometheusResponse{} } -func (*PrometheusResponse) ProtoMessage() {} -func (*PrometheusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{1} -} -func (m *PrometheusResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusResponse.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 *PrometheusResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusResponse.Merge(m, src) -} -func (m *PrometheusResponse) XXX_Size() int { - return m.Size() -} -func (m *PrometheusResponse) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusResponse proto.InternalMessageInfo - -func (m *PrometheusResponse) GetStatus() string { - if m != nil { - return m.Status - } - return "" -} - -func (m *PrometheusResponse) GetData() PrometheusData { - if m != nil { - return m.Data - } - return PrometheusData{} -} - -func (m *PrometheusResponse) GetErrorType() string { - if m != nil { - return m.ErrorType - } - return "" -} - -func (m *PrometheusResponse) GetError() string { - if m != nil { - return m.Error - } - return "" -} - -func (m *PrometheusResponse) GetHeaders() []*tripperware.PrometheusResponseHeader { - if m != nil { - return m.Headers - } - return nil -} - -func (m *PrometheusResponse) GetWarnings() []string { - if m != nil { - return m.Warnings - } - return nil -} - -type PrometheusData struct { - ResultType string `protobuf:"bytes,1,opt,name=ResultType,proto3" json:"resultType"` - Result []tripperware.SampleStream `protobuf:"bytes,2,rep,name=Result,proto3" json:"result"` - Stats *tripperware.PrometheusResponseStats `protobuf:"bytes,3,opt,name=stats,proto3" json:"stats,omitempty"` -} - -func (m *PrometheusData) Reset() { *m = PrometheusData{} } -func (*PrometheusData) ProtoMessage() {} -func (*PrometheusData) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{2} -} -func (m *PrometheusData) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *PrometheusData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_PrometheusData.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 *PrometheusData) XXX_Merge(src proto.Message) { - xxx_messageInfo_PrometheusData.Merge(m, src) -} -func (m *PrometheusData) XXX_Size() int { - return m.Size() -} -func (m *PrometheusData) XXX_DiscardUnknown() { - xxx_messageInfo_PrometheusData.DiscardUnknown(m) -} - -var xxx_messageInfo_PrometheusData proto.InternalMessageInfo - -func (m *PrometheusData) GetResultType() string { - if m != nil { - return m.ResultType - } - return "" -} - -func (m *PrometheusData) GetResult() []tripperware.SampleStream { - if m != nil { - return m.Result - } - return nil -} - -func (m *PrometheusData) GetStats() *tripperware.PrometheusResponseStats { - if m != nil { - return m.Stats - } - return nil -} - -type CachedResponse struct { - Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key"` - // List of cached responses; non-overlapping and in order. - Extents []Extent `protobuf:"bytes,2,rep,name=extents,proto3" json:"extents"` -} - -func (m *CachedResponse) Reset() { *m = CachedResponse{} } -func (*CachedResponse) ProtoMessage() {} -func (*CachedResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{3} -} -func (m *CachedResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *CachedResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_CachedResponse.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 *CachedResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_CachedResponse.Merge(m, src) -} -func (m *CachedResponse) XXX_Size() int { - return m.Size() -} -func (m *CachedResponse) XXX_DiscardUnknown() { - xxx_messageInfo_CachedResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_CachedResponse proto.InternalMessageInfo - -func (m *CachedResponse) GetKey() string { - if m != nil { - return m.Key - } - return "" -} - -func (m *CachedResponse) GetExtents() []Extent { - if m != nil { - return m.Extents - } - return nil -} - -type Extent struct { - Start int64 `protobuf:"varint,1,opt,name=start,proto3" json:"start"` - End int64 `protobuf:"varint,2,opt,name=end,proto3" json:"end"` - TraceId string `protobuf:"bytes,4,opt,name=trace_id,json=traceId,proto3" json:"-"` - Response *types.Any `protobuf:"bytes,5,opt,name=response,proto3" json:"response"` -} - -func (m *Extent) Reset() { *m = Extent{} } -func (*Extent) ProtoMessage() {} -func (*Extent) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{4} -} -func (m *Extent) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Extent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_Extent.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 *Extent) XXX_Merge(src proto.Message) { - xxx_messageInfo_Extent.Merge(m, src) -} -func (m *Extent) XXX_Size() int { - return m.Size() -} -func (m *Extent) XXX_DiscardUnknown() { - xxx_messageInfo_Extent.DiscardUnknown(m) -} - -var xxx_messageInfo_Extent proto.InternalMessageInfo - -func (m *Extent) GetStart() int64 { - if m != nil { - return m.Start - } - return 0 -} - -func (m *Extent) GetEnd() int64 { - if m != nil { - return m.End - } - return 0 -} - -func (m *Extent) GetTraceId() string { - if m != nil { - return m.TraceId - } - return "" -} - -func (m *Extent) GetResponse() *types.Any { - if m != nil { - return m.Response - } - return nil -} - -type CachingOptions struct { - Disabled bool `protobuf:"varint,1,opt,name=disabled,proto3" json:"disabled,omitempty"` -} - -func (m *CachingOptions) Reset() { *m = CachingOptions{} } -func (*CachingOptions) ProtoMessage() {} -func (*CachingOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_79b02382e213d0b2, []int{5} -} -func (m *CachingOptions) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *CachingOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_CachingOptions.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 *CachingOptions) XXX_Merge(src proto.Message) { - xxx_messageInfo_CachingOptions.Merge(m, src) -} -func (m *CachingOptions) XXX_Size() int { - return m.Size() -} -func (m *CachingOptions) XXX_DiscardUnknown() { - xxx_messageInfo_CachingOptions.DiscardUnknown(m) -} - -var xxx_messageInfo_CachingOptions proto.InternalMessageInfo - -func (m *CachingOptions) GetDisabled() bool { - if m != nil { - return m.Disabled - } - return false -} - -func init() { - proto.RegisterType((*PrometheusRequest)(nil), "queryrange.PrometheusRequest") - proto.RegisterType((*PrometheusResponse)(nil), "queryrange.PrometheusResponse") - proto.RegisterType((*PrometheusData)(nil), "queryrange.PrometheusData") - proto.RegisterType((*CachedResponse)(nil), "queryrange.CachedResponse") - proto.RegisterType((*Extent)(nil), "queryrange.Extent") - proto.RegisterType((*CachingOptions)(nil), "queryrange.CachingOptions") -} - -func init() { proto.RegisterFile("queryrange.proto", fileDescriptor_79b02382e213d0b2) } - -var fileDescriptor_79b02382e213d0b2 = []byte{ - // 792 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xcd, 0x4e, 0xe3, 0x48, - 0x10, 0x8e, 0xe3, 0xc4, 0x71, 0x9a, 0x55, 0x60, 0x1b, 0xc4, 0x3a, 0x39, 0xd8, 0x51, 0xc4, 0x4a, - 0x59, 0x89, 0x75, 0x24, 0x56, 0x7b, 0xdc, 0xd5, 0x62, 0x7e, 0xc4, 0xee, 0x85, 0x95, 0x19, 0x69, - 0xa4, 0xb9, 0x8c, 0x3a, 0x71, 0x8f, 0x63, 0x48, 0x6c, 0xd3, 0xdd, 0x16, 0xe4, 0x36, 0x8f, 0x30, - 0xc7, 0x79, 0x84, 0x39, 0xcc, 0x83, 0x70, 0xe4, 0x30, 0x07, 0x4e, 0x66, 0x08, 0x97, 0x91, 0x4f, - 0x3c, 0xc2, 0xc8, 0xdd, 0xed, 0xc4, 0x80, 0xd0, 0x5c, 0xac, 0xaa, 0xea, 0xaf, 0xca, 0xf5, 0x7d, - 0xd5, 0xd5, 0x60, 0xed, 0x3c, 0xc1, 0x64, 0x46, 0x50, 0xe8, 0x63, 0x3b, 0x26, 0x11, 0x8b, 0x20, - 0x58, 0x46, 0x3a, 0x1b, 0x7e, 0xe4, 0x47, 0x3c, 0x3c, 0xc8, 0x2d, 0x81, 0xe8, 0x98, 0x7e, 0x14, - 0xf9, 0x13, 0x3c, 0xe0, 0xde, 0x30, 0x79, 0x37, 0xf0, 0x12, 0x82, 0x58, 0x10, 0x85, 0xf2, 0xbc, - 0xfd, 0xf4, 0x1c, 0x85, 0x33, 0x79, 0xb4, 0xe7, 0x07, 0x6c, 0x9c, 0x0c, 0xed, 0x51, 0x34, 0x1d, - 0x8c, 0x22, 0xc2, 0xf0, 0x65, 0x4c, 0xa2, 0x53, 0x3c, 0x62, 0xd2, 0x1b, 0xc4, 0x67, 0xfe, 0x20, - 0x6f, 0x20, 0xc0, 0x64, 0xc0, 0x48, 0x10, 0xc7, 0x98, 0x5c, 0x20, 0x82, 0x79, 0x4c, 0x16, 0xe9, - 0xdd, 0x56, 0xc1, 0xcf, 0xff, 0x93, 0x68, 0x8a, 0xd9, 0x18, 0x27, 0xd4, 0xc5, 0xe7, 0x09, 0xa6, - 0x0c, 0x42, 0x50, 0x8b, 0x11, 0x1b, 0x1b, 0x4a, 0x57, 0xe9, 0x37, 0x5d, 0x6e, 0xc3, 0x0d, 0x50, - 0xa7, 0x0c, 0x11, 0x66, 0x54, 0xbb, 0x4a, 0x5f, 0x75, 0x85, 0x03, 0xd7, 0x80, 0x8a, 0x43, 0xcf, - 0x50, 0x79, 0x2c, 0x37, 0xf3, 0x5c, 0xca, 0x70, 0x6c, 0xd4, 0x78, 0x88, 0xdb, 0xf0, 0x2f, 0xd0, - 0x60, 0xc1, 0x14, 0x47, 0x09, 0x33, 0xea, 0x5d, 0xa5, 0xbf, 0xb2, 0xd3, 0xb6, 0x05, 0x2f, 0xbb, - 0xe0, 0x65, 0xef, 0x4b, 0xde, 0x8e, 0x7e, 0x95, 0x5a, 0x95, 0x8f, 0xb7, 0x96, 0xe2, 0x16, 0x39, - 0xf9, 0xaf, 0x79, 0xcf, 0x86, 0xc6, 0xfb, 0x11, 0x0e, 0x3c, 0x02, 0xad, 0x11, 0x1a, 0x8d, 0x83, - 0xd0, 0x3f, 0x8e, 0xf3, 0x4c, 0x6a, 0x34, 0x78, 0xed, 0x8e, 0x5d, 0x9a, 0xc3, 0xde, 0x23, 0x84, - 0x53, 0xcb, 0x8b, 0xbb, 0x4f, 0xf2, 0xe0, 0x01, 0x68, 0x1c, 0x61, 0xe4, 0x61, 0x42, 0x0d, 0xbd, - 0xab, 0xf6, 0x57, 0x76, 0xb6, 0xec, 0x92, 0x5e, 0xf6, 0x33, 0x7d, 0x04, 0xd8, 0xa9, 0x67, 0xa9, - 0xa5, 0xfc, 0xee, 0x16, 0xb9, 0x52, 0x21, 0x46, 0x8d, 0xa6, 0x68, 0x93, 0x3b, 0xbd, 0xb4, 0x0a, - 0x60, 0xb9, 0x02, 0x8d, 0xa3, 0x90, 0x62, 0xd8, 0x03, 0xda, 0x09, 0x43, 0x2c, 0xa1, 0x42, 0x64, - 0x07, 0x64, 0xa9, 0xa5, 0x51, 0x1e, 0x71, 0xe5, 0x09, 0x3c, 0x04, 0xb5, 0x7d, 0xc4, 0x10, 0x57, - 0xfc, 0x09, 0xaf, 0x65, 0xc5, 0x1c, 0xe1, 0x6c, 0xe6, 0xbc, 0xb2, 0xd4, 0x6a, 0x79, 0x88, 0xa1, - 0xed, 0x68, 0x1a, 0x30, 0x3c, 0x8d, 0xd9, 0xcc, 0xe5, 0xf9, 0xf0, 0x4f, 0xd0, 0x3c, 0x20, 0x24, - 0x22, 0xaf, 0x66, 0x31, 0xe6, 0xa3, 0x6a, 0x3a, 0xbf, 0x64, 0xa9, 0xb5, 0x8e, 0x8b, 0x60, 0x29, - 0x63, 0x89, 0x84, 0xbf, 0x81, 0x3a, 0x77, 0xf8, 0x28, 0x9b, 0xce, 0x7a, 0x96, 0x5a, 0xab, 0x3c, - 0xa5, 0x04, 0x17, 0x08, 0x78, 0xb8, 0x54, 0xb0, 0xce, 0x15, 0xfc, 0xf5, 0x45, 0x05, 0x05, 0xff, - 0x17, 0x24, 0xdc, 0x01, 0xfa, 0x6b, 0x44, 0xc2, 0x20, 0xf4, 0xa9, 0xa1, 0x75, 0xd5, 0x7e, 0xd3, - 0xd9, 0xcc, 0x52, 0x0b, 0x5e, 0xc8, 0x58, 0xe9, 0xc7, 0x0b, 0x5c, 0xef, 0x8b, 0x02, 0x5a, 0x8f, - 0xe5, 0x80, 0x36, 0x00, 0x2e, 0xa6, 0xc9, 0x84, 0x71, 0xc6, 0x42, 0xe0, 0x56, 0x96, 0x5a, 0x80, - 0x2c, 0xa2, 0x6e, 0x09, 0x01, 0x77, 0x81, 0x26, 0x3c, 0xa3, 0xca, 0xbb, 0x6f, 0x3f, 0xea, 0xfe, - 0x04, 0x4d, 0xe3, 0x09, 0x3e, 0x61, 0x04, 0xa3, 0xa9, 0xd3, 0x92, 0x4a, 0x6b, 0xa2, 0x94, 0x2b, - 0x13, 0xe1, 0x71, 0x31, 0x7c, 0x95, 0x0f, 0x6b, 0xeb, 0x07, 0xfc, 0xf3, 0x09, 0x53, 0x21, 0x29, - 0x4f, 0x2b, 0x4b, 0x2a, 0xee, 0xcd, 0x29, 0x68, 0xe5, 0x97, 0x17, 0x7b, 0x8b, 0x2b, 0xd3, 0x06, - 0xea, 0x19, 0x9e, 0x49, 0x3a, 0x8d, 0x2c, 0xb5, 0x72, 0xd7, 0xcd, 0x3f, 0xf9, 0x82, 0xe1, 0x4b, - 0x86, 0x43, 0x46, 0x25, 0x03, 0x58, 0xbe, 0x2c, 0x07, 0xfc, 0xc8, 0x59, 0x95, 0xad, 0x17, 0x50, - 0xb7, 0x30, 0x7a, 0x9f, 0x15, 0xa0, 0x09, 0x10, 0xb4, 0x8a, 0x35, 0xcf, 0x7f, 0xa3, 0x3a, 0xcd, - 0x2c, 0xb5, 0x44, 0xa0, 0xd8, 0xf8, 0xb6, 0xd8, 0x78, 0xfe, 0x0a, 0x88, 0x2e, 0x70, 0xe8, 0x89, - 0xd5, 0xef, 0x02, 0x9d, 0x11, 0x34, 0xc2, 0x6f, 0x03, 0x4f, 0xde, 0x99, 0x62, 0xbe, 0x3c, 0xfc, - 0xaf, 0x07, 0xff, 0x06, 0x3a, 0x91, 0x74, 0xe4, 0x4b, 0xb0, 0xf1, 0xec, 0x25, 0xd8, 0x0d, 0x67, - 0xce, 0x4f, 0x59, 0x6a, 0x2d, 0x90, 0xee, 0xc2, 0xfa, 0xaf, 0xa6, 0xab, 0x6b, 0xb5, 0xde, 0xb6, - 0x90, 0xa6, 0xb4, 0xc1, 0x1d, 0xa0, 0x7b, 0x01, 0x45, 0xc3, 0x09, 0xf6, 0x78, 0xe3, 0xba, 0xbb, - 0xf0, 0x9d, 0x7f, 0xae, 0xef, 0xcc, 0xca, 0xcd, 0x9d, 0x59, 0x79, 0xb8, 0x33, 0x95, 0xf7, 0x73, - 0x53, 0xf9, 0x34, 0x37, 0x95, 0xab, 0xb9, 0xa9, 0x5c, 0xcf, 0x4d, 0xe5, 0xeb, 0xdc, 0x54, 0xbe, - 0xcd, 0xcd, 0xca, 0xc3, 0xdc, 0x54, 0x3e, 0xdc, 0x9b, 0x95, 0xeb, 0x7b, 0xb3, 0x72, 0x73, 0x6f, - 0x56, 0xde, 0x94, 0x9e, 0xee, 0xa1, 0xc6, 0x7b, 0xfb, 0xe3, 0x7b, 0x00, 0x00, 0x00, 0xff, 0xff, - 0x72, 0xed, 0xdd, 0xa8, 0xe1, 0x05, 0x00, 0x00, -} - -func (this *PrometheusRequest) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusRequest) - if !ok { - that2, ok := that.(PrometheusRequest) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Path != that1.Path { - return false - } - if this.Start != that1.Start { - return false - } - if this.End != that1.End { - return false - } - if this.Step != that1.Step { - return false - } - if this.Timeout != that1.Timeout { - return false - } - if this.Query != that1.Query { - return false - } - if !this.CachingOptions.Equal(&that1.CachingOptions) { - return false - } - if len(this.Headers) != len(that1.Headers) { - return false - } - for i := range this.Headers { - if !this.Headers[i].Equal(that1.Headers[i]) { - return false - } - } - if this.Stats != that1.Stats { - return false - } - return true -} -func (this *PrometheusResponse) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusResponse) - if !ok { - that2, ok := that.(PrometheusResponse) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Status != that1.Status { - return false - } - if !this.Data.Equal(&that1.Data) { - return false - } - if this.ErrorType != that1.ErrorType { - return false - } - if this.Error != that1.Error { - return false - } - if len(this.Headers) != len(that1.Headers) { - return false - } - for i := range this.Headers { - if !this.Headers[i].Equal(that1.Headers[i]) { - return false - } - } - if len(this.Warnings) != len(that1.Warnings) { - return false - } - for i := range this.Warnings { - if this.Warnings[i] != that1.Warnings[i] { - return false - } - } - return true -} -func (this *PrometheusData) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*PrometheusData) - if !ok { - that2, ok := that.(PrometheusData) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.ResultType != that1.ResultType { - return false - } - if len(this.Result) != len(that1.Result) { - return false - } - for i := range this.Result { - if !this.Result[i].Equal(&that1.Result[i]) { - return false - } - } - if !this.Stats.Equal(that1.Stats) { - return false - } - return true -} -func (this *CachedResponse) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*CachedResponse) - if !ok { - that2, ok := that.(CachedResponse) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Key != that1.Key { - return false - } - if len(this.Extents) != len(that1.Extents) { - return false - } - for i := range this.Extents { - if !this.Extents[i].Equal(&that1.Extents[i]) { - return false - } - } - return true -} -func (this *Extent) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*Extent) - if !ok { - that2, ok := that.(Extent) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Start != that1.Start { - return false - } - if this.End != that1.End { - return false - } - if this.TraceId != that1.TraceId { - return false - } - if !this.Response.Equal(that1.Response) { - return false - } - return true -} -func (this *CachingOptions) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*CachingOptions) - if !ok { - that2, ok := that.(CachingOptions) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Disabled != that1.Disabled { - return false - } - return true -} -func (this *PrometheusRequest) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 13) - s = append(s, "&queryrange.PrometheusRequest{") - s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") - s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") - s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") - s = append(s, "Step: "+fmt.Sprintf("%#v", this.Step)+",\n") - s = append(s, "Timeout: "+fmt.Sprintf("%#v", this.Timeout)+",\n") - s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") - s = append(s, "CachingOptions: "+strings.Replace(this.CachingOptions.GoString(), `&`, ``, 1)+",\n") - if this.Headers != nil { - s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") - } - s = append(s, "Stats: "+fmt.Sprintf("%#v", this.Stats)+",\n") - s = append(s, "}") - return strings.Join(s, "") -} -func (this *PrometheusResponse) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 10) - s = append(s, "&queryrange.PrometheusResponse{") - s = append(s, "Status: "+fmt.Sprintf("%#v", this.Status)+",\n") - s = append(s, "Data: "+strings.Replace(this.Data.GoString(), `&`, ``, 1)+",\n") - s = append(s, "ErrorType: "+fmt.Sprintf("%#v", this.ErrorType)+",\n") - s = append(s, "Error: "+fmt.Sprintf("%#v", this.Error)+",\n") - if this.Headers != nil { - s = append(s, "Headers: "+fmt.Sprintf("%#v", this.Headers)+",\n") - } - s = append(s, "Warnings: "+fmt.Sprintf("%#v", this.Warnings)+",\n") - s = append(s, "}") - return strings.Join(s, "") -} -func (this *PrometheusData) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 7) - s = append(s, "&queryrange.PrometheusData{") - s = append(s, "ResultType: "+fmt.Sprintf("%#v", this.ResultType)+",\n") - if this.Result != nil { - vs := make([]*tripperware.SampleStream, len(this.Result)) - for i := range vs { - vs[i] = &this.Result[i] - } - s = append(s, "Result: "+fmt.Sprintf("%#v", vs)+",\n") - } - if this.Stats != nil { - s = append(s, "Stats: "+fmt.Sprintf("%#v", this.Stats)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *CachedResponse) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 6) - s = append(s, "&queryrange.CachedResponse{") - s = append(s, "Key: "+fmt.Sprintf("%#v", this.Key)+",\n") - if this.Extents != nil { - vs := make([]*Extent, len(this.Extents)) - for i := range vs { - vs[i] = &this.Extents[i] - } - s = append(s, "Extents: "+fmt.Sprintf("%#v", vs)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *Extent) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 8) - s = append(s, "&queryrange.Extent{") - s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") - s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") - s = append(s, "TraceId: "+fmt.Sprintf("%#v", this.TraceId)+",\n") - if this.Response != nil { - s = append(s, "Response: "+fmt.Sprintf("%#v", this.Response)+",\n") - } - s = append(s, "}") - return strings.Join(s, "") -} -func (this *CachingOptions) GoString() string { - if this == nil { - return "nil" - } - s := make([]string, 0, 5) - s = append(s, "&queryrange.CachingOptions{") - s = append(s, "Disabled: "+fmt.Sprintf("%#v", this.Disabled)+",\n") - s = append(s, "}") - return strings.Join(s, "") -} -func valueToGoStringQueryrange(v interface{}, typ string) string { - rv := reflect.ValueOf(v) - if rv.IsNil() { - return "nil" - } - pv := reflect.Indirect(rv).Interface() - return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv) -} -func (m *PrometheusRequest) 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 *PrometheusRequest) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Stats) > 0 { - i -= len(m.Stats) - copy(dAtA[i:], m.Stats) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Stats))) - i-- - dAtA[i] = 0x4a - } - if len(m.Headers) > 0 { - for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Headers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x42 - } - } - { - size, err := m.CachingOptions.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x3a - if len(m.Query) > 0 { - i -= len(m.Query) - copy(dAtA[i:], m.Query) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Query))) - i-- - dAtA[i] = 0x32 - } - n2, err2 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Timeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Timeout):]) - if err2 != nil { - return 0, err2 - } - i -= n2 - i = encodeVarintQueryrange(dAtA, i, uint64(n2)) - i-- - dAtA[i] = 0x2a - if m.Step != 0 { - i = encodeVarintQueryrange(dAtA, i, uint64(m.Step)) - i-- - dAtA[i] = 0x20 - } - if m.End != 0 { - i = encodeVarintQueryrange(dAtA, i, uint64(m.End)) - i-- - dAtA[i] = 0x18 - } - if m.Start != 0 { - i = encodeVarintQueryrange(dAtA, i, uint64(m.Start)) - i-- - dAtA[i] = 0x10 - } - if len(m.Path) > 0 { - i -= len(m.Path) - copy(dAtA[i:], m.Path) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Path))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *PrometheusResponse) 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 *PrometheusResponse) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Warnings) > 0 { - for iNdEx := len(m.Warnings) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Warnings[iNdEx]) - copy(dAtA[i:], m.Warnings[iNdEx]) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Warnings[iNdEx]))) - i-- - dAtA[i] = 0x32 - } - } - if len(m.Headers) > 0 { - for iNdEx := len(m.Headers) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Headers[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x2a - } - } - if len(m.Error) > 0 { - i -= len(m.Error) - copy(dAtA[i:], m.Error) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Error))) - i-- - dAtA[i] = 0x22 - } - if len(m.ErrorType) > 0 { - i -= len(m.ErrorType) - copy(dAtA[i:], m.ErrorType) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.ErrorType))) - i-- - dAtA[i] = 0x1a - } - { - size, err := m.Data.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - if len(m.Status) > 0 { - i -= len(m.Status) - copy(dAtA[i:], m.Status) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Status))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *PrometheusData) 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 *PrometheusData) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *PrometheusData) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Stats != nil { - { - size, err := m.Stats.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1a - } - if len(m.Result) > 0 { - for iNdEx := len(m.Result) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Result[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - } - if len(m.ResultType) > 0 { - i -= len(m.ResultType) - copy(dAtA[i:], m.ResultType) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.ResultType))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *CachedResponse) 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 *CachedResponse) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *CachedResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Extents) > 0 { - for iNdEx := len(m.Extents) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.Extents[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - } - if len(m.Key) > 0 { - i -= len(m.Key) - copy(dAtA[i:], m.Key) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Key))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *Extent) 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 *Extent) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Extent) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Response != nil { - { - size, err := m.Response.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQueryrange(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x2a - } - if len(m.TraceId) > 0 { - i -= len(m.TraceId) - copy(dAtA[i:], m.TraceId) - i = encodeVarintQueryrange(dAtA, i, uint64(len(m.TraceId))) - i-- - dAtA[i] = 0x22 - } - if m.End != 0 { - i = encodeVarintQueryrange(dAtA, i, uint64(m.End)) - i-- - dAtA[i] = 0x10 - } - if m.Start != 0 { - i = encodeVarintQueryrange(dAtA, i, uint64(m.Start)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func (m *CachingOptions) 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 *CachingOptions) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *CachingOptions) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Disabled { - i-- - if m.Disabled { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func encodeVarintQueryrange(dAtA []byte, offset int, v uint64) int { - offset -= sovQueryrange(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return base -} -func (m *PrometheusRequest) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Path) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - if m.Start != 0 { - n += 1 + sovQueryrange(uint64(m.Start)) - } - if m.End != 0 { - n += 1 + sovQueryrange(uint64(m.End)) - } - if m.Step != 0 { - n += 1 + sovQueryrange(uint64(m.Step)) - } - l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.Timeout) - n += 1 + l + sovQueryrange(uint64(l)) - l = len(m.Query) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - l = m.CachingOptions.Size() - n += 1 + l + sovQueryrange(uint64(l)) - if len(m.Headers) > 0 { - for _, e := range m.Headers { - l = e.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - } - l = len(m.Stats) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - return n -} - -func (m *PrometheusResponse) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Status) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - l = m.Data.Size() - n += 1 + l + sovQueryrange(uint64(l)) - l = len(m.ErrorType) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - l = len(m.Error) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - if len(m.Headers) > 0 { - for _, e := range m.Headers { - l = e.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - } - if len(m.Warnings) > 0 { - for _, s := range m.Warnings { - l = len(s) - n += 1 + l + sovQueryrange(uint64(l)) - } - } - return n -} - -func (m *PrometheusData) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.ResultType) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - if len(m.Result) > 0 { - for _, e := range m.Result { - l = e.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - } - if m.Stats != nil { - l = m.Stats.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - return n -} - -func (m *CachedResponse) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Key) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - if len(m.Extents) > 0 { - for _, e := range m.Extents { - l = e.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - } - return n -} - -func (m *Extent) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Start != 0 { - n += 1 + sovQueryrange(uint64(m.Start)) - } - if m.End != 0 { - n += 1 + sovQueryrange(uint64(m.End)) - } - l = len(m.TraceId) - if l > 0 { - n += 1 + l + sovQueryrange(uint64(l)) - } - if m.Response != nil { - l = m.Response.Size() - n += 1 + l + sovQueryrange(uint64(l)) - } - return n -} - -func (m *CachingOptions) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Disabled { - n += 2 - } - return n -} - -func sovQueryrange(x uint64) (n int) { - return (math_bits.Len64(x|1) + 6) / 7 -} -func sozQueryrange(x uint64) (n int) { - return sovQueryrange(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (this *PrometheusRequest) String() string { - if this == nil { - return "nil" - } - repeatedStringForHeaders := "[]*PrometheusRequestHeader{" - for _, f := range this.Headers { - repeatedStringForHeaders += strings.Replace(fmt.Sprintf("%v", f), "PrometheusRequestHeader", "tripperware.PrometheusRequestHeader", 1) + "," - } - repeatedStringForHeaders += "}" - s := strings.Join([]string{`&PrometheusRequest{`, - `Path:` + fmt.Sprintf("%v", this.Path) + `,`, - `Start:` + fmt.Sprintf("%v", this.Start) + `,`, - `End:` + fmt.Sprintf("%v", this.End) + `,`, - `Step:` + fmt.Sprintf("%v", this.Step) + `,`, - `Timeout:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Timeout), "Duration", "duration.Duration", 1), `&`, ``, 1) + `,`, - `Query:` + fmt.Sprintf("%v", this.Query) + `,`, - `CachingOptions:` + strings.Replace(strings.Replace(this.CachingOptions.String(), "CachingOptions", "CachingOptions", 1), `&`, ``, 1) + `,`, - `Headers:` + repeatedStringForHeaders + `,`, - `Stats:` + fmt.Sprintf("%v", this.Stats) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusResponse) String() string { - if this == nil { - return "nil" - } - repeatedStringForHeaders := "[]*PrometheusResponseHeader{" - for _, f := range this.Headers { - repeatedStringForHeaders += strings.Replace(fmt.Sprintf("%v", f), "PrometheusResponseHeader", "tripperware.PrometheusResponseHeader", 1) + "," - } - repeatedStringForHeaders += "}" - s := strings.Join([]string{`&PrometheusResponse{`, - `Status:` + fmt.Sprintf("%v", this.Status) + `,`, - `Data:` + strings.Replace(strings.Replace(this.Data.String(), "PrometheusData", "PrometheusData", 1), `&`, ``, 1) + `,`, - `ErrorType:` + fmt.Sprintf("%v", this.ErrorType) + `,`, - `Error:` + fmt.Sprintf("%v", this.Error) + `,`, - `Headers:` + repeatedStringForHeaders + `,`, - `Warnings:` + fmt.Sprintf("%v", this.Warnings) + `,`, - `}`, - }, "") - return s -} -func (this *PrometheusData) String() string { - if this == nil { - return "nil" - } - repeatedStringForResult := "[]SampleStream{" - for _, f := range this.Result { - repeatedStringForResult += fmt.Sprintf("%v", f) + "," - } - repeatedStringForResult += "}" - s := strings.Join([]string{`&PrometheusData{`, - `ResultType:` + fmt.Sprintf("%v", this.ResultType) + `,`, - `Result:` + repeatedStringForResult + `,`, - `Stats:` + strings.Replace(fmt.Sprintf("%v", this.Stats), "PrometheusResponseStats", "tripperware.PrometheusResponseStats", 1) + `,`, - `}`, - }, "") - return s -} -func (this *CachedResponse) String() string { - if this == nil { - return "nil" - } - repeatedStringForExtents := "[]Extent{" - for _, f := range this.Extents { - repeatedStringForExtents += strings.Replace(strings.Replace(f.String(), "Extent", "Extent", 1), `&`, ``, 1) + "," - } - repeatedStringForExtents += "}" - s := strings.Join([]string{`&CachedResponse{`, - `Key:` + fmt.Sprintf("%v", this.Key) + `,`, - `Extents:` + repeatedStringForExtents + `,`, - `}`, - }, "") - return s -} -func (this *Extent) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&Extent{`, - `Start:` + fmt.Sprintf("%v", this.Start) + `,`, - `End:` + fmt.Sprintf("%v", this.End) + `,`, - `TraceId:` + fmt.Sprintf("%v", this.TraceId) + `,`, - `Response:` + strings.Replace(fmt.Sprintf("%v", this.Response), "Any", "types.Any", 1) + `,`, - `}`, - }, "") - return s -} -func (this *CachingOptions) String() string { - if this == nil { - return "nil" - } - s := strings.Join([]string{`&CachingOptions{`, - `Disabled:` + fmt.Sprintf("%v", this.Disabled) + `,`, - `}`, - }, "") - return s -} -func valueToStringQueryrange(v interface{}) string { - rv := reflect.ValueOf(v) - if rv.IsNil() { - return "nil" - } - pv := reflect.Indirect(rv).Interface() - return fmt.Sprintf("*%v", pv) -} -func (m *PrometheusRequest) 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: PrometheusRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Path", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Path = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) - } - m.Start = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Start |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) - } - m.End = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.End |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Step", wireType) - } - m.Step = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Step |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Timeout", 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 err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.Timeout, dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Query", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Query = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CachingOptions", 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 err := m.CachingOptions.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Headers = append(m.Headers, &tripperware.PrometheusRequestHeader{}) - if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 9: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Stats = string(dAtA[iNdEx:postIndex]) - 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 *PrometheusResponse) 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: PrometheusResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Status = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Data", 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 err := m.Data.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ErrorType", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ErrorType = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Error = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Headers", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Headers = append(m.Headers, &tripperware.PrometheusResponseHeader{}) - if err := m.Headers[len(m.Headers)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Warnings", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Warnings = append(m.Warnings, string(dAtA[iNdEx:postIndex])) - 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 *PrometheusData) 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: PrometheusData: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: PrometheusData: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ResultType", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ResultType = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Result = append(m.Result, tripperware.SampleStream{}) - if err := m.Result[len(m.Result)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Stats", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 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.Stats == nil { - m.Stats = &tripperware.PrometheusResponseStats{} - } - if err := m.Stats.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 *CachedResponse) 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: CachedResponse: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: CachedResponse: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Key = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Extents", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Extents = append(m.Extents, Extent{}) - if err := m.Extents[len(m.Extents)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipQueryrange(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthQueryrange - } - if (iNdEx + skippy) < 0 { - return ErrInvalidLengthQueryrange - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Extent) 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: Extent: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Extent: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Start", wireType) - } - m.Start = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Start |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field End", wireType) - } - m.End = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.End |= int64(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TraceId", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.TraceId = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Response", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQueryrange - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQueryrange - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Response == nil { - m.Response = &types.Any{} - } - if err := m.Response.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 *CachingOptions) 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: CachingOptions: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: CachingOptions: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Disabled", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQueryrange - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Disabled = bool(v != 0) - 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 skipQueryrange(dAtA []byte) (n int, err error) { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowQueryrange - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowQueryrange - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - iNdEx++ - if dAtA[iNdEx-1] < 0x80 { - break - } - } - return iNdEx, nil - case 1: - iNdEx += 8 - return iNdEx, nil - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowQueryrange - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if length < 0 { - return 0, ErrInvalidLengthQueryrange - } - iNdEx += length - if iNdEx < 0 { - return 0, ErrInvalidLengthQueryrange - } - return iNdEx, nil - case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowQueryrange - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipQueryrange(dAtA[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - if iNdEx < 0 { - return 0, ErrInvalidLengthQueryrange - } - } - return iNdEx, nil - case 4: - return iNdEx, nil - case 5: - iNdEx += 4 - return iNdEx, nil - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - } - panic("unreachable") -} - -var ( - ErrInvalidLengthQueryrange = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowQueryrange = fmt.Errorf("proto: integer overflow") -) diff --git a/pkg/querier/tripperware/queryrange/queryrange.proto b/pkg/querier/tripperware/queryrange/queryrange.proto deleted file mode 100644 index 3642409504..0000000000 --- a/pkg/querier/tripperware/queryrange/queryrange.proto +++ /dev/null @@ -1,60 +0,0 @@ -syntax = "proto3"; - -package queryrange; - -option go_package = "queryrange"; - -import "gogoproto/gogo.proto"; -import "google/protobuf/duration.proto"; -import "google/protobuf/any.proto"; -import "github.com/cortexproject/cortex/pkg/querier/tripperware/query.proto"; - -option (gogoproto.marshaler_all) = true; -option (gogoproto.unmarshaler_all) = true; - -message PrometheusRequest { - string path = 1; - int64 start = 2; - int64 end = 3; - int64 step = 4; - google.protobuf.Duration timeout = 5 [(gogoproto.stdduration) = true, (gogoproto.nullable) = false]; - string query = 6; - CachingOptions cachingOptions = 7 [(gogoproto.nullable) = false]; - repeated tripperware.PrometheusRequestHeader Headers = 8 [(gogoproto.jsontag) = "-"]; - string stats = 9; -} - -message PrometheusResponse { - string Status = 1 [(gogoproto.jsontag) = "status"]; - PrometheusData Data = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "data,omitempty"]; - string ErrorType = 3 [(gogoproto.jsontag) = "errorType,omitempty"]; - string Error = 4 [(gogoproto.jsontag) = "error,omitempty"]; - repeated tripperware.PrometheusResponseHeader Headers = 5 [(gogoproto.jsontag) = "-"]; - repeated string Warnings = 6 [(gogoproto.jsontag) = "warnings,omitempty"]; -} - -message PrometheusData { - string ResultType = 1 [(gogoproto.jsontag) = "resultType"]; - repeated tripperware.SampleStream Result = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "result"]; - tripperware.PrometheusResponseStats stats = 3 [(gogoproto.jsontag) = "stats,omitempty"]; -} - -message CachedResponse { - string key = 1 [(gogoproto.jsontag) = "key"]; - - // List of cached responses; non-overlapping and in order. - repeated Extent extents = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "extents"]; -} - -message Extent { - int64 start = 1 [(gogoproto.jsontag) = "start"]; - int64 end = 2 [(gogoproto.jsontag) = "end"]; - // reserved the previous key to ensure cache transition - reserved 3; - string trace_id = 4 [(gogoproto.jsontag) = "-"]; - google.protobuf.Any response = 5 [(gogoproto.jsontag) = "response"]; -} - -message CachingOptions { - bool disabled = 1; -} diff --git a/pkg/querier/tripperware/queryrange/results_cache.go b/pkg/querier/tripperware/queryrange/results_cache.go index 5d5f073ed7..128a24130c 100644 --- a/pkg/querier/tripperware/queryrange/results_cache.go +++ b/pkg/querier/tripperware/queryrange/results_cache.go @@ -85,13 +85,19 @@ type PrometheusResponseExtractor struct{} // Extract extracts response for specific a range from a response. func (PrometheusResponseExtractor) Extract(start, end int64, from tripperware.Response) tripperware.Response { - promRes := from.(*PrometheusResponse) - return &PrometheusResponse{ + promRes := from.(*tripperware.PrometheusResponse) + return &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: promRes.Data.ResultType, - Result: extractMatrix(start, end, promRes.Data.Result), - Stats: extractStats(start, end, promRes.Data.Stats), + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: extractSampleStreams(start, end, promRes.Data.Result.GetMatrix().GetSampleStreams()), + }, + }, + }, + Stats: extractStats(start, end, promRes.Data.Stats), }, Headers: promRes.Headers, Warnings: promRes.Warnings, @@ -101,10 +107,10 @@ func (PrometheusResponseExtractor) Extract(start, end int64, from tripperware.Re // ResponseWithoutHeaders is useful in caching data without headers since // we anyways do not need headers for sending back the response so this saves some space by reducing size of the objects. func (PrometheusResponseExtractor) ResponseWithoutHeaders(resp tripperware.Response) tripperware.Response { - promRes := resp.(*PrometheusResponse) - return &PrometheusResponse{ + promRes := resp.(*tripperware.PrometheusResponse) + return &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: promRes.Data.ResultType, Result: promRes.Data.Result, Stats: promRes.Data.Stats, @@ -115,10 +121,10 @@ func (PrometheusResponseExtractor) ResponseWithoutHeaders(resp tripperware.Respo // ResponseWithoutStats is returns the response without the stats information func (PrometheusResponseExtractor) ResponseWithoutStats(resp tripperware.Response) tripperware.Response { - promRes := resp.(*PrometheusResponse) - return &PrometheusResponse{ + promRes := resp.(*tripperware.PrometheusResponse) + return &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: promRes.Data.ResultType, Result: promRes.Data.Result, }, @@ -223,7 +229,7 @@ func (s resultsCache) Do(ctx context.Context, r tripperware.Request) (tripperwar var ( key = s.splitter.GenerateCacheKey(tenant.JoinTenantIDs(tenantIDs), r) - extents []Extent + extents []tripperware.Extent response tripperware.Response ) @@ -378,7 +384,7 @@ func getHeaderValuesWithName(r tripperware.Response, headerName string) (headerV return } -func (s resultsCache) handleMiss(ctx context.Context, r tripperware.Request, maxCacheTime int64) (tripperware.Response, []Extent, error) { +func (s resultsCache) handleMiss(ctx context.Context, r tripperware.Request, maxCacheTime int64) (tripperware.Response, []tripperware.Extent, error) { level.Debug(util_log.WithContext(ctx, s.logger)).Log("msg", "handle miss", "start", r.GetStart(), "spanID", jaegerSpanID(ctx)) response, err := s.next.Do(ctx, r) if err != nil { @@ -386,7 +392,7 @@ func (s resultsCache) handleMiss(ctx context.Context, r tripperware.Request, max } if !s.shouldCacheResponse(ctx, r, response, maxCacheTime) { - return response, []Extent{}, nil + return response, []tripperware.Extent{}, nil } extent, err := toExtent(ctx, r, s.extractor.ResponseWithoutHeaders(response)) @@ -394,13 +400,13 @@ func (s resultsCache) handleMiss(ctx context.Context, r tripperware.Request, max return nil, nil, err } - extents := []Extent{ + extents := []tripperware.Extent{ extent, } return response, extents, nil } -func (s resultsCache) handleHit(ctx context.Context, r tripperware.Request, extents []Extent, maxCacheTime int64) (tripperware.Response, []Extent, error) { +func (s resultsCache) handleHit(ctx context.Context, r tripperware.Request, extents []tripperware.Extent, maxCacheTime int64) (tripperware.Response, []tripperware.Extent, error) { var ( reqResps []tripperware.RequestResponse err error @@ -452,7 +458,7 @@ func (s resultsCache) handleHit(ctx context.Context, r tripperware.Request, exte if err != nil { return nil, nil, err } - mergedExtents := make([]Extent, 0, len(extents)) + mergedExtents := make([]tripperware.Extent, 0, len(extents)) for i := 1; i < len(extents); i++ { if accumulator.End+r.GetStep() < extents[i].Start { @@ -473,7 +479,7 @@ func (s resultsCache) handleHit(ctx context.Context, r tripperware.Request, exte accumulator.TraceId = jaegerTraceID(ctx) accumulator.End = extents[i].End - currentRes, err := extents[i].toResponse() + currentRes, err := extents[i].ToResponse() if err != nil { return nil, nil, err } @@ -495,15 +501,15 @@ func (s resultsCache) handleHit(ctx context.Context, r tripperware.Request, exte type accumulator struct { tripperware.Response - Extent + tripperware.Extent } -func merge(extents []Extent, acc *accumulator) ([]Extent, error) { +func merge(extents []tripperware.Extent, acc *accumulator) ([]tripperware.Extent, error) { any, err := types.MarshalAny(acc.Response) if err != nil { return nil, err } - return append(extents, Extent{ + return append(extents, tripperware.Extent{ Start: acc.Extent.Start, End: acc.Extent.End, Response: any, @@ -511,8 +517,8 @@ func merge(extents []Extent, acc *accumulator) ([]Extent, error) { }), nil } -func newAccumulator(base Extent) (*accumulator, error) { - res, err := base.toResponse() +func newAccumulator(base tripperware.Extent) (*accumulator, error) { + res, err := base.ToResponse() if err != nil { return nil, err } @@ -522,12 +528,12 @@ func newAccumulator(base Extent) (*accumulator, error) { }, nil } -func toExtent(ctx context.Context, req tripperware.Request, res tripperware.Response) (Extent, error) { +func toExtent(ctx context.Context, req tripperware.Request, res tripperware.Response) (tripperware.Extent, error) { any, err := types.MarshalAny(res) if err != nil { - return Extent{}, err + return tripperware.Extent{}, err } - return Extent{ + return tripperware.Extent{ Start: req.GetStart(), End: req.GetEnd(), Response: any, @@ -537,7 +543,7 @@ func toExtent(ctx context.Context, req tripperware.Request, res tripperware.Resp // partition calculates the required requests to satisfy req given the cached data. // extents must be in order by start time. -func (s resultsCache) partition(req tripperware.Request, extents []Extent) ([]tripperware.Request, []tripperware.Response, error) { +func (s resultsCache) partition(req tripperware.Request, extents []tripperware.Extent) ([]tripperware.Request, []tripperware.Response, error) { var requests []tripperware.Request var cachedResponses []tripperware.Response start := req.GetStart() @@ -563,7 +569,7 @@ func (s resultsCache) partition(req tripperware.Request, extents []Extent) ([]tr r := req.WithStartEnd(start, extent.Start) requests = append(requests, r) } - res, err := extent.toResponse() + res, err := extent.ToResponse() if err != nil { return nil, nil, err } @@ -587,13 +593,13 @@ func (s resultsCache) partition(req tripperware.Request, extents []Extent) ([]tr return requests, cachedResponses, nil } -func (s resultsCache) filterRecentExtents(req tripperware.Request, maxCacheFreshness time.Duration, extents []Extent) ([]Extent, error) { +func (s resultsCache) filterRecentExtents(req tripperware.Request, maxCacheFreshness time.Duration, extents []tripperware.Extent) ([]tripperware.Extent, error) { maxCacheTime := (int64(model.Now().Add(-maxCacheFreshness)) / req.GetStep()) * req.GetStep() for i := range extents { // Never cache data for the latest freshness period. if extents[i].End > maxCacheTime { extents[i].End = maxCacheTime - res, err := extents[i].toResponse() + res, err := extents[i].ToResponse() if err != nil { return nil, err } @@ -608,13 +614,13 @@ func (s resultsCache) filterRecentExtents(req tripperware.Request, maxCacheFresh return extents, nil } -func (s resultsCache) get(ctx context.Context, key string) ([]Extent, bool) { +func (s resultsCache) get(ctx context.Context, key string) ([]tripperware.Extent, bool) { found, bufs, _ := s.cache.Fetch(ctx, []string{cache.HashKey(key)}) if len(found) != 1 { return nil, false } - var resp CachedResponse + var resp tripperware.CachedResponse log, ctx := spanlogger.New(ctx, "unmarshal-extent") //nolint:ineffassign,staticcheck defer log.Finish() @@ -640,8 +646,8 @@ func (s resultsCache) get(ctx context.Context, key string) ([]Extent, bool) { return resp.Extents, true } -func (s resultsCache) put(ctx context.Context, key string, extents []Extent) { - buf, err := proto.Marshal(&CachedResponse{ +func (s resultsCache) put(ctx context.Context, key string, extents []tripperware.Extent) { + buf, err := proto.Marshal(&tripperware.CachedResponse{ Key: key, Extents: extents, }) @@ -702,7 +708,7 @@ func extractStats(start, end int64, stats *tripperware.PrometheusResponseStats) return result } -func extractMatrix(start, end int64, matrix []tripperware.SampleStream) []tripperware.SampleStream { +func extractSampleStreams(start, end int64, matrix []tripperware.SampleStream) []tripperware.SampleStream { result := make([]tripperware.SampleStream, 0, len(matrix)) for _, stream := range matrix { extracted, ok := extractSampleStream(start, end, stream) @@ -728,20 +734,3 @@ func extractSampleStream(start, end int64, stream tripperware.SampleStream) (tri } return result, true } - -func (e *Extent) toResponse() (tripperware.Response, error) { - msg, err := types.EmptyAny(e.Response) - if err != nil { - return nil, err - } - - if err := types.UnmarshalAny(e.Response, msg); err != nil { - return nil, err - } - - resp, ok := msg.(tripperware.Response) - if !ok { - return nil, fmt.Errorf("bad cached type") - } - return resp, nil -} diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index abc216c4bd..721bbfe3c2 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -3,6 +3,7 @@ package queryrange import ( "context" "fmt" + "net/http" "strconv" "testing" "time" @@ -28,7 +29,7 @@ const ( ) var ( - parsedRequest = &PrometheusRequest{ + parsedRequest = &tripperware.PrometheusRequest{ Path: "/api/v1/query_range", Start: 1536673680 * 1e3, End: 1536716898 * 1e3, @@ -36,28 +37,24 @@ var ( Query: "sum(container_memory_rss) by (namespace)", Stats: "all", } - reqHeaders = []*tripperware.PrometheusRequestHeader{ - { - Name: "Test-Header", - Values: []string{"test"}, - }, - } - noCacheRequest = &PrometheusRequest{ + reqHeaders = http.Header(map[string][]string{"Test-Header": {"test"}}) + + noCacheRequest = &tripperware.PrometheusRequest{ Path: "/api/v1/query_range", Start: 1536673680 * 1e3, End: 1536716898 * 1e3, Step: 120 * 1e3, Query: "sum(container_memory_rss) by (namespace)", - CachingOptions: CachingOptions{Disabled: true}, + CachingOptions: tripperware.CachingOptions{Disabled: true}, } - noCacheRequestWithStats = &PrometheusRequest{ + noCacheRequestWithStats = &tripperware.PrometheusRequest{ Path: "/api/v1/query_range", Start: 1536673680 * 1e3, End: 1536716898 * 1e3, Step: 120 * 1e3, Stats: "all", Query: "sum(container_memory_rss) by (namespace)", - CachingOptions: CachingOptions{Disabled: true}, + CachingOptions: tripperware.CachingOptions{Disabled: true}, } respHeaders = []*tripperware.PrometheusResponseHeader{ { @@ -65,37 +62,49 @@ var ( Values: []string{"application/json"}, }, } - parsedResponse = &PrometheusResponse{ + parsedResponse = &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "foo", Value: "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 137, TimestampMs: 1536673680000}, - {Value: 137, TimestampMs: 1536673780000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "foo", Value: "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 137, TimestampMs: 1536673680000}, + {Value: 137, TimestampMs: 1536673780000}, + }, + }, + }, }, }, }, }, } - parsedResponseWithWarnings = &PrometheusResponse{ + parsedResponseWithWarnings = &tripperware.PrometheusResponse{ Status: "success", Warnings: []string{"test-warn"}, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "foo", Value: "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 137, TimestampMs: 1536673680000}, - {Value: 137, TimestampMs: 1536673780000}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "foo", Value: "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 137, TimestampMs: 1536673680000}, + {Value: 137, TimestampMs: 1536673780000}, + }, + }, + }, }, }, }, @@ -103,11 +112,11 @@ var ( } ) -func mkAPIResponse(start, end, step int64) *PrometheusResponse { +func mkAPIResponse(start, end, step int64) *tripperware.PrometheusResponse { return mkAPIResponseWithStats(start, end, step, false) } -func mkAPIResponseWithStats(start, end, step int64, withStats bool) *PrometheusResponse { +func mkAPIResponseWithStats(start, end, step int64, withStats bool) *tripperware.PrometheusResponse { var samples []cortexpb.Sample var stats *tripperware.PrometheusResponseStats if withStats { @@ -129,42 +138,48 @@ func mkAPIResponseWithStats(start, end, step int64, withStats bool) *PrometheusR } } - return &PrometheusResponse{ + return &tripperware.PrometheusResponse{ Status: StatusSuccess, - Data: PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: matrix, Stats: stats, - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "foo", Value: "bar"}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "foo", Value: "bar"}, + }, + Samples: samples, + }, + }, }, - Samples: samples, }, }, }, } } -func mkExtentWithStats(start, end int64) Extent { +func mkExtentWithStats(start, end int64) tripperware.Extent { return mkExtentWithStepWithStats(start, end, 10, true) } -func mkExtent(start, end int64) Extent { +func mkExtent(start, end int64) tripperware.Extent { return mkExtentWithStepWithStats(start, end, 10, false) } -func mkExtentWithStep(start, end, step int64) Extent { +func mkExtentWithStep(start, end, step int64) tripperware.Extent { return mkExtentWithStepWithStats(start, end, step, false) } -func mkExtentWithStepWithStats(start, end, step int64, withStats bool) Extent { +func mkExtentWithStepWithStats(start, end, step int64, withStats bool) tripperware.Extent { res := mkAPIResponseWithStats(start, end, step, withStats) any, err := types.MarshalAny(res) if err != nil { panic(err) } - return Extent{ + return tripperware.Extent{ Start: start, End: end, Response: any, @@ -267,8 +282,8 @@ func TestShouldCache(t *testing.T) { // Tests only for cacheControlHeader { name: "does not contain the cacheControl header", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{ + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{ Headers: []*tripperware.PrometheusResponseHeader{ { Name: "meaninglessheader", @@ -280,8 +295,8 @@ func TestShouldCache(t *testing.T) { }, { name: "does contain the cacheControl header which has the value", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{ + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{ Headers: []*tripperware.PrometheusResponseHeader{ { Name: cacheControlHeader, @@ -293,8 +308,8 @@ func TestShouldCache(t *testing.T) { }, { name: "cacheControl header contains extra values but still good", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{ + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{ Headers: []*tripperware.PrometheusResponseHeader{ { Name: cacheControlHeader, @@ -306,22 +321,22 @@ func TestShouldCache(t *testing.T) { }, { name: "broken response", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "nil headers", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{ + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{ Headers: []*tripperware.PrometheusResponseHeader{nil}, }), expected: true, }, { name: "had cacheControl header but no values", - request: &PrometheusRequest{Query: "metric"}, - input: tripperware.Response(&PrometheusResponse{ + request: &tripperware.PrometheusRequest{Query: "metric"}, + input: tripperware.Response(&tripperware.PrometheusResponse{ Headers: []*tripperware.PrometheusResponseHeader{{Name: cacheControlHeader}}, }), expected: true, @@ -329,151 +344,151 @@ func TestShouldCache(t *testing.T) { // @ modifier on vector selectors. { name: "@ modifier on vector selector, before end, before maxCacheTime", - request: &PrometheusRequest{Query: "metric @ 123", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ 123", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on vector selector, after end, before maxCacheTime", - request: &PrometheusRequest{Query: "metric @ 127", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ 127", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on vector selector, before end, after maxCacheTime", - request: &PrometheusRequest{Query: "metric @ 151", End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ 151", End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on vector selector, after end, after maxCacheTime", - request: &PrometheusRequest{Query: "metric @ 151", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ 151", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on vector selector with start() before maxCacheTime", - request: &PrometheusRequest{Query: "metric @ start()", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ start()", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on vector selector with end() after maxCacheTime", - request: &PrometheusRequest{Query: "metric @ end()", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric @ end()", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, // @ modifier on matrix selectors. { name: "@ modifier on matrix selector, before end, before maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ 123)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ 123)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on matrix selector, after end, before maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ 127)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ 127)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on matrix selector, before end, after maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ 151)", End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ 151)", End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on matrix selector, after end, after maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ 151)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ 151)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on matrix selector with start() before maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ start())", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ start())", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on matrix selector with end() after maxCacheTime", - request: &PrometheusRequest{Query: "rate(metric[5m] @ end())", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] @ end())", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, // @ modifier on subqueries. { name: "@ modifier on subqueries, before end, before maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 123)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 123)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on subqueries, after end, before maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 127)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 127)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on subqueries, before end, after maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 151)", End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 151)", End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on subqueries, after end, after maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 151)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ 151)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, { name: "@ modifier on subqueries with start() before maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ start())", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ start())", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "@ modifier on subqueries with end() after maxCacheTime", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ end())", Start: 100000, End: 200000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] @ end())", Start: 100000, End: 200000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, // offset on vector selectors. { name: "positive offset on vector selector", - request: &PrometheusRequest{Query: "metric offset 10ms", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric offset 10ms", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "negative offset on vector selector", - request: &PrometheusRequest{Query: "metric offset -10ms", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "metric offset -10ms", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, // offset on matrix selectors. { name: "positive offset on matrix selector", - request: &PrometheusRequest{Query: "rate(metric[5m] offset 10ms)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] offset 10ms)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "negative offset on matrix selector", - request: &PrometheusRequest{Query: "rate(metric[5m] offset -10ms)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "rate(metric[5m] offset -10ms)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, // offset on subqueries. { name: "positive offset on subqueries", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] offset 10ms)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] offset 10ms)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: true, }, { name: "negative offset on subqueries", - request: &PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] offset -10ms)", End: 125000}, - input: tripperware.Response(&PrometheusResponse{}), + request: &tripperware.PrometheusRequest{Query: "sum_over_time(rate(metric[1m])[10m:1m] offset -10ms)", End: 125000}, + input: tripperware.Response(&tripperware.PrometheusResponse{}), expected: false, }, } { @@ -492,17 +507,17 @@ func TestPartition(t *testing.T) { for _, tc := range []struct { name string input tripperware.Request - prevCachedResponse []Extent + prevCachedResponse []tripperware.Extent expectedRequests []tripperware.Request expectedCachedResponse []tripperware.Response }{ { name: "Test a complete hit.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(0, 100), }, expectedCachedResponse: []tripperware.Response{ @@ -512,30 +527,30 @@ func TestPartition(t *testing.T) { { name: "Test with a complete miss.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(110, 210), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 100, }}, }, { name: "Test a partial hit.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(50, 100), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 50, }, @@ -546,16 +561,16 @@ func TestPartition(t *testing.T) { }, { name: "Test multiple partial hits.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 200, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(50, 120), mkExtent(160, 250), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 120, End: 160, }, @@ -567,16 +582,16 @@ func TestPartition(t *testing.T) { }, { name: "Partial hits with tiny gap.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 160, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(50, 120), mkExtent(122, 130), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 120, End: 160, }, @@ -587,15 +602,15 @@ func TestPartition(t *testing.T) { }, { name: "Extent is outside the range and the request has a single step (same start and end).", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(50, 90), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 100, End: 100, }, @@ -604,11 +619,11 @@ func TestPartition(t *testing.T) { { name: "Test when hit has a large step and only a single sample extent.", // If there is a only a single sample in the split interval, start and end will be the same. - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtent(100, 100), }, expectedCachedResponse: []tripperware.Response{ @@ -617,11 +632,11 @@ func TestPartition(t *testing.T) { }, { name: "[Stats] Test a complete hit.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(0, 100), }, expectedCachedResponse: []tripperware.Response{ @@ -631,30 +646,30 @@ func TestPartition(t *testing.T) { { name: "[Stats] Test with a complete miss.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(110, 210), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 100, }}, }, { name: "[stats] Test a partial hit.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(50, 100), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 50, }, @@ -665,16 +680,16 @@ func TestPartition(t *testing.T) { }, { name: "[stats] Test multiple partial hits.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 200, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(50, 120), mkExtentWithStats(160, 250), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 120, End: 160, }, @@ -686,16 +701,16 @@ func TestPartition(t *testing.T) { }, { name: "[stats] Partial hits with tiny gap.", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 160, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(50, 120), mkExtentWithStats(122, 130), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 120, End: 160, }, @@ -706,15 +721,15 @@ func TestPartition(t *testing.T) { }, { name: "[stats] Extent is outside the range and the request has a single step (same start and end).", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(50, 90), }, expectedRequests: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 100, End: 100, }, @@ -723,11 +738,11 @@ func TestPartition(t *testing.T) { { name: "[stats] Test when hit has a large step and only a single sample extent.", // If there is a only a single sample in the split interval, start and end will be the same. - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 100, }, - prevCachedResponse: []Extent{ + prevCachedResponse: []tripperware.Extent{ mkExtentWithStats(100, 100), }, expectedCachedResponse: []tripperware.Response{ @@ -755,24 +770,24 @@ func TestHandleHit(t *testing.T) { for _, tc := range []struct { name string input tripperware.Request - cachedEntry []Extent - expectedUpdatedCachedEntry []Extent + cachedEntry []tripperware.Extent + expectedUpdatedCachedEntry []tripperware.Extent }{ { name: "Should drop tiny extent that overlaps with non-tiny request only", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 120, Step: 5, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 50, 5), mkExtentWithStep(60, 65, 5), mkExtentWithStep(100, 105, 5), mkExtentWithStep(110, 150, 5), mkExtentWithStep(160, 165, 5), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 50, 5), mkExtentWithStep(60, 65, 5), mkExtentWithStep(100, 150, 5), @@ -781,12 +796,12 @@ func TestHandleHit(t *testing.T) { }, { name: "Should replace tiny extents that are cover by bigger request", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 200, Step: 5, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 50, 5), mkExtentWithStep(60, 65, 5), mkExtentWithStep(100, 105, 5), @@ -795,7 +810,7 @@ func TestHandleHit(t *testing.T) { mkExtentWithStep(220, 225, 5), mkExtentWithStep(240, 250, 5), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 50, 5), mkExtentWithStep(60, 65, 5), mkExtentWithStep(100, 200, 5), @@ -805,12 +820,12 @@ func TestHandleHit(t *testing.T) { }, { name: "Should not drop tiny extent that completely overlaps with tiny request", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 105, Step: 5, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 50, 5), mkExtentWithStep(60, 65, 5), mkExtentWithStep(100, 105, 5), @@ -820,12 +835,12 @@ func TestHandleHit(t *testing.T) { }, { name: "Should not drop tiny extent that partially center-overlaps with tiny request", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 106, End: 108, Step: 2, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 64, 2), mkExtentWithStep(104, 110, 2), mkExtentWithStep(160, 166, 2), @@ -834,17 +849,17 @@ func TestHandleHit(t *testing.T) { }, { name: "Should not drop tiny extent that partially left-overlaps with tiny request", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 106, Step: 2, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 64, 2), mkExtentWithStep(104, 110, 2), mkExtentWithStep(160, 166, 2), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 64, 2), mkExtentWithStep(100, 110, 2), mkExtentWithStep(160, 166, 2), @@ -852,17 +867,17 @@ func TestHandleHit(t *testing.T) { }, { name: "Should not drop tiny extent that partially right-overlaps with tiny request", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 106, Step: 2, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 64, 2), mkExtentWithStep(98, 102, 2), mkExtentWithStep(160, 166, 2), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 64, 2), mkExtentWithStep(98, 106, 2), mkExtentWithStep(160, 166, 2), @@ -870,57 +885,57 @@ func TestHandleHit(t *testing.T) { }, { name: "Should merge fragmented extents if request fills the hole", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 40, End: 80, Step: 20, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 20, 20), mkExtentWithStep(80, 100, 20), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(0, 100, 20), }, }, { name: "Should left-extend extent if request starts earlier than extent in cache", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 40, End: 80, Step: 20, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 160, 20), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(40, 160, 20), }, }, { name: "Should right-extend extent if request ends later than extent in cache", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 100, End: 180, Step: 20, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 160, 20), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 180, 20), }, }, { name: "Should not throw error if complete-overlapped smaller Extent is erroneous", - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ // This request is carefully created such that cachedEntry is not used to fulfill // the request. Start: 160, End: 180, Step: 20, }, - cachedEntry: []Extent{ + cachedEntry: []tripperware.Extent{ { Start: 60, End: 80, @@ -932,7 +947,7 @@ func TestHandleHit(t *testing.T) { }, mkExtentWithStep(60, 160, 20), }, - expectedUpdatedCachedEntry: []Extent{ + expectedUpdatedCachedEntry: []tripperware.Extent{ mkExtentWithStep(60, 180, 20), }, }, @@ -1051,7 +1066,7 @@ func TestResultsCacheMaxFreshness(t *testing.T) { for i, tc := range []struct { fakeLimits tripperware.Limits Handler tripperware.HandlerFunc - expectedResponse *PrometheusResponse + expectedResponse *tripperware.PrometheusResponse }{ { fakeLimits: mockLimits{maxCacheFreshness: 5 * time.Second}, @@ -1096,7 +1111,7 @@ func TestResultsCacheMaxFreshness(t *testing.T) { // fill cache key := constSplitter(day).GenerateCacheKey("1", req) - rc.(*resultsCache).put(ctx, key, []Extent{mkExtent(int64(modelNow)-(600*1e3), int64(modelNow))}) + rc.(*resultsCache).put(ctx, key, []tripperware.Extent{mkExtent(int64(modelNow)-(600*1e3), int64(modelNow))}) resp, err := rc.Do(ctx, req) require.NoError(t, err) @@ -1127,13 +1142,13 @@ func Test_resultsCache_MissingData(t *testing.T) { ctx := context.Background() // fill up the cache - rc.put(ctx, "empty", []Extent{{ + rc.put(ctx, "empty", []tripperware.Extent{{ Start: 100, End: 200, Response: nil, }}) - rc.put(ctx, "notempty", []Extent{mkExtent(100, 120)}) - rc.put(ctx, "mixed", []Extent{mkExtent(100, 120), { + rc.put(ctx, "notempty", []tripperware.Extent{mkExtent(100, 120)}) + rc.put(ctx, "mixed", []tripperware.Extent{mkExtent(100, 120), { Start: 120, End: 200, Response: nil, @@ -1161,14 +1176,14 @@ func TestConstSplitter_generateCacheKey(t *testing.T) { interval time.Duration want string }{ - {"0", &PrometheusRequest{Start: 0, Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:0"}, - {"<30m", &PrometheusRequest{Start: toMs(10 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:0"}, - {"30m", &PrometheusRequest{Start: toMs(30 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:1"}, - {"91m", &PrometheusRequest{Start: toMs(91 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:3"}, - {"0", &PrometheusRequest{Start: 0, Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:0"}, - {"<1d", &PrometheusRequest{Start: toMs(22 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:0"}, - {"4d", &PrometheusRequest{Start: toMs(4 * 24 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:4"}, - {"3d5h", &PrometheusRequest{Start: toMs(77 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:3"}, + {"0", &tripperware.PrometheusRequest{Start: 0, Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:0"}, + {"<30m", &tripperware.PrometheusRequest{Start: toMs(10 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:0"}, + {"30m", &tripperware.PrometheusRequest{Start: toMs(30 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:1"}, + {"91m", &tripperware.PrometheusRequest{Start: toMs(91 * time.Minute), Step: 10, Query: "foo{}"}, 30 * time.Minute, "fake:foo{}:10:3"}, + {"0", &tripperware.PrometheusRequest{Start: 0, Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:0"}, + {"<1d", &tripperware.PrometheusRequest{Start: toMs(22 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:0"}, + {"4d", &tripperware.PrometheusRequest{Start: toMs(4 * 24 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:4"}, + {"3d5h", &tripperware.PrometheusRequest{Start: toMs(77 * time.Hour), Step: 10, Query: "foo{}"}, 24 * time.Hour, "fake:foo{}:10:3"}, } for _, tt := range tests { tt := tt @@ -1206,7 +1221,7 @@ func TestResultsCacheShouldCacheFunc(t *testing.T) { { name: "check cache based on request", shouldCache: func(r tripperware.Request) bool { - if v, ok := r.(*PrometheusRequest); ok { + if v, ok := r.(*tripperware.PrometheusRequest); ok { return !v.CachingOptions.Disabled } return false diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index e3feb76b26..4d7e4a9a68 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -72,14 +72,14 @@ func TestSplitQuery(t *testing.T) { interval time.Duration }{ { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 60 * 60 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 60 * 60 * seconds, Step: 15 * seconds, @@ -89,14 +89,14 @@ func TestSplitQuery(t *testing.T) { interval: day, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 60 * 60 * seconds, End: 60 * 60 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 60 * 60 * seconds, End: 60 * 60 * seconds, Step: 15 * seconds, @@ -106,14 +106,14 @@ func TestSplitQuery(t *testing.T) { interval: day, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 60 * 60 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 60 * 60 * seconds, Step: 15 * seconds, @@ -123,14 +123,14 @@ func TestSplitQuery(t *testing.T) { interval: 3 * time.Hour, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 24 * 3600 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 24 * 3600 * seconds, Step: 15 * seconds, @@ -140,14 +140,14 @@ func TestSplitQuery(t *testing.T) { interval: day, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 3 * 3600 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: 3 * 3600 * seconds, Step: 15 * seconds, @@ -157,20 +157,20 @@ func TestSplitQuery(t *testing.T) { interval: 3 * time.Hour, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 2 * 24 * 3600 * seconds, Step: 15 * seconds, Query: "foo @ start()", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: (24 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo @ 0.000", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 24 * 3600 * seconds, End: 2 * 24 * 3600 * seconds, Step: 15 * seconds, @@ -180,20 +180,20 @@ func TestSplitQuery(t *testing.T) { interval: day, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 2 * 3 * 3600 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 0, End: (3 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 3 * 3600 * seconds, End: 2 * 3 * 3600 * seconds, Step: 15 * seconds, @@ -203,26 +203,26 @@ func TestSplitQuery(t *testing.T) { interval: 3 * time.Hour, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 3 * 3600 * seconds, End: 3 * 24 * 3600 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 3 * 3600 * seconds, End: (24 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 24 * 3600 * seconds, End: (2 * 24 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 2 * 24 * 3600 * seconds, End: 3 * 24 * 3600 * seconds, Step: 15 * seconds, @@ -232,26 +232,26 @@ func TestSplitQuery(t *testing.T) { interval: day, }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 2 * 3600 * seconds, End: 3 * 3 * 3600 * seconds, Step: 15 * seconds, Query: "foo", }, expected: []tripperware.Request{ - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 2 * 3600 * seconds, End: (3 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 3 * 3600 * seconds, End: (2 * 3 * 3600 * seconds) - (15 * seconds), Step: 15 * seconds, Query: "foo", }, - &PrometheusRequest{ + &tripperware.PrometheusRequest{ Start: 2 * 3 * 3600 * seconds, End: 3 * 3 * 3600 * seconds, Step: 15 * seconds, diff --git a/pkg/querier/tripperware/queryrange/step_align_test.go b/pkg/querier/tripperware/queryrange/step_align_test.go index b5e4ce5a98..ac197b5b46 100644 --- a/pkg/querier/tripperware/queryrange/step_align_test.go +++ b/pkg/querier/tripperware/queryrange/step_align_test.go @@ -12,15 +12,15 @@ import ( func TestStepAlign(t *testing.T) { for i, tc := range []struct { - input, expected *PrometheusRequest + input, expected *tripperware.PrometheusRequest }{ { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 0, End: 100, Step: 10, }, - expected: &PrometheusRequest{ + expected: &tripperware.PrometheusRequest{ Start: 0, End: 100, Step: 10, @@ -28,12 +28,12 @@ func TestStepAlign(t *testing.T) { }, { - input: &PrometheusRequest{ + input: &tripperware.PrometheusRequest{ Start: 2, End: 102, Step: 10, }, - expected: &PrometheusRequest{ + expected: &tripperware.PrometheusRequest{ Start: 0, End: 100, Step: 10, @@ -43,10 +43,10 @@ func TestStepAlign(t *testing.T) { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - var result *PrometheusRequest + var result *tripperware.PrometheusRequest s := stepAlign{ next: tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - result = req.(*PrometheusRequest) + result = req.(*tripperware.PrometheusRequest) return nil, nil }), }