From b1d47931b1cb0ff76de902e8a82593c5954c1b1b Mon Sep 17 00:00:00 2001 From: afayek Date: Wed, 28 Jun 2023 17:00:21 -0700 Subject: [PATCH 01/38] replace json data conversion with protobuf for querier handler Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 2 +- pkg/querier/handler/handler.go | 1982 +++++++++++++++++ .../tripperware/queryrange/query_range.go | 3 +- 3 files changed, 1985 insertions(+), 2 deletions(-) create mode 100644 pkg/querier/handler/handler.go diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 17bd42d7c44..4cc1b7c5d2a 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -8,6 +8,7 @@ import ( "path" "sync" + v1 "github.com/cortexproject/cortex/pkg/querier/handler" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gorilla/mux" @@ -21,7 +22,6 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" - v1 "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go new file mode 100644 index 00000000000..5c55a770344 --- /dev/null +++ b/pkg/querier/handler/handler.go @@ -0,0 +1,1982 @@ +// Copyright 2016 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package handler + +import ( + "context" + "fmt" + "github.com/gogo/protobuf/proto" + "math" + "math/rand" + "net" + "net/http" + "net/url" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "time" + "unsafe" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/regexp" + jsoniter "github.com/json-iterator/go" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/common/route" + "golang.org/x/exp/slices" + + "github.com/cortexproject/cortex/pkg/cortexpb" + github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/textparse" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/scrape" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/util/httputil" + "github.com/prometheus/prometheus/util/jsonutil" + "github.com/prometheus/prometheus/util/stats" +) + +type status string + +const ( + statusSuccess status = "success" + statusError status = "error" + + // Non-standard status code (originally introduced by nginx) for the case when a client closes + // the connection while the server is still processing the request. + statusClientClosedConnection = 499 +) + +type errorType string + +const ( + errorNone errorType = "" + errorTimeout errorType = "timeout" + errorCanceled errorType = "canceled" + errorExec errorType = "execution" + errorBadData errorType = "bad_data" + errorInternal errorType = "internal" + errorUnavailable errorType = "unavailable" + errorNotFound errorType = "not_found" +) + +var LocalhostRepresentations = []string{"127.0.0.1", "localhost", "::1"} + +type apiError struct { + typ errorType + err error +} + +func (e *apiError) Error() string { + return fmt.Sprintf("%s: %s", e.typ, e.err) +} + +// ScrapePoolsRetriever provide the list of all scrape pools. +type ScrapePoolsRetriever interface { + ScrapePools() []string +} + +// TargetRetriever provides the list of active/dropped targets to scrape or not. +type TargetRetriever interface { + TargetsActive() map[string][]*scrape.Target + TargetsDropped() map[string][]*scrape.Target +} + +// AlertmanagerRetriever provides a list of all/dropped AlertManager URLs. +type AlertmanagerRetriever interface { + Alertmanagers() []*url.URL + DroppedAlertmanagers() []*url.URL +} + +// RulesRetriever provides a list of active rules and alerts. +type RulesRetriever interface { + RuleGroups() []*rules.Group + AlertingRules() []*rules.AlertingRule +} + +type StatsRenderer func(context.Context, *stats.Statistics, string) stats.QueryStats + +func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) stats.QueryStats { + if param != "" { + return stats.NewQueryStats(s) + } + return nil +} + +// PrometheusVersion contains build information about Prometheus. +type PrometheusVersion struct { + Version string `json:"version"` + Revision string `json:"revision"` + Branch string `json:"branch"` + BuildUser string `json:"buildUser"` + BuildDate string `json:"buildDate"` + GoVersion string `json:"goVersion"` +} + +// RuntimeInfo contains runtime information about Prometheus. +type RuntimeInfo struct { + StartTime time.Time `json:"startTime"` + CWD string `json:"CWD"` + ReloadConfigSuccess bool `json:"reloadConfigSuccess"` + LastConfigTime time.Time `json:"lastConfigTime"` + CorruptionCount int64 `json:"corruptionCount"` + GoroutineCount int `json:"goroutineCount"` + GOMAXPROCS int `json:"GOMAXPROCS"` + GOMEMLIMIT int64 `json:"GOMEMLIMIT"` + GOGC string `json:"GOGC"` + GODEBUG string `json:"GODEBUG"` + StorageRetention string `json:"storageRetention"` +} + +type response struct { + Status status `json:"status"` + Data interface{} `json:"data,omitempty"` + ErrorType errorType `json:"errorType,omitempty"` + Error string `json:"error,omitempty"` + Warnings []string `json:"warnings,omitempty"` +} + +type apiFuncResult struct { + data interface{} + err *apiError + warnings storage.Warnings + finalizer func() +} + +type apiFunc func(r *http.Request) apiFuncResult + +// TSDBAdminStats defines the tsdb interfaces used by the v1 API for admin operations as well as statistics. +type TSDBAdminStats interface { + CleanTombstones() error + Delete(mint, maxt int64, ms ...*labels.Matcher) error + Snapshot(dir string, withHead bool) error + Stats(statsByLabelName string, limit int) (*tsdb.Stats, error) + WALReplayStatus() (tsdb.WALReplayStatus, error) +} + +// QueryEngine defines the interface for the *promql.Engine, so it can be replaced, wrapped or mocked. +type QueryEngine interface { + SetQueryLogger(l promql.QueryLogger) + NewInstantQuery(ctx context.Context, q storage.Queryable, opts *promql.QueryOpts, qs string, ts time.Time) (promql.Query, error) + NewRangeQuery(ctx context.Context, q storage.Queryable, opts *promql.QueryOpts, qs string, start, end time.Time, interval time.Duration) (promql.Query, error) +} + +// API can register a set of endpoints in a router and handle +// them using the provided storage and query engine. +type API struct { + Queryable storage.SampleAndChunkQueryable + QueryEngine QueryEngine + ExemplarQueryable storage.ExemplarQueryable + + scrapePoolsRetriever func(context.Context) ScrapePoolsRetriever + targetRetriever func(context.Context) TargetRetriever + alertmanagerRetriever func(context.Context) AlertmanagerRetriever + rulesRetriever func(context.Context) RulesRetriever + now func() time.Time + config func() config.Config + flagsMap map[string]string + ready func(http.HandlerFunc) http.HandlerFunc + globalURLOptions GlobalURLOptions + + db TSDBAdminStats + dbDir string + enableAdmin bool + logger log.Logger + CORSOrigin *regexp.Regexp + buildInfo *PrometheusVersion + runtimeInfo func() (RuntimeInfo, error) + gatherer prometheus.Gatherer + isAgent bool + statsRenderer StatsRenderer + + remoteWriteHandler http.Handler + remoteReadHandler http.Handler +} + +func init() { + jsoniter.RegisterTypeEncoderFunc("promql.Series", marshalSeriesJSON, marshalSeriesJSONIsEmpty) + jsoniter.RegisterTypeEncoderFunc("promql.Sample", marshalSampleJSON, marshalSampleJSONIsEmpty) + jsoniter.RegisterTypeEncoderFunc("promql.FPoint", marshalFPointJSON, marshalPointJSONIsEmpty) + jsoniter.RegisterTypeEncoderFunc("promql.HPoint", marshalHPointJSON, marshalPointJSONIsEmpty) + jsoniter.RegisterTypeEncoderFunc("exemplar.Exemplar", marshalExemplarJSON, marshalExemplarJSONEmpty) +} + +// NewAPI returns an initialized API type. +func NewAPI( + qe QueryEngine, + q storage.SampleAndChunkQueryable, + ap storage.Appendable, + eq storage.ExemplarQueryable, + spsr func(context.Context) ScrapePoolsRetriever, + tr func(context.Context) TargetRetriever, + ar func(context.Context) AlertmanagerRetriever, + configFunc func() config.Config, + flagsMap map[string]string, + globalURLOptions GlobalURLOptions, + readyFunc func(http.HandlerFunc) http.HandlerFunc, + db TSDBAdminStats, + dbDir string, + enableAdmin bool, + logger log.Logger, + rr func(context.Context) RulesRetriever, + remoteReadSampleLimit int, + remoteReadConcurrencyLimit int, + remoteReadMaxBytesInFrame int, + isAgent bool, + corsOrigin *regexp.Regexp, + runtimeInfo func() (RuntimeInfo, error), + buildInfo *PrometheusVersion, + gatherer prometheus.Gatherer, + registerer prometheus.Registerer, + statsRenderer StatsRenderer, +) *API { + a := &API{ + QueryEngine: qe, + Queryable: q, + ExemplarQueryable: eq, + + scrapePoolsRetriever: spsr, + targetRetriever: tr, + alertmanagerRetriever: ar, + + now: time.Now, + config: configFunc, + flagsMap: flagsMap, + ready: readyFunc, + globalURLOptions: globalURLOptions, + db: db, + dbDir: dbDir, + enableAdmin: enableAdmin, + rulesRetriever: rr, + logger: logger, + CORSOrigin: corsOrigin, + runtimeInfo: runtimeInfo, + buildInfo: buildInfo, + gatherer: gatherer, + isAgent: isAgent, + statsRenderer: defaultStatsRenderer, + + remoteReadHandler: remote.NewReadHandler(logger, registerer, q, configFunc, remoteReadSampleLimit, remoteReadConcurrencyLimit, remoteReadMaxBytesInFrame), + } + + if statsRenderer != nil { + a.statsRenderer = statsRenderer + } + + if ap != nil { + a.remoteWriteHandler = remote.NewWriteHandler(logger, ap) + } + + return a +} + +func setUnavailStatusOnTSDBNotReady(r apiFuncResult) apiFuncResult { + if r.err != nil && errors.Cause(r.err.err) == tsdb.ErrNotReady { + r.err.typ = errorUnavailable + } + return r +} + +// Register the API's endpoints in the given router. +func (api *API) Register(r *route.Router) { + wrap := func(f apiFunc) http.HandlerFunc { + hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + httputil.SetCORS(w, api.CORSOrigin, r) + result := setUnavailStatusOnTSDBNotReady(f(r)) + if result.finalizer != nil { + defer result.finalizer() + } + if result.err != nil { + api.respondError(w, result.err, result.data) + return + } + + if result.data != nil { + api.respond(w, result.data, result.warnings) + return + } + w.WriteHeader(http.StatusNoContent) + }) + return api.ready(httputil.CompressionHandler{ + Handler: hf, + }.ServeHTTP) + } + + wrapAgent := func(f apiFunc) http.HandlerFunc { + return wrap(func(r *http.Request) apiFuncResult { + if api.isAgent { + return apiFuncResult{nil, &apiError{errorExec, errors.New("unavailable with Prometheus Agent")}, nil, nil} + } + return f(r) + }) + } + + r.Options("/*path", wrap(api.options)) + + r.Get("/query", wrapAgent(api.query)) + r.Post("/query", wrapAgent(api.query)) + r.Get("/query_range", wrapAgent(api.queryRange)) + r.Post("/query_range", wrapAgent(api.queryRange)) + r.Get("/query_exemplars", wrapAgent(api.queryExemplars)) + r.Post("/query_exemplars", wrapAgent(api.queryExemplars)) + + r.Get("/format_query", wrapAgent(api.formatQuery)) + r.Post("/format_query", wrapAgent(api.formatQuery)) + + r.Get("/labels", wrapAgent(api.labelNames)) + r.Post("/labels", wrapAgent(api.labelNames)) + r.Get("/label/:name/values", wrapAgent(api.labelValues)) + + r.Get("/series", wrapAgent(api.series)) + r.Post("/series", wrapAgent(api.series)) + r.Del("/series", wrapAgent(api.dropSeries)) + + r.Get("/scrape_pools", wrap(api.scrapePools)) + r.Get("/targets", wrap(api.targets)) + r.Get("/targets/metadata", wrap(api.targetMetadata)) + r.Get("/alertmanagers", wrapAgent(api.alertmanagers)) + + r.Get("/metadata", wrap(api.metricMetadata)) + + r.Get("/status/config", wrap(api.serveConfig)) + r.Get("/status/runtimeinfo", wrap(api.serveRuntimeInfo)) + r.Get("/status/buildinfo", wrap(api.serveBuildInfo)) + r.Get("/status/flags", wrap(api.serveFlags)) + r.Get("/status/tsdb", wrapAgent(api.serveTSDBStatus)) + r.Get("/status/walreplay", api.serveWALReplayStatus) + r.Post("/read", api.ready(api.remoteRead)) + r.Post("/write", api.ready(api.remoteWrite)) + + r.Get("/alerts", wrapAgent(api.alerts)) + r.Get("/rules", wrapAgent(api.rules)) + + // Admin APIs + r.Post("/admin/tsdb/delete_series", wrapAgent(api.deleteSeries)) + r.Post("/admin/tsdb/clean_tombstones", wrapAgent(api.cleanTombstones)) + r.Post("/admin/tsdb/snapshot", wrapAgent(api.snapshot)) + + r.Put("/admin/tsdb/delete_series", wrapAgent(api.deleteSeries)) + r.Put("/admin/tsdb/clean_tombstones", wrapAgent(api.cleanTombstones)) + r.Put("/admin/tsdb/snapshot", wrapAgent(api.snapshot)) +} + +type queryData struct { + ResultType parser.ValueType `json:"resultType"` + Result parser.Value `json:"result"` + Stats stats.QueryStats `json:"stats,omitempty"` +} + +func invalidParamError(err error, parameter string) apiFuncResult { + return apiFuncResult{nil, &apiError{ + errorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), + }, nil, nil} +} + +func (api *API) options(*http.Request) apiFuncResult { + return apiFuncResult{nil, nil, nil, nil} +} + +func (api *API) query(r *http.Request) (result apiFuncResult) { + ts, err := parseTimeParam(r, "time", api.now()) + if err != nil { + return invalidParamError(err, "time") + } + ctx := r.Context() + if to := r.FormValue("timeout"); to != "" { + var cancel context.CancelFunc + timeout, err := parseDuration(to) + if err != nil { + return invalidParamError(err, "timeout") + } + + ctx, cancel = context.WithTimeout(ctx, timeout) + defer cancel() + } + + opts, err := extractQueryOpts(r) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + qry, err := api.QueryEngine.NewInstantQuery(ctx, api.Queryable, opts, r.FormValue("query"), ts) + if err != nil { + return invalidParamError(err, "query") + } + + // From now on, we must only return with a finalizer in the result (to + // be called by the caller) or call qry.Close ourselves (which is + // required in the case of a panic). + defer func() { + if result.finalizer == nil { + qry.Close() + } + }() + + ctx = httputil.ContextFromRequest(ctx, r) + + res := qry.Exec(ctx) + if res.Err != nil { + return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + } + + // Optional stats field in response if parameter "stats" is not empty. + sr := api.statsRenderer + if sr == nil { + sr = defaultStatsRenderer + } + qs := sr(ctx, qry.Stats(), r.FormValue("stats")) + + return apiFuncResult{&queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + }, nil, res.Warnings, qry.Close} +} + +func (api *API) formatQuery(r *http.Request) (result apiFuncResult) { + expr, err := parser.ParseExpr(r.FormValue("query")) + if err != nil { + return invalidParamError(err, "query") + } + + return apiFuncResult{expr.Pretty(0), nil, nil, nil} +} + +func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { + opts := &promql.QueryOpts{ + EnablePerStepStats: r.FormValue("stats") == "all", + } + if strDuration := r.FormValue("lookback_delta"); strDuration != "" { + duration, err := parseDuration(strDuration) + if err != nil { + return nil, fmt.Errorf("error parsing lookback delta duration: %w", err) + } + opts.LookbackDelta = duration + } + return opts, nil +} + +func (api *API) queryRange(r *http.Request) (result apiFuncResult) { + start, err := parseTime(r.FormValue("start")) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTime(r.FormValue("end")) + if err != nil { + return invalidParamError(err, "end") + } + if end.Before(start) { + return invalidParamError(errors.New("end timestamp must not be before start time"), "end") + } + + step, err := parseDuration(r.FormValue("step")) + if err != nil { + return invalidParamError(err, "step") + } + + if step <= 0 { + return invalidParamError(errors.New("zero or negative query resolution step widths are not accepted. Try a positive integer"), "step") + } + + // For safety, limit the number of returned points per timeseries. + // This is sufficient for 60s resolution for a week or 1h resolution for a year. + if end.Sub(start)/step > 11000 { + err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + + ctx := r.Context() + if to := r.FormValue("timeout"); to != "" { + var cancel context.CancelFunc + timeout, err := parseDuration(to) + if err != nil { + return invalidParamError(err, "timeout") + } + + ctx, cancel = context.WithTimeout(ctx, timeout) + defer cancel() + } + + opts, err := extractQueryOpts(r) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + qry, err := api.QueryEngine.NewRangeQuery(ctx, api.Queryable, opts, r.FormValue("query"), start, end, step) + if err != nil { + return invalidParamError(err, "query") + } + // From now on, we must only return with a finalizer in the result (to + // be called by the caller) or call qry.Close ourselves (which is + // required in the case of a panic). + defer func() { + if result.finalizer == nil { + qry.Close() + } + }() + + ctx = httputil.ContextFromRequest(ctx, r) + + res := qry.Exec(ctx) + if res.Err != nil { + return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + } + + // Optional stats field in response if parameter "stats" is not empty. + sr := api.statsRenderer + if sr == nil { + sr = defaultStatsRenderer + } + qs := sr(ctx, qry.Stats(), r.FormValue("stats")) + + return apiFuncResult{&queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + }, nil, res.Warnings, qry.Close} +} + +func (api *API) queryExemplars(r *http.Request) apiFuncResult { + start, err := parseTimeParam(r, "start", minTime) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTimeParam(r, "end", maxTime) + if err != nil { + return invalidParamError(err, "end") + } + if end.Before(start) { + err := errors.New("end timestamp must not be before start timestamp") + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + + expr, err := parser.ParseExpr(r.FormValue("query")) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + + selectors := parser.ExtractSelectors(expr) + if len(selectors) < 1 { + return apiFuncResult{nil, nil, nil, nil} + } + + ctx := r.Context() + eq, err := api.ExemplarQueryable.ExemplarQuerier(ctx) + if err != nil { + return apiFuncResult{nil, returnAPIError(err), nil, nil} + } + + res, err := eq.Select(timestamp.FromTime(start), timestamp.FromTime(end), selectors...) + if err != nil { + return apiFuncResult{nil, returnAPIError(err), nil, nil} + } + + return apiFuncResult{res, nil, nil, nil} +} + +func returnAPIError(err error) *apiError { + if err == nil { + return nil + } + + cause := errors.Unwrap(err) + if cause == nil { + cause = err + } + + switch cause.(type) { + case promql.ErrQueryCanceled: + return &apiError{errorCanceled, err} + case promql.ErrQueryTimeout: + return &apiError{errorTimeout, err} + case promql.ErrStorage: + return &apiError{errorInternal, err} + } + + if errors.Is(err, context.Canceled) { + return &apiError{errorCanceled, err} + } + + return &apiError{errorExec, err} +} + +func (api *API) labelNames(r *http.Request) apiFuncResult { + start, err := parseTimeParam(r, "start", minTime) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTimeParam(r, "end", maxTime) + if err != nil { + return invalidParamError(err, "end") + } + + matcherSets, err := parseMatchersParam(r.Form["match[]"]) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + + q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) + if err != nil { + return apiFuncResult{nil, returnAPIError(err), nil, nil} + } + defer q.Close() + + var ( + names []string + warnings storage.Warnings + ) + if len(matcherSets) > 0 { + labelNamesSet := make(map[string]struct{}) + + for _, matchers := range matcherSets { + vals, callWarnings, err := q.LabelNames(matchers...) + if err != nil { + return apiFuncResult{nil, returnAPIError(err), warnings, nil} + } + + warnings = append(warnings, callWarnings...) + for _, val := range vals { + labelNamesSet[val] = struct{}{} + } + } + + // Convert the map to an array. + names = make([]string, 0, len(labelNamesSet)) + for key := range labelNamesSet { + names = append(names, key) + } + slices.Sort(names) + } else { + names, warnings, err = q.LabelNames() + if err != nil { + return apiFuncResult{nil, &apiError{errorExec, err}, warnings, nil} + } + } + + if names == nil { + names = []string{} + } + return apiFuncResult{names, nil, warnings, nil} +} + +func (api *API) labelValues(r *http.Request) (result apiFuncResult) { + ctx := r.Context() + name := route.Param(ctx, "name") + + if !model.LabelNameRE.MatchString(name) { + return apiFuncResult{nil, &apiError{errorBadData, errors.Errorf("invalid label name: %q", name)}, nil, nil} + } + + start, err := parseTimeParam(r, "start", minTime) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTimeParam(r, "end", maxTime) + if err != nil { + return invalidParamError(err, "end") + } + + matcherSets, err := parseMatchersParam(r.Form["match[]"]) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + + q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) + if err != nil { + return apiFuncResult{nil, &apiError{errorExec, err}, nil, nil} + } + // From now on, we must only return with a finalizer in the result (to + // be called by the caller) or call q.Close ourselves (which is required + // in the case of a panic). + defer func() { + if result.finalizer == nil { + q.Close() + } + }() + closer := func() { + q.Close() + } + + var ( + vals []string + warnings storage.Warnings + ) + if len(matcherSets) > 0 { + var callWarnings storage.Warnings + labelValuesSet := make(map[string]struct{}) + for _, matchers := range matcherSets { + vals, callWarnings, err = q.LabelValues(name, matchers...) + if err != nil { + return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} + } + warnings = append(warnings, callWarnings...) + for _, val := range vals { + labelValuesSet[val] = struct{}{} + } + } + + vals = make([]string, 0, len(labelValuesSet)) + for val := range labelValuesSet { + vals = append(vals, val) + } + } else { + vals, warnings, err = q.LabelValues(name) + if err != nil { + return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} + } + + if vals == nil { + vals = []string{} + } + } + + slices.Sort(vals) + + return apiFuncResult{vals, nil, warnings, closer} +} + +var ( + minTime = time.Unix(math.MinInt64/1000+62135596801, 0).UTC() + maxTime = time.Unix(math.MaxInt64/1000-62135596801, 999999999).UTC() + + minTimeFormatted = minTime.Format(time.RFC3339Nano) + maxTimeFormatted = maxTime.Format(time.RFC3339Nano) +) + +func (api *API) series(r *http.Request) (result apiFuncResult) { + if err := r.ParseForm(); err != nil { + return apiFuncResult{nil, &apiError{errorBadData, errors.Wrapf(err, "error parsing form values")}, nil, nil} + } + if len(r.Form["match[]"]) == 0 { + return apiFuncResult{nil, &apiError{errorBadData, errors.New("no match[] parameter provided")}, nil, nil} + } + + start, err := parseTimeParam(r, "start", minTime) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTimeParam(r, "end", maxTime) + if err != nil { + return invalidParamError(err, "end") + } + + matcherSets, err := parseMatchersParam(r.Form["match[]"]) + if err != nil { + return invalidParamError(err, "match[]") + } + + q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) + if err != nil { + return apiFuncResult{nil, returnAPIError(err), nil, nil} + } + // From now on, we must only return with a finalizer in the result (to + // be called by the caller) or call q.Close ourselves (which is required + // in the case of a panic). + defer func() { + if result.finalizer == nil { + q.Close() + } + }() + closer := func() { + q.Close() + } + + hints := &storage.SelectHints{ + Start: timestamp.FromTime(start), + End: timestamp.FromTime(end), + Func: "series", // There is no series function, this token is used for lookups that don't need samples. + } + var set storage.SeriesSet + + if len(matcherSets) > 1 { + var sets []storage.SeriesSet + for _, mset := range matcherSets { + // We need to sort this select results to merge (deduplicate) the series sets later. + s := q.Select(true, hints, mset...) + sets = append(sets, s) + } + set = storage.NewMergeSeriesSet(sets, storage.ChainedSeriesMerge) + } else { + // At this point at least one match exists. + set = q.Select(false, hints, matcherSets[0]...) + } + + metrics := []labels.Labels{} + for set.Next() { + metrics = append(metrics, set.At().Labels()) + } + + warnings := set.Warnings() + if set.Err() != nil { + return apiFuncResult{nil, returnAPIError(set.Err()), warnings, closer} + } + + return apiFuncResult{metrics, nil, warnings, closer} +} + +func (api *API) dropSeries(_ *http.Request) apiFuncResult { + return apiFuncResult{nil, &apiError{errorInternal, errors.New("not implemented")}, nil, nil} +} + +// Target has the information for one target. +type Target struct { + // Labels before any processing. + DiscoveredLabels map[string]string `json:"discoveredLabels"` + // Any labels that are added to this target and its metrics. + Labels map[string]string `json:"labels"` + + ScrapePool string `json:"scrapePool"` + ScrapeURL string `json:"scrapeUrl"` + GlobalURL string `json:"globalUrl"` + + LastError string `json:"lastError"` + LastScrape time.Time `json:"lastScrape"` + LastScrapeDuration float64 `json:"lastScrapeDuration"` + Health scrape.TargetHealth `json:"health"` + + ScrapeInterval string `json:"scrapeInterval"` + ScrapeTimeout string `json:"scrapeTimeout"` +} + +type ScrapePoolsDiscovery struct { + ScrapePools []string `json:"scrapePools"` +} + +// DroppedTarget has the information for one target that was dropped during relabelling. +type DroppedTarget struct { + // Labels before any processing. + DiscoveredLabels map[string]string `json:"discoveredLabels"` +} + +// TargetDiscovery has all the active targets. +type TargetDiscovery struct { + ActiveTargets []*Target `json:"activeTargets"` + DroppedTargets []*DroppedTarget `json:"droppedTargets"` +} + +// GlobalURLOptions contains fields used for deriving the global URL for local targets. +type GlobalURLOptions struct { + ListenAddress string + Host string + Scheme string +} + +// sanitizeSplitHostPort acts like net.SplitHostPort. +// Additionally, if there is no port in the host passed as input, we return the +// original host, making sure that IPv6 addresses are not surrounded by square +// brackets. +func sanitizeSplitHostPort(input string) (string, string, error) { + host, port, err := net.SplitHostPort(input) + if err != nil && strings.HasSuffix(err.Error(), "missing port in address") { + var errWithPort error + host, _, errWithPort = net.SplitHostPort(input + ":80") + if errWithPort == nil { + err = nil + } + } + return host, port, err +} + +func getGlobalURL(u *url.URL, opts GlobalURLOptions) (*url.URL, error) { + host, port, err := sanitizeSplitHostPort(u.Host) + if err != nil { + return u, err + } + + for _, lhr := range LocalhostRepresentations { + if host == lhr { + _, ownPort, err := net.SplitHostPort(opts.ListenAddress) + if err != nil { + return u, err + } + + if port == ownPort { + // Only in the case where the target is on localhost and its port is + // the same as the one we're listening on, we know for sure that + // we're monitoring our own process and that we need to change the + // scheme, hostname, and port to the externally reachable ones as + // well. We shouldn't need to touch the path at all, since if a + // path prefix is defined, the path under which we scrape ourselves + // should already contain the prefix. + u.Scheme = opts.Scheme + u.Host = opts.Host + } else { + // Otherwise, we only know that localhost is not reachable + // externally, so we replace only the hostname by the one in the + // external URL. It could be the wrong hostname for the service on + // this port, but it's still the best possible guess. + host, _, err := sanitizeSplitHostPort(opts.Host) + if err != nil { + return u, err + } + u.Host = host + if port != "" { + u.Host = net.JoinHostPort(u.Host, port) + } + } + break + } + } + + return u, nil +} + +func (api *API) scrapePools(r *http.Request) apiFuncResult { + names := api.scrapePoolsRetriever(r.Context()).ScrapePools() + sort.Strings(names) + res := &ScrapePoolsDiscovery{ScrapePools: names} + return apiFuncResult{data: res, err: nil, warnings: nil, finalizer: nil} +} + +func (api *API) targets(r *http.Request) apiFuncResult { + sortKeys := func(targets map[string][]*scrape.Target) ([]string, int) { + var n int + keys := make([]string, 0, len(targets)) + for k := range targets { + keys = append(keys, k) + n += len(targets[k]) + } + slices.Sort(keys) + return keys, n + } + + scrapePool := r.URL.Query().Get("scrapePool") + state := strings.ToLower(r.URL.Query().Get("state")) + showActive := state == "" || state == "any" || state == "active" + showDropped := state == "" || state == "any" || state == "dropped" + res := &TargetDiscovery{} + + if showActive { + targetsActive := api.targetRetriever(r.Context()).TargetsActive() + activeKeys, numTargets := sortKeys(targetsActive) + res.ActiveTargets = make([]*Target, 0, numTargets) + + for _, key := range activeKeys { + if scrapePool != "" && key != scrapePool { + continue + } + for _, target := range targetsActive[key] { + lastErrStr := "" + lastErr := target.LastError() + if lastErr != nil { + lastErrStr = lastErr.Error() + } + + globalURL, err := getGlobalURL(target.URL(), api.globalURLOptions) + + res.ActiveTargets = append(res.ActiveTargets, &Target{ + DiscoveredLabels: target.DiscoveredLabels().Map(), + Labels: target.Labels().Map(), + ScrapePool: key, + ScrapeURL: target.URL().String(), + GlobalURL: globalURL.String(), + LastError: func() string { + switch { + case err == nil && lastErrStr == "": + return "" + case err != nil: + return errors.Wrapf(err, lastErrStr).Error() + default: + return lastErrStr + } + }(), + LastScrape: target.LastScrape(), + LastScrapeDuration: target.LastScrapeDuration().Seconds(), + Health: target.Health(), + ScrapeInterval: target.GetValue(model.ScrapeIntervalLabel), + ScrapeTimeout: target.GetValue(model.ScrapeTimeoutLabel), + }) + } + } + } else { + res.ActiveTargets = []*Target{} + } + if showDropped { + targetsDropped := api.targetRetriever(r.Context()).TargetsDropped() + droppedKeys, numTargets := sortKeys(targetsDropped) + res.DroppedTargets = make([]*DroppedTarget, 0, numTargets) + for _, key := range droppedKeys { + if scrapePool != "" && key != scrapePool { + continue + } + for _, target := range targetsDropped[key] { + res.DroppedTargets = append(res.DroppedTargets, &DroppedTarget{ + DiscoveredLabels: target.DiscoveredLabels().Map(), + }) + } + } + } else { + res.DroppedTargets = []*DroppedTarget{} + } + return apiFuncResult{res, nil, nil, nil} +} + +func matchLabels(lset labels.Labels, matchers []*labels.Matcher) bool { + for _, m := range matchers { + if !m.Matches(lset.Get(m.Name)) { + return false + } + } + return true +} + +func (api *API) targetMetadata(r *http.Request) apiFuncResult { + limit := -1 + if s := r.FormValue("limit"); s != "" { + var err error + if limit, err = strconv.Atoi(s); err != nil { + return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a number")}, nil, nil} + } + } + + matchTarget := r.FormValue("match_target") + var matchers []*labels.Matcher + var err error + if matchTarget != "" { + matchers, err = parser.ParseMetricSelector(matchTarget) + if err != nil { + return invalidParamError(err, "match_target") + } + } + + metric := r.FormValue("metric") + res := []metricMetadata{} + for _, tt := range api.targetRetriever(r.Context()).TargetsActive() { + for _, t := range tt { + if limit >= 0 && len(res) >= limit { + break + } + // Filter targets that don't satisfy the label matchers. + if matchTarget != "" && !matchLabels(t.Labels(), matchers) { + continue + } + // If no metric is specified, get the full list for the target. + if metric == "" { + for _, md := range t.MetadataList() { + res = append(res, metricMetadata{ + Target: t.Labels(), + Metric: md.Metric, + Type: md.Type, + Help: md.Help, + Unit: md.Unit, + }) + } + continue + } + // Get metadata for the specified metric. + if md, ok := t.Metadata(metric); ok { + res = append(res, metricMetadata{ + Target: t.Labels(), + Type: md.Type, + Help: md.Help, + Unit: md.Unit, + }) + } + } + } + + return apiFuncResult{res, nil, nil, nil} +} + +type metricMetadata struct { + Target labels.Labels `json:"target"` + Metric string `json:"metric,omitempty"` + Type textparse.MetricType `json:"type"` + Help string `json:"help"` + Unit string `json:"unit"` +} + +// AlertmanagerDiscovery has all the active Alertmanagers. +type AlertmanagerDiscovery struct { + ActiveAlertmanagers []*AlertmanagerTarget `json:"activeAlertmanagers"` + DroppedAlertmanagers []*AlertmanagerTarget `json:"droppedAlertmanagers"` +} + +// AlertmanagerTarget has info on one AM. +type AlertmanagerTarget struct { + URL string `json:"url"` +} + +func (api *API) alertmanagers(r *http.Request) apiFuncResult { + urls := api.alertmanagerRetriever(r.Context()).Alertmanagers() + droppedURLS := api.alertmanagerRetriever(r.Context()).DroppedAlertmanagers() + ams := &AlertmanagerDiscovery{ActiveAlertmanagers: make([]*AlertmanagerTarget, len(urls)), DroppedAlertmanagers: make([]*AlertmanagerTarget, len(droppedURLS))} + for i, url := range urls { + ams.ActiveAlertmanagers[i] = &AlertmanagerTarget{URL: url.String()} + } + for i, url := range droppedURLS { + ams.DroppedAlertmanagers[i] = &AlertmanagerTarget{URL: url.String()} + } + return apiFuncResult{ams, nil, nil, nil} +} + +// AlertDiscovery has info for all active alerts. +type AlertDiscovery struct { + Alerts []*Alert `json:"alerts"` +} + +// Alert has info for an alert. +type Alert struct { + Labels labels.Labels `json:"labels"` + Annotations labels.Labels `json:"annotations"` + State string `json:"state"` + ActiveAt *time.Time `json:"activeAt,omitempty"` + KeepFiringSince *time.Time `json:"keepFiringSince,omitempty"` + Value string `json:"value"` +} + +func (api *API) alerts(r *http.Request) apiFuncResult { + alertingRules := api.rulesRetriever(r.Context()).AlertingRules() + alerts := []*Alert{} + + for _, alertingRule := range alertingRules { + alerts = append( + alerts, + rulesAlertsToAPIAlerts(alertingRule.ActiveAlerts())..., + ) + } + + res := &AlertDiscovery{Alerts: alerts} + + return apiFuncResult{res, nil, nil, nil} +} + +func rulesAlertsToAPIAlerts(rulesAlerts []*rules.Alert) []*Alert { + apiAlerts := make([]*Alert, len(rulesAlerts)) + for i, ruleAlert := range rulesAlerts { + apiAlerts[i] = &Alert{ + Labels: ruleAlert.Labels, + Annotations: ruleAlert.Annotations, + State: ruleAlert.State.String(), + ActiveAt: &ruleAlert.ActiveAt, + Value: strconv.FormatFloat(ruleAlert.Value, 'e', -1, 64), + } + if !ruleAlert.KeepFiringSince.IsZero() { + apiAlerts[i].KeepFiringSince = &ruleAlert.KeepFiringSince + } + } + + return apiAlerts +} + +type metadata struct { + Type textparse.MetricType `json:"type"` + Help string `json:"help"` + Unit string `json:"unit"` +} + +func (api *API) metricMetadata(r *http.Request) apiFuncResult { + metrics := map[string]map[metadata]struct{}{} + + limit := -1 + if s := r.FormValue("limit"); s != "" { + var err error + if limit, err = strconv.Atoi(s); err != nil { + return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a number")}, nil, nil} + } + } + + metric := r.FormValue("metric") + for _, tt := range api.targetRetriever(r.Context()).TargetsActive() { + for _, t := range tt { + + if metric == "" { + for _, mm := range t.MetadataList() { + m := metadata{Type: mm.Type, Help: mm.Help, Unit: mm.Unit} + ms, ok := metrics[mm.Metric] + + if !ok { + ms = map[metadata]struct{}{} + metrics[mm.Metric] = ms + } + ms[m] = struct{}{} + } + continue + } + + if md, ok := t.Metadata(metric); ok { + m := metadata{Type: md.Type, Help: md.Help, Unit: md.Unit} + ms, ok := metrics[md.Metric] + + if !ok { + ms = map[metadata]struct{}{} + metrics[md.Metric] = ms + } + ms[m] = struct{}{} + } + } + } + + // Put the elements from the pseudo-set into a slice for marshaling. + res := map[string][]metadata{} + for name, set := range metrics { + if limit >= 0 && len(res) >= limit { + break + } + + s := []metadata{} + for metadata := range set { + s = append(s, metadata) + } + res[name] = s + } + + return apiFuncResult{res, nil, nil, nil} +} + +// RuleDiscovery has info for all rules +type RuleDiscovery struct { + RuleGroups []*RuleGroup `json:"groups"` +} + +// RuleGroup has info for rules which are part of a group +type RuleGroup struct { + Name string `json:"name"` + File string `json:"file"` + // In order to preserve rule ordering, while exposing type (alerting or recording) + // specific properties, both alerting and recording rules are exposed in the + // same array. + Rules []Rule `json:"rules"` + Interval float64 `json:"interval"` + Limit int `json:"limit"` + EvaluationTime float64 `json:"evaluationTime"` + LastEvaluation time.Time `json:"lastEvaluation"` +} + +type Rule interface{} + +type AlertingRule struct { + // State can be "pending", "firing", "inactive". + State string `json:"state"` + Name string `json:"name"` + Query string `json:"query"` + Duration float64 `json:"duration"` + KeepFiringFor float64 `json:"keepFiringFor"` + Labels labels.Labels `json:"labels"` + Annotations labels.Labels `json:"annotations"` + Alerts []*Alert `json:"alerts"` + Health rules.RuleHealth `json:"health"` + LastError string `json:"lastError,omitempty"` + EvaluationTime float64 `json:"evaluationTime"` + LastEvaluation time.Time `json:"lastEvaluation"` + // Type of an alertingRule is always "alerting". + Type string `json:"type"` +} + +type RecordingRule struct { + Name string `json:"name"` + Query string `json:"query"` + Labels labels.Labels `json:"labels,omitempty"` + Health rules.RuleHealth `json:"health"` + LastError string `json:"lastError,omitempty"` + EvaluationTime float64 `json:"evaluationTime"` + LastEvaluation time.Time `json:"lastEvaluation"` + // Type of a recordingRule is always "recording". + Type string `json:"type"` +} + +func (api *API) rules(r *http.Request) apiFuncResult { + if err := r.ParseForm(); err != nil { + return apiFuncResult{nil, &apiError{errorBadData, errors.Wrapf(err, "error parsing form values")}, nil, nil} + } + + queryFormToSet := func(values []string) map[string]struct{} { + set := make(map[string]struct{}, len(values)) + for _, v := range values { + set[v] = struct{}{} + } + return set + } + + rnSet := queryFormToSet(r.Form["rule_name[]"]) + rgSet := queryFormToSet(r.Form["rule_group[]"]) + fSet := queryFormToSet(r.Form["file[]"]) + + ruleGroups := api.rulesRetriever(r.Context()).RuleGroups() + res := &RuleDiscovery{RuleGroups: make([]*RuleGroup, 0, len(ruleGroups))} + typ := strings.ToLower(r.URL.Query().Get("type")) + + if typ != "" && typ != "alert" && typ != "record" { + return invalidParamError(errors.Errorf("not supported value %q", typ), "type") + } + + returnAlerts := typ == "" || typ == "alert" + returnRecording := typ == "" || typ == "record" + + rgs := make([]*RuleGroup, 0, len(ruleGroups)) + for _, grp := range ruleGroups { + if len(rgSet) > 0 { + if _, ok := rgSet[grp.Name()]; !ok { + continue + } + } + + if len(fSet) > 0 { + if _, ok := fSet[grp.File()]; !ok { + continue + } + } + + apiRuleGroup := &RuleGroup{ + Name: grp.Name(), + File: grp.File(), + Interval: grp.Interval().Seconds(), + Limit: grp.Limit(), + Rules: []Rule{}, + EvaluationTime: grp.GetEvaluationTime().Seconds(), + LastEvaluation: grp.GetLastEvaluation(), + } + for _, rr := range grp.Rules() { + var enrichedRule Rule + + if len(rnSet) > 0 { + if _, ok := rnSet[rr.Name()]; !ok { + continue + } + } + + lastError := "" + if rr.LastError() != nil { + lastError = rr.LastError().Error() + } + switch rule := rr.(type) { + case *rules.AlertingRule: + if !returnAlerts { + break + } + enrichedRule = AlertingRule{ + State: rule.State().String(), + Name: rule.Name(), + Query: rule.Query().String(), + Duration: rule.HoldDuration().Seconds(), + KeepFiringFor: rule.KeepFiringFor().Seconds(), + Labels: rule.Labels(), + Annotations: rule.Annotations(), + Alerts: rulesAlertsToAPIAlerts(rule.ActiveAlerts()), + Health: rule.Health(), + LastError: lastError, + EvaluationTime: rule.GetEvaluationDuration().Seconds(), + LastEvaluation: rule.GetEvaluationTimestamp(), + Type: "alerting", + } + case *rules.RecordingRule: + if !returnRecording { + break + } + enrichedRule = RecordingRule{ + Name: rule.Name(), + Query: rule.Query().String(), + Labels: rule.Labels(), + Health: rule.Health(), + LastError: lastError, + EvaluationTime: rule.GetEvaluationDuration().Seconds(), + LastEvaluation: rule.GetEvaluationTimestamp(), + Type: "recording", + } + default: + err := errors.Errorf("failed to assert type of rule '%v'", rule.Name()) + return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} + } + + if enrichedRule != nil { + apiRuleGroup.Rules = append(apiRuleGroup.Rules, enrichedRule) + } + } + + // If the rule group response has no rules, skip it - this means we filtered all the rules of this group. + if len(apiRuleGroup.Rules) > 0 { + rgs = append(rgs, apiRuleGroup) + } + } + res.RuleGroups = rgs + return apiFuncResult{res, nil, nil, nil} +} + +type prometheusConfig struct { + YAML string `json:"yaml"` +} + +func (api *API) serveRuntimeInfo(_ *http.Request) apiFuncResult { + status, err := api.runtimeInfo() + if err != nil { + return apiFuncResult{status, &apiError{errorInternal, err}, nil, nil} + } + return apiFuncResult{status, nil, nil, nil} +} + +func (api *API) serveBuildInfo(_ *http.Request) apiFuncResult { + return apiFuncResult{api.buildInfo, nil, nil, nil} +} + +func (api *API) serveConfig(_ *http.Request) apiFuncResult { + cfg := &prometheusConfig{ + YAML: api.config().String(), + } + return apiFuncResult{cfg, nil, nil, nil} +} + +func (api *API) serveFlags(_ *http.Request) apiFuncResult { + return apiFuncResult{api.flagsMap, nil, nil, nil} +} + +// TSDBStat holds the information about individual cardinality. +type TSDBStat struct { + Name string `json:"name"` + Value uint64 `json:"value"` +} + +// HeadStats has information about the TSDB head. +type HeadStats struct { + NumSeries uint64 `json:"numSeries"` + NumLabelPairs int `json:"numLabelPairs"` + ChunkCount int64 `json:"chunkCount"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` +} + +// TSDBStatus has information of cardinality statistics from postings. +type TSDBStatus struct { + HeadStats HeadStats `json:"headStats"` + SeriesCountByMetricName []TSDBStat `json:"seriesCountByMetricName"` + LabelValueCountByLabelName []TSDBStat `json:"labelValueCountByLabelName"` + MemoryInBytesByLabelName []TSDBStat `json:"memoryInBytesByLabelName"` + SeriesCountByLabelValuePair []TSDBStat `json:"seriesCountByLabelValuePair"` +} + +// TSDBStatsFromIndexStats converts a index.Stat slice to a TSDBStat slice. +func TSDBStatsFromIndexStats(stats []index.Stat) []TSDBStat { + result := make([]TSDBStat, 0, len(stats)) + for _, item := range stats { + item := TSDBStat{Name: item.Name, Value: item.Count} + result = append(result, item) + } + return result +} + +func (api *API) serveTSDBStatus(r *http.Request) apiFuncResult { + limit := 10 + if s := r.FormValue("limit"); s != "" { + var err error + if limit, err = strconv.Atoi(s); err != nil || limit < 1 { + return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a positive number")}, nil, nil} + } + } + s, err := api.db.Stats(labels.MetricName, limit) + if err != nil { + return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} + } + metrics, err := api.gatherer.Gather() + if err != nil { + return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("error gathering runtime status: %s", err)}, nil, nil} + } + chunkCount := int64(math.NaN()) + for _, mF := range metrics { + if *mF.Name == "prometheus_tsdb_head_chunks" { + m := *mF.Metric[0] + if m.Gauge != nil { + chunkCount = int64(m.Gauge.GetValue()) + break + } + } + } + return apiFuncResult{TSDBStatus{ + HeadStats: HeadStats{ + NumSeries: s.NumSeries, + ChunkCount: chunkCount, + MinTime: s.MinTime, + MaxTime: s.MaxTime, + NumLabelPairs: s.IndexPostingStats.NumLabelPairs, + }, + SeriesCountByMetricName: TSDBStatsFromIndexStats(s.IndexPostingStats.CardinalityMetricsStats), + LabelValueCountByLabelName: TSDBStatsFromIndexStats(s.IndexPostingStats.CardinalityLabelStats), + MemoryInBytesByLabelName: TSDBStatsFromIndexStats(s.IndexPostingStats.LabelValueStats), + SeriesCountByLabelValuePair: TSDBStatsFromIndexStats(s.IndexPostingStats.LabelValuePairsStats), + }, nil, nil, nil} +} + +type walReplayStatus struct { + Min int `json:"min"` + Max int `json:"max"` + Current int `json:"current"` +} + +func (api *API) serveWALReplayStatus(w http.ResponseWriter, r *http.Request) { + httputil.SetCORS(w, api.CORSOrigin, r) + status, err := api.db.WALReplayStatus() + if err != nil { + api.respondError(w, &apiError{errorInternal, err}, nil) + } + api.respond(w, walReplayStatus{ + Min: status.Min, + Max: status.Max, + Current: status.Current, + }, nil) +} + +func (api *API) remoteRead(w http.ResponseWriter, r *http.Request) { + // This is only really for tests - this will never be nil IRL. + if api.remoteReadHandler != nil { + api.remoteReadHandler.ServeHTTP(w, r) + } else { + http.Error(w, "not found", http.StatusNotFound) + } +} + +func (api *API) remoteWrite(w http.ResponseWriter, r *http.Request) { + if api.remoteWriteHandler != nil { + api.remoteWriteHandler.ServeHTTP(w, r) + } else { + http.Error(w, "remote write receiver needs to be enabled with --web.enable-remote-write-receiver", http.StatusNotFound) + } +} + +func (api *API) deleteSeries(r *http.Request) apiFuncResult { + if !api.enableAdmin { + return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} + } + if err := r.ParseForm(); err != nil { + return apiFuncResult{nil, &apiError{errorBadData, errors.Wrap(err, "error parsing form values")}, nil, nil} + } + if len(r.Form["match[]"]) == 0 { + return apiFuncResult{nil, &apiError{errorBadData, errors.New("no match[] parameter provided")}, nil, nil} + } + + start, err := parseTimeParam(r, "start", minTime) + if err != nil { + return invalidParamError(err, "start") + } + end, err := parseTimeParam(r, "end", maxTime) + if err != nil { + return invalidParamError(err, "end") + } + + for _, s := range r.Form["match[]"] { + matchers, err := parser.ParseMetricSelector(s) + if err != nil { + return invalidParamError(err, "match[]") + } + if err := api.db.Delete(timestamp.FromTime(start), timestamp.FromTime(end), matchers...); err != nil { + return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} + } + } + + return apiFuncResult{nil, nil, nil, nil} +} + +func (api *API) snapshot(r *http.Request) apiFuncResult { + if !api.enableAdmin { + return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} + } + var ( + skipHead bool + err error + ) + if r.FormValue("skip_head") != "" { + skipHead, err = strconv.ParseBool(r.FormValue("skip_head")) + if err != nil { + return invalidParamError(errors.Wrapf(err, "unable to parse boolean"), "skip_head") + } + } + + var ( + snapdir = filepath.Join(api.dbDir, "snapshots") + name = fmt.Sprintf("%s-%016x", + time.Now().UTC().Format("20060102T150405Z0700"), + rand.Int63()) + dir = filepath.Join(snapdir, name) + ) + if err := os.MkdirAll(dir, 0o777); err != nil { + return apiFuncResult{nil, &apiError{errorInternal, errors.Wrap(err, "create snapshot directory")}, nil, nil} + } + if err := api.db.Snapshot(dir, !skipHead); err != nil { + return apiFuncResult{nil, &apiError{errorInternal, errors.Wrap(err, "create snapshot")}, nil, nil} + } + + return apiFuncResult{struct { + Name string `json:"name"` + }{name}, nil, nil, nil} +} + +func (api *API) cleanTombstones(*http.Request) apiFuncResult { + if !api.enableAdmin { + return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} + } + if err := api.db.CleanTombstones(); err != nil { + return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} + } + + return apiFuncResult{nil, nil, nil, nil} +} + +func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { + statusMessage := statusSuccess + var warningStrings []string + for _, warning := range warnings { + warningStrings = append(warningStrings, warning.Error()) + } + var prometheusResponse = queryrange.PrometheusResponse{ + Status: string(statusMessage), + Data: extractPrometheusData(data), + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, + } + + b, err := proto.Marshal(&prometheusResponse) + if err != nil { + level.Error(api.logger).Log("msg", "error marshaling json response", "err", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + w.Header().Set("Content-Type", "application/protobuf") + w.WriteHeader(http.StatusOK) + if n, err := w.Write(b); err != nil { + level.Error(api.logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) + } +} + +func (api *API) respondError(w http.ResponseWriter, apiErr *apiError, data interface{}) { + json := jsoniter.ConfigCompatibleWithStandardLibrary + b, err := json.Marshal(&response{ + Status: statusError, + ErrorType: apiErr.typ, + Error: apiErr.err.Error(), + Data: data, + }) + if err != nil { + level.Error(api.logger).Log("msg", "error marshaling json response", "err", err) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + var code int + switch apiErr.typ { + case errorBadData: + code = http.StatusBadRequest + case errorExec: + code = http.StatusUnprocessableEntity + case errorCanceled: + code = statusClientClosedConnection + case errorTimeout: + code = http.StatusServiceUnavailable + case errorInternal: + code = http.StatusInternalServerError + case errorNotFound: + code = http.StatusNotFound + default: + code = http.StatusInternalServerError + } + + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(code) + if n, err := w.Write(b); err != nil { + level.Error(api.logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) + } +} + +func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) (time.Time, error) { + val := r.FormValue(paramName) + if val == "" { + return defaultValue, nil + } + result, err := parseTime(val) + if err != nil { + return time.Time{}, errors.Wrapf(err, "Invalid time value for '%s'", paramName) + } + return result, nil +} + +func parseTime(s string) (time.Time, error) { + if t, err := strconv.ParseFloat(s, 64); err == nil { + s, ns := math.Modf(t) + ns = math.Round(ns*1000) / 1000 + return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil + } + if t, err := time.Parse(time.RFC3339Nano, s); err == nil { + return t, nil + } + + // Stdlib's time parser can only handle 4 digit years. As a workaround until + // that is fixed we want to at least support our own boundary times. + // Context: https://github.com/prometheus/client_golang/issues/614 + // Upstream issue: https://github.com/golang/go/issues/20555 + switch s { + case minTimeFormatted: + return minTime, nil + case maxTimeFormatted: + return maxTime, nil + } + return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) +} + +func parseDuration(s string) (time.Duration, error) { + if d, err := strconv.ParseFloat(s, 64); err == nil { + ts := d * float64(time.Second) + if ts > float64(math.MaxInt64) || ts < float64(math.MinInt64) { + return 0, errors.Errorf("cannot parse %q to a valid duration. It overflows int64", s) + } + return time.Duration(ts), nil + } + if d, err := model.ParseDuration(s); err == nil { + return time.Duration(d), nil + } + return 0, errors.Errorf("cannot parse %q to a valid duration", s) +} + +func parseMatchersParam(matchers []string) ([][]*labels.Matcher, error) { + var matcherSets [][]*labels.Matcher + for _, s := range matchers { + matchers, err := parser.ParseMetricSelector(s) + if err != nil { + return nil, err + } + matcherSets = append(matcherSets, matchers) + } + +OUTER: + for _, ms := range matcherSets { + for _, lm := range ms { + if lm != nil && !lm.Matches("") { + continue OUTER + } + } + return nil, errors.New("match[] must contain at least one non-empty matcher") + } + return matcherSets, nil +} + +// marshalSeriesJSON writes something like the following: +// +// { +// "metric" : { +// "__name__" : "up", +// "job" : "prometheus", +// "instance" : "localhost:9090" +// }, +// "values": [ +// [ 1435781451.781, "1" ], +// < more values> +// ], +// "histograms": [ +// [ 1435781451.781, { < histogram, see jsonutil.MarshalHistogram > } ], +// < more histograms > +// ], +// }, +func marshalSeriesJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { + s := *((*promql.Series)(ptr)) + stream.WriteObjectStart() + stream.WriteObjectField(`metric`) + m, err := s.Metric.MarshalJSON() + if err != nil { + stream.Error = err + return + } + stream.SetBuffer(append(stream.Buffer(), m...)) + + for i, p := range s.Floats { + stream.WriteMore() + if i == 0 { + stream.WriteObjectField(`values`) + stream.WriteArrayStart() + } + marshalFPointJSON(unsafe.Pointer(&p), stream) + } + if len(s.Floats) > 0 { + stream.WriteArrayEnd() + } + for i, p := range s.Histograms { + stream.WriteMore() + if i == 0 { + stream.WriteObjectField(`histograms`) + stream.WriteArrayStart() + } + marshalHPointJSON(unsafe.Pointer(&p), stream) + } + if len(s.Histograms) > 0 { + stream.WriteArrayEnd() + } + stream.WriteObjectEnd() +} + +func marshalSeriesJSONIsEmpty(unsafe.Pointer) bool { + return false +} + +// marshalSampleJSON writes something like the following for normal value samples: +// +// { +// "metric" : { +// "__name__" : "up", +// "job" : "prometheus", +// "instance" : "localhost:9090" +// }, +// "value": [ 1435781451.781, "1.234" ] +// }, +// +// For histogram samples, it writes something like this: +// +// { +// "metric" : { +// "__name__" : "up", +// "job" : "prometheus", +// "instance" : "localhost:9090" +// }, +// "histogram": [ 1435781451.781, { < histogram, see jsonutil.MarshalHistogram > } ] +// }, +func marshalSampleJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { + s := *((*promql.Sample)(ptr)) + stream.WriteObjectStart() + stream.WriteObjectField(`metric`) + m, err := s.Metric.MarshalJSON() + if err != nil { + stream.Error = err + return + } + stream.SetBuffer(append(stream.Buffer(), m...)) + stream.WriteMore() + if s.H == nil { + stream.WriteObjectField(`value`) + } else { + stream.WriteObjectField(`histogram`) + } + stream.WriteArrayStart() + jsonutil.MarshalTimestamp(s.T, stream) + stream.WriteMore() + if s.H == nil { + jsonutil.MarshalFloat(s.F, stream) + } else { + jsonutil.MarshalHistogram(s.H, stream) + } + stream.WriteArrayEnd() + stream.WriteObjectEnd() +} + +func marshalSampleJSONIsEmpty(unsafe.Pointer) bool { + return false +} + +// marshalFPointJSON writes `[ts, "1.234"]`. +func marshalFPointJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { + p := *((*promql.FPoint)(ptr)) + stream.WriteArrayStart() + jsonutil.MarshalTimestamp(p.T, stream) + stream.WriteMore() + jsonutil.MarshalFloat(p.F, stream) + stream.WriteArrayEnd() +} + +// marshalHPointJSON writes `[ts, { < histogram, see jsonutil.MarshalHistogram > } ]`. +func marshalHPointJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { + p := *((*promql.HPoint)(ptr)) + stream.WriteArrayStart() + jsonutil.MarshalTimestamp(p.T, stream) + stream.WriteMore() + jsonutil.MarshalHistogram(p.H, stream) + stream.WriteArrayEnd() +} + +func marshalPointJSONIsEmpty(unsafe.Pointer) bool { + return false +} + +// marshalExemplarJSON writes. +// +// { +// labels: , +// value: "", +// timestamp: +// } +func marshalExemplarJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { + p := *((*exemplar.Exemplar)(ptr)) + stream.WriteObjectStart() + + // "labels" key. + stream.WriteObjectField(`labels`) + lbls, err := p.Labels.MarshalJSON() + if err != nil { + stream.Error = err + return + } + stream.SetBuffer(append(stream.Buffer(), lbls...)) + + // "value" key. + stream.WriteMore() + stream.WriteObjectField(`value`) + jsonutil.MarshalFloat(p.Value, stream) + + // "timestamp" key. + stream.WriteMore() + stream.WriteObjectField(`timestamp`) + jsonutil.MarshalTimestamp(p.Ts, stream) + + stream.WriteObjectEnd() +} + +func marshalExemplarJSONEmpty(unsafe.Pointer) bool { + return false +} + +func extractPrometheusData(in interface{}) queryrange.PrometheusData { + data, _ := in.(*queryData) + var resultType string = "" + if data != nil { + resultType = string(data.ResultType) + + if resultType == "matrix" { + samplesLen := len(data.Result.(promql.Matrix)[0].Floats) + samples := make([]cortexpb.Sample, samplesLen) + for i := 0; i < samplesLen; i++ { + samples[i] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[0].Floats[i].F, + TimestampMs: data.Result.(promql.Matrix)[0].Floats[i].T, + } + } + + labelsLen := len(data.Result.(promql.Matrix)[0].Metric) + labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for i := 0; i < labelsLen; i++ { + labels[i] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[0].Metric[i].Name, + Value: data.Result.(promql.Matrix)[0].Metric[i].Value, + } + } + sampleStream := tripperware.SampleStream{Labels: labels, Samples: samples} + + return queryrange.PrometheusData{ + ResultType: resultType, + Result: []tripperware.SampleStream{sampleStream}, + Stats: nil, + } + } + } + return queryrange.PrometheusData{ + ResultType: resultType, + Result: []tripperware.SampleStream{}, + Stats: nil, + } +} diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 82eb0c12fc6..08958f3a66b 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "fmt" + "github.com/gogo/protobuf/proto" "io" "net/http" "net/url" @@ -276,7 +277,7 @@ func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ t log.LogFields(otlog.Int("bytes", len(buf))) var resp PrometheusResponse - if err := json.Unmarshal(buf, &resp); err != nil { + if err := proto.Unmarshal(buf, &resp); err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } From c54f8f55fbf94479e51786d8144ec9d5c44aa9e1 Mon Sep 17 00:00:00 2001 From: afayek Date: Tue, 11 Jul 2023 22:57:34 -0700 Subject: [PATCH 02/38] add stats to PrometheusResponse created from query data Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 112 ++++++++++++++++++++++----------- 1 file changed, 77 insertions(+), 35 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 5c55a770344..d4428e0ea9f 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -16,6 +16,7 @@ package handler import ( "context" "fmt" + "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" "github.com/gogo/protobuf/proto" "math" "math/rand" @@ -553,11 +554,11 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return apiFuncResult{&queryData{ + return apiFuncResult{createPrometheusResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }, nil, res.Warnings, qry.Close} + }), nil, res.Warnings, qry.Close} } func (api *API) queryExemplars(r *http.Request) apiFuncResult { @@ -1638,17 +1639,27 @@ func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storag for _, warning := range warnings { warningStrings = append(warningStrings, warning.Error()) } - var prometheusResponse = queryrange.PrometheusResponse{ - Status: string(statusMessage), - Data: extractPrometheusData(data), - ErrorType: "", - Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, + var b []byte + var err error + switch data.(type) { + case queryrange.PrometheusResponse: + prometheusResponse, _ := data.(queryrange.PrometheusResponse) + prometheusResponse.Status = string(statusMessage) + b, err = proto.Marshal(&prometheusResponse) + case instantquery.PrometheusInstantQueryResponse: + prometheusInstantQueryResponse, _ := data.(instantquery.PrometheusInstantQueryResponse) + prometheusInstantQueryResponse.Status = string(statusMessage) + b, err = proto.Marshal(&prometheusInstantQueryResponse) + default: + json := jsoniter.ConfigCompatibleWithStandardLibrary + b, err = json.Marshal(&response{ + Status: statusMessage, + Data: data, + Warnings: warningStrings, + }) } - - b, err := proto.Marshal(&prometheusResponse) if err != nil { - level.Error(api.logger).Log("msg", "error marshaling json response", "err", err) + level.Error(api.logger).Log("msg", "error marshaling protobuf response", "err", err) http.Error(w, err.Error(), http.StatusInternalServerError) return } @@ -1941,42 +1952,73 @@ func marshalExemplarJSONEmpty(unsafe.Pointer) bool { return false } -func extractPrometheusData(in interface{}) queryrange.PrometheusData { - data, _ := in.(*queryData) - var resultType string = "" +func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResponse { + data, _ := queryResult.(*queryData) if data != nil { - resultType = string(data.ResultType) + sampleStreamsLen := len(data.Result.(promql.Matrix)) + sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) - if resultType == "matrix" { - samplesLen := len(data.Result.(promql.Matrix)[0].Floats) + for i := 0; i < sampleStreamsLen; i++ { + samplesLen := len(data.Result.(promql.Matrix)[i].Floats) samples := make([]cortexpb.Sample, samplesLen) - for i := 0; i < samplesLen; i++ { - samples[i] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[0].Floats[i].F, - TimestampMs: data.Result.(promql.Matrix)[0].Floats[i].T, + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, } } - labelsLen := len(data.Result.(promql.Matrix)[0].Metric) + labelsLen := len(data.Result.(promql.Matrix)[i].Metric) labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for i := 0; i < labelsLen; i++ { - labels[i] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Matrix)[0].Metric[i].Name, - Value: data.Result.(promql.Matrix)[0].Metric[i].Value, + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[i].Metric[j].Name, + Value: data.Result.(promql.Matrix)[i].Metric[j].Value, } } - sampleStream := tripperware.SampleStream{Labels: labels, Samples: samples} - return queryrange.PrometheusData{ - ResultType: resultType, - Result: []tripperware.SampleStream{sampleStream}, - Stats: nil, + sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} + } + + var stats *tripperware.PrometheusResponseStats + + if data.Stats != nil { + builtin := data.Stats.Builtin() + queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) + queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) + for i := 0; i < queryableSamplesStatsPerStepLen; i++ { + queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, + TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, + } + } + + statSamples := tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, + TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, } + + stats = &tripperware.PrometheusResponseStats{Samples: &statSamples} + } + + return queryrange.PrometheusResponse{ + Status: "", + Data: queryrange.PrometheusData{ + ResultType: string(data.ResultType), + Result: sampleStreams, + Stats: stats, + }, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, } } - return queryrange.PrometheusData{ - ResultType: resultType, - Result: []tripperware.SampleStream{}, - Stats: nil, + + return queryrange.PrometheusResponse{ + Status: "", + Data: queryrange.PrometheusData{}, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, } } From cabd489cbbb1dd52d7c3d8f5a7e56b7366845729 Mon Sep 17 00:00:00 2001 From: afayek Date: Wed, 12 Jul 2023 09:50:17 -0700 Subject: [PATCH 03/38] add conversion from query data to PrometheusInstantQueryResponse in querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 124 ++++++++++++++++-- .../tripperware/instantquery/instant_query.go | 3 +- 2 files changed, 115 insertions(+), 12 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index d4428e0ea9f..a31c544ec29 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -452,11 +452,11 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return apiFuncResult{&queryData{ + return apiFuncResult{createPrometheusInstantQueryResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }, nil, res.Warnings, qry.Close} + }), nil, res.Warnings, qry.Close} } func (api *API) formatQuery(r *http.Request) (result apiFuncResult) { @@ -1959,15 +1959,6 @@ func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResp sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) for i := 0; i < sampleStreamsLen; i++ { - samplesLen := len(data.Result.(promql.Matrix)[i].Floats) - samples := make([]cortexpb.Sample, samplesLen) - for j := 0; j < samplesLen; j++ { - samples[j] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[i].Floats[j].F, - TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, - } - } - labelsLen := len(data.Result.(promql.Matrix)[i].Metric) labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) for j := 0; j < labelsLen; j++ { @@ -1977,6 +1968,14 @@ func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResp } } + samplesLen := len(data.Result.(promql.Matrix)[i].Floats) + samples := make([]cortexpb.Sample, samplesLen) + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, + } + } sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} } @@ -2022,3 +2021,106 @@ func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResp Headers: []*tripperware.PrometheusResponseHeader{}, } } + +func createPrometheusInstantQueryResponse(queryResult interface{}) instantquery.PrometheusInstantQueryResponse { + data, _ := queryResult.(*queryData) + if data != nil { + var instantQueryResult instantquery.PrometheusInstantQueryResult + + switch string(data.ResultType) { + case "matrix": + matrixSamplesLen := len(data.Result.(promql.Matrix)) + matrixSamples := make([]tripperware.SampleStream, matrixSamplesLen) + + for i := 0; i < matrixSamplesLen; i++ { + labelsLen := len(data.Result.(promql.Matrix)[i].Metric) + labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[i].Metric[j].Name, + Value: data.Result.(promql.Matrix)[i].Metric[j].Value, + } + } + + samplesLen := len(data.Result.(promql.Matrix)[i].Floats) + samples := make([]cortexpb.Sample, samplesLen) + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, + } + } + + matrixSamples[i] = tripperware.SampleStream{Labels: labels, Samples: samples} + } + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{Matrix: &instantquery.Matrix{SampleStreams: matrixSamples}} + + case "vector": + vectorSamplesLen := len(data.Result.(promql.Vector)) + vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) + + for i := 0; i < vectorSamplesLen; i++ { + labelsLen := len(data.Result.(promql.Vector)[i].Metric) + labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Vector)[i].Metric[j].Name, + Value: data.Result.(promql.Vector)[i].Metric[j].Value, + } + } + + vectorSamples[i] = &instantquery.Sample{Labels: labels, + Sample: cortexpb.Sample{ + TimestampMs: data.Result.(promql.Vector)[i].T, + Value: data.Result.(promql.Vector)[i].F, + }, + } + } + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{Vector: &instantquery.Vector{Samples: vectorSamples}} + + default: + // TODO: add scalar and string instant query responses + } + + var stats *tripperware.PrometheusResponseStats + + if data.Stats != nil { + builtin := data.Stats.Builtin() + queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) + queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) + for i := 0; i < queryableSamplesStatsPerStepLen; i++ { + queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, + TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, + } + } + + statSamples := tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, + TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + } + + stats = &tripperware.PrometheusResponseStats{Samples: &statSamples} + } + + return instantquery.PrometheusInstantQueryResponse{ + Status: "", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: string(data.ResultType), + Result: instantQueryResult, + Stats: stats, + }, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, + } + } + + return instantquery.PrometheusInstantQueryResponse{ + Status: "", + Data: instantquery.PrometheusInstantQueryData{}, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, + } +} diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 135670ec377..e5f0ccef573 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "fmt" + "github.com/gogo/protobuf/proto" "io" "net/http" "net/url" @@ -172,7 +173,7 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ } var resp PrometheusInstantQueryResponse - if err := json.Unmarshal(buf, &resp); err != nil { + if err := proto.Unmarshal(buf, &resp); err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } From b84307584b2688bf01dab40a2ff24ae1506f8324 Mon Sep 17 00:00:00 2001 From: afayek Date: Thu, 13 Jul 2023 15:15:36 -0700 Subject: [PATCH 04/38] remove endpoints not used by query api and add it as a handler for /query and /query_range Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 61 +- pkg/querier/handler/handler.go | 1489 +------------------------------- 2 files changed, 60 insertions(+), 1490 deletions(-) diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 4cc1b7c5d2a..d988e45b530 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -8,7 +8,7 @@ import ( "path" "sync" - v1 "github.com/cortexproject/cortex/pkg/querier/handler" + qapi "github.com/cortexproject/cortex/pkg/querier/handler" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gorilla/mux" @@ -22,6 +22,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" + v1api "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" @@ -193,28 +194,28 @@ func NewQuerierHandler( Help: "Current number of inflight requests to the querier.", }, []string{"method", "route"}) - api := v1.NewAPI( + v1api := v1api.NewAPI( engine, querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. nil, // No remote write support. exemplarQueryable, - func(ctx context.Context) v1.ScrapePoolsRetriever { return nil }, - func(context.Context) v1.TargetRetriever { return &querier.DummyTargetRetriever{} }, - func(context.Context) v1.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, + func(ctx context.Context) v1api.ScrapePoolsRetriever { return nil }, + func(context.Context) v1api.TargetRetriever { return &querier.DummyTargetRetriever{} }, + func(context.Context) v1api.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, func() config.Config { return config.Config{} }, map[string]string{}, // TODO: include configuration flags - v1.GlobalURLOptions{}, + v1api.GlobalURLOptions{}, func(f http.HandlerFunc) http.HandlerFunc { return f }, nil, // Only needed for admin APIs. "", // This is for snapshots, which is disabled when admin APIs are disabled. Hence empty. false, // Disable admin APIs. logger, - func(context.Context) v1.RulesRetriever { return &querier.DummyRulesRetriever{} }, + func(context.Context) v1api.RulesRetriever { return &querier.DummyRulesRetriever{} }, 0, 0, 0, // Remote read samples and concurrency limit. false, regexp.MustCompile(".*"), - func() (v1.RuntimeInfo, error) { return v1.RuntimeInfo{}, errors.New("not implemented") }, - &v1.PrometheusVersion{ + func() (v1api.RuntimeInfo, error) { return v1api.RuntimeInfo{}, errors.New("not implemented") }, + &v1api.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, @@ -230,6 +231,26 @@ func NewQuerierHandler( false, ) + queryapi := qapi.NewAPI( + engine, + querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. + func(f http.HandlerFunc) http.HandlerFunc { return f }, + logger, + false, + regexp.MustCompile(".*"), + nil, + ) + + queryapi := qapi.NewAPI( + engine, + querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. + func(f http.HandlerFunc) http.HandlerFunc { return f }, + logger, + false, + regexp.MustCompile(".*"), + nil, + ) + router := mux.NewRouter() // Use a separate metric for the querier in order to differentiate requests from the query-frontend when @@ -248,18 +269,22 @@ func NewQuerierHandler( legacyPrefix := path.Join(cfg.ServerPrefix, cfg.LegacyHTTPPrefix) promRouter := route.New().WithPrefix(path.Join(prefix, "/api/v1")) - api.Register(promRouter) + v1api.Register(promRouter) + queryPromRouter := route.New().WithPrefix(path.Join(prefix, "/api/v1")) + queryapi.Register(queryPromRouter) legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) - api.Register(legacyPromRouter) + v1api.Register(legacyPromRouter) + queryLegacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) + queryapi.Register(queryLegacyPromRouter) // TODO(gotjosh): This custom handler is temporary until we're able to vendor the changes in: // https://github.com/prometheus/prometheus/pull/7125/files router.Path(path.Join(prefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(prefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(prefix, "/api/v1/read")).Methods("POST").Handler(promRouter) - router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(promRouter) - router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(promRouter) + router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(queryPromRouter) + router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(queryPromRouter) router.Path(path.Join(prefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/labels")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(promRouter) @@ -271,8 +296,8 @@ func NewQuerierHandler( router.Path(path.Join(legacyPrefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Methods("POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(queryLegacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(queryLegacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/labels")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(legacyPromRouter) @@ -293,14 +318,14 @@ type buildInfoHandler struct { } type buildInfoResponse struct { - Status string `json:"status"` - Data *v1.PrometheusVersion `json:"data"` + Status string `json:"status"` + Data *v1api.PrometheusVersion `json:"data"` } func (h *buildInfoHandler) ServeHTTP(writer http.ResponseWriter, _ *http.Request) { infoResponse := buildInfoResponse{ Status: "success", - Data: &v1.PrometheusVersion{ + Data: &v1api.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index a31c544ec29..127675a575d 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -1,65 +1,31 @@ -// Copyright 2016 The Prometheus Authors -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - package handler import ( "context" "fmt" "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" - "github.com/gogo/protobuf/proto" - "math" - "math/rand" - "net" - "net/http" - "net/url" - "os" - "path/filepath" - "sort" - "strconv" - "strings" - "time" - "unsafe" - "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/gogo/protobuf/proto" "github.com/grafana/regexp" jsoniter "github.com/json-iterator/go" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/common/route" - "golang.org/x/exp/slices" + "math" + "net/http" + "strconv" + "time" "github.com/cortexproject/cortex/pkg/cortexpb" github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" - "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/textparse" - "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" - "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/util/httputil" - "github.com/prometheus/prometheus/util/jsonutil" "github.com/prometheus/prometheus/util/stats" ) @@ -87,8 +53,6 @@ const ( errorNotFound errorType = "not_found" ) -var LocalhostRepresentations = []string{"127.0.0.1", "localhost", "::1"} - type apiError struct { typ errorType err error @@ -98,29 +62,6 @@ func (e *apiError) Error() string { return fmt.Sprintf("%s: %s", e.typ, e.err) } -// ScrapePoolsRetriever provide the list of all scrape pools. -type ScrapePoolsRetriever interface { - ScrapePools() []string -} - -// TargetRetriever provides the list of active/dropped targets to scrape or not. -type TargetRetriever interface { - TargetsActive() map[string][]*scrape.Target - TargetsDropped() map[string][]*scrape.Target -} - -// AlertmanagerRetriever provides a list of all/dropped AlertManager URLs. -type AlertmanagerRetriever interface { - Alertmanagers() []*url.URL - DroppedAlertmanagers() []*url.URL -} - -// RulesRetriever provides a list of active rules and alerts. -type RulesRetriever interface { - RuleGroups() []*rules.Group - AlertingRules() []*rules.AlertingRule -} - type StatsRenderer func(context.Context, *stats.Statistics, string) stats.QueryStats func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) stats.QueryStats { @@ -130,31 +71,6 @@ func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) return nil } -// PrometheusVersion contains build information about Prometheus. -type PrometheusVersion struct { - Version string `json:"version"` - Revision string `json:"revision"` - Branch string `json:"branch"` - BuildUser string `json:"buildUser"` - BuildDate string `json:"buildDate"` - GoVersion string `json:"goVersion"` -} - -// RuntimeInfo contains runtime information about Prometheus. -type RuntimeInfo struct { - StartTime time.Time `json:"startTime"` - CWD string `json:"CWD"` - ReloadConfigSuccess bool `json:"reloadConfigSuccess"` - LastConfigTime time.Time `json:"lastConfigTime"` - CorruptionCount int64 `json:"corruptionCount"` - GoroutineCount int `json:"goroutineCount"` - GOMAXPROCS int `json:"GOMAXPROCS"` - GOMEMLIMIT int64 `json:"GOMEMLIMIT"` - GOGC string `json:"GOGC"` - GODEBUG string `json:"GODEBUG"` - StorageRetention string `json:"storageRetention"` -} - type response struct { Status status `json:"status"` Data interface{} `json:"data,omitempty"` @@ -172,15 +88,6 @@ type apiFuncResult struct { type apiFunc func(r *http.Request) apiFuncResult -// TSDBAdminStats defines the tsdb interfaces used by the v1 API for admin operations as well as statistics. -type TSDBAdminStats interface { - CleanTombstones() error - Delete(mint, maxt int64, ms ...*labels.Matcher) error - Snapshot(dir string, withHead bool) error - Stats(statsByLabelName string, limit int) (*tsdb.Stats, error) - WALReplayStatus() (tsdb.WALReplayStatus, error) -} - // QueryEngine defines the interface for the *promql.Engine, so it can be replaced, wrapped or mocked. type QueryEngine interface { SetQueryLogger(l promql.QueryLogger) @@ -191,109 +98,41 @@ type QueryEngine interface { // API can register a set of endpoints in a router and handle // them using the provided storage and query engine. type API struct { - Queryable storage.SampleAndChunkQueryable - QueryEngine QueryEngine - ExemplarQueryable storage.ExemplarQueryable - - scrapePoolsRetriever func(context.Context) ScrapePoolsRetriever - targetRetriever func(context.Context) TargetRetriever - alertmanagerRetriever func(context.Context) AlertmanagerRetriever - rulesRetriever func(context.Context) RulesRetriever - now func() time.Time - config func() config.Config - flagsMap map[string]string - ready func(http.HandlerFunc) http.HandlerFunc - globalURLOptions GlobalURLOptions - - db TSDBAdminStats - dbDir string - enableAdmin bool + Queryable storage.SampleAndChunkQueryable + QueryEngine QueryEngine + now func() time.Time + ready func(http.HandlerFunc) http.HandlerFunc logger log.Logger CORSOrigin *regexp.Regexp - buildInfo *PrometheusVersion - runtimeInfo func() (RuntimeInfo, error) - gatherer prometheus.Gatherer isAgent bool statsRenderer StatsRenderer - - remoteWriteHandler http.Handler - remoteReadHandler http.Handler -} - -func init() { - jsoniter.RegisterTypeEncoderFunc("promql.Series", marshalSeriesJSON, marshalSeriesJSONIsEmpty) - jsoniter.RegisterTypeEncoderFunc("promql.Sample", marshalSampleJSON, marshalSampleJSONIsEmpty) - jsoniter.RegisterTypeEncoderFunc("promql.FPoint", marshalFPointJSON, marshalPointJSONIsEmpty) - jsoniter.RegisterTypeEncoderFunc("promql.HPoint", marshalHPointJSON, marshalPointJSONIsEmpty) - jsoniter.RegisterTypeEncoderFunc("exemplar.Exemplar", marshalExemplarJSON, marshalExemplarJSONEmpty) } // NewAPI returns an initialized API type. func NewAPI( qe QueryEngine, q storage.SampleAndChunkQueryable, - ap storage.Appendable, - eq storage.ExemplarQueryable, - spsr func(context.Context) ScrapePoolsRetriever, - tr func(context.Context) TargetRetriever, - ar func(context.Context) AlertmanagerRetriever, - configFunc func() config.Config, - flagsMap map[string]string, - globalURLOptions GlobalURLOptions, readyFunc func(http.HandlerFunc) http.HandlerFunc, - db TSDBAdminStats, - dbDir string, - enableAdmin bool, logger log.Logger, - rr func(context.Context) RulesRetriever, - remoteReadSampleLimit int, - remoteReadConcurrencyLimit int, - remoteReadMaxBytesInFrame int, isAgent bool, corsOrigin *regexp.Regexp, - runtimeInfo func() (RuntimeInfo, error), - buildInfo *PrometheusVersion, - gatherer prometheus.Gatherer, - registerer prometheus.Registerer, statsRenderer StatsRenderer, ) *API { a := &API{ - QueryEngine: qe, - Queryable: q, - ExemplarQueryable: eq, - - scrapePoolsRetriever: spsr, - targetRetriever: tr, - alertmanagerRetriever: ar, - - now: time.Now, - config: configFunc, - flagsMap: flagsMap, - ready: readyFunc, - globalURLOptions: globalURLOptions, - db: db, - dbDir: dbDir, - enableAdmin: enableAdmin, - rulesRetriever: rr, - logger: logger, - CORSOrigin: corsOrigin, - runtimeInfo: runtimeInfo, - buildInfo: buildInfo, - gatherer: gatherer, - isAgent: isAgent, - statsRenderer: defaultStatsRenderer, - - remoteReadHandler: remote.NewReadHandler(logger, registerer, q, configFunc, remoteReadSampleLimit, remoteReadConcurrencyLimit, remoteReadMaxBytesInFrame), + QueryEngine: qe, + Queryable: q, + now: time.Now, + ready: readyFunc, + logger: logger, + CORSOrigin: corsOrigin, + isAgent: isAgent, + statsRenderer: defaultStatsRenderer, } if statsRenderer != nil { a.statsRenderer = statsRenderer } - if ap != nil { - a.remoteWriteHandler = remote.NewWriteHandler(logger, ap) - } - return a } @@ -338,53 +177,10 @@ func (api *API) Register(r *route.Router) { }) } - r.Options("/*path", wrap(api.options)) - r.Get("/query", wrapAgent(api.query)) r.Post("/query", wrapAgent(api.query)) r.Get("/query_range", wrapAgent(api.queryRange)) r.Post("/query_range", wrapAgent(api.queryRange)) - r.Get("/query_exemplars", wrapAgent(api.queryExemplars)) - r.Post("/query_exemplars", wrapAgent(api.queryExemplars)) - - r.Get("/format_query", wrapAgent(api.formatQuery)) - r.Post("/format_query", wrapAgent(api.formatQuery)) - - r.Get("/labels", wrapAgent(api.labelNames)) - r.Post("/labels", wrapAgent(api.labelNames)) - r.Get("/label/:name/values", wrapAgent(api.labelValues)) - - r.Get("/series", wrapAgent(api.series)) - r.Post("/series", wrapAgent(api.series)) - r.Del("/series", wrapAgent(api.dropSeries)) - - r.Get("/scrape_pools", wrap(api.scrapePools)) - r.Get("/targets", wrap(api.targets)) - r.Get("/targets/metadata", wrap(api.targetMetadata)) - r.Get("/alertmanagers", wrapAgent(api.alertmanagers)) - - r.Get("/metadata", wrap(api.metricMetadata)) - - r.Get("/status/config", wrap(api.serveConfig)) - r.Get("/status/runtimeinfo", wrap(api.serveRuntimeInfo)) - r.Get("/status/buildinfo", wrap(api.serveBuildInfo)) - r.Get("/status/flags", wrap(api.serveFlags)) - r.Get("/status/tsdb", wrapAgent(api.serveTSDBStatus)) - r.Get("/status/walreplay", api.serveWALReplayStatus) - r.Post("/read", api.ready(api.remoteRead)) - r.Post("/write", api.ready(api.remoteWrite)) - - r.Get("/alerts", wrapAgent(api.alerts)) - r.Get("/rules", wrapAgent(api.rules)) - - // Admin APIs - r.Post("/admin/tsdb/delete_series", wrapAgent(api.deleteSeries)) - r.Post("/admin/tsdb/clean_tombstones", wrapAgent(api.cleanTombstones)) - r.Post("/admin/tsdb/snapshot", wrapAgent(api.snapshot)) - - r.Put("/admin/tsdb/delete_series", wrapAgent(api.deleteSeries)) - r.Put("/admin/tsdb/clean_tombstones", wrapAgent(api.cleanTombstones)) - r.Put("/admin/tsdb/snapshot", wrapAgent(api.snapshot)) } type queryData struct { @@ -399,10 +195,6 @@ func invalidParamError(err error, parameter string) apiFuncResult { }, nil, nil} } -func (api *API) options(*http.Request) apiFuncResult { - return apiFuncResult{nil, nil, nil, nil} -} - func (api *API) query(r *http.Request) (result apiFuncResult) { ts, err := parseTimeParam(r, "time", api.now()) if err != nil { @@ -459,15 +251,6 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { }), nil, res.Warnings, qry.Close} } -func (api *API) formatQuery(r *http.Request) (result apiFuncResult) { - expr, err := parser.ParseExpr(r.FormValue("query")) - if err != nil { - return invalidParamError(err, "query") - } - - return apiFuncResult{expr.Pretty(0), nil, nil, nil} -} - func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { opts := &promql.QueryOpts{ EnablePerStepStats: r.FormValue("stats") == "all", @@ -561,44 +344,6 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { }), nil, res.Warnings, qry.Close} } -func (api *API) queryExemplars(r *http.Request) apiFuncResult { - start, err := parseTimeParam(r, "start", minTime) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTimeParam(r, "end", maxTime) - if err != nil { - return invalidParamError(err, "end") - } - if end.Before(start) { - err := errors.New("end timestamp must not be before start timestamp") - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} - } - - expr, err := parser.ParseExpr(r.FormValue("query")) - if err != nil { - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} - } - - selectors := parser.ExtractSelectors(expr) - if len(selectors) < 1 { - return apiFuncResult{nil, nil, nil, nil} - } - - ctx := r.Context() - eq, err := api.ExemplarQueryable.ExemplarQuerier(ctx) - if err != nil { - return apiFuncResult{nil, returnAPIError(err), nil, nil} - } - - res, err := eq.Select(timestamp.FromTime(start), timestamp.FromTime(end), selectors...) - if err != nil { - return apiFuncResult{nil, returnAPIError(err), nil, nil} - } - - return apiFuncResult{res, nil, nil, nil} -} - func returnAPIError(err error) *apiError { if err == nil { return nil @@ -625,141 +370,6 @@ func returnAPIError(err error) *apiError { return &apiError{errorExec, err} } -func (api *API) labelNames(r *http.Request) apiFuncResult { - start, err := parseTimeParam(r, "start", minTime) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTimeParam(r, "end", maxTime) - if err != nil { - return invalidParamError(err, "end") - } - - matcherSets, err := parseMatchersParam(r.Form["match[]"]) - if err != nil { - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} - } - - q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) - if err != nil { - return apiFuncResult{nil, returnAPIError(err), nil, nil} - } - defer q.Close() - - var ( - names []string - warnings storage.Warnings - ) - if len(matcherSets) > 0 { - labelNamesSet := make(map[string]struct{}) - - for _, matchers := range matcherSets { - vals, callWarnings, err := q.LabelNames(matchers...) - if err != nil { - return apiFuncResult{nil, returnAPIError(err), warnings, nil} - } - - warnings = append(warnings, callWarnings...) - for _, val := range vals { - labelNamesSet[val] = struct{}{} - } - } - - // Convert the map to an array. - names = make([]string, 0, len(labelNamesSet)) - for key := range labelNamesSet { - names = append(names, key) - } - slices.Sort(names) - } else { - names, warnings, err = q.LabelNames() - if err != nil { - return apiFuncResult{nil, &apiError{errorExec, err}, warnings, nil} - } - } - - if names == nil { - names = []string{} - } - return apiFuncResult{names, nil, warnings, nil} -} - -func (api *API) labelValues(r *http.Request) (result apiFuncResult) { - ctx := r.Context() - name := route.Param(ctx, "name") - - if !model.LabelNameRE.MatchString(name) { - return apiFuncResult{nil, &apiError{errorBadData, errors.Errorf("invalid label name: %q", name)}, nil, nil} - } - - start, err := parseTimeParam(r, "start", minTime) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTimeParam(r, "end", maxTime) - if err != nil { - return invalidParamError(err, "end") - } - - matcherSets, err := parseMatchersParam(r.Form["match[]"]) - if err != nil { - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} - } - - q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) - if err != nil { - return apiFuncResult{nil, &apiError{errorExec, err}, nil, nil} - } - // From now on, we must only return with a finalizer in the result (to - // be called by the caller) or call q.Close ourselves (which is required - // in the case of a panic). - defer func() { - if result.finalizer == nil { - q.Close() - } - }() - closer := func() { - q.Close() - } - - var ( - vals []string - warnings storage.Warnings - ) - if len(matcherSets) > 0 { - var callWarnings storage.Warnings - labelValuesSet := make(map[string]struct{}) - for _, matchers := range matcherSets { - vals, callWarnings, err = q.LabelValues(name, matchers...) - if err != nil { - return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} - } - warnings = append(warnings, callWarnings...) - for _, val := range vals { - labelValuesSet[val] = struct{}{} - } - } - - vals = make([]string, 0, len(labelValuesSet)) - for val := range labelValuesSet { - vals = append(vals, val) - } - } else { - vals, warnings, err = q.LabelValues(name) - if err != nil { - return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} - } - - if vals == nil { - vals = []string{} - } - } - - slices.Sort(vals) - - return apiFuncResult{vals, nil, warnings, closer} -} - var ( minTime = time.Unix(math.MinInt64/1000+62135596801, 0).UTC() maxTime = time.Unix(math.MaxInt64/1000-62135596801, 999999999).UTC() @@ -768,871 +378,6 @@ var ( maxTimeFormatted = maxTime.Format(time.RFC3339Nano) ) -func (api *API) series(r *http.Request) (result apiFuncResult) { - if err := r.ParseForm(); err != nil { - return apiFuncResult{nil, &apiError{errorBadData, errors.Wrapf(err, "error parsing form values")}, nil, nil} - } - if len(r.Form["match[]"]) == 0 { - return apiFuncResult{nil, &apiError{errorBadData, errors.New("no match[] parameter provided")}, nil, nil} - } - - start, err := parseTimeParam(r, "start", minTime) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTimeParam(r, "end", maxTime) - if err != nil { - return invalidParamError(err, "end") - } - - matcherSets, err := parseMatchersParam(r.Form["match[]"]) - if err != nil { - return invalidParamError(err, "match[]") - } - - q, err := api.Queryable.Querier(r.Context(), timestamp.FromTime(start), timestamp.FromTime(end)) - if err != nil { - return apiFuncResult{nil, returnAPIError(err), nil, nil} - } - // From now on, we must only return with a finalizer in the result (to - // be called by the caller) or call q.Close ourselves (which is required - // in the case of a panic). - defer func() { - if result.finalizer == nil { - q.Close() - } - }() - closer := func() { - q.Close() - } - - hints := &storage.SelectHints{ - Start: timestamp.FromTime(start), - End: timestamp.FromTime(end), - Func: "series", // There is no series function, this token is used for lookups that don't need samples. - } - var set storage.SeriesSet - - if len(matcherSets) > 1 { - var sets []storage.SeriesSet - for _, mset := range matcherSets { - // We need to sort this select results to merge (deduplicate) the series sets later. - s := q.Select(true, hints, mset...) - sets = append(sets, s) - } - set = storage.NewMergeSeriesSet(sets, storage.ChainedSeriesMerge) - } else { - // At this point at least one match exists. - set = q.Select(false, hints, matcherSets[0]...) - } - - metrics := []labels.Labels{} - for set.Next() { - metrics = append(metrics, set.At().Labels()) - } - - warnings := set.Warnings() - if set.Err() != nil { - return apiFuncResult{nil, returnAPIError(set.Err()), warnings, closer} - } - - return apiFuncResult{metrics, nil, warnings, closer} -} - -func (api *API) dropSeries(_ *http.Request) apiFuncResult { - return apiFuncResult{nil, &apiError{errorInternal, errors.New("not implemented")}, nil, nil} -} - -// Target has the information for one target. -type Target struct { - // Labels before any processing. - DiscoveredLabels map[string]string `json:"discoveredLabels"` - // Any labels that are added to this target and its metrics. - Labels map[string]string `json:"labels"` - - ScrapePool string `json:"scrapePool"` - ScrapeURL string `json:"scrapeUrl"` - GlobalURL string `json:"globalUrl"` - - LastError string `json:"lastError"` - LastScrape time.Time `json:"lastScrape"` - LastScrapeDuration float64 `json:"lastScrapeDuration"` - Health scrape.TargetHealth `json:"health"` - - ScrapeInterval string `json:"scrapeInterval"` - ScrapeTimeout string `json:"scrapeTimeout"` -} - -type ScrapePoolsDiscovery struct { - ScrapePools []string `json:"scrapePools"` -} - -// DroppedTarget has the information for one target that was dropped during relabelling. -type DroppedTarget struct { - // Labels before any processing. - DiscoveredLabels map[string]string `json:"discoveredLabels"` -} - -// TargetDiscovery has all the active targets. -type TargetDiscovery struct { - ActiveTargets []*Target `json:"activeTargets"` - DroppedTargets []*DroppedTarget `json:"droppedTargets"` -} - -// GlobalURLOptions contains fields used for deriving the global URL for local targets. -type GlobalURLOptions struct { - ListenAddress string - Host string - Scheme string -} - -// sanitizeSplitHostPort acts like net.SplitHostPort. -// Additionally, if there is no port in the host passed as input, we return the -// original host, making sure that IPv6 addresses are not surrounded by square -// brackets. -func sanitizeSplitHostPort(input string) (string, string, error) { - host, port, err := net.SplitHostPort(input) - if err != nil && strings.HasSuffix(err.Error(), "missing port in address") { - var errWithPort error - host, _, errWithPort = net.SplitHostPort(input + ":80") - if errWithPort == nil { - err = nil - } - } - return host, port, err -} - -func getGlobalURL(u *url.URL, opts GlobalURLOptions) (*url.URL, error) { - host, port, err := sanitizeSplitHostPort(u.Host) - if err != nil { - return u, err - } - - for _, lhr := range LocalhostRepresentations { - if host == lhr { - _, ownPort, err := net.SplitHostPort(opts.ListenAddress) - if err != nil { - return u, err - } - - if port == ownPort { - // Only in the case where the target is on localhost and its port is - // the same as the one we're listening on, we know for sure that - // we're monitoring our own process and that we need to change the - // scheme, hostname, and port to the externally reachable ones as - // well. We shouldn't need to touch the path at all, since if a - // path prefix is defined, the path under which we scrape ourselves - // should already contain the prefix. - u.Scheme = opts.Scheme - u.Host = opts.Host - } else { - // Otherwise, we only know that localhost is not reachable - // externally, so we replace only the hostname by the one in the - // external URL. It could be the wrong hostname for the service on - // this port, but it's still the best possible guess. - host, _, err := sanitizeSplitHostPort(opts.Host) - if err != nil { - return u, err - } - u.Host = host - if port != "" { - u.Host = net.JoinHostPort(u.Host, port) - } - } - break - } - } - - return u, nil -} - -func (api *API) scrapePools(r *http.Request) apiFuncResult { - names := api.scrapePoolsRetriever(r.Context()).ScrapePools() - sort.Strings(names) - res := &ScrapePoolsDiscovery{ScrapePools: names} - return apiFuncResult{data: res, err: nil, warnings: nil, finalizer: nil} -} - -func (api *API) targets(r *http.Request) apiFuncResult { - sortKeys := func(targets map[string][]*scrape.Target) ([]string, int) { - var n int - keys := make([]string, 0, len(targets)) - for k := range targets { - keys = append(keys, k) - n += len(targets[k]) - } - slices.Sort(keys) - return keys, n - } - - scrapePool := r.URL.Query().Get("scrapePool") - state := strings.ToLower(r.URL.Query().Get("state")) - showActive := state == "" || state == "any" || state == "active" - showDropped := state == "" || state == "any" || state == "dropped" - res := &TargetDiscovery{} - - if showActive { - targetsActive := api.targetRetriever(r.Context()).TargetsActive() - activeKeys, numTargets := sortKeys(targetsActive) - res.ActiveTargets = make([]*Target, 0, numTargets) - - for _, key := range activeKeys { - if scrapePool != "" && key != scrapePool { - continue - } - for _, target := range targetsActive[key] { - lastErrStr := "" - lastErr := target.LastError() - if lastErr != nil { - lastErrStr = lastErr.Error() - } - - globalURL, err := getGlobalURL(target.URL(), api.globalURLOptions) - - res.ActiveTargets = append(res.ActiveTargets, &Target{ - DiscoveredLabels: target.DiscoveredLabels().Map(), - Labels: target.Labels().Map(), - ScrapePool: key, - ScrapeURL: target.URL().String(), - GlobalURL: globalURL.String(), - LastError: func() string { - switch { - case err == nil && lastErrStr == "": - return "" - case err != nil: - return errors.Wrapf(err, lastErrStr).Error() - default: - return lastErrStr - } - }(), - LastScrape: target.LastScrape(), - LastScrapeDuration: target.LastScrapeDuration().Seconds(), - Health: target.Health(), - ScrapeInterval: target.GetValue(model.ScrapeIntervalLabel), - ScrapeTimeout: target.GetValue(model.ScrapeTimeoutLabel), - }) - } - } - } else { - res.ActiveTargets = []*Target{} - } - if showDropped { - targetsDropped := api.targetRetriever(r.Context()).TargetsDropped() - droppedKeys, numTargets := sortKeys(targetsDropped) - res.DroppedTargets = make([]*DroppedTarget, 0, numTargets) - for _, key := range droppedKeys { - if scrapePool != "" && key != scrapePool { - continue - } - for _, target := range targetsDropped[key] { - res.DroppedTargets = append(res.DroppedTargets, &DroppedTarget{ - DiscoveredLabels: target.DiscoveredLabels().Map(), - }) - } - } - } else { - res.DroppedTargets = []*DroppedTarget{} - } - return apiFuncResult{res, nil, nil, nil} -} - -func matchLabels(lset labels.Labels, matchers []*labels.Matcher) bool { - for _, m := range matchers { - if !m.Matches(lset.Get(m.Name)) { - return false - } - } - return true -} - -func (api *API) targetMetadata(r *http.Request) apiFuncResult { - limit := -1 - if s := r.FormValue("limit"); s != "" { - var err error - if limit, err = strconv.Atoi(s); err != nil { - return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a number")}, nil, nil} - } - } - - matchTarget := r.FormValue("match_target") - var matchers []*labels.Matcher - var err error - if matchTarget != "" { - matchers, err = parser.ParseMetricSelector(matchTarget) - if err != nil { - return invalidParamError(err, "match_target") - } - } - - metric := r.FormValue("metric") - res := []metricMetadata{} - for _, tt := range api.targetRetriever(r.Context()).TargetsActive() { - for _, t := range tt { - if limit >= 0 && len(res) >= limit { - break - } - // Filter targets that don't satisfy the label matchers. - if matchTarget != "" && !matchLabels(t.Labels(), matchers) { - continue - } - // If no metric is specified, get the full list for the target. - if metric == "" { - for _, md := range t.MetadataList() { - res = append(res, metricMetadata{ - Target: t.Labels(), - Metric: md.Metric, - Type: md.Type, - Help: md.Help, - Unit: md.Unit, - }) - } - continue - } - // Get metadata for the specified metric. - if md, ok := t.Metadata(metric); ok { - res = append(res, metricMetadata{ - Target: t.Labels(), - Type: md.Type, - Help: md.Help, - Unit: md.Unit, - }) - } - } - } - - return apiFuncResult{res, nil, nil, nil} -} - -type metricMetadata struct { - Target labels.Labels `json:"target"` - Metric string `json:"metric,omitempty"` - Type textparse.MetricType `json:"type"` - Help string `json:"help"` - Unit string `json:"unit"` -} - -// AlertmanagerDiscovery has all the active Alertmanagers. -type AlertmanagerDiscovery struct { - ActiveAlertmanagers []*AlertmanagerTarget `json:"activeAlertmanagers"` - DroppedAlertmanagers []*AlertmanagerTarget `json:"droppedAlertmanagers"` -} - -// AlertmanagerTarget has info on one AM. -type AlertmanagerTarget struct { - URL string `json:"url"` -} - -func (api *API) alertmanagers(r *http.Request) apiFuncResult { - urls := api.alertmanagerRetriever(r.Context()).Alertmanagers() - droppedURLS := api.alertmanagerRetriever(r.Context()).DroppedAlertmanagers() - ams := &AlertmanagerDiscovery{ActiveAlertmanagers: make([]*AlertmanagerTarget, len(urls)), DroppedAlertmanagers: make([]*AlertmanagerTarget, len(droppedURLS))} - for i, url := range urls { - ams.ActiveAlertmanagers[i] = &AlertmanagerTarget{URL: url.String()} - } - for i, url := range droppedURLS { - ams.DroppedAlertmanagers[i] = &AlertmanagerTarget{URL: url.String()} - } - return apiFuncResult{ams, nil, nil, nil} -} - -// AlertDiscovery has info for all active alerts. -type AlertDiscovery struct { - Alerts []*Alert `json:"alerts"` -} - -// Alert has info for an alert. -type Alert struct { - Labels labels.Labels `json:"labels"` - Annotations labels.Labels `json:"annotations"` - State string `json:"state"` - ActiveAt *time.Time `json:"activeAt,omitempty"` - KeepFiringSince *time.Time `json:"keepFiringSince,omitempty"` - Value string `json:"value"` -} - -func (api *API) alerts(r *http.Request) apiFuncResult { - alertingRules := api.rulesRetriever(r.Context()).AlertingRules() - alerts := []*Alert{} - - for _, alertingRule := range alertingRules { - alerts = append( - alerts, - rulesAlertsToAPIAlerts(alertingRule.ActiveAlerts())..., - ) - } - - res := &AlertDiscovery{Alerts: alerts} - - return apiFuncResult{res, nil, nil, nil} -} - -func rulesAlertsToAPIAlerts(rulesAlerts []*rules.Alert) []*Alert { - apiAlerts := make([]*Alert, len(rulesAlerts)) - for i, ruleAlert := range rulesAlerts { - apiAlerts[i] = &Alert{ - Labels: ruleAlert.Labels, - Annotations: ruleAlert.Annotations, - State: ruleAlert.State.String(), - ActiveAt: &ruleAlert.ActiveAt, - Value: strconv.FormatFloat(ruleAlert.Value, 'e', -1, 64), - } - if !ruleAlert.KeepFiringSince.IsZero() { - apiAlerts[i].KeepFiringSince = &ruleAlert.KeepFiringSince - } - } - - return apiAlerts -} - -type metadata struct { - Type textparse.MetricType `json:"type"` - Help string `json:"help"` - Unit string `json:"unit"` -} - -func (api *API) metricMetadata(r *http.Request) apiFuncResult { - metrics := map[string]map[metadata]struct{}{} - - limit := -1 - if s := r.FormValue("limit"); s != "" { - var err error - if limit, err = strconv.Atoi(s); err != nil { - return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a number")}, nil, nil} - } - } - - metric := r.FormValue("metric") - for _, tt := range api.targetRetriever(r.Context()).TargetsActive() { - for _, t := range tt { - - if metric == "" { - for _, mm := range t.MetadataList() { - m := metadata{Type: mm.Type, Help: mm.Help, Unit: mm.Unit} - ms, ok := metrics[mm.Metric] - - if !ok { - ms = map[metadata]struct{}{} - metrics[mm.Metric] = ms - } - ms[m] = struct{}{} - } - continue - } - - if md, ok := t.Metadata(metric); ok { - m := metadata{Type: md.Type, Help: md.Help, Unit: md.Unit} - ms, ok := metrics[md.Metric] - - if !ok { - ms = map[metadata]struct{}{} - metrics[md.Metric] = ms - } - ms[m] = struct{}{} - } - } - } - - // Put the elements from the pseudo-set into a slice for marshaling. - res := map[string][]metadata{} - for name, set := range metrics { - if limit >= 0 && len(res) >= limit { - break - } - - s := []metadata{} - for metadata := range set { - s = append(s, metadata) - } - res[name] = s - } - - return apiFuncResult{res, nil, nil, nil} -} - -// RuleDiscovery has info for all rules -type RuleDiscovery struct { - RuleGroups []*RuleGroup `json:"groups"` -} - -// RuleGroup has info for rules which are part of a group -type RuleGroup struct { - Name string `json:"name"` - File string `json:"file"` - // In order to preserve rule ordering, while exposing type (alerting or recording) - // specific properties, both alerting and recording rules are exposed in the - // same array. - Rules []Rule `json:"rules"` - Interval float64 `json:"interval"` - Limit int `json:"limit"` - EvaluationTime float64 `json:"evaluationTime"` - LastEvaluation time.Time `json:"lastEvaluation"` -} - -type Rule interface{} - -type AlertingRule struct { - // State can be "pending", "firing", "inactive". - State string `json:"state"` - Name string `json:"name"` - Query string `json:"query"` - Duration float64 `json:"duration"` - KeepFiringFor float64 `json:"keepFiringFor"` - Labels labels.Labels `json:"labels"` - Annotations labels.Labels `json:"annotations"` - Alerts []*Alert `json:"alerts"` - Health rules.RuleHealth `json:"health"` - LastError string `json:"lastError,omitempty"` - EvaluationTime float64 `json:"evaluationTime"` - LastEvaluation time.Time `json:"lastEvaluation"` - // Type of an alertingRule is always "alerting". - Type string `json:"type"` -} - -type RecordingRule struct { - Name string `json:"name"` - Query string `json:"query"` - Labels labels.Labels `json:"labels,omitempty"` - Health rules.RuleHealth `json:"health"` - LastError string `json:"lastError,omitempty"` - EvaluationTime float64 `json:"evaluationTime"` - LastEvaluation time.Time `json:"lastEvaluation"` - // Type of a recordingRule is always "recording". - Type string `json:"type"` -} - -func (api *API) rules(r *http.Request) apiFuncResult { - if err := r.ParseForm(); err != nil { - return apiFuncResult{nil, &apiError{errorBadData, errors.Wrapf(err, "error parsing form values")}, nil, nil} - } - - queryFormToSet := func(values []string) map[string]struct{} { - set := make(map[string]struct{}, len(values)) - for _, v := range values { - set[v] = struct{}{} - } - return set - } - - rnSet := queryFormToSet(r.Form["rule_name[]"]) - rgSet := queryFormToSet(r.Form["rule_group[]"]) - fSet := queryFormToSet(r.Form["file[]"]) - - ruleGroups := api.rulesRetriever(r.Context()).RuleGroups() - res := &RuleDiscovery{RuleGroups: make([]*RuleGroup, 0, len(ruleGroups))} - typ := strings.ToLower(r.URL.Query().Get("type")) - - if typ != "" && typ != "alert" && typ != "record" { - return invalidParamError(errors.Errorf("not supported value %q", typ), "type") - } - - returnAlerts := typ == "" || typ == "alert" - returnRecording := typ == "" || typ == "record" - - rgs := make([]*RuleGroup, 0, len(ruleGroups)) - for _, grp := range ruleGroups { - if len(rgSet) > 0 { - if _, ok := rgSet[grp.Name()]; !ok { - continue - } - } - - if len(fSet) > 0 { - if _, ok := fSet[grp.File()]; !ok { - continue - } - } - - apiRuleGroup := &RuleGroup{ - Name: grp.Name(), - File: grp.File(), - Interval: grp.Interval().Seconds(), - Limit: grp.Limit(), - Rules: []Rule{}, - EvaluationTime: grp.GetEvaluationTime().Seconds(), - LastEvaluation: grp.GetLastEvaluation(), - } - for _, rr := range grp.Rules() { - var enrichedRule Rule - - if len(rnSet) > 0 { - if _, ok := rnSet[rr.Name()]; !ok { - continue - } - } - - lastError := "" - if rr.LastError() != nil { - lastError = rr.LastError().Error() - } - switch rule := rr.(type) { - case *rules.AlertingRule: - if !returnAlerts { - break - } - enrichedRule = AlertingRule{ - State: rule.State().String(), - Name: rule.Name(), - Query: rule.Query().String(), - Duration: rule.HoldDuration().Seconds(), - KeepFiringFor: rule.KeepFiringFor().Seconds(), - Labels: rule.Labels(), - Annotations: rule.Annotations(), - Alerts: rulesAlertsToAPIAlerts(rule.ActiveAlerts()), - Health: rule.Health(), - LastError: lastError, - EvaluationTime: rule.GetEvaluationDuration().Seconds(), - LastEvaluation: rule.GetEvaluationTimestamp(), - Type: "alerting", - } - case *rules.RecordingRule: - if !returnRecording { - break - } - enrichedRule = RecordingRule{ - Name: rule.Name(), - Query: rule.Query().String(), - Labels: rule.Labels(), - Health: rule.Health(), - LastError: lastError, - EvaluationTime: rule.GetEvaluationDuration().Seconds(), - LastEvaluation: rule.GetEvaluationTimestamp(), - Type: "recording", - } - default: - err := errors.Errorf("failed to assert type of rule '%v'", rule.Name()) - return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} - } - - if enrichedRule != nil { - apiRuleGroup.Rules = append(apiRuleGroup.Rules, enrichedRule) - } - } - - // If the rule group response has no rules, skip it - this means we filtered all the rules of this group. - if len(apiRuleGroup.Rules) > 0 { - rgs = append(rgs, apiRuleGroup) - } - } - res.RuleGroups = rgs - return apiFuncResult{res, nil, nil, nil} -} - -type prometheusConfig struct { - YAML string `json:"yaml"` -} - -func (api *API) serveRuntimeInfo(_ *http.Request) apiFuncResult { - status, err := api.runtimeInfo() - if err != nil { - return apiFuncResult{status, &apiError{errorInternal, err}, nil, nil} - } - return apiFuncResult{status, nil, nil, nil} -} - -func (api *API) serveBuildInfo(_ *http.Request) apiFuncResult { - return apiFuncResult{api.buildInfo, nil, nil, nil} -} - -func (api *API) serveConfig(_ *http.Request) apiFuncResult { - cfg := &prometheusConfig{ - YAML: api.config().String(), - } - return apiFuncResult{cfg, nil, nil, nil} -} - -func (api *API) serveFlags(_ *http.Request) apiFuncResult { - return apiFuncResult{api.flagsMap, nil, nil, nil} -} - -// TSDBStat holds the information about individual cardinality. -type TSDBStat struct { - Name string `json:"name"` - Value uint64 `json:"value"` -} - -// HeadStats has information about the TSDB head. -type HeadStats struct { - NumSeries uint64 `json:"numSeries"` - NumLabelPairs int `json:"numLabelPairs"` - ChunkCount int64 `json:"chunkCount"` - MinTime int64 `json:"minTime"` - MaxTime int64 `json:"maxTime"` -} - -// TSDBStatus has information of cardinality statistics from postings. -type TSDBStatus struct { - HeadStats HeadStats `json:"headStats"` - SeriesCountByMetricName []TSDBStat `json:"seriesCountByMetricName"` - LabelValueCountByLabelName []TSDBStat `json:"labelValueCountByLabelName"` - MemoryInBytesByLabelName []TSDBStat `json:"memoryInBytesByLabelName"` - SeriesCountByLabelValuePair []TSDBStat `json:"seriesCountByLabelValuePair"` -} - -// TSDBStatsFromIndexStats converts a index.Stat slice to a TSDBStat slice. -func TSDBStatsFromIndexStats(stats []index.Stat) []TSDBStat { - result := make([]TSDBStat, 0, len(stats)) - for _, item := range stats { - item := TSDBStat{Name: item.Name, Value: item.Count} - result = append(result, item) - } - return result -} - -func (api *API) serveTSDBStatus(r *http.Request) apiFuncResult { - limit := 10 - if s := r.FormValue("limit"); s != "" { - var err error - if limit, err = strconv.Atoi(s); err != nil || limit < 1 { - return apiFuncResult{nil, &apiError{errorBadData, errors.New("limit must be a positive number")}, nil, nil} - } - } - s, err := api.db.Stats(labels.MetricName, limit) - if err != nil { - return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} - } - metrics, err := api.gatherer.Gather() - if err != nil { - return apiFuncResult{nil, &apiError{errorInternal, fmt.Errorf("error gathering runtime status: %s", err)}, nil, nil} - } - chunkCount := int64(math.NaN()) - for _, mF := range metrics { - if *mF.Name == "prometheus_tsdb_head_chunks" { - m := *mF.Metric[0] - if m.Gauge != nil { - chunkCount = int64(m.Gauge.GetValue()) - break - } - } - } - return apiFuncResult{TSDBStatus{ - HeadStats: HeadStats{ - NumSeries: s.NumSeries, - ChunkCount: chunkCount, - MinTime: s.MinTime, - MaxTime: s.MaxTime, - NumLabelPairs: s.IndexPostingStats.NumLabelPairs, - }, - SeriesCountByMetricName: TSDBStatsFromIndexStats(s.IndexPostingStats.CardinalityMetricsStats), - LabelValueCountByLabelName: TSDBStatsFromIndexStats(s.IndexPostingStats.CardinalityLabelStats), - MemoryInBytesByLabelName: TSDBStatsFromIndexStats(s.IndexPostingStats.LabelValueStats), - SeriesCountByLabelValuePair: TSDBStatsFromIndexStats(s.IndexPostingStats.LabelValuePairsStats), - }, nil, nil, nil} -} - -type walReplayStatus struct { - Min int `json:"min"` - Max int `json:"max"` - Current int `json:"current"` -} - -func (api *API) serveWALReplayStatus(w http.ResponseWriter, r *http.Request) { - httputil.SetCORS(w, api.CORSOrigin, r) - status, err := api.db.WALReplayStatus() - if err != nil { - api.respondError(w, &apiError{errorInternal, err}, nil) - } - api.respond(w, walReplayStatus{ - Min: status.Min, - Max: status.Max, - Current: status.Current, - }, nil) -} - -func (api *API) remoteRead(w http.ResponseWriter, r *http.Request) { - // This is only really for tests - this will never be nil IRL. - if api.remoteReadHandler != nil { - api.remoteReadHandler.ServeHTTP(w, r) - } else { - http.Error(w, "not found", http.StatusNotFound) - } -} - -func (api *API) remoteWrite(w http.ResponseWriter, r *http.Request) { - if api.remoteWriteHandler != nil { - api.remoteWriteHandler.ServeHTTP(w, r) - } else { - http.Error(w, "remote write receiver needs to be enabled with --web.enable-remote-write-receiver", http.StatusNotFound) - } -} - -func (api *API) deleteSeries(r *http.Request) apiFuncResult { - if !api.enableAdmin { - return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} - } - if err := r.ParseForm(); err != nil { - return apiFuncResult{nil, &apiError{errorBadData, errors.Wrap(err, "error parsing form values")}, nil, nil} - } - if len(r.Form["match[]"]) == 0 { - return apiFuncResult{nil, &apiError{errorBadData, errors.New("no match[] parameter provided")}, nil, nil} - } - - start, err := parseTimeParam(r, "start", minTime) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTimeParam(r, "end", maxTime) - if err != nil { - return invalidParamError(err, "end") - } - - for _, s := range r.Form["match[]"] { - matchers, err := parser.ParseMetricSelector(s) - if err != nil { - return invalidParamError(err, "match[]") - } - if err := api.db.Delete(timestamp.FromTime(start), timestamp.FromTime(end), matchers...); err != nil { - return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} - } - } - - return apiFuncResult{nil, nil, nil, nil} -} - -func (api *API) snapshot(r *http.Request) apiFuncResult { - if !api.enableAdmin { - return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} - } - var ( - skipHead bool - err error - ) - if r.FormValue("skip_head") != "" { - skipHead, err = strconv.ParseBool(r.FormValue("skip_head")) - if err != nil { - return invalidParamError(errors.Wrapf(err, "unable to parse boolean"), "skip_head") - } - } - - var ( - snapdir = filepath.Join(api.dbDir, "snapshots") - name = fmt.Sprintf("%s-%016x", - time.Now().UTC().Format("20060102T150405Z0700"), - rand.Int63()) - dir = filepath.Join(snapdir, name) - ) - if err := os.MkdirAll(dir, 0o777); err != nil { - return apiFuncResult{nil, &apiError{errorInternal, errors.Wrap(err, "create snapshot directory")}, nil, nil} - } - if err := api.db.Snapshot(dir, !skipHead); err != nil { - return apiFuncResult{nil, &apiError{errorInternal, errors.Wrap(err, "create snapshot")}, nil, nil} - } - - return apiFuncResult{struct { - Name string `json:"name"` - }{name}, nil, nil, nil} -} - -func (api *API) cleanTombstones(*http.Request) apiFuncResult { - if !api.enableAdmin { - return apiFuncResult{nil, &apiError{errorUnavailable, errors.New("admin APIs disabled")}, nil, nil} - } - if err := api.db.CleanTombstones(); err != nil { - return apiFuncResult{nil, &apiError{errorInternal, err}, nil, nil} - } - - return apiFuncResult{nil, nil, nil, nil} -} - func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { statusMessage := statusSuccess var warningStrings []string @@ -1650,13 +395,6 @@ func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storag prometheusInstantQueryResponse, _ := data.(instantquery.PrometheusInstantQueryResponse) prometheusInstantQueryResponse.Status = string(statusMessage) b, err = proto.Marshal(&prometheusInstantQueryResponse) - default: - json := jsoniter.ConfigCompatibleWithStandardLibrary - b, err = json.Marshal(&response{ - Status: statusMessage, - Data: data, - Warnings: warningStrings, - }) } if err != nil { level.Error(api.logger).Log("msg", "error marshaling protobuf response", "err", err) @@ -1759,199 +497,6 @@ func parseDuration(s string) (time.Duration, error) { return 0, errors.Errorf("cannot parse %q to a valid duration", s) } -func parseMatchersParam(matchers []string) ([][]*labels.Matcher, error) { - var matcherSets [][]*labels.Matcher - for _, s := range matchers { - matchers, err := parser.ParseMetricSelector(s) - if err != nil { - return nil, err - } - matcherSets = append(matcherSets, matchers) - } - -OUTER: - for _, ms := range matcherSets { - for _, lm := range ms { - if lm != nil && !lm.Matches("") { - continue OUTER - } - } - return nil, errors.New("match[] must contain at least one non-empty matcher") - } - return matcherSets, nil -} - -// marshalSeriesJSON writes something like the following: -// -// { -// "metric" : { -// "__name__" : "up", -// "job" : "prometheus", -// "instance" : "localhost:9090" -// }, -// "values": [ -// [ 1435781451.781, "1" ], -// < more values> -// ], -// "histograms": [ -// [ 1435781451.781, { < histogram, see jsonutil.MarshalHistogram > } ], -// < more histograms > -// ], -// }, -func marshalSeriesJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { - s := *((*promql.Series)(ptr)) - stream.WriteObjectStart() - stream.WriteObjectField(`metric`) - m, err := s.Metric.MarshalJSON() - if err != nil { - stream.Error = err - return - } - stream.SetBuffer(append(stream.Buffer(), m...)) - - for i, p := range s.Floats { - stream.WriteMore() - if i == 0 { - stream.WriteObjectField(`values`) - stream.WriteArrayStart() - } - marshalFPointJSON(unsafe.Pointer(&p), stream) - } - if len(s.Floats) > 0 { - stream.WriteArrayEnd() - } - for i, p := range s.Histograms { - stream.WriteMore() - if i == 0 { - stream.WriteObjectField(`histograms`) - stream.WriteArrayStart() - } - marshalHPointJSON(unsafe.Pointer(&p), stream) - } - if len(s.Histograms) > 0 { - stream.WriteArrayEnd() - } - stream.WriteObjectEnd() -} - -func marshalSeriesJSONIsEmpty(unsafe.Pointer) bool { - return false -} - -// marshalSampleJSON writes something like the following for normal value samples: -// -// { -// "metric" : { -// "__name__" : "up", -// "job" : "prometheus", -// "instance" : "localhost:9090" -// }, -// "value": [ 1435781451.781, "1.234" ] -// }, -// -// For histogram samples, it writes something like this: -// -// { -// "metric" : { -// "__name__" : "up", -// "job" : "prometheus", -// "instance" : "localhost:9090" -// }, -// "histogram": [ 1435781451.781, { < histogram, see jsonutil.MarshalHistogram > } ] -// }, -func marshalSampleJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { - s := *((*promql.Sample)(ptr)) - stream.WriteObjectStart() - stream.WriteObjectField(`metric`) - m, err := s.Metric.MarshalJSON() - if err != nil { - stream.Error = err - return - } - stream.SetBuffer(append(stream.Buffer(), m...)) - stream.WriteMore() - if s.H == nil { - stream.WriteObjectField(`value`) - } else { - stream.WriteObjectField(`histogram`) - } - stream.WriteArrayStart() - jsonutil.MarshalTimestamp(s.T, stream) - stream.WriteMore() - if s.H == nil { - jsonutil.MarshalFloat(s.F, stream) - } else { - jsonutil.MarshalHistogram(s.H, stream) - } - stream.WriteArrayEnd() - stream.WriteObjectEnd() -} - -func marshalSampleJSONIsEmpty(unsafe.Pointer) bool { - return false -} - -// marshalFPointJSON writes `[ts, "1.234"]`. -func marshalFPointJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { - p := *((*promql.FPoint)(ptr)) - stream.WriteArrayStart() - jsonutil.MarshalTimestamp(p.T, stream) - stream.WriteMore() - jsonutil.MarshalFloat(p.F, stream) - stream.WriteArrayEnd() -} - -// marshalHPointJSON writes `[ts, { < histogram, see jsonutil.MarshalHistogram > } ]`. -func marshalHPointJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { - p := *((*promql.HPoint)(ptr)) - stream.WriteArrayStart() - jsonutil.MarshalTimestamp(p.T, stream) - stream.WriteMore() - jsonutil.MarshalHistogram(p.H, stream) - stream.WriteArrayEnd() -} - -func marshalPointJSONIsEmpty(unsafe.Pointer) bool { - return false -} - -// marshalExemplarJSON writes. -// -// { -// labels: , -// value: "", -// timestamp: -// } -func marshalExemplarJSON(ptr unsafe.Pointer, stream *jsoniter.Stream) { - p := *((*exemplar.Exemplar)(ptr)) - stream.WriteObjectStart() - - // "labels" key. - stream.WriteObjectField(`labels`) - lbls, err := p.Labels.MarshalJSON() - if err != nil { - stream.Error = err - return - } - stream.SetBuffer(append(stream.Buffer(), lbls...)) - - // "value" key. - stream.WriteMore() - stream.WriteObjectField(`value`) - jsonutil.MarshalFloat(p.Value, stream) - - // "timestamp" key. - stream.WriteMore() - stream.WriteObjectField(`timestamp`) - jsonutil.MarshalTimestamp(p.Ts, stream) - - stream.WriteObjectEnd() -} - -func marshalExemplarJSONEmpty(unsafe.Pointer) bool { - return false -} - func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResponse { data, _ := queryResult.(*queryData) if data != nil { From f1eddd972be5e2aafcf63c1a01e6350e4eafc230 Mon Sep 17 00:00:00 2001 From: afayek Date: Thu, 13 Jul 2023 16:25:18 -0700 Subject: [PATCH 05/38] add snappy compression Signed-off-by: Ahmed Hassan --- pkg/querier/handler/compression.go | 99 +++++++++++++++++++ pkg/querier/handler/handler.go | 2 +- .../tripperware/instantquery/instant_query.go | 3 +- pkg/querier/tripperware/query.go | 4 + .../tripperware/queryrange/query_range.go | 3 +- 5 files changed, 106 insertions(+), 5 deletions(-) create mode 100644 pkg/querier/handler/compression.go diff --git a/pkg/querier/handler/compression.go b/pkg/querier/handler/compression.go new file mode 100644 index 00000000000..b700b778f26 --- /dev/null +++ b/pkg/querier/handler/compression.go @@ -0,0 +1,99 @@ +package handler + +import ( + "net/http" + "strings" + + "github.com/golang/snappy" + "github.com/klauspost/compress/gzhttp" + "github.com/klauspost/compress/zlib" +) + +const ( + acceptEncodingHeader = "Accept-Encoding" + contentEncodingHeader = "Content-Encoding" + snappyEncoding = "snappy" + gzipEncoding = "gzip" + deflateEncoding = "deflate" +) + +// Wrapper around http.ResponseWriter which adds deflate compression +type deflatedResponseWriter struct { + http.ResponseWriter + writer *zlib.Writer +} + +// Writes HTTP response content data. +func (c *deflatedResponseWriter) Write(p []byte) (int, error) { + return c.writer.Write(p) +} + +// Close Closes the deflatedResponseWriter and ensures to flush all data before. +func (c *deflatedResponseWriter) Close() { + c.writer.Close() +} + +// Constructs a new deflatedResponseWriter to compress the original writer using 'deflate' compression. +func newDeflateResponseWriter(writer http.ResponseWriter) *deflatedResponseWriter { + return &deflatedResponseWriter{ + ResponseWriter: writer, + writer: zlib.NewWriter(writer), + } +} + +// Wrapper around http.ResponseWriter which adds deflate compression +type snappyResponseWriter struct { + http.ResponseWriter + writer *snappy.Writer +} + +// Writes HTTP response content data. +func (c *snappyResponseWriter) Write(p []byte) (int, error) { + return c.writer.Write(p) +} + +// Close Closes the snappyResponseWriter and ensures to flush all data before. +func (c *snappyResponseWriter) Close() { + c.writer.Close() +} + +// Constructs a new snappyResponseWriter to compress the original writer using 'deflate' compression. +func newSnappyResponseWriter(writer http.ResponseWriter) *snappyResponseWriter { + return &snappyResponseWriter{ + ResponseWriter: writer, + writer: snappy.NewBufferedWriter(writer), + } +} + +// CompressionHandler is a wrapper around http.Handler which adds suitable +// response compression based on the client's Accept-Encoding headers. +type CompressionHandler struct { + Handler http.Handler +} + +// ServeHTTP adds compression to the original http.Handler's ServeHTTP() method. +func (c CompressionHandler) ServeHTTP(writer http.ResponseWriter, req *http.Request) { + encodings := strings.Split(req.Header.Get(acceptEncodingHeader), ",") + for _, encoding := range encodings { + switch strings.TrimSpace(encoding) { + case gzipEncoding: + gzhttp.GzipHandler(c.Handler).ServeHTTP(writer, req) + return + case snappyEncoding: + compWriter := newSnappyResponseWriter(writer) + writer.Header().Set(contentEncodingHeader, snappyEncoding) + c.Handler.ServeHTTP(compWriter, req) + compWriter.Close() + return + case deflateEncoding: + compWriter := newDeflateResponseWriter(writer) + writer.Header().Set(contentEncodingHeader, deflateEncoding) + c.Handler.ServeHTTP(compWriter, req) + compWriter.Close() + return + default: + c.Handler.ServeHTTP(writer, req) + return + } + } +} diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 127675a575d..5a60583392c 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -163,7 +163,7 @@ func (api *API) Register(r *route.Router) { } w.WriteHeader(http.StatusNoContent) }) - return api.ready(httputil.CompressionHandler{ + return api.ready(CompressionHandler{ Handler: hf, }.ServeHTTP) } diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index e5f0ccef573..e23039156fd 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -210,8 +210,7 @@ func (instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } - // Always ask gzip to the querier - h.Set("Accept-Encoding", "gzip") + h.Set("Accept-Encoding", "snappy") req := &http.Request{ Method: "GET", diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index 0e8237c74eb..80fb5ff96d9 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -5,6 +5,7 @@ import ( "compress/gzip" "context" "fmt" + "github.com/golang/snappy" "io" "net/http" "sort" @@ -242,6 +243,9 @@ func BodyBuffer(res *http.Response, logger log.Logger) ([]byte, error) { defer runutil.CloseWithLogOnErr(logger, gReader, "close gzip reader") return io.ReadAll(gReader) + } else if strings.EqualFold(res.Header.Get("Content-Encoding"), "snappy") { + sReader := snappy.NewReader(buf) + return io.ReadAll(sReader) } return buf.Bytes(), nil diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 08958f3a66b..36a777da947 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -244,8 +244,7 @@ func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) } } - // Always ask gzip to the querier - h.Set("Accept-Encoding", "gzip") + h.Set("Accept-Encoding", "snappy") req := &http.Request{ Method: "GET", From 855375f182101b5677b85250f38d06e82a899733 Mon Sep 17 00:00:00 2001 From: afayek Date: Fri, 14 Jul 2023 09:29:08 -0700 Subject: [PATCH 06/38] improve code readability in querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/compression.go | 6 - pkg/querier/handler/handler.go | 266 +++++++++++++---------------- 2 files changed, 123 insertions(+), 149 deletions(-) diff --git a/pkg/querier/handler/compression.go b/pkg/querier/handler/compression.go index b700b778f26..a4db3191923 100644 --- a/pkg/querier/handler/compression.go +++ b/pkg/querier/handler/compression.go @@ -23,17 +23,14 @@ type deflatedResponseWriter struct { writer *zlib.Writer } -// Writes HTTP response content data. func (c *deflatedResponseWriter) Write(p []byte) (int, error) { return c.writer.Write(p) } -// Close Closes the deflatedResponseWriter and ensures to flush all data before. func (c *deflatedResponseWriter) Close() { c.writer.Close() } -// Constructs a new deflatedResponseWriter to compress the original writer using 'deflate' compression. func newDeflateResponseWriter(writer http.ResponseWriter) *deflatedResponseWriter { return &deflatedResponseWriter{ ResponseWriter: writer, @@ -47,17 +44,14 @@ type snappyResponseWriter struct { writer *snappy.Writer } -// Writes HTTP response content data. func (c *snappyResponseWriter) Write(p []byte) (int, error) { return c.writer.Write(p) } -// Close Closes the snappyResponseWriter and ensures to flush all data before. func (c *snappyResponseWriter) Close() { c.writer.Close() } -// Constructs a new snappyResponseWriter to compress the original writer using 'deflate' compression. func newSnappyResponseWriter(writer http.ResponseWriter) *snappyResponseWriter { return &snappyResponseWriter{ ResponseWriter: writer, diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 5a60583392c..972c817f384 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -395,6 +395,10 @@ func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storag prometheusInstantQueryResponse, _ := data.(instantquery.PrometheusInstantQueryResponse) prometheusInstantQueryResponse.Status = string(statusMessage) b, err = proto.Marshal(&prometheusInstantQueryResponse) + default: + level.Error(api.logger).Log("msg", "error asserting response type") + http.Error(w, "error asserting response type", http.StatusInternalServerError) + return } if err != nil { level.Error(api.logger).Log("msg", "error marshaling protobuf response", "err", err) @@ -497,175 +501,151 @@ func parseDuration(s string) (time.Duration, error) { return 0, errors.Errorf("cannot parse %q to a valid duration", s) } -func createPrometheusResponse(queryResult interface{}) queryrange.PrometheusResponse { - data, _ := queryResult.(*queryData) - if data != nil { - sampleStreamsLen := len(data.Result.(promql.Matrix)) - sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) - - for i := 0; i < sampleStreamsLen; i++ { - labelsLen := len(data.Result.(promql.Matrix)[i].Metric) - labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Matrix)[i].Metric[j].Name, - Value: data.Result.(promql.Matrix)[i].Metric[j].Value, - } - } - - samplesLen := len(data.Result.(promql.Matrix)[i].Floats) - samples := make([]cortexpb.Sample, samplesLen) - for j := 0; j < samplesLen; j++ { - samples[j] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[i].Floats[j].F, - TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, - } - } - sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} +func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { + if data == nil { + return queryrange.PrometheusResponse{ + Status: "", + Data: queryrange.PrometheusData{}, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, } + } - var stats *tripperware.PrometheusResponseStats - - if data.Stats != nil { - builtin := data.Stats.Builtin() - queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) - queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) - for i := 0; i < queryableSamplesStatsPerStepLen; i++ { - queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, - TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, - } - } + sampleStreams := getSampleStreams(data) - statSamples := tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, - TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, - } + var stats *tripperware.PrometheusResponseStats + if data.Stats != nil { + builtin := data.Stats.Builtin() + stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} + } - stats = &tripperware.PrometheusResponseStats{Samples: &statSamples} - } + return queryrange.PrometheusResponse{ + Status: "", + Data: queryrange.PrometheusData{ + ResultType: string(data.ResultType), + Result: *sampleStreams, + Stats: stats, + }, + ErrorType: "", + Error: "", + Headers: []*tripperware.PrometheusResponseHeader{}, + } +} - return queryrange.PrometheusResponse{ - Status: "", - Data: queryrange.PrometheusData{ - ResultType: string(data.ResultType), - Result: sampleStreams, - Stats: stats, - }, +func createPrometheusInstantQueryResponse(data *queryData) instantquery.PrometheusInstantQueryResponse { + if data == nil { + return instantquery.PrometheusInstantQueryResponse{ + Status: "", + Data: instantquery.PrometheusInstantQueryData{}, ErrorType: "", Error: "", Headers: []*tripperware.PrometheusResponseHeader{}, } } - return queryrange.PrometheusResponse{ - Status: "", - Data: queryrange.PrometheusData{}, + var instantQueryResult instantquery.PrometheusInstantQueryResult + switch string(data.ResultType) { + case "matrix": + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ + Matrix: &instantquery.Matrix{ + SampleStreams: *getSampleStreams(data), + }, + } + case "vector": + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ + Vector: &instantquery.Vector{ + Samples: *getSamples(data), + }, + } + default: + // TODO: add scalar and string instant query responses + } + + var stats *tripperware.PrometheusResponseStats + if data.Stats != nil { + builtin := data.Stats.Builtin() + stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} + } + + return instantquery.PrometheusInstantQueryResponse{ + Status: "", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: string(data.ResultType), + Result: instantQueryResult, + Stats: stats, + }, ErrorType: "", Error: "", Headers: []*tripperware.PrometheusResponseHeader{}, } } -func createPrometheusInstantQueryResponse(queryResult interface{}) instantquery.PrometheusInstantQueryResponse { - data, _ := queryResult.(*queryData) - if data != nil { - var instantQueryResult instantquery.PrometheusInstantQueryResult - - switch string(data.ResultType) { - case "matrix": - matrixSamplesLen := len(data.Result.(promql.Matrix)) - matrixSamples := make([]tripperware.SampleStream, matrixSamplesLen) - - for i := 0; i < matrixSamplesLen; i++ { - labelsLen := len(data.Result.(promql.Matrix)[i].Metric) - labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Matrix)[i].Metric[j].Name, - Value: data.Result.(promql.Matrix)[i].Metric[j].Value, - } - } - - samplesLen := len(data.Result.(promql.Matrix)[i].Floats) - samples := make([]cortexpb.Sample, samplesLen) - for j := 0; j < samplesLen; j++ { - samples[j] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[i].Floats[j].F, - TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, - } - } - - matrixSamples[i] = tripperware.SampleStream{Labels: labels, Samples: samples} - } - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{Matrix: &instantquery.Matrix{SampleStreams: matrixSamples}} - - case "vector": - vectorSamplesLen := len(data.Result.(promql.Vector)) - vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) - - for i := 0; i < vectorSamplesLen; i++ { - labelsLen := len(data.Result.(promql.Vector)[i].Metric) - labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Vector)[i].Metric[j].Name, - Value: data.Result.(promql.Vector)[i].Metric[j].Value, - } - } - - vectorSamples[i] = &instantquery.Sample{Labels: labels, - Sample: cortexpb.Sample{ - TimestampMs: data.Result.(promql.Vector)[i].T, - Value: data.Result.(promql.Vector)[i].F, - }, - } - } - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{Vector: &instantquery.Vector{Samples: vectorSamples}} - - default: - // TODO: add scalar and string instant query responses +func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { + queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) + queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) + for i := 0; i < queryableSamplesStatsPerStepLen; i++ { + queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, + TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, } + } + + statSamples := tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, + TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + } - var stats *tripperware.PrometheusResponseStats - - if data.Stats != nil { - builtin := data.Stats.Builtin() - queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) - queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) - for i := 0; i < queryableSamplesStatsPerStepLen; i++ { - queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, - TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, - } + return &statSamples +} + +func getSampleStreams(data *queryData) *[]tripperware.SampleStream { + sampleStreamsLen := len(data.Result.(promql.Matrix)) + sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) + + for i := 0; i < sampleStreamsLen; i++ { + labelsLen := len(data.Result.(promql.Matrix)[i].Metric) + labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[i].Metric[j].Name, + Value: data.Result.(promql.Matrix)[i].Metric[j].Value, } + } - statSamples := tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, - TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + samplesLen := len(data.Result.(promql.Matrix)[i].Floats) + samples := make([]cortexpb.Sample, samplesLen) + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, } + } + sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} + } + return &sampleStreams +} - stats = &tripperware.PrometheusResponseStats{Samples: &statSamples} +func getSamples(data *queryData) *[]*instantquery.Sample { + vectorSamplesLen := len(data.Result.(promql.Vector)) + vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) + + for i := 0; i < vectorSamplesLen; i++ { + labelsLen := len(data.Result.(promql.Vector)[i].Metric) + labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Vector)[i].Metric[j].Name, + Value: data.Result.(promql.Vector)[i].Metric[j].Value, + } } - return instantquery.PrometheusInstantQueryResponse{ - Status: "", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: string(data.ResultType), - Result: instantQueryResult, - Stats: stats, + vectorSamples[i] = &instantquery.Sample{Labels: labels, + Sample: cortexpb.Sample{ + TimestampMs: data.Result.(promql.Vector)[i].T, + Value: data.Result.(promql.Vector)[i].F, }, - ErrorType: "", - Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, } } - - return instantquery.PrometheusInstantQueryResponse{ - Status: "", - Data: instantquery.PrometheusInstantQueryData{}, - ErrorType: "", - Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, - } + return &vectorSamples } From 6884a45f50f191bcef8612cf9691727e75569ca3 Mon Sep 17 00:00:00 2001 From: afayek Date: Mon, 17 Jul 2023 17:06:52 -0700 Subject: [PATCH 07/38] add Query and QueryRange handler functions directly to router Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 37 ++++++-- pkg/querier/handler/handler.go | 149 +++++++++++---------------------- 2 files changed, 78 insertions(+), 108 deletions(-) diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index d988e45b530..a669975df81 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -3,6 +3,7 @@ package api import ( "context" "encoding/json" + "github.com/prometheus/prometheus/util/httputil" "html/template" "net/http" "path" @@ -27,6 +28,7 @@ import ( "github.com/weaveworks/common/middleware" "github.com/cortexproject/cortex/pkg/querier" + "github.com/cortexproject/cortex/pkg/querier/handler" "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/util" ) @@ -270,21 +272,40 @@ func NewQuerierHandler( promRouter := route.New().WithPrefix(path.Join(prefix, "/api/v1")) v1api.Register(promRouter) - queryPromRouter := route.New().WithPrefix(path.Join(prefix, "/api/v1")) - queryapi.Register(queryPromRouter) legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) v1api.Register(legacyPromRouter) - queryLegacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) - queryapi.Register(queryLegacyPromRouter) + + wrap := func(f qapi.ApiFunc) http.HandlerFunc { + hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + httputil.SetCORS(w, queryapi.CORSOrigin, r) + result := qapi.SetUnavailStatusOnTSDBNotReady(f(r)) + if result.Finalizer != nil { + defer result.Finalizer() + } + if result.Err != nil { + queryapi.RespondError(w, result.Err, result.Data) + return + } + + if result.Data != nil { + queryapi.Respond(w, result.Data, result.Warnings) + return + } + w.WriteHeader(http.StatusNoContent) + }) + return queryapi.Ready(handler.CompressionHandler{ + Handler: hf, + }.ServeHTTP) + } // TODO(gotjosh): This custom handler is temporary until we're able to vendor the changes in: // https://github.com/prometheus/prometheus/pull/7125/files router.Path(path.Join(prefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(prefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(prefix, "/api/v1/read")).Methods("POST").Handler(promRouter) - router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(queryPromRouter) - router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(queryPromRouter) + router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) + router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) router.Path(path.Join(prefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/labels")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(promRouter) @@ -296,8 +317,8 @@ func NewQuerierHandler( router.Path(path.Join(legacyPrefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Methods("POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(queryLegacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(queryLegacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/labels")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(legacyPromRouter) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 972c817f384..7ec7db834eb 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -11,7 +11,7 @@ import ( jsoniter "github.com/json-iterator/go" "github.com/pkg/errors" "github.com/prometheus/common/model" - "github.com/prometheus/common/route" + v1 "github.com/prometheus/prometheus/web/api/v1" "math" "net/http" "strconv" @@ -62,8 +62,6 @@ func (e *apiError) Error() string { return fmt.Sprintf("%s: %s", e.typ, e.err) } -type StatsRenderer func(context.Context, *stats.Statistics, string) stats.QueryStats - func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) stats.QueryStats { if param != "" { return stats.NewQueryStats(s) @@ -79,124 +77,75 @@ type response struct { Warnings []string `json:"warnings,omitempty"` } -type apiFuncResult struct { - data interface{} - err *apiError - warnings storage.Warnings - finalizer func() +type ApiFuncResult struct { + Data interface{} + Err *apiError + Warnings storage.Warnings + Finalizer func() } -type apiFunc func(r *http.Request) apiFuncResult - -// QueryEngine defines the interface for the *promql.Engine, so it can be replaced, wrapped or mocked. -type QueryEngine interface { - SetQueryLogger(l promql.QueryLogger) - NewInstantQuery(ctx context.Context, q storage.Queryable, opts *promql.QueryOpts, qs string, ts time.Time) (promql.Query, error) - NewRangeQuery(ctx context.Context, q storage.Queryable, opts *promql.QueryOpts, qs string, start, end time.Time, interval time.Duration) (promql.Query, error) -} +type ApiFunc func(r *http.Request) ApiFuncResult -// API can register a set of endpoints in a router and handle -// them using the provided storage and query engine. type API struct { Queryable storage.SampleAndChunkQueryable - QueryEngine QueryEngine - now func() time.Time - ready func(http.HandlerFunc) http.HandlerFunc - logger log.Logger + QueryEngine v1.QueryEngine + Now func() time.Time + Ready func(http.HandlerFunc) http.HandlerFunc + Logger log.Logger CORSOrigin *regexp.Regexp - isAgent bool - statsRenderer StatsRenderer + IsAgent bool + StatsRenderer v1.StatsRenderer } // NewAPI returns an initialized API type. func NewAPI( - qe QueryEngine, + qe v1.QueryEngine, q storage.SampleAndChunkQueryable, readyFunc func(http.HandlerFunc) http.HandlerFunc, logger log.Logger, isAgent bool, corsOrigin *regexp.Regexp, - statsRenderer StatsRenderer, + statsRenderer v1.StatsRenderer, ) *API { a := &API{ QueryEngine: qe, Queryable: q, - now: time.Now, - ready: readyFunc, - logger: logger, + Now: time.Now, + Ready: readyFunc, + Logger: logger, CORSOrigin: corsOrigin, - isAgent: isAgent, - statsRenderer: defaultStatsRenderer, + IsAgent: isAgent, + StatsRenderer: defaultStatsRenderer, } if statsRenderer != nil { - a.statsRenderer = statsRenderer + a.StatsRenderer = statsRenderer } return a } -func setUnavailStatusOnTSDBNotReady(r apiFuncResult) apiFuncResult { - if r.err != nil && errors.Cause(r.err.err) == tsdb.ErrNotReady { - r.err.typ = errorUnavailable +func SetUnavailStatusOnTSDBNotReady(r ApiFuncResult) ApiFuncResult { + if r.Err != nil && errors.Cause(r.Err.err) == tsdb.ErrNotReady { + r.Err.typ = errorUnavailable } return r } -// Register the API's endpoints in the given router. -func (api *API) Register(r *route.Router) { - wrap := func(f apiFunc) http.HandlerFunc { - hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - httputil.SetCORS(w, api.CORSOrigin, r) - result := setUnavailStatusOnTSDBNotReady(f(r)) - if result.finalizer != nil { - defer result.finalizer() - } - if result.err != nil { - api.respondError(w, result.err, result.data) - return - } - - if result.data != nil { - api.respond(w, result.data, result.warnings) - return - } - w.WriteHeader(http.StatusNoContent) - }) - return api.ready(CompressionHandler{ - Handler: hf, - }.ServeHTTP) - } - - wrapAgent := func(f apiFunc) http.HandlerFunc { - return wrap(func(r *http.Request) apiFuncResult { - if api.isAgent { - return apiFuncResult{nil, &apiError{errorExec, errors.New("unavailable with Prometheus Agent")}, nil, nil} - } - return f(r) - }) - } - - r.Get("/query", wrapAgent(api.query)) - r.Post("/query", wrapAgent(api.query)) - r.Get("/query_range", wrapAgent(api.queryRange)) - r.Post("/query_range", wrapAgent(api.queryRange)) -} - type queryData struct { ResultType parser.ValueType `json:"resultType"` Result parser.Value `json:"result"` Stats stats.QueryStats `json:"stats,omitempty"` } -func invalidParamError(err error, parameter string) apiFuncResult { - return apiFuncResult{nil, &apiError{ +func invalidParamError(err error, parameter string) ApiFuncResult { + return ApiFuncResult{nil, &apiError{ errorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), }, nil, nil} } -func (api *API) query(r *http.Request) (result apiFuncResult) { - ts, err := parseTimeParam(r, "time", api.now()) +func (api *API) Query(r *http.Request) (result ApiFuncResult) { + ts, err := parseTimeParam(r, "time", api.Now()) if err != nil { return invalidParamError(err, "time") } @@ -214,18 +163,18 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { opts, err := extractQueryOpts(r) if err != nil { - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } qry, err := api.QueryEngine.NewInstantQuery(ctx, api.Queryable, opts, r.FormValue("query"), ts) if err != nil { return invalidParamError(err, "query") } - // From now on, we must only return with a finalizer in the result (to + // From now on, we must only return with a Finalizer in the result (to // be called by the caller) or call qry.Close ourselves (which is // required in the case of a panic). defer func() { - if result.finalizer == nil { + if result.Finalizer == nil { qry.Close() } }() @@ -234,17 +183,17 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { res := qry.Exec(ctx) if res.Err != nil { - return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + return ApiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} } // Optional stats field in response if parameter "stats" is not empty. - sr := api.statsRenderer + sr := api.StatsRenderer if sr == nil { sr = defaultStatsRenderer } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return apiFuncResult{createPrometheusInstantQueryResponse(&queryData{ + return ApiFuncResult{createPrometheusInstantQueryResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, @@ -265,7 +214,7 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { return opts, nil } -func (api *API) queryRange(r *http.Request) (result apiFuncResult) { +func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { start, err := parseTime(r.FormValue("start")) if err != nil { return invalidParamError(err, "start") @@ -291,7 +240,7 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { // This is sufficient for 60s resolution for a week or 1h resolution for a year. if end.Sub(start)/step > 11000 { err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } ctx := r.Context() @@ -308,17 +257,17 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { opts, err := extractQueryOpts(r) if err != nil { - return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } qry, err := api.QueryEngine.NewRangeQuery(ctx, api.Queryable, opts, r.FormValue("query"), start, end, step) if err != nil { return invalidParamError(err, "query") } - // From now on, we must only return with a finalizer in the result (to + // From now on, we must only return with a Finalizer in the result (to // be called by the caller) or call qry.Close ourselves (which is // required in the case of a panic). defer func() { - if result.finalizer == nil { + if result.Finalizer == nil { qry.Close() } }() @@ -327,17 +276,17 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { res := qry.Exec(ctx) if res.Err != nil { - return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + return ApiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} } // Optional stats field in response if parameter "stats" is not empty. - sr := api.statsRenderer + sr := api.StatsRenderer if sr == nil { sr = defaultStatsRenderer } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return apiFuncResult{createPrometheusResponse(&queryData{ + return ApiFuncResult{createPrometheusResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, @@ -378,7 +327,7 @@ var ( maxTimeFormatted = maxTime.Format(time.RFC3339Nano) ) -func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { +func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { statusMessage := statusSuccess var warningStrings []string for _, warning := range warnings { @@ -396,12 +345,12 @@ func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storag prometheusInstantQueryResponse.Status = string(statusMessage) b, err = proto.Marshal(&prometheusInstantQueryResponse) default: - level.Error(api.logger).Log("msg", "error asserting response type") + level.Error(api.Logger).Log("msg", "error asserting response type") http.Error(w, "error asserting response type", http.StatusInternalServerError) return } if err != nil { - level.Error(api.logger).Log("msg", "error marshaling protobuf response", "err", err) + level.Error(api.Logger).Log("msg", "error marshaling protobuf response", "err", err) http.Error(w, err.Error(), http.StatusInternalServerError) return } @@ -409,11 +358,11 @@ func (api *API) respond(w http.ResponseWriter, data interface{}, warnings storag w.Header().Set("Content-Type", "application/protobuf") w.WriteHeader(http.StatusOK) if n, err := w.Write(b); err != nil { - level.Error(api.logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) + level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) } } -func (api *API) respondError(w http.ResponseWriter, apiErr *apiError, data interface{}) { +func (api *API) RespondError(w http.ResponseWriter, apiErr *apiError, data interface{}) { json := jsoniter.ConfigCompatibleWithStandardLibrary b, err := json.Marshal(&response{ Status: statusError, @@ -422,7 +371,7 @@ func (api *API) respondError(w http.ResponseWriter, apiErr *apiError, data inter Data: data, }) if err != nil { - level.Error(api.logger).Log("msg", "error marshaling json response", "err", err) + level.Error(api.Logger).Log("msg", "error marshaling json response", "err", err) http.Error(w, err.Error(), http.StatusInternalServerError) return } @@ -448,7 +397,7 @@ func (api *API) respondError(w http.ResponseWriter, apiErr *apiError, data inter w.Header().Set("Content-Type", "application/json") w.WriteHeader(code) if n, err := w.Write(b); err != nil { - level.Error(api.logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) + level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) } } From f83b7318b08719a3f32b915d41a67bfacd15a946 Mon Sep 17 00:00:00 2001 From: afayek Date: Fri, 21 Jul 2023 11:21:33 -0700 Subject: [PATCH 08/38] add scalar and string result types to instant query handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 7ec7db834eb..f20d935a0b0 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -508,7 +508,11 @@ func createPrometheusInstantQueryResponse(data *queryData) instantquery.Promethe }, } default: - // TODO: add scalar and string instant query responses + rawBytes, err := jsoniter.Marshal(data) + if err != nil { + // TODO: handler error + } + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_RawBytes{RawBytes: rawBytes} } var stats *tripperware.PrometheusResponseStats From 112312be9b42ba161af163cc66b747d97f42550c Mon Sep 17 00:00:00 2001 From: afayek Date: Thu, 27 Jul 2023 22:43:55 -0700 Subject: [PATCH 09/38] reuse time parsing functions for querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 69 ++++++---------------------------- 1 file changed, 11 insertions(+), 58 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index f20d935a0b0..c5efdf6d303 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -4,17 +4,16 @@ import ( "context" "fmt" "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" + "github.com/cortexproject/cortex/pkg/util" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/protobuf/proto" "github.com/grafana/regexp" jsoniter "github.com/json-iterator/go" "github.com/pkg/errors" - "github.com/prometheus/common/model" v1 "github.com/prometheus/prometheus/web/api/v1" "math" "net/http" - "strconv" "time" "github.com/cortexproject/cortex/pkg/cortexpb" @@ -145,14 +144,15 @@ func invalidParamError(err error, parameter string) ApiFuncResult { } func (api *API) Query(r *http.Request) (result ApiFuncResult) { - ts, err := parseTimeParam(r, "time", api.Now()) + tms, err := instantquery.ParseTimeParam(r, "time", api.Now().Unix()) + ts := time.Unix(tms/1000, (tms%1000)*10e6) if err != nil { return invalidParamError(err, "time") } ctx := r.Context() if to := r.FormValue("timeout"); to != "" { var cancel context.CancelFunc - timeout, err := parseDuration(to) + timeout, err := time.ParseDuration(to) if err != nil { return invalidParamError(err, "timeout") } @@ -205,7 +205,7 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { EnablePerStepStats: r.FormValue("stats") == "all", } if strDuration := r.FormValue("lookback_delta"); strDuration != "" { - duration, err := parseDuration(strDuration) + duration, err := time.ParseDuration(strDuration) if err != nil { return nil, fmt.Errorf("error parsing lookback delta duration: %w", err) } @@ -215,11 +215,13 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { } func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { - start, err := parseTime(r.FormValue("start")) + startMs, err := util.ParseTime(r.FormValue("start")) + start := time.Unix(startMs/1000, (startMs%1000)*10e6) if err != nil { return invalidParamError(err, "start") } - end, err := parseTime(r.FormValue("end")) + endMs, err := util.ParseTime(r.FormValue("end")) + end := time.Unix(endMs/1000, (endMs%1000)*10e6) if err != nil { return invalidParamError(err, "end") } @@ -227,7 +229,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { return invalidParamError(errors.New("end timestamp must not be before start time"), "end") } - step, err := parseDuration(r.FormValue("step")) + step, err := time.ParseDuration(r.FormValue("step")) if err != nil { return invalidParamError(err, "step") } @@ -246,7 +248,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { ctx := r.Context() if to := r.FormValue("timeout"); to != "" { var cancel context.CancelFunc - timeout, err := parseDuration(to) + timeout, err := time.ParseDuration(to) if err != nil { return invalidParamError(err, "timeout") } @@ -401,55 +403,6 @@ func (api *API) RespondError(w http.ResponseWriter, apiErr *apiError, data inter } } -func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) (time.Time, error) { - val := r.FormValue(paramName) - if val == "" { - return defaultValue, nil - } - result, err := parseTime(val) - if err != nil { - return time.Time{}, errors.Wrapf(err, "Invalid time value for '%s'", paramName) - } - return result, nil -} - -func parseTime(s string) (time.Time, error) { - if t, err := strconv.ParseFloat(s, 64); err == nil { - s, ns := math.Modf(t) - ns = math.Round(ns*1000) / 1000 - return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil - } - if t, err := time.Parse(time.RFC3339Nano, s); err == nil { - return t, nil - } - - // Stdlib's time parser can only handle 4 digit years. As a workaround until - // that is fixed we want to at least support our own boundary times. - // Context: https://github.com/prometheus/client_golang/issues/614 - // Upstream issue: https://github.com/golang/go/issues/20555 - switch s { - case minTimeFormatted: - return minTime, nil - case maxTimeFormatted: - return maxTime, nil - } - return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) -} - -func parseDuration(s string) (time.Duration, error) { - if d, err := strconv.ParseFloat(s, 64); err == nil { - ts := d * float64(time.Second) - if ts > float64(math.MaxInt64) || ts < float64(math.MinInt64) { - return 0, errors.Errorf("cannot parse %q to a valid duration. It overflows int64", s) - } - return time.Duration(ts), nil - } - if d, err := model.ParseDuration(s); err == nil { - return time.Duration(d), nil - } - return 0, errors.Errorf("cannot parse %q to a valid duration", s) -} - func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { if data == nil { return queryrange.PrometheusResponse{ From 97174832ec5d0920932b18e581a8b17dba8f90e1 Mon Sep 17 00:00:00 2001 From: afayek Date: Fri, 28 Jul 2023 15:44:46 -0700 Subject: [PATCH 10/38] improve querier handler code readability Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 15 +++---- pkg/querier/handler/handler.go | 71 +++++++++++++++------------------- 2 files changed, 37 insertions(+), 49 deletions(-) diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index a669975df81..04c898aad66 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -3,7 +3,6 @@ package api import ( "context" "encoding/json" - "github.com/prometheus/prometheus/util/httputil" "html/template" "net/http" "path" @@ -246,10 +245,7 @@ func NewQuerierHandler( queryapi := qapi.NewAPI( engine, querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. - func(f http.HandlerFunc) http.HandlerFunc { return f }, logger, - false, - regexp.MustCompile(".*"), nil, ) @@ -278,8 +274,7 @@ func NewQuerierHandler( wrap := func(f qapi.ApiFunc) http.HandlerFunc { hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - httputil.SetCORS(w, queryapi.CORSOrigin, r) - result := qapi.SetUnavailStatusOnTSDBNotReady(f(r)) + result := f(r) if result.Finalizer != nil { defer result.Finalizer() } @@ -294,9 +289,9 @@ func NewQuerierHandler( } w.WriteHeader(http.StatusNoContent) }) - return queryapi.Ready(handler.CompressionHandler{ + return handler.CompressionHandler{ Handler: hf, - }.ServeHTTP) + }.ServeHTTP } // TODO(gotjosh): This custom handler is temporary until we're able to vendor the changes in: @@ -317,8 +312,8 @@ func NewQuerierHandler( router.Path(path.Join(legacyPrefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Methods("POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) router.Path(path.Join(legacyPrefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/labels")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(legacyPromRouter) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index c5efdf6d303..87d6b8bdc83 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -8,12 +8,13 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/protobuf/proto" - "github.com/grafana/regexp" jsoniter "github.com/json-iterator/go" "github.com/pkg/errors" + "github.com/prometheus/common/model" v1 "github.com/prometheus/prometheus/web/api/v1" "math" "net/http" + "strconv" "time" "github.com/cortexproject/cortex/pkg/cortexpb" @@ -23,7 +24,6 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/util/httputil" "github.com/prometheus/prometheus/util/stats" ) @@ -89,10 +89,7 @@ type API struct { Queryable storage.SampleAndChunkQueryable QueryEngine v1.QueryEngine Now func() time.Time - Ready func(http.HandlerFunc) http.HandlerFunc Logger log.Logger - CORSOrigin *regexp.Regexp - IsAgent bool StatsRenderer v1.StatsRenderer } @@ -100,20 +97,14 @@ type API struct { func NewAPI( qe v1.QueryEngine, q storage.SampleAndChunkQueryable, - readyFunc func(http.HandlerFunc) http.HandlerFunc, logger log.Logger, - isAgent bool, - corsOrigin *regexp.Regexp, statsRenderer v1.StatsRenderer, ) *API { a := &API{ QueryEngine: qe, Queryable: q, Now: time.Now, - Ready: readyFunc, Logger: logger, - CORSOrigin: corsOrigin, - IsAgent: isAgent, StatsRenderer: defaultStatsRenderer, } @@ -124,13 +115,6 @@ func NewAPI( return a } -func SetUnavailStatusOnTSDBNotReady(r ApiFuncResult) ApiFuncResult { - if r.Err != nil && errors.Cause(r.Err.err) == tsdb.ErrNotReady { - r.Err.typ = errorUnavailable - } - return r -} - type queryData struct { ResultType parser.ValueType `json:"resultType"` Result parser.Value `json:"result"` @@ -145,14 +129,14 @@ func invalidParamError(err error, parameter string) ApiFuncResult { func (api *API) Query(r *http.Request) (result ApiFuncResult) { tms, err := instantquery.ParseTimeParam(r, "time", api.Now().Unix()) - ts := time.Unix(tms/1000, (tms%1000)*10e6) + ts := time.Unix(tms/1000, (tms%1000)*10e6).UTC() if err != nil { return invalidParamError(err, "time") } ctx := r.Context() if to := r.FormValue("timeout"); to != "" { var cancel context.CancelFunc - timeout, err := time.ParseDuration(to) + timeout, err := parseDuration(to) if err != nil { return invalidParamError(err, "timeout") } @@ -216,12 +200,12 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { startMs, err := util.ParseTime(r.FormValue("start")) - start := time.Unix(startMs/1000, (startMs%1000)*10e6) + start := time.Unix(startMs/1000, (startMs%1000)*10e6).UTC() if err != nil { return invalidParamError(err, "start") } endMs, err := util.ParseTime(r.FormValue("end")) - end := time.Unix(endMs/1000, (endMs%1000)*10e6) + end := time.Unix(endMs/1000, (endMs%1000)*10e6).UTC() if err != nil { return invalidParamError(err, "end") } @@ -229,7 +213,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { return invalidParamError(errors.New("end timestamp must not be before start time"), "end") } - step, err := time.ParseDuration(r.FormValue("step")) + step, err := parseDuration(r.FormValue("step")) if err != nil { return invalidParamError(err, "step") } @@ -248,7 +232,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { ctx := r.Context() if to := r.FormValue("timeout"); to != "" { var cancel context.CancelFunc - timeout, err := time.ParseDuration(to) + timeout, err := parseDuration(to) if err != nil { return invalidParamError(err, "timeout") } @@ -295,6 +279,20 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { }), nil, res.Warnings, qry.Close} } +func parseDuration(s string) (time.Duration, error) { + if d, err := strconv.ParseFloat(s, 64); err == nil { + ts := d * float64(time.Second) + if ts > float64(math.MaxInt64) || ts < float64(math.MinInt64) { + return 0, errors.Errorf("cannot parse %q to a valid duration. It overflows int64", s) + } + return time.Duration(ts), nil + } + if d, err := model.ParseDuration(s); err == nil { + return time.Duration(d), nil + } + return 0, errors.Errorf("cannot parse %q to a valid duration", s) +} + func returnAPIError(err error) *apiError { if err == nil { return nil @@ -330,22 +328,17 @@ var ( ) func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { - statusMessage := statusSuccess var warningStrings []string for _, warning := range warnings { warningStrings = append(warningStrings, warning.Error()) } var b []byte var err error - switch data.(type) { + switch resp := data.(type) { case queryrange.PrometheusResponse: - prometheusResponse, _ := data.(queryrange.PrometheusResponse) - prometheusResponse.Status = string(statusMessage) - b, err = proto.Marshal(&prometheusResponse) + b, err = proto.Marshal(&resp) case instantquery.PrometheusInstantQueryResponse: - prometheusInstantQueryResponse, _ := data.(instantquery.PrometheusInstantQueryResponse) - prometheusInstantQueryResponse.Status = string(statusMessage) - b, err = proto.Marshal(&prometheusInstantQueryResponse) + b, err = proto.Marshal(&resp) default: level.Error(api.Logger).Log("msg", "error asserting response type") http.Error(w, "error asserting response type", http.StatusInternalServerError) @@ -406,7 +399,7 @@ func (api *API) RespondError(w http.ResponseWriter, apiErr *apiError, data inter func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { if data == nil { return queryrange.PrometheusResponse{ - Status: "", + Status: string(statusSuccess), Data: queryrange.PrometheusData{}, ErrorType: "", Error: "", @@ -423,7 +416,7 @@ func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { } return queryrange.PrometheusResponse{ - Status: "", + Status: string(statusSuccess), Data: queryrange.PrometheusData{ ResultType: string(data.ResultType), Result: *sampleStreams, @@ -438,7 +431,7 @@ func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { func createPrometheusInstantQueryResponse(data *queryData) instantquery.PrometheusInstantQueryResponse { if data == nil { return instantquery.PrometheusInstantQueryResponse{ - Status: "", + Status: string(statusSuccess), Data: instantquery.PrometheusInstantQueryData{}, ErrorType: "", Error: "", @@ -447,14 +440,14 @@ func createPrometheusInstantQueryResponse(data *queryData) instantquery.Promethe } var instantQueryResult instantquery.PrometheusInstantQueryResult - switch string(data.ResultType) { - case "matrix": + switch data.Result.Type() { + case parser.ValueTypeMatrix: instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ Matrix: &instantquery.Matrix{ SampleStreams: *getSampleStreams(data), }, } - case "vector": + case parser.ValueTypeVector: instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ Vector: &instantquery.Vector{ Samples: *getSamples(data), @@ -475,7 +468,7 @@ func createPrometheusInstantQueryResponse(data *queryData) instantquery.Promethe } return instantquery.PrometheusInstantQueryResponse{ - Status: "", + Status: string(statusSuccess), Data: instantquery.PrometheusInstantQueryData{ ResultType: string(data.ResultType), Result: instantQueryResult, From 581847aeed438228067be252bf5bd69cad2fc033 Mon Sep 17 00:00:00 2001 From: afayek Date: Wed, 2 Aug 2023 10:56:59 -0700 Subject: [PATCH 11/38] reuse thanos api struct definitions for querier handler Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 17 +- pkg/querier/handler/handler.go | 104 +++---- .../v2/interceptors/logging/doc.go | 36 +++ .../v2/interceptors/logging/interceptors.go | 135 +++++++++ .../v2/interceptors/logging/logging.go | 139 +++++++++ .../v2/interceptors/logging/options.go | 94 ++++++ .../v2/interceptors/logging/payload.go | 152 ++++++++++ .../thanos-io/thanos/pkg/api/api.go | 278 ++++++++++++++++++ .../thanos-io/thanos/pkg/logging/grpc.go | 207 +++++++++++++ .../thanos-io/thanos/pkg/logging/http.go | 195 ++++++++++++ .../thanos-io/thanos/pkg/logging/logger.go | 59 ++++ .../thanos-io/thanos/pkg/logging/options.go | 245 +++++++++++++++ .../thanos/pkg/logging/yaml_parser.go | 40 +++ .../thanos-io/thanos/pkg/prober/combiner.go | 56 ++++ .../thanos-io/thanos/pkg/prober/grpc.go | 47 +++ .../thanos-io/thanos/pkg/prober/http.go | 81 +++++ .../thanos/pkg/prober/intrumentation.go | 82 ++++++ .../thanos-io/thanos/pkg/prober/prober.go | 25 ++ .../thanos-io/thanos/pkg/server/http/http.go | 145 +++++++++ .../thanos/pkg/server/http/option.go | 63 ++++ .../thanos-io/thanos/pkg/server/http/utils.go | 32 ++ vendor/golang.org/x/net/http2/h2c/h2c.go | 240 +++++++++++++++ .../google.golang.org/grpc/health/client.go | 117 ++++++++ .../google.golang.org/grpc/health/logging.go | 23 ++ .../google.golang.org/grpc/health/server.go | 163 ++++++++++ vendor/modules.txt | 11 +- 26 files changed, 2711 insertions(+), 75 deletions(-) create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go create mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/api/api.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/http.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/logger.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/options.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/http.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/prober.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/http.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/option.go create mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go create mode 100644 vendor/golang.org/x/net/http2/h2c/h2c.go create mode 100644 vendor/google.golang.org/grpc/health/client.go create mode 100644 vendor/google.golang.org/grpc/health/logging.go create mode 100644 vendor/google.golang.org/grpc/health/server.go diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 04c898aad66..b76f5f48b20 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -3,6 +3,7 @@ package api import ( "context" "encoding/json" + thanos_api "github.com/thanos-io/thanos/pkg/api" "html/template" "net/http" "path" @@ -272,19 +273,19 @@ func NewQuerierHandler( legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) v1api.Register(legacyPromRouter) - wrap := func(f qapi.ApiFunc) http.HandlerFunc { + wrap := func(f thanos_api.ApiFunc) http.HandlerFunc { hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - result := f(r) - if result.Finalizer != nil { - defer result.Finalizer() + data, warnings, err, finalizer := f(r) + if finalizer != nil { + defer finalizer() } - if result.Err != nil { - queryapi.RespondError(w, result.Err, result.Data) + if err != nil { + queryapi.RespondError(w, err, data) return } - if result.Data != nil { - queryapi.Respond(w, result.Data, result.Warnings) + if data != nil { + queryapi.Respond(w, data, warnings) return } w.WriteHeader(http.StatusNoContent) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 87d6b8bdc83..1b512fcd4ad 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -26,6 +26,7 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/httputil" "github.com/prometheus/prometheus/util/stats" + thanos_api "github.com/thanos-io/thanos/pkg/api" ) type status string @@ -39,28 +40,10 @@ const ( statusClientClosedConnection = 499 ) -type errorType string - const ( - errorNone errorType = "" - errorTimeout errorType = "timeout" - errorCanceled errorType = "canceled" - errorExec errorType = "execution" - errorBadData errorType = "bad_data" - errorInternal errorType = "internal" - errorUnavailable errorType = "unavailable" - errorNotFound errorType = "not_found" + errorNotFound thanos_api.ErrorType = "not_found" ) -type apiError struct { - typ errorType - err error -} - -func (e *apiError) Error() string { - return fmt.Sprintf("%s: %s", e.typ, e.err) -} - func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) stats.QueryStats { if param != "" { return stats.NewQueryStats(s) @@ -69,22 +52,13 @@ func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) } type response struct { - Status status `json:"status"` - Data interface{} `json:"data,omitempty"` - ErrorType errorType `json:"errorType,omitempty"` - Error string `json:"error,omitempty"` - Warnings []string `json:"warnings,omitempty"` -} - -type ApiFuncResult struct { - Data interface{} - Err *apiError - Warnings storage.Warnings - Finalizer func() + Status status `json:"status"` + Data interface{} `json:"data,omitempty"` + ErrorType thanos_api.ErrorType `json:"errorType,omitempty"` + Error string `json:"error,omitempty"` + Warnings []string `json:"warnings,omitempty"` } -type ApiFunc func(r *http.Request) ApiFuncResult - type API struct { Queryable storage.SampleAndChunkQueryable QueryEngine v1.QueryEngine @@ -121,13 +95,13 @@ type queryData struct { Stats stats.QueryStats `json:"stats,omitempty"` } -func invalidParamError(err error, parameter string) ApiFuncResult { - return ApiFuncResult{nil, &apiError{ - errorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), - }, nil, nil} +func invalidParamError(err error, parameter string) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { + return nil, nil, &thanos_api.ApiError{ + thanos_api.ErrorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), + }, nil } -func (api *API) Query(r *http.Request) (result ApiFuncResult) { +func (api *API) Query(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { tms, err := instantquery.ParseTimeParam(r, "time", api.Now().Unix()) ts := time.Unix(tms/1000, (tms%1000)*10e6).UTC() if err != nil { @@ -147,7 +121,7 @@ func (api *API) Query(r *http.Request) (result ApiFuncResult) { opts, err := extractQueryOpts(r) if err != nil { - return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil } qry, err := api.QueryEngine.NewInstantQuery(ctx, api.Queryable, opts, r.FormValue("query"), ts) if err != nil { @@ -158,7 +132,7 @@ func (api *API) Query(r *http.Request) (result ApiFuncResult) { // be called by the caller) or call qry.Close ourselves (which is // required in the case of a panic). defer func() { - if result.Finalizer == nil { + if finalizer == nil { qry.Close() } }() @@ -167,7 +141,7 @@ func (api *API) Query(r *http.Request) (result ApiFuncResult) { res := qry.Exec(ctx) if res.Err != nil { - return ApiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + return nil, res.Warnings, returnAPIError(res.Err), qry.Close } // Optional stats field in response if parameter "stats" is not empty. @@ -177,11 +151,11 @@ func (api *API) Query(r *http.Request) (result ApiFuncResult) { } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return ApiFuncResult{createPrometheusInstantQueryResponse(&queryData{ + return createPrometheusInstantQueryResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }), nil, res.Warnings, qry.Close} + }), res.Warnings, nil, qry.Close } func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { @@ -198,7 +172,7 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { return opts, nil } -func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { +func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { startMs, err := util.ParseTime(r.FormValue("start")) start := time.Unix(startMs/1000, (startMs%1000)*10e6).UTC() if err != nil { @@ -226,7 +200,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { // This is sufficient for 60s resolution for a week or 1h resolution for a year. if end.Sub(start)/step > 11000 { err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") - return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil } ctx := r.Context() @@ -243,7 +217,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { opts, err := extractQueryOpts(r) if err != nil { - return ApiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil } qry, err := api.QueryEngine.NewRangeQuery(ctx, api.Queryable, opts, r.FormValue("query"), start, end, step) if err != nil { @@ -253,7 +227,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { // be called by the caller) or call qry.Close ourselves (which is // required in the case of a panic). defer func() { - if result.Finalizer == nil { + if finalizer == nil { qry.Close() } }() @@ -262,7 +236,7 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { res := qry.Exec(ctx) if res.Err != nil { - return ApiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} + return nil, res.Warnings, returnAPIError(res.Err), qry.Close } // Optional stats field in response if parameter "stats" is not empty. @@ -272,11 +246,11 @@ func (api *API) QueryRange(r *http.Request) (result ApiFuncResult) { } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return ApiFuncResult{createPrometheusResponse(&queryData{ + return createPrometheusResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }), nil, res.Warnings, qry.Close} + }), res.Warnings, nil, qry.Close } func parseDuration(s string) (time.Duration, error) { @@ -293,7 +267,7 @@ func parseDuration(s string) (time.Duration, error) { return 0, errors.Errorf("cannot parse %q to a valid duration", s) } -func returnAPIError(err error) *apiError { +func returnAPIError(err error) *thanos_api.ApiError { if err == nil { return nil } @@ -305,18 +279,18 @@ func returnAPIError(err error) *apiError { switch cause.(type) { case promql.ErrQueryCanceled: - return &apiError{errorCanceled, err} + return &thanos_api.ApiError{thanos_api.ErrorCanceled, err} case promql.ErrQueryTimeout: - return &apiError{errorTimeout, err} + return &thanos_api.ApiError{thanos_api.ErrorTimeout, err} case promql.ErrStorage: - return &apiError{errorInternal, err} + return &thanos_api.ApiError{thanos_api.ErrorInternal, err} } if errors.Is(err, context.Canceled) { - return &apiError{errorCanceled, err} + return &thanos_api.ApiError{thanos_api.ErrorCanceled, err} } - return &apiError{errorExec, err} + return &thanos_api.ApiError{thanos_api.ErrorExec, err} } var ( @@ -357,12 +331,12 @@ func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storag } } -func (api *API) RespondError(w http.ResponseWriter, apiErr *apiError, data interface{}) { +func (api *API) RespondError(w http.ResponseWriter, apiErr *thanos_api.ApiError, data interface{}) { json := jsoniter.ConfigCompatibleWithStandardLibrary b, err := json.Marshal(&response{ Status: statusError, - ErrorType: apiErr.typ, - Error: apiErr.err.Error(), + ErrorType: apiErr.Typ, + Error: apiErr.Err.Error(), Data: data, }) if err != nil { @@ -372,16 +346,16 @@ func (api *API) RespondError(w http.ResponseWriter, apiErr *apiError, data inter } var code int - switch apiErr.typ { - case errorBadData: + switch apiErr.Typ { + case thanos_api.ErrorBadData: code = http.StatusBadRequest - case errorExec: + case thanos_api.ErrorExec: code = http.StatusUnprocessableEntity - case errorCanceled: + case thanos_api.ErrorCanceled: code = statusClientClosedConnection - case errorTimeout: + case thanos_api.ErrorTimeout: code = http.StatusServiceUnavailable - case errorInternal: + case thanos_api.ErrorInternal: code = http.StatusInternalServerError case errorNotFound: code = http.StatusNotFound diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go new file mode 100644 index 00000000000..378baed7766 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go @@ -0,0 +1,36 @@ +// Copyright 2017 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +// +/* +logging is a "parent" package for gRPC logging middlewares. + +The gRPC logging middleware populates request-scoped data to `grpc_ctxtags.Tags` that relate to the current gRPC call +(e.g. service and method names). + +Once the gRPC logging middleware has added the gRPC specific Tags to the ctx they will then be written with the logs +that are made using the `ctx_logrus` or `ctx_zap` loggers. + +All logging middleware will emit a final log statement. It is based on the error returned by the handler function, +the gRPC status code, an error (if any) and it emit at a level controlled via `WithLevels`. + +This parent package + +This particular package is intended for use by other middleware, logging or otherwise. It contains interfaces that other +logging middlewares *could* share . This allows code to be shared between different implementations. + +Field names + +All field names of loggers follow the OpenTracing semantics definitions, with `grpc.` prefix if needed: +https://github.com/opentracing/specification/blob/master/semantic_conventions.md + +Implementations: + +* providers/logrus +* providers/zap +* providers/kit +* providers/zerolog + +See relevant packages below. +*/ +package logging diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go new file mode 100644 index 00000000000..654223f2afd --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go @@ -0,0 +1,135 @@ +package logging + +import ( + "context" + "fmt" + "io" + "time" + + "google.golang.org/grpc" + + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" +) + +// extractFields returns all fields from tags. +func extractFields(tags tags.Tags) Fields { + var fields Fields + for k, v := range tags.Values() { + fields = append(fields, k, v) + } + return fields +} + +type reporter struct { + ctx context.Context + typ interceptors.GRPCType + service, method string + startCallLogged bool + opts *options + logger Logger + kind string +} + +func (c *reporter) logMessage(logger Logger, err error, msg string, duration time.Duration) { + code := c.opts.codeFunc(err) + logger = logger.With("grpc.code", code.String()) + if err != nil { + logger = logger.With("grpc.error", fmt.Sprintf("%v", err)) + } + logger = logger.With(extractFields(tags.Extract(c.ctx))...) + logger.With(c.opts.durationFieldFunc(duration)...).Log(c.opts.levelFunc(code), msg) +} + +func (c *reporter) PostCall(err error, duration time.Duration) { + switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { + case LogFinishCall, LogStartAndFinishCall: + if err == io.EOF { + err = nil + } + c.logMessage(c.logger, err, "finished call", duration) + default: + return + } +} + +func (c *reporter) PostMsgSend(_ interface{}, err error, duration time.Duration) { + if c.startCallLogged { + return + } + switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { + case LogStartAndFinishCall: + c.startCallLogged = true + c.logMessage(c.logger, err, "started call", duration) + } +} + +func (c *reporter) PostMsgReceive(_ interface{}, err error, duration time.Duration) { + if c.startCallLogged { + return + } + switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { + case LogStartAndFinishCall: + c.startCallLogged = true + c.logMessage(c.logger, err, "started call", duration) + } +} + +type reportable struct { + opts *options + logger Logger +} + +func (r *reportable) ServerReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { + return r.reporter(ctx, typ, service, method, KindServerFieldValue) +} + +func (r *reportable) ClientReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { + return r.reporter(ctx, typ, service, method, KindClientFieldValue) +} + +func (r *reportable) reporter(ctx context.Context, typ interceptors.GRPCType, service string, method string, kind string) (interceptors.Reporter, context.Context) { + fields := commonFields(kind, typ, service, method) + fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) + if d, ok := ctx.Deadline(); ok { + fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) + } + return &reporter{ + ctx: ctx, + typ: typ, + service: service, + method: method, + startCallLogged: false, + opts: r.opts, + logger: r.logger.With(fields...), + kind: kind, + }, ctx +} + +// UnaryClientInterceptor returns a new unary client interceptor that optionally logs the execution of external gRPC calls. +// Logger will use all tags (from tags package) available in current context as fields. +func UnaryClientInterceptor(logger Logger, opts ...Option) grpc.UnaryClientInterceptor { + o := evaluateClientOpt(opts) + return interceptors.UnaryClientInterceptor(&reportable{logger: logger, opts: o}) +} + +// StreamClientInterceptor returns a new streaming client interceptor that optionally logs the execution of external gRPC calls. +// Logger will use all tags (from tags package) available in current context as fields. +func StreamClientInterceptor(logger Logger, opts ...Option) grpc.StreamClientInterceptor { + o := evaluateClientOpt(opts) + return interceptors.StreamClientInterceptor(&reportable{logger: logger, opts: o}) +} + +// UnaryServerInterceptor returns a new unary server interceptors that optionally logs endpoint handling. +// Logger will use all tags (from tags package) available in current context as fields. +func UnaryServerInterceptor(logger Logger, opts ...Option) grpc.UnaryServerInterceptor { + o := evaluateServerOpt(opts) + return interceptors.UnaryServerInterceptor(&reportable{logger: logger, opts: o}) +} + +// StreamServerInterceptor returns a new stream server interceptors that optionally logs endpoint handling. +// Logger will use all tags (from tags package) available in current context as fields. +func StreamServerInterceptor(logger Logger, opts ...Option) grpc.StreamServerInterceptor { + o := evaluateServerOpt(opts) + return interceptors.StreamServerInterceptor(&reportable{logger: logger, opts: o}) +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go new file mode 100644 index 00000000000..67d6861bda1 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go @@ -0,0 +1,139 @@ +// Copyright 2017 Michal Witkowski. All Rights Reserved. +// See LICENSE for licensing terms. + +package logging + +import ( + "context" + "io" + + "github.com/golang/protobuf/proto" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" +) + +// Decision defines rules for enabling start and end of logging. +type Decision int + +const ( + // NoLogCall - Logging is disabled. + NoLogCall Decision = iota + // LogFinishCall - Only finish logs of request is enabled. + LogFinishCall + // LogStartAndFinishCall - Logging of start and end of request is enabled. + LogStartAndFinishCall +) + +var ( + // SystemTag is tag representing an event inside gRPC call. + SystemTag = []string{"protocol", "grpc"} + // ComponentFieldKey is a tag representing the client/server that is calling. + ComponentFieldKey = "grpc.component" + KindServerFieldValue = "server" + KindClientFieldValue = "client" + ServiceFieldKey = "grpc.service" + MethodFieldKey = "grpc.method" + MethodTypeFieldKey = "grpc.method_type" +) + +func commonFields(kind string, typ interceptors.GRPCType, service string, method string) Fields { + return Fields{ + SystemTag[0], SystemTag[1], + ComponentFieldKey, kind, + ServiceFieldKey, service, + MethodFieldKey, method, + MethodTypeFieldKey, string(typ), + } +} + +// Fields represents logging fields. It has to have even number of elements (pairs). +type Fields []string + +// ErrorToCode function determines the error code of an error +// This makes using custom errors with grpc middleware easier +type ErrorToCode func(err error) codes.Code + +func DefaultErrorToCode(err error) codes.Code { + return status.Code(err) +} + +// Decider function defines rules for suppressing any interceptor logs +type Decider func(fullMethodName string, err error) Decision + +// DefaultDeciderMethod is the default implementation of decider to see if you should log the call +// by default this if always true so all calls are logged +func DefaultDeciderMethod(_ string, _ error) Decision { + return LogStartAndFinishCall +} + +// ServerPayloadLoggingDecider is a user-provided function for deciding whether to log the server-side +// request/response payloads +type ServerPayloadLoggingDecider func(ctx context.Context, fullMethodName string, servingObject interface{}) bool + +// ClientPayloadLoggingDecider is a user-provided function for deciding whether to log the client-side +// request/response payloads +type ClientPayloadLoggingDecider func(ctx context.Context, fullMethodName string) bool + +// JsonPbMarshaller is a marshaller that serializes protobuf messages. +type JsonPbMarshaler interface { + Marshal(out io.Writer, pb proto.Message) error +} + +// Logger is unified interface that we used for all our interceptors. Official implementations are available under +// provider/ directory as separate modules. +type Logger interface { + // Log logs the fields for given log level. We can assume users (middleware library) will put fields in pairs and + // those will be unique. + Log(Level, string) + // With returns mockLogger with given fields appended. We can assume users (middleware library) will put fields in pairs + // and those will be unique. + With(fields ...string) Logger +} + +// Level represents logging level. +type Level string + +const ( + DEBUG = Level("debug") + INFO = Level("info") + WARNING = Level("warning") + ERROR = Level("error") +) + +// CodeToLevel function defines the mapping between gRPC return codes and interceptor log level. +type CodeToLevel func(code codes.Code) Level + +// DefaultServerCodeToLevel is the helper mapper that maps gRPC return codes to log levels for server side. +func DefaultServerCodeToLevel(code codes.Code) Level { + switch code { + case codes.OK, codes.NotFound, codes.Canceled, codes.AlreadyExists, codes.InvalidArgument, codes.Unauthenticated: + return INFO + + case codes.DeadlineExceeded, codes.PermissionDenied, codes.ResourceExhausted, codes.FailedPrecondition, codes.Aborted, + codes.OutOfRange, codes.Unavailable: + return WARNING + + case codes.Unknown, codes.Unimplemented, codes.Internal, codes.DataLoss: + return ERROR + + default: + return ERROR + } +} + +// DefaultClientCodeToLevel is the helper mapper that maps gRPC return codes to log levels for client side. +func DefaultClientCodeToLevel(code codes.Code) Level { + switch code { + case codes.OK, codes.Canceled, codes.InvalidArgument, codes.NotFound, codes.AlreadyExists, codes.ResourceExhausted, + codes.FailedPrecondition, codes.Aborted, codes.OutOfRange: + return DEBUG + case codes.Unknown, codes.DeadlineExceeded, codes.PermissionDenied, codes.Unauthenticated: + return INFO + case codes.Unimplemented, codes.Internal, codes.Unavailable, codes.DataLoss: + return WARNING + default: + return INFO + } +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go new file mode 100644 index 00000000000..b1aa328cf25 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go @@ -0,0 +1,94 @@ +package logging + +import ( + "fmt" + "time" +) + +var ( + defaultOptions = &options{ + shouldLog: DefaultDeciderMethod, + codeFunc: DefaultErrorToCode, + durationFieldFunc: DefaultDurationToFields, + // levelFunc depends if it's client or server. + levelFunc: nil, + } +) + +type options struct { + levelFunc CodeToLevel + shouldLog Decider + codeFunc ErrorToCode + durationFieldFunc DurationToFields +} + +type Option func(*options) + +// DurationToFields function defines how to produce duration fields for logging. +type DurationToFields func(duration time.Duration) Fields + +func evaluateServerOpt(opts []Option) *options { + optCopy := &options{} + *optCopy = *defaultOptions + optCopy.levelFunc = DefaultServerCodeToLevel + for _, o := range opts { + o(optCopy) + } + return optCopy +} + +func evaluateClientOpt(opts []Option) *options { + optCopy := &options{} + *optCopy = *defaultOptions + optCopy.levelFunc = DefaultClientCodeToLevel + for _, o := range opts { + o(optCopy) + } + return optCopy +} + +// WithDecider customizes the function for deciding if the gRPC interceptor logs should log. +func WithDecider(f Decider) Option { + return func(o *options) { + o.shouldLog = f + } +} + +// WithLevels customizes the function for mapping gRPC return codes and interceptor log level statements. +func WithLevels(f CodeToLevel) Option { + return func(o *options) { + o.levelFunc = f + } +} + +// WithCodes customizes the function for mapping errors to error codes. +func WithCodes(f ErrorToCode) Option { + return func(o *options) { + o.codeFunc = f + } +} + +// WithDurationField customizes the function for mapping request durations to log fields. +func WithDurationField(f DurationToFields) Option { + return func(o *options) { + o.durationFieldFunc = f + } +} + +// DefaultDurationToFields is the default implementation of converting request duration to a field. +var DefaultDurationToFields = DurationToTimeMillisFields + +// DurationToTimeMillisFields converts the duration to milliseconds and uses the key `grpc.time_ms`. +func DurationToTimeMillisFields(duration time.Duration) Fields { + return Fields{"grpc.time_ms", fmt.Sprintf("%v", durationToMilliseconds(duration))} +} + +// DurationToDurationField uses a Duration field to log the request duration +// and leaves it up to Log's encoder settings to determine how that is output. +func DurationToDurationField(duration time.Duration) Fields { + return Fields{"grpc.duration", duration.String()} +} + +func durationToMilliseconds(duration time.Duration) float32 { + return float32(duration.Nanoseconds()/1000) / 1000 +} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go new file mode 100644 index 00000000000..ea5e2ceecc0 --- /dev/null +++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go @@ -0,0 +1,152 @@ +package logging + +import ( + "bytes" + "context" + "fmt" + "time" + + "github.com/golang/protobuf/jsonpb" + "github.com/golang/protobuf/proto" + "google.golang.org/grpc" + + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" +) + +var ( + // JsonPbMarshaller is the marshaller used for serializing protobuf messages. + // If needed, this variable can be reassigned with a different marshaller with the same Marshal() signature. + JsonPbMarshaller JsonPbMarshaler = &jsonpb.Marshaler{} +) + +type serverPayloadReporter struct { + ctx context.Context + logger Logger +} + +func (c *serverPayloadReporter) PostCall(error, time.Duration) {} + +func (c *serverPayloadReporter) PostMsgSend(req interface{}, err error, duration time.Duration) { + if err != nil { + return + } + logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) + // For server send message is the response. + logProtoMessageAsJson(logger.With("grpc.send.duration", duration.String()), req, "grpc.response.content", "response payload logged as grpc.response.content field") +} + +func (c *serverPayloadReporter) PostMsgReceive(reply interface{}, err error, duration time.Duration) { + if err != nil { + return + } + logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) + // For server recv message is the request. + logProtoMessageAsJson(logger.With("grpc.recv.duration", duration.String()), reply, "grpc.request.content", "request payload logged as grpc.request.content field") +} + +type clientPayloadReporter struct { + ctx context.Context + logger Logger +} + +func (c *clientPayloadReporter) PostCall(error, time.Duration) {} + +func (c *clientPayloadReporter) PostMsgSend(req interface{}, err error, duration time.Duration) { + if err != nil { + return + } + logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) + logProtoMessageAsJson(logger.With("grpc.send.duration", duration.String()), req, "grpc.request.content", "request payload logged as grpc.request.content field") +} + +func (c *clientPayloadReporter) PostMsgReceive(reply interface{}, err error, duration time.Duration) { + if err != nil { + return + } + logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) + logProtoMessageAsJson(logger.With("grpc.recv.duration", duration.String()), reply, "grpc.response.content", "response payload logged as grpc.response.content field") +} + +type payloadReportable struct { + clientDecider ClientPayloadLoggingDecider + serverDecider ServerPayloadLoggingDecider + logger Logger +} + +func (r *payloadReportable) ServerReporter(ctx context.Context, req interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { + if !r.serverDecider(ctx, interceptors.FullMethod(service, method), req) { + return interceptors.NoopReporter{}, ctx + } + fields := commonFields(KindServerFieldValue, typ, service, method) + fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) + if d, ok := ctx.Deadline(); ok { + fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) + } + return &serverPayloadReporter{ + ctx: ctx, + logger: r.logger.With(fields...), + }, ctx +} +func (r *payloadReportable) ClientReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { + if !r.clientDecider(ctx, interceptors.FullMethod(service, method)) { + return interceptors.NoopReporter{}, ctx + } + fields := commonFields(KindClientFieldValue, typ, service, method) + fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) + if d, ok := ctx.Deadline(); ok { + fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) + } + return &clientPayloadReporter{ + ctx: ctx, + logger: r.logger.With(fields...), + }, ctx +} + +// PayloadUnaryServerInterceptor returns a new unary server interceptors that logs the payloads of requests on INFO level. +// Logger tags will be used from tags context. +func PayloadUnaryServerInterceptor(logger Logger, decider ServerPayloadLoggingDecider) grpc.UnaryServerInterceptor { + return interceptors.UnaryServerInterceptor(&payloadReportable{logger: logger, serverDecider: decider}) +} + +// PayloadStreamServerInterceptor returns a new server server interceptors that logs the payloads of requests on INFO level. +// Logger tags will be used from tags context. +func PayloadStreamServerInterceptor(logger Logger, decider ServerPayloadLoggingDecider) grpc.StreamServerInterceptor { + return interceptors.StreamServerInterceptor(&payloadReportable{logger: logger, serverDecider: decider}) +} + +// PayloadUnaryClientInterceptor returns a new unary client interceptor that logs the paylods of requests and responses on INFO level. +// Logger tags will be used from tags context. +func PayloadUnaryClientInterceptor(logger Logger, decider ClientPayloadLoggingDecider) grpc.UnaryClientInterceptor { + return interceptors.UnaryClientInterceptor(&payloadReportable{logger: logger, clientDecider: decider}) +} + +// PayloadStreamClientInterceptor returns a new streaming client interceptor that logs the paylods of requests and responses on INFO level. +// Logger tags will be used from tags context. +func PayloadStreamClientInterceptor(logger Logger, decider ClientPayloadLoggingDecider) grpc.StreamClientInterceptor { + return interceptors.StreamClientInterceptor(&payloadReportable{logger: logger, clientDecider: decider}) +} + +func logProtoMessageAsJson(logger Logger, pbMsg interface{}, key string, msg string) { + if p, ok := pbMsg.(proto.Message); ok { + payload, err := (&jsonpbObjectMarshaler{pb: p}).marshalJSON() + if err != nil { + logger = logger.With(key, err.Error()) + } else { + logger = logger.With(key, string(payload)) + } + logger.Log(INFO, msg) + } +} + +type jsonpbObjectMarshaler struct { + pb proto.Message +} + +func (j *jsonpbObjectMarshaler) marshalJSON() ([]byte, error) { + b := &bytes.Buffer{} + if err := JsonPbMarshaller.Marshal(b, j.pb); err != nil { + return nil, fmt.Errorf("jsonpb serializer failed: %v", err) + } + return b.Bytes(), nil +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/api/api.go b/vendor/github.com/thanos-io/thanos/pkg/api/api.go new file mode 100644 index 00000000000..6be81c8b158 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/api/api.go @@ -0,0 +1,278 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2016 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// This package is a modified copy from +// github.com/prometheus/prometheus/web/api/v1@2121b4628baa7d9d9406aa468712a6a332e77aff. + +package api + +import ( + "encoding/json" + "fmt" + "net/http" + "os" + "runtime" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/klauspost/compress/gzhttp" + "github.com/opentracing/opentracing-go" + "github.com/prometheus/common/route" + "github.com/prometheus/common/version" + + extpromhttp "github.com/thanos-io/thanos/pkg/extprom/http" + "github.com/thanos-io/thanos/pkg/logging" + "github.com/thanos-io/thanos/pkg/server/http/middleware" + "github.com/thanos-io/thanos/pkg/tracing" +) + +type status string + +const ( + StatusSuccess status = "success" + StatusError status = "error" +) + +type ErrorType string + +const ( + ErrorNone ErrorType = "" + ErrorTimeout ErrorType = "timeout" + ErrorCanceled ErrorType = "canceled" + ErrorExec ErrorType = "execution" + ErrorBadData ErrorType = "bad_data" + ErrorInternal ErrorType = "internal" +) + +var corsHeaders = map[string]string{ + "Access-Control-Allow-Headers": "Accept, Accept-Encoding, Authorization, Content-Type, Origin", + "Access-Control-Allow-Methods": "GET, OPTIONS", + "Access-Control-Allow-Origin": "*", + "Access-Control-Expose-Headers": "Date", +} + +// ThanosVersion contains build information about Thanos. +type ThanosVersion struct { + Version string `json:"version"` + Revision string `json:"revision"` + Branch string `json:"branch"` + BuildUser string `json:"buildUser"` + BuildDate string `json:"buildDate"` + GoVersion string `json:"goVersion"` +} + +var BuildInfo = &ThanosVersion{ + Version: version.Version, + Revision: version.Revision, + Branch: version.Branch, + BuildUser: version.BuildUser, + BuildDate: version.BuildDate, + GoVersion: version.GoVersion, +} + +type ApiError struct { + Typ ErrorType + Err error +} + +func (e *ApiError) Error() string { + return fmt.Sprintf("%s: %s", e.Typ, e.Err) +} + +// RuntimeInfo contains runtime information about Thanos. +type RuntimeInfo struct { + StartTime time.Time `json:"startTime"` + CWD string `json:"CWD"` + GoroutineCount int `json:"goroutineCount"` + GOMAXPROCS int `json:"GOMAXPROCS"` + GOGC string `json:"GOGC"` + GODEBUG string `json:"GODEBUG"` +} + +// RuntimeInfoFn returns updated runtime information about Thanos. +type RuntimeInfoFn func() RuntimeInfo + +type response struct { + Status status `json:"status"` + Data interface{} `json:"data,omitempty"` + ErrorType ErrorType `json:"errorType,omitempty"` + Error string `json:"error,omitempty"` + Warnings []string `json:"warnings,omitempty"` +} + +// SetCORS enables cross-site script calls. +func SetCORS(w http.ResponseWriter) { + for h, v := range corsHeaders { + w.Header().Set(h, v) + } +} + +type ApiFunc func(r *http.Request) (interface{}, []error, *ApiError, func()) + +type BaseAPI struct { + logger log.Logger + flagsMap map[string]string + runtimeInfo RuntimeInfoFn + buildInfo *ThanosVersion + Now func() time.Time + disableCORS bool +} + +// NewBaseAPI returns a new initialized BaseAPI type. +func NewBaseAPI(logger log.Logger, disableCORS bool, flagsMap map[string]string) *BaseAPI { + + return &BaseAPI{ + logger: logger, + flagsMap: flagsMap, + runtimeInfo: GetRuntimeInfoFunc(logger), + buildInfo: BuildInfo, + disableCORS: disableCORS, + Now: time.Now, + } +} + +// Register registers the common API endpoints. +func (api *BaseAPI) Register(r *route.Router, tracer opentracing.Tracer, logger log.Logger, ins extpromhttp.InstrumentationMiddleware, logMiddleware *logging.HTTPServerMiddleware) { + instr := GetInstr(tracer, logger, ins, logMiddleware, api.disableCORS) + + r.Options("/*path", instr("options", api.options)) + + r.Get("/status/flags", instr("status_flags", api.flags)) + r.Get("/status/runtimeinfo", instr("status_runtime", api.serveRuntimeInfo)) + r.Get("/status/buildinfo", instr("status_build", api.serveBuildInfo)) +} + +func (api *BaseAPI) options(r *http.Request) (interface{}, []error, *ApiError, func()) { + return nil, nil, nil, func() {} +} + +func (api *BaseAPI) flags(r *http.Request) (interface{}, []error, *ApiError, func()) { + return api.flagsMap, nil, nil, func() {} +} + +func (api *BaseAPI) serveRuntimeInfo(r *http.Request) (interface{}, []error, *ApiError, func()) { + return api.runtimeInfo(), nil, nil, func() {} +} + +func (api *BaseAPI) serveBuildInfo(r *http.Request) (interface{}, []error, *ApiError, func()) { + return api.buildInfo, nil, nil, func() {} +} + +func GetRuntimeInfoFunc(logger log.Logger) RuntimeInfoFn { + CWD, err := os.Getwd() + if err != nil { + CWD = "" + level.Warn(logger).Log("msg", "failed to retrieve current working directory", "err", err) + } + + birth := time.Now() + + return func() RuntimeInfo { + return RuntimeInfo{ + StartTime: birth, + CWD: CWD, + GoroutineCount: runtime.NumGoroutine(), + GOMAXPROCS: runtime.GOMAXPROCS(0), + GOGC: os.Getenv("GOGC"), + GODEBUG: os.Getenv("GODEBUG"), + } + } +} + +type InstrFunc func(name string, f ApiFunc) http.HandlerFunc + +// GetInstr returns a http HandlerFunc with the instrumentation middleware. +func GetInstr( + tracer opentracing.Tracer, + logger log.Logger, + ins extpromhttp.InstrumentationMiddleware, + logMiddleware *logging.HTTPServerMiddleware, + disableCORS bool, +) InstrFunc { + instr := func(name string, f ApiFunc) http.HandlerFunc { + hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if !disableCORS { + SetCORS(w) + } + if data, warnings, err, releaseResources := f(r); err != nil { + RespondError(w, err, data) + releaseResources() + } else if data != nil { + Respond(w, data, warnings) + releaseResources() + } else { + w.WriteHeader(http.StatusNoContent) + releaseResources() + } + }) + + return tracing.HTTPMiddleware(tracer, name, logger, + ins.NewHandler(name, + gzhttp.GzipHandler( + middleware.RequestID( + logMiddleware.HTTPMiddleware(name, hf), + ), + ), + ), + ) + } + return instr +} + +func Respond(w http.ResponseWriter, data interface{}, warnings []error) { + w.Header().Set("Content-Type", "application/json") + if len(warnings) > 0 { + w.Header().Set("Cache-Control", "no-store") + } + w.WriteHeader(http.StatusOK) + + resp := &response{ + Status: StatusSuccess, + Data: data, + } + for _, warn := range warnings { + resp.Warnings = append(resp.Warnings, warn.Error()) + } + _ = json.NewEncoder(w).Encode(resp) +} + +func RespondError(w http.ResponseWriter, apiErr *ApiError, data interface{}) { + w.Header().Set("Content-Type", "application/json") + w.Header().Set("Cache-Control", "no-store") + + var code int + switch apiErr.Typ { + case ErrorBadData: + code = http.StatusBadRequest + case ErrorExec: + code = 422 + case ErrorCanceled, ErrorTimeout: + code = http.StatusServiceUnavailable + case ErrorInternal: + code = http.StatusInternalServerError + default: + code = http.StatusInternalServerError + } + w.WriteHeader(code) + + _ = json.NewEncoder(w).Encode(&response{ + Status: StatusError, + ErrorType: apiErr.Typ, + Error: apiErr.Err.Error(), + Data: data, + }) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go b/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go new file mode 100644 index 00000000000..44387d395f4 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go @@ -0,0 +1,207 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package logging + +import ( + "fmt" + "math/rand" + "sort" + "strings" + "time" + + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" + grpc_logging "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging" + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" + "github.com/oklog/ulid" + "google.golang.org/grpc/status" + "gopkg.in/yaml.v2" +) + +// NewRequestConfig parses the string into a req logging config structure. +// Raise an error if unmarshalling is not possible, or values are not valid. +func NewRequestConfig(configYAML []byte) (*RequestConfig, error) { + reqLogConfig := &RequestConfig{} + if err := yaml.UnmarshalStrict(configYAML, reqLogConfig); err != nil { + return nil, err + } + return reqLogConfig, nil +} + +// checkOptionsConfigEmpty checks if the OptionsConfig struct is empty and valid. +// If invalid combination is present, return an error. +func checkOptionsConfigEmpty(optcfg OptionsConfig) (bool, error) { + if optcfg.Level == "" && !optcfg.Decision.LogEnd && !optcfg.Decision.LogStart { + return true, nil + } + if optcfg.Level == "" && (optcfg.Decision.LogStart || optcfg.Decision.LogEnd) { + return false, fmt.Errorf("level field is empty") + } + return false, nil +} + +// fillGlobalOptionConfig configures all the method to have global config for logging. +func fillGlobalOptionConfig(reqLogConfig *RequestConfig, isgRPC bool) (string, bool, bool, error) { + globalLevel := "ERROR" + globalStart, globalEnd := false, false + + globalOptionConfig := reqLogConfig.Options + isEmpty, err := checkOptionsConfigEmpty(globalOptionConfig) + + // If the decision for logging is enabled with empty level field. + if err != nil { + return "", false, false, err + } + if !isEmpty { + globalLevel = globalOptionConfig.Level + globalStart = globalOptionConfig.Decision.LogStart + globalEnd = globalOptionConfig.Decision.LogEnd + } + + protocolOptionConfig := reqLogConfig.HTTP.Options + if isgRPC { + // gRPC config overrides the global config. + protocolOptionConfig = reqLogConfig.GRPC.Options + } + + isEmpty, err = checkOptionsConfigEmpty(protocolOptionConfig) + // If the decision for logging is enabled with empty level field. + if err != nil { + return "", false, false, err + } + + if !isEmpty { + globalLevel = protocolOptionConfig.Level + globalStart = protocolOptionConfig.Decision.LogStart + globalEnd = protocolOptionConfig.Decision.LogEnd + } + return globalLevel, globalStart, globalEnd, nil +} + +// getGRPCLoggingOption returns the logging ENUM based on logStart and logEnd values. +func getGRPCLoggingOption(logStart, logEnd bool) (grpc_logging.Decision, error) { + if !logStart && !logEnd { + return grpc_logging.NoLogCall, nil + } + if !logStart && logEnd { + return grpc_logging.LogFinishCall, nil + } + if logStart && logEnd { + return grpc_logging.LogStartAndFinishCall, nil + } + return -1, fmt.Errorf("log start call is not supported") +} + +// validateLevel validates the list of level entries. +// Raise an error if empty or log level not in uppercase. +func validateLevel(level string) error { + if level == "" { + return fmt.Errorf("level field in YAML file is empty") + } + if level == "INFO" || level == "DEBUG" || level == "ERROR" || level == "WARNING" { + return nil + } + return fmt.Errorf("the format of level is invalid. Expected INFO/DEBUG/ERROR/WARNING, got this %v", level) +} + +// NewGRPCOption adds in the config options and returns tags for logging middleware. +func NewGRPCOption(configYAML []byte) ([]tags.Option, []grpc_logging.Option, error) { + + // Configure tagOpts and logOpts. + tagOpts := []tags.Option{ + tags.WithFieldExtractor(func(_ string, req interface{}) map[string]string { + tagMap := tags.TagBasedRequestFieldExtractor("request-id")("", req) + // If a request-id exists for a given request. + if tagMap != nil { + if _, ok := tagMap["request-id"]; ok { + return tagMap + } + } + entropy := ulid.Monotonic(rand.New(rand.NewSource(time.Now().UnixNano())), 0) + reqID := ulid.MustNew(ulid.Timestamp(time.Now()), entropy) + tagMap = make(map[string]string) + tagMap["request-id"] = reqID.String() + return tagMap + }), + } + logOpts := []grpc_logging.Option{ + grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { + return grpc_logging.NoLogCall + }), + grpc_logging.WithLevels(DefaultCodeToLevelGRPC), + } + + // Unmarshal YAML. + // if req logging is disabled. + if len(configYAML) == 0 { + return tagOpts, logOpts, nil + } + + reqLogConfig, err := NewRequestConfig(configYAML) + // If unmarshalling is an issue. + if err != nil { + return tagOpts, logOpts, err + } + + globalLevel, globalStart, globalEnd, err := fillGlobalOptionConfig(reqLogConfig, true) + // If global options have invalid entries. + if err != nil { + return tagOpts, logOpts, err + } + + // If the level entry does not matches our entries. + if err := validateLevel(globalLevel); err != nil { + return tagOpts, logOpts, err + } + + // If the combination is valid, use them, otherwise return error. + reqLogDecision, err := getGRPCLoggingOption(globalStart, globalEnd) + if err != nil { + return tagOpts, logOpts, err + } + + if len(reqLogConfig.GRPC.Config) == 0 { + logOpts = []grpc_logging.Option{ + grpc_logging.WithDecider(func(_ string, err error) grpc_logging.Decision { + + runtimeLevel := grpc_logging.DefaultServerCodeToLevel(status.Code(err)) + for _, lvl := range MapAllowedLevels[globalLevel] { + if string(runtimeLevel) == strings.ToLower(lvl) { + return reqLogDecision + } + } + return grpc_logging.NoLogCall + }), + grpc_logging.WithLevels(DefaultCodeToLevelGRPC), + } + return tagOpts, logOpts, nil + } + + logOpts = []grpc_logging.Option{ + grpc_logging.WithLevels(DefaultCodeToLevelGRPC), + } + + methodNameSlice := []string{} + + for _, eachConfig := range reqLogConfig.GRPC.Config { + eachConfigMethodName := interceptors.FullMethod(eachConfig.Service, eachConfig.Method) + methodNameSlice = append(methodNameSlice, eachConfigMethodName) + } + + logOpts = append(logOpts, []grpc_logging.Option{ + grpc_logging.WithDecider(func(runtimeMethodName string, err error) grpc_logging.Decision { + + idx := sort.SearchStrings(methodNameSlice, runtimeMethodName) + if idx < len(methodNameSlice) && methodNameSlice[idx] == runtimeMethodName { + runtimeLevel := grpc_logging.DefaultServerCodeToLevel(status.Code(err)) + for _, lvl := range MapAllowedLevels[globalLevel] { + if string(runtimeLevel) == strings.ToLower(lvl) { + return reqLogDecision + } + } + } + return grpc_logging.NoLogCall + }), + }...) + return tagOpts, logOpts, nil +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/http.go b/vendor/github.com/thanos-io/thanos/pkg/logging/http.go new file mode 100644 index 00000000000..36c4395f763 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/logging/http.go @@ -0,0 +1,195 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package logging + +import ( + "fmt" + "net" + "sort" + "strings" + + "net/http" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + + httputil "github.com/thanos-io/thanos/pkg/server/http" +) + +type HTTPServerMiddleware struct { + opts *options + logger log.Logger +} + +func (m *HTTPServerMiddleware) preCall(name string, start time.Time, r *http.Request) { + logger := m.opts.filterLog(m.logger) + level.Debug(logger).Log("http.start_time", start.String(), "http.method", fmt.Sprintf("%s %s", r.Method, r.URL), "http.request_id", r.Header.Get("X-Request-ID"), "thanos.method_name", name, "msg", "started call") +} + +func (m *HTTPServerMiddleware) postCall(name string, start time.Time, wrapped *httputil.ResponseWriterWithStatus, r *http.Request) { + status := wrapped.Status() + logger := log.With(m.logger, "http.method", fmt.Sprintf("%s %s", r.Method, r.URL), "http.request_id", r.Header.Get("X-Request-ID"), "http.status_code", fmt.Sprintf("%d", status), + "http.time_ms", fmt.Sprintf("%v", durationToMilliseconds(time.Since(start))), "http.remote_addr", r.RemoteAddr, "thanos.method_name", name) + + logger = m.opts.filterLog(logger) + m.opts.levelFunc(logger, status).Log("msg", "finished call") +} + +func (m *HTTPServerMiddleware) HTTPMiddleware(name string, next http.Handler) http.HandlerFunc { + return func(w http.ResponseWriter, r *http.Request) { + wrapped := httputil.WrapResponseWriterWithStatus(w) + start := time.Now() + hostPort := r.Host + if hostPort == "" { + hostPort = r.URL.Host + } + + var port string + var err error + // Try to extract port if there is ':' as part of 'hostPort'. + if strings.Contains(hostPort, ":") { + _, port, err = net.SplitHostPort(hostPort) + if err != nil { + level.Error(m.logger).Log("msg", "failed to parse host port for http log decision", "err", err) + next.ServeHTTP(w, r) + return + } + } + + deciderURL := r.URL.String() + if len(port) > 0 { + deciderURL = net.JoinHostPort(deciderURL, port) + } + decision := m.opts.shouldLog(deciderURL, nil) + + switch decision { + case NoLogCall: + next.ServeHTTP(w, r) + + case LogStartAndFinishCall: + m.preCall(name, start, r) + next.ServeHTTP(wrapped, r) + m.postCall(name, start, wrapped, r) + + case LogFinishCall: + next.ServeHTTP(wrapped, r) + m.postCall(name, start, wrapped, r) + } + } +} + +// NewHTTPServerMiddleware returns an http middleware. +func NewHTTPServerMiddleware(logger log.Logger, opts ...Option) *HTTPServerMiddleware { + o := evaluateOpt(opts) + return &HTTPServerMiddleware{ + logger: log.With(logger, "protocol", "http", "http.component", "server"), + opts: o, + } +} + +// getHTTPLoggingOption returns the logging ENUM based on logStart and logEnd values. +func getHTTPLoggingOption(logStart, logEnd bool) (Decision, error) { + if !logStart && !logEnd { + return NoLogCall, nil + } + if !logStart && logEnd { + return LogFinishCall, nil + } + if logStart && logEnd { + return LogStartAndFinishCall, nil + } + return -1, fmt.Errorf("log start call is not supported") +} + +// getLevel returns the level based logger. +func getLevel(lvl string) level.Option { + switch lvl { + case "INFO": + return level.AllowInfo() + case "DEBUG": + return level.AllowDebug() + case "WARN": + return level.AllowWarn() + case "ERROR": + return level.AllowError() + default: + return level.AllowAll() + } +} + +// NewHTTPOption returns a http config option. +func NewHTTPOption(configYAML []byte) ([]Option, error) { + // Define a black config option. + logOpts := []Option{ + WithDecider(func(_ string, err error) Decision { + return NoLogCall + }), + } + + // If req logging is disabled. + if len(configYAML) == 0 { + return logOpts, nil + } + + reqLogConfig, err := NewRequestConfig(configYAML) + // If unmarshalling is an issue. + if err != nil { + return logOpts, err + } + + globalLevel, globalStart, globalEnd, err := fillGlobalOptionConfig(reqLogConfig, false) + + // If global options have invalid entries. + if err != nil { + return logOpts, err + } + // If the level entry does not matches our entries. + if err := validateLevel(globalLevel); err != nil { + // fmt.Printf("HTTP") + return logOpts, err + } + + // If the combination is valid, use them, otherwise return error. + reqLogDecision, err := getHTTPLoggingOption(globalStart, globalEnd) + if err != nil { + return logOpts, err + } + + logOpts = []Option{ + WithFilter(func(logger log.Logger) log.Logger { + return level.NewFilter(logger, getLevel(globalLevel)) + }), + WithLevels(DefaultCodeToLevel), + } + + if len(reqLogConfig.HTTP.Config) == 0 { + logOpts = append(logOpts, []Option{WithDecider(func(_ string, err error) Decision { + return reqLogDecision + }), + }...) + return logOpts, nil + } + + methodNameSlice := []string{} + + for _, eachConfig := range reqLogConfig.HTTP.Config { + eachConfigName := fmt.Sprintf("%v:%v", eachConfig.Path, eachConfig.Port) + methodNameSlice = append(methodNameSlice, eachConfigName) + } + + sort.Strings(methodNameSlice) + + logOpts = append(logOpts, []Option{ + WithDecider(func(runtimeMethodName string, err error) Decision { + idx := sort.SearchStrings(methodNameSlice, runtimeMethodName) + if idx < len(methodNameSlice) && methodNameSlice[idx] == runtimeMethodName { + return reqLogDecision + } + return NoLogCall + }), + }...) + return logOpts, nil + +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go b/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go new file mode 100644 index 00000000000..cca8853da58 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go @@ -0,0 +1,59 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package logging + +import ( + "os" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" +) + +const ( + LogFormatLogfmt = "logfmt" + LogFormatJSON = "json" +) + +// NewLogger returns a log.Logger that prints in the provided format at the +// provided level with a UTC timestamp and the caller of the log entry. If non +// empty, the debug name is also appended as a field to all log lines. Panics +// if the log level is not error, warn, info or debug. Log level is expected to +// be validated before passed to this function. +func NewLogger(logLevel, logFormat, debugName string) log.Logger { + var ( + logger log.Logger + lvl level.Option + ) + + switch logLevel { + case "error": + lvl = level.AllowError() + case "warn": + lvl = level.AllowWarn() + case "info": + lvl = level.AllowInfo() + case "debug": + lvl = level.AllowDebug() + default: + // This enum is already checked and enforced by flag validations, so + // this should never happen. + panic("unexpected log level") + } + + logger = log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) + if logFormat == LogFormatJSON { + logger = log.NewJSONLogger(log.NewSyncWriter(os.Stderr)) + } + + // Sort the logger chain to avoid expensive log.Valuer evaluation for disallowed level. + // Ref: https://github.com/go-kit/log/issues/14#issuecomment-945038252 + logger = log.With(logger, "ts", log.DefaultTimestampUTC, "caller", log.Caller(5)) + logger = level.NewFilter(logger, lvl) + + if debugName != "" { + logger = log.With(logger, "name", debugName) + } + + return logger +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/options.go b/vendor/github.com/thanos-io/thanos/pkg/logging/options.go new file mode 100644 index 00000000000..76b0f8be7d3 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/logging/options.go @@ -0,0 +1,245 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package logging + +import ( + "fmt" + "math/rand" + "time" + + extflag "github.com/efficientgo/tools/extkingpin" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + grpc_logging "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging" + "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" + "github.com/oklog/ulid" + "google.golang.org/grpc/codes" +) + +// Decision defines rules for enabling start and end of logging. +type Decision int + +const ( + // NoLogCall - Logging is disabled. + NoLogCall Decision = iota + // LogFinishCall - Only finish logs of request is enabled. + LogFinishCall + // LogStartAndFinishCall - Logging of start and end of request is enabled. + LogStartAndFinishCall +) + +var defaultOptions = &options{ + shouldLog: DefaultDeciderMethod, + codeFunc: DefaultErrorToCode, + levelFunc: DefaultCodeToLevel, + durationFieldFunc: DurationToTimeMillisFields, + filterLog: DefaultFilterLogging, +} + +func evaluateOpt(opts []Option) *options { + optCopy := &options{} + *optCopy = *defaultOptions + optCopy.levelFunc = DefaultCodeToLevel + for _, o := range opts { + o(optCopy) + } + return optCopy +} + +// WithDecider customizes the function for deciding if the HTTP Middlewares/Tripperwares should log. +func WithDecider(f Decider) Option { + return func(o *options) { + o.shouldLog = f + } +} + +// WithLevels customizes the function for mapping HTTP response codes and interceptor log level statements. +func WithLevels(f CodeToLevel) Option { + return func(o *options) { + o.levelFunc = f + } +} + +// WithFilter customizes the function for deciding which level of logging should be allowed. +// Follows go-kit Allow convention. +func WithFilter(f FilterLogging) Option { + return func(o *options) { + o.filterLog = f + } +} + +// Interface for the additional methods. + +// Types for the Options. +type Option func(*options) + +// Fields represents logging fields. It has to have even number of elements (pairs). +type Fields []string + +// ErrorToCode function determines the error code of the error +// for the http response. +type ErrorToCode func(err error) int + +// DefaultErrorToCode returns an InternalServerError. +func DefaultErrorToCode(_ error) int { + return 500 +} + +// Decider function defines rules for suppressing the logging. +type Decider func(methodName string, err error) Decision + +// DefaultDeciderMethod is the default implementation of decider to see if you should log the call +// by default this is set to LogStartAndFinishCall. +func DefaultDeciderMethod(_ string, _ error) Decision { + return LogStartAndFinishCall +} + +// CodeToLevel function defines the mapping between HTTP Response codes to log levels for server side. +type CodeToLevel func(logger log.Logger, code int) log.Logger + +// DurationToFields function defines how to produce duration fields for logging. +type DurationToFields func(duration time.Duration) Fields + +// FilterLogging makes sure only the logs with level=lvl gets logged, or filtered. +type FilterLogging func(logger log.Logger) log.Logger + +// DefaultFilterLogging allows logs from all levels to be logged in output. +func DefaultFilterLogging(logger log.Logger) log.Logger { + return level.NewFilter(logger, level.AllowAll()) +} + +type options struct { + levelFunc CodeToLevel + shouldLog Decider + codeFunc ErrorToCode + durationFieldFunc DurationToFields + filterLog FilterLogging +} + +// DefaultCodeToLevel is the helper mapper that maps HTTP Response codes to log levels. +func DefaultCodeToLevel(logger log.Logger, code int) log.Logger { + if code >= 200 && code < 500 { + return level.Debug(logger) + } + return level.Error(logger) +} + +// DefaultCodeToLevelGRPC is the helper mapper that maps gRPC Response codes to log levels. +func DefaultCodeToLevelGRPC(c codes.Code) grpc_logging.Level { + switch c { + case codes.Unknown, codes.Unimplemented, codes.Internal, codes.DataLoss: + return grpc_logging.ERROR + default: + return grpc_logging.DEBUG + } +} + +// DurationToTimeMillisFields converts the duration to milliseconds and uses the key `http.time_ms`. +func DurationToTimeMillisFields(duration time.Duration) Fields { + return Fields{"http.time_ms", fmt.Sprintf("%v", durationToMilliseconds(duration))} +} + +func durationToMilliseconds(duration time.Duration) float32 { + return float32(duration.Nanoseconds()/1000) / 1000 +} + +// LogDecision defines mapping of flag options to the logging decision. +var LogDecision = map[string]Decision{ + "NoLogCall": NoLogCall, + "LogFinishCall": LogFinishCall, + "LogStartAndFinishCall": LogStartAndFinishCall, +} + +// MapAllowedLevels allows to map a given level to a list of allowed level. +// Convention taken from go-kit/level v0.10.0 https://godoc.org/github.com/go-kit/log/level#AllowAll. +var MapAllowedLevels = map[string][]string{ + "DEBUG": {"INFO", "DEBUG", "WARN", "ERROR"}, + "ERROR": {"ERROR"}, + "INFO": {"INFO", "WARN", "ERROR"}, + "WARN": {"WARN", "ERROR"}, +} + +// TODO: @yashrsharma44 - To be deprecated in the next release. +func ParseHTTPOptions(flagDecision string, reqLogConfig *extflag.PathOrContent) ([]Option, error) { + // Default Option: No Logging. + logOpts := []Option{WithDecider(func(_ string, _ error) Decision { + return NoLogCall + })} + + // If flag is incorrectly parsed. + configYAML, err := reqLogConfig.Content() + if err != nil { + return logOpts, fmt.Errorf("getting request logging config failed. %v", err) + } + + // Check if the user enables request logging through flags and YAML. + if len(configYAML) != 0 && flagDecision != "" { + return logOpts, fmt.Errorf("both log.request.decision and request.logging have been enabled, please use only one of the flags") + } + // If old flag is enabled. + if len(flagDecision) > 0 { + logOpts := []Option{WithDecider(func(_ string, _ error) Decision { + return LogDecision[flagDecision] + })} + return logOpts, nil + } + return NewHTTPOption(configYAML) +} + +// TODO: @yashrsharma44 - To be deprecated in the next release. +func ParsegRPCOptions(flagDecision string, reqLogConfig *extflag.PathOrContent) ([]tags.Option, []grpc_logging.Option, error) { + // Default Option: No Logging. + logOpts := []grpc_logging.Option{grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { + return grpc_logging.NoLogCall + })} + + configYAML, err := reqLogConfig.Content() + if err != nil { + return []tags.Option{}, logOpts, fmt.Errorf("getting request logging config failed. %v", err) + } + + // Check if the user enables request logging through flags and YAML. + if len(configYAML) != 0 && flagDecision != "" { + return []tags.Option{}, logOpts, fmt.Errorf("both log.request.decision and request.logging-config have been enabled, please use only one of the flags") + } + + // If the old flag is empty, use the new YAML config. + if flagDecision == "" { + tagOpts, logOpts, err := NewGRPCOption(configYAML) + if err != nil { + return []tags.Option{}, logOpts, err + } + return tagOpts, logOpts, nil + } + + tagOpts := []tags.Option{ + tags.WithFieldExtractor(func(_ string, req interface{}) map[string]string { + tagMap := tags.TagBasedRequestFieldExtractor("request-id")("", req) + // If a request-id exists for a given request. + if tagMap != nil { + if _, ok := tagMap["request-id"]; ok { + return tagMap + } + } + entropy := ulid.Monotonic(rand.New(rand.NewSource(time.Now().UnixNano())), 0) + reqID := ulid.MustNew(ulid.Timestamp(time.Now()), entropy) + tagMap = make(map[string]string) + tagMap["request-id"] = reqID.String() + return tagMap + }), + } + logOpts = []grpc_logging.Option{grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { + switch flagDecision { + case "NoLogCall": + return grpc_logging.NoLogCall + case "LogFinishCall": + return grpc_logging.LogFinishCall + case "LogStartAndFinishCall": + return grpc_logging.LogStartAndFinishCall + default: + return grpc_logging.NoLogCall + } + })} + return tagOpts, logOpts, nil +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go b/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go new file mode 100644 index 00000000000..ed3c893ed9f --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go @@ -0,0 +1,40 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package logging + +type RequestConfig struct { + HTTP HTTPProtocolConfigs `yaml:"http"` + GRPC GRPCProtocolConfigs `yaml:"grpc"` + Options OptionsConfig `yaml:"options"` +} + +type HTTPProtocolConfigs struct { + Options OptionsConfig `yaml:"options"` + Config []HTTPProtocolConfig `yaml:"config"` +} + +type GRPCProtocolConfigs struct { + Options OptionsConfig `yaml:"options"` + Config []GRPCProtocolConfig `yaml:"config"` +} + +type OptionsConfig struct { + Level string `yaml:"level"` + Decision DecisionConfig `yaml:"decision"` +} + +type DecisionConfig struct { + LogStart bool `yaml:"log_start"` + LogEnd bool `yaml:"log_end"` +} + +type HTTPProtocolConfig struct { + Path string `yaml:"path"` + Port uint64 `yaml:"port"` +} + +type GRPCProtocolConfig struct { + Service string `yaml:"service"` + Method string `yaml:"method"` +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go b/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go new file mode 100644 index 00000000000..348e71cf4cb --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go @@ -0,0 +1,56 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package prober + +import "sync" + +type combined struct { + mu sync.Mutex + probes []Probe +} + +// Combine folds given probes into one, reflects their statuses in a thread-safe way. +func Combine(probes ...Probe) Probe { + return &combined{probes: probes} +} + +// Ready sets components status to ready. +func (p *combined) Ready() { + p.mu.Lock() + defer p.mu.Unlock() + + for _, probe := range p.probes { + probe.Ready() + } +} + +// NotReady sets components status to not ready with given error as a cause. +func (p *combined) NotReady(err error) { + p.mu.Lock() + defer p.mu.Unlock() + + for _, probe := range p.probes { + probe.NotReady(err) + } +} + +// Healthy sets components status to healthy. +func (p *combined) Healthy() { + p.mu.Lock() + defer p.mu.Unlock() + + for _, probe := range p.probes { + probe.Healthy() + } +} + +// NotHealthy sets components status to not healthy with given error as a cause. +func (p *combined) NotHealthy(err error) { + p.mu.Lock() + defer p.mu.Unlock() + + for _, probe := range p.probes { + probe.NotHealthy(err) + } +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go b/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go new file mode 100644 index 00000000000..6151e040338 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go @@ -0,0 +1,47 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package prober + +import ( + "google.golang.org/grpc/health" + grpc_health "google.golang.org/grpc/health/grpc_health_v1" +) + +// GRPCProbe represents health and readiness status of given component, and provides GRPC integration. +type GRPCProbe struct { + h *health.Server +} + +// NewGRPC creates a Probe that wrapped around grpc/healt.Server which reflects status of server. +func NewGRPC() *GRPCProbe { + h := health.NewServer() + h.SetServingStatus("", grpc_health.HealthCheckResponse_NOT_SERVING) + + return &GRPCProbe{h: h} +} + +// HealthServer returns a gRPC health server which responds readiness and liveness checks. +func (p *GRPCProbe) HealthServer() *health.Server { + return p.h +} + +// Ready sets components status to ready. +func (p *GRPCProbe) Ready() { + p.h.SetServingStatus("", grpc_health.HealthCheckResponse_SERVING) +} + +// NotReady sets components status to not ready with given error as a cause. +func (p *GRPCProbe) NotReady(err error) { + p.h.SetServingStatus("", grpc_health.HealthCheckResponse_NOT_SERVING) +} + +// Healthy sets components status to healthy. +func (p *GRPCProbe) Healthy() { + p.h.Resume() +} + +// NotHealthy sets components status to not healthy with given error as a cause. +func (p *GRPCProbe) NotHealthy(err error) { + p.h.Shutdown() +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/http.go b/vendor/github.com/thanos-io/thanos/pkg/prober/http.go new file mode 100644 index 00000000000..18f9c98af17 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/prober/http.go @@ -0,0 +1,81 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package prober + +import ( + "io" + "net/http" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "go.uber.org/atomic" +) + +type check func() bool + +// HTTPProbe represents health and readiness status of given component, and provides HTTP integration. +type HTTPProbe struct { + ready atomic.Uint32 + healthy atomic.Uint32 +} + +// NewHTTP returns HTTPProbe representing readiness and healthiness of given component. +func NewHTTP() *HTTPProbe { + return &HTTPProbe{} +} + +// HealthyHandler returns a HTTP Handler which responds health checks. +func (p *HTTPProbe) HealthyHandler(logger log.Logger) http.HandlerFunc { + return p.handler(logger, p.isHealthy) +} + +// ReadyHandler returns a HTTP Handler which responds readiness checks. +func (p *HTTPProbe) ReadyHandler(logger log.Logger) http.HandlerFunc { + return p.handler(logger, p.IsReady) +} + +func (p *HTTPProbe) handler(logger log.Logger, c check) http.HandlerFunc { + return func(w http.ResponseWriter, _ *http.Request) { + if !c() { + http.Error(w, "NOT OK", http.StatusServiceUnavailable) + return + } + if _, err := io.WriteString(w, "OK"); err != nil { + level.Error(logger).Log("msg", "failed to write probe response", "err", err) + } + } +} + +// IsReady returns true if component is ready. +func (p *HTTPProbe) IsReady() bool { + ready := p.ready.Load() + return ready > 0 +} + +// isHealthy returns true if component is healthy. +func (p *HTTPProbe) isHealthy() bool { + healthy := p.healthy.Load() + return healthy > 0 +} + +// Ready sets components status to ready. +func (p *HTTPProbe) Ready() { + p.ready.Swap(1) +} + +// NotReady sets components status to not ready with given error as a cause. +func (p *HTTPProbe) NotReady(err error) { + p.ready.Swap(0) + +} + +// Healthy sets components status to healthy. +func (p *HTTPProbe) Healthy() { + p.healthy.Swap(1) +} + +// NotHealthy sets components status to not healthy with given error as a cause. +func (p *HTTPProbe) NotHealthy(err error) { + p.healthy.Swap(0) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go b/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go new file mode 100644 index 00000000000..7c8d78ff85b --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go @@ -0,0 +1,82 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package prober + +import ( + "sync" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/thanos-io/thanos/pkg/component" +) + +const ( + ready = "ready" + notReady = "not-ready" + healthy = "healthy" +) + +// InstrumentationProbe stores instrumentation state of Probe. +// This is created with an intention to combine with other Probe's using prober.Combine. +type InstrumentationProbe struct { + component component.Component + logger log.Logger + + statusMetric *prometheus.GaugeVec + mu sync.Mutex + statusString string +} + +// NewInstrumentation returns InstrumentationProbe records readiness and healthiness for given component. +func NewInstrumentation(component component.Component, logger log.Logger, reg prometheus.Registerer) *InstrumentationProbe { + p := InstrumentationProbe{ + component: component, + logger: logger, + statusMetric: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "status", + Help: "Represents status (0 indicates failure, 1 indicates success) of the component.", + ConstLabels: map[string]string{"component": component.String()}, + }, + []string{"check"}, + ), + } + return &p +} + +// Ready records the component status when Ready is called, if combined with other Probes. +func (p *InstrumentationProbe) Ready() { + p.statusMetric.WithLabelValues(ready).Set(1) + p.mu.Lock() + defer p.mu.Unlock() + if p.statusString != ready { + level.Info(p.logger).Log("msg", "changing probe status", "status", ready) + p.statusString = ready + } +} + +// NotReady records the component status when NotReady is called, if combined with other Probes. +func (p *InstrumentationProbe) NotReady(err error) { + p.statusMetric.WithLabelValues(ready).Set(0) + p.mu.Lock() + defer p.mu.Unlock() + if p.statusString != notReady { + level.Warn(p.logger).Log("msg", "changing probe status", "status", notReady, "reason", err) + p.statusString = notReady + } +} + +// Healthy records the component status when Healthy is called, if combined with other Probes. +func (p *InstrumentationProbe) Healthy() { + p.statusMetric.WithLabelValues(healthy).Set(1) + level.Info(p.logger).Log("msg", "changing probe status", "status", "healthy") +} + +// NotHealthy records the component status when NotHealthy is called, if combined with other Probes. +func (p *InstrumentationProbe) NotHealthy(err error) { + p.statusMetric.WithLabelValues(healthy).Set(0) + level.Info(p.logger).Log("msg", "changing probe status", "status", "not-healthy", "reason", err) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go b/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go new file mode 100644 index 00000000000..db03e6e046f --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go @@ -0,0 +1,25 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package prober + +// Prober represents health and readiness status of given component. +// +// From Kubernetes documentation https://kubernetes.io/docs/tasks/configure-pod-container/configure-liveness-readiness-probes/ : +// +// liveness: Many applications running for long periods of time eventually transition to broken states, +// (healthy) and cannot recover except by being restarted. +// Kubernetes provides liveness probes to detect and remedy such situations. +// +// readiness: Sometimes, applications are temporarily unable to serve traffic. +// (ready) For example, an application might need to load large data or configuration files during startup, +// or depend on external services after startup. In such cases, you don’t want to kill the application, +// but you don’t want to send it requests either. Kubernetes provides readiness probes to detect +// and mitigate these situations. A pod with containers reporting that they are not ready +// does not receive traffic through Kubernetes Services. +type Probe interface { + Healthy() + NotHealthy(err error) + Ready() + NotReady(err error) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go new file mode 100644 index 00000000000..795d6c4fdb1 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go @@ -0,0 +1,145 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package http + +import ( + "context" + "net/http" + "net/http/pprof" + + "github.com/felixge/fgprof" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promhttp" + toolkit_web "github.com/prometheus/exporter-toolkit/web" + "golang.org/x/net/http2" + "golang.org/x/net/http2/h2c" + + "github.com/thanos-io/thanos/pkg/component" + "github.com/thanos-io/thanos/pkg/prober" +) + +// A Server defines parameters for serve HTTP requests, a wrapper around http.Server. +type Server struct { + logger log.Logger + comp component.Component + prober *prober.HTTPProbe + + mux *http.ServeMux + srv *http.Server + + opts options +} + +// New creates a new Server. +func New(logger log.Logger, reg *prometheus.Registry, comp component.Component, prober *prober.HTTPProbe, opts ...Option) *Server { + options := options{} + for _, o := range opts { + o.apply(&options) + } + + mux := http.NewServeMux() + if options.mux != nil { + mux = options.mux + } + + registerMetrics(mux, reg) + registerProbes(mux, prober, logger) + registerProfiler(mux) + + var h http.Handler + if options.enableH2C { + h2s := &http2.Server{} + h = h2c.NewHandler(mux, h2s) + } else { + h = mux + } + + return &Server{ + logger: log.With(logger, "service", "http/server", "component", comp.String()), + comp: comp, + prober: prober, + mux: mux, + srv: &http.Server{Addr: options.listen, Handler: h}, + opts: options, + } +} + +// ListenAndServe listens on the TCP network address and handles requests on incoming connections. +func (s *Server) ListenAndServe() error { + level.Info(s.logger).Log("msg", "listening for requests and metrics", "address", s.opts.listen) + err := toolkit_web.Validate(s.opts.tlsConfigPath) + if err != nil { + return errors.Wrap(err, "server could not be started") + } + + flags := &toolkit_web.FlagConfig{ + WebListenAddresses: &([]string{s.opts.listen}), + WebSystemdSocket: ofBool(false), + WebConfigFile: &s.opts.tlsConfigPath, + } + + return errors.Wrap(toolkit_web.ListenAndServe(s.srv, flags, s.logger), "serve HTTP and metrics") +} + +// Shutdown gracefully shuts down the server by waiting, +// for specified amount of time (by gracePeriod) for connections to return to idle and then shut down. +func (s *Server) Shutdown(err error) { + level.Info(s.logger).Log("msg", "internal server is shutting down", "err", err) + if err == http.ErrServerClosed { + level.Warn(s.logger).Log("msg", "internal server closed unexpectedly") + return + } + + if s.opts.gracePeriod == 0 { + s.srv.Close() + level.Info(s.logger).Log("msg", "internal server is shutdown", "err", err) + return + } + + ctx, cancel := context.WithTimeout(context.Background(), s.opts.gracePeriod) + defer cancel() + + if err := s.srv.Shutdown(ctx); err != nil { + level.Error(s.logger).Log("msg", "internal server shut down failed", "err", err) + return + } + level.Info(s.logger).Log("msg", "internal server is shutdown gracefully", "err", err) +} + +// Handle registers the handler for the given pattern. +func (s *Server) Handle(pattern string, handler http.Handler) { + s.mux.Handle(pattern, handler) +} + +func registerProfiler(mux *http.ServeMux) { + mux.HandleFunc("/debug/pprof/", pprof.Index) + mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) + mux.HandleFunc("/debug/pprof/profile", pprof.Profile) + mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + mux.Handle("/debug/fgprof", fgprof.Handler()) +} + +func registerMetrics(mux *http.ServeMux, g prometheus.Gatherer) { + if g != nil { + mux.Handle("/metrics", promhttp.HandlerFor(g, promhttp.HandlerOpts{ + EnableOpenMetrics: true, + })) + } +} + +func registerProbes(mux *http.ServeMux, p *prober.HTTPProbe, logger log.Logger) { + if p != nil { + mux.Handle("/-/healthy", p.HealthyHandler(logger)) + mux.Handle("/-/ready", p.ReadyHandler(logger)) + } +} + +// Helper for exporter toolkit FlagConfig. +func ofBool(i bool) *bool { + return &i +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go new file mode 100644 index 00000000000..d08699f9dff --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go @@ -0,0 +1,63 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package http + +import ( + "net/http" + "time" +) + +type options struct { + gracePeriod time.Duration + listen string + tlsConfigPath string + mux *http.ServeMux + enableH2C bool +} + +// Option overrides behavior of Server. +type Option interface { + apply(*options) +} + +type optionFunc func(*options) + +func (f optionFunc) apply(o *options) { + f(o) +} + +// WithGracePeriod sets shutdown grace period for HTTP server. +// Server waits connections to drain for specified amount of time. +func WithGracePeriod(t time.Duration) Option { + return optionFunc(func(o *options) { + o.gracePeriod = t + }) +} + +// WithListen sets address to listen for HTTP server. +// Server accepts incoming TCP connections on given address. +func WithListen(s string) Option { + return optionFunc(func(o *options) { + o.listen = s + }) +} + +func WithTLSConfig(tls string) Option { + return optionFunc(func(o *options) { + o.tlsConfigPath = tls + }) +} + +func WithEnableH2C(enableH2C bool) Option { + return optionFunc(func(o *options) { + o.enableH2C = enableH2C + }) +} + +// WithMux overrides the server's default mux. +func WithMux(mux *http.ServeMux) Option { + return optionFunc(func(o *options) { + o.mux = mux + }) +} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go new file mode 100644 index 00000000000..d8da1dc3c54 --- /dev/null +++ b/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go @@ -0,0 +1,32 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package http + +import "net/http" + +// ResponseWriterWithStatus wraps around http.ResponseWriter to capture the status code of the response. +type ResponseWriterWithStatus struct { + http.ResponseWriter + status int + isHeaderWritten bool +} + +// WrapResponseWriterWithStatus wraps the http.ResponseWriter for extracting status. +func WrapResponseWriterWithStatus(w http.ResponseWriter) *ResponseWriterWithStatus { + return &ResponseWriterWithStatus{ResponseWriter: w} +} + +// Status returns http response status. +func (r *ResponseWriterWithStatus) Status() int { + return r.status +} + +// WriteHeader writes the header. +func (r *ResponseWriterWithStatus) WriteHeader(code int) { + if !r.isHeaderWritten { + r.status = code + r.ResponseWriter.WriteHeader(code) + r.isHeaderWritten = true + } +} diff --git a/vendor/golang.org/x/net/http2/h2c/h2c.go b/vendor/golang.org/x/net/http2/h2c/h2c.go new file mode 100644 index 00000000000..a72bbed1bc3 --- /dev/null +++ b/vendor/golang.org/x/net/http2/h2c/h2c.go @@ -0,0 +1,240 @@ +// Copyright 2018 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package h2c implements the unencrypted "h2c" form of HTTP/2. +// +// The h2c protocol is the non-TLS version of HTTP/2 which is not available from +// net/http or golang.org/x/net/http2. +package h2c + +import ( + "bufio" + "bytes" + "encoding/base64" + "errors" + "fmt" + "io" + "log" + "net" + "net/http" + "net/textproto" + "os" + "strings" + + "golang.org/x/net/http/httpguts" + "golang.org/x/net/http2" +) + +var ( + http2VerboseLogs bool +) + +func init() { + e := os.Getenv("GODEBUG") + if strings.Contains(e, "http2debug=1") || strings.Contains(e, "http2debug=2") { + http2VerboseLogs = true + } +} + +// h2cHandler is a Handler which implements h2c by hijacking the HTTP/1 traffic +// that should be h2c traffic. There are two ways to begin a h2c connection +// (RFC 7540 Section 3.2 and 3.4): (1) Starting with Prior Knowledge - this +// works by starting an h2c connection with a string of bytes that is valid +// HTTP/1, but unlikely to occur in practice and (2) Upgrading from HTTP/1 to +// h2c - this works by using the HTTP/1 Upgrade header to request an upgrade to +// h2c. When either of those situations occur we hijack the HTTP/1 connection, +// convert it to a HTTP/2 connection and pass the net.Conn to http2.ServeConn. +type h2cHandler struct { + Handler http.Handler + s *http2.Server +} + +// NewHandler returns an http.Handler that wraps h, intercepting any h2c +// traffic. If a request is an h2c connection, it's hijacked and redirected to +// s.ServeConn. Otherwise the returned Handler just forwards requests to h. This +// works because h2c is designed to be parseable as valid HTTP/1, but ignored by +// any HTTP server that does not handle h2c. Therefore we leverage the HTTP/1 +// compatible parts of the Go http library to parse and recognize h2c requests. +// Once a request is recognized as h2c, we hijack the connection and convert it +// to an HTTP/2 connection which is understandable to s.ServeConn. (s.ServeConn +// understands HTTP/2 except for the h2c part of it.) +// +// The first request on an h2c connection is read entirely into memory before +// the Handler is called. To limit the memory consumed by this request, wrap +// the result of NewHandler in an http.MaxBytesHandler. +func NewHandler(h http.Handler, s *http2.Server) http.Handler { + return &h2cHandler{ + Handler: h, + s: s, + } +} + +// extractServer extracts existing http.Server instance from http.Request or create an empty http.Server +func extractServer(r *http.Request) *http.Server { + server, ok := r.Context().Value(http.ServerContextKey).(*http.Server) + if ok { + return server + } + return new(http.Server) +} + +// ServeHTTP implement the h2c support that is enabled by h2c.GetH2CHandler. +func (s h2cHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + // Handle h2c with prior knowledge (RFC 7540 Section 3.4) + if r.Method == "PRI" && len(r.Header) == 0 && r.URL.Path == "*" && r.Proto == "HTTP/2.0" { + if http2VerboseLogs { + log.Print("h2c: attempting h2c with prior knowledge.") + } + conn, err := initH2CWithPriorKnowledge(w) + if err != nil { + if http2VerboseLogs { + log.Printf("h2c: error h2c with prior knowledge: %v", err) + } + return + } + defer conn.Close() + s.s.ServeConn(conn, &http2.ServeConnOpts{ + Context: r.Context(), + BaseConfig: extractServer(r), + Handler: s.Handler, + SawClientPreface: true, + }) + return + } + // Handle Upgrade to h2c (RFC 7540 Section 3.2) + if isH2CUpgrade(r.Header) { + conn, settings, err := h2cUpgrade(w, r) + if err != nil { + if http2VerboseLogs { + log.Printf("h2c: error h2c upgrade: %v", err) + } + w.WriteHeader(http.StatusInternalServerError) + return + } + defer conn.Close() + s.s.ServeConn(conn, &http2.ServeConnOpts{ + Context: r.Context(), + BaseConfig: extractServer(r), + Handler: s.Handler, + UpgradeRequest: r, + Settings: settings, + }) + return + } + s.Handler.ServeHTTP(w, r) + return +} + +// initH2CWithPriorKnowledge implements creating a h2c connection with prior +// knowledge (Section 3.4) and creates a net.Conn suitable for http2.ServeConn. +// All we have to do is look for the client preface that is suppose to be part +// of the body, and reforward the client preface on the net.Conn this function +// creates. +func initH2CWithPriorKnowledge(w http.ResponseWriter) (net.Conn, error) { + hijacker, ok := w.(http.Hijacker) + if !ok { + return nil, errors.New("h2c: connection does not support Hijack") + } + conn, rw, err := hijacker.Hijack() + if err != nil { + return nil, err + } + + const expectedBody = "SM\r\n\r\n" + + buf := make([]byte, len(expectedBody)) + n, err := io.ReadFull(rw, buf) + if err != nil { + return nil, fmt.Errorf("h2c: error reading client preface: %s", err) + } + + if string(buf[:n]) == expectedBody { + return newBufConn(conn, rw), nil + } + + conn.Close() + return nil, errors.New("h2c: invalid client preface") +} + +// h2cUpgrade establishes a h2c connection using the HTTP/1 upgrade (Section 3.2). +func h2cUpgrade(w http.ResponseWriter, r *http.Request) (_ net.Conn, settings []byte, err error) { + settings, err = getH2Settings(r.Header) + if err != nil { + return nil, nil, err + } + hijacker, ok := w.(http.Hijacker) + if !ok { + return nil, nil, errors.New("h2c: connection does not support Hijack") + } + + body, err := io.ReadAll(r.Body) + if err != nil { + return nil, nil, err + } + r.Body = io.NopCloser(bytes.NewBuffer(body)) + + conn, rw, err := hijacker.Hijack() + if err != nil { + return nil, nil, err + } + + rw.Write([]byte("HTTP/1.1 101 Switching Protocols\r\n" + + "Connection: Upgrade\r\n" + + "Upgrade: h2c\r\n\r\n")) + return newBufConn(conn, rw), settings, nil +} + +// isH2CUpgrade returns true if the header properly request an upgrade to h2c +// as specified by Section 3.2. +func isH2CUpgrade(h http.Header) bool { + return httpguts.HeaderValuesContainsToken(h[textproto.CanonicalMIMEHeaderKey("Upgrade")], "h2c") && + httpguts.HeaderValuesContainsToken(h[textproto.CanonicalMIMEHeaderKey("Connection")], "HTTP2-Settings") +} + +// getH2Settings returns the settings in the HTTP2-Settings header. +func getH2Settings(h http.Header) ([]byte, error) { + vals, ok := h[textproto.CanonicalMIMEHeaderKey("HTTP2-Settings")] + if !ok { + return nil, errors.New("missing HTTP2-Settings header") + } + if len(vals) != 1 { + return nil, fmt.Errorf("expected 1 HTTP2-Settings. Got: %v", vals) + } + settings, err := base64.RawURLEncoding.DecodeString(vals[0]) + if err != nil { + return nil, err + } + return settings, nil +} + +func newBufConn(conn net.Conn, rw *bufio.ReadWriter) net.Conn { + rw.Flush() + if rw.Reader.Buffered() == 0 { + // If there's no buffered data to be read, + // we can just discard the bufio.ReadWriter. + return conn + } + return &bufConn{conn, rw.Reader} +} + +// bufConn wraps a net.Conn, but reads drain the bufio.Reader first. +type bufConn struct { + net.Conn + *bufio.Reader +} + +func (c *bufConn) Read(p []byte) (int, error) { + if c.Reader == nil { + return c.Conn.Read(p) + } + n := c.Reader.Buffered() + if n == 0 { + c.Reader = nil + return c.Conn.Read(p) + } + if n < len(p) { + p = p[:n] + } + return c.Reader.Read(p) +} diff --git a/vendor/google.golang.org/grpc/health/client.go b/vendor/google.golang.org/grpc/health/client.go new file mode 100644 index 00000000000..b5bee483802 --- /dev/null +++ b/vendor/google.golang.org/grpc/health/client.go @@ -0,0 +1,117 @@ +/* + * + * Copyright 2018 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package health + +import ( + "context" + "fmt" + "io" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/connectivity" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/backoff" + "google.golang.org/grpc/status" +) + +var ( + backoffStrategy = backoff.DefaultExponential + backoffFunc = func(ctx context.Context, retries int) bool { + d := backoffStrategy.Backoff(retries) + timer := time.NewTimer(d) + select { + case <-timer.C: + return true + case <-ctx.Done(): + timer.Stop() + return false + } + } +) + +func init() { + internal.HealthCheckFunc = clientHealthCheck +} + +const healthCheckMethod = "/grpc.health.v1.Health/Watch" + +// This function implements the protocol defined at: +// https://github.com/grpc/grpc/blob/master/doc/health-checking.md +func clientHealthCheck(ctx context.Context, newStream func(string) (interface{}, error), setConnectivityState func(connectivity.State, error), service string) error { + tryCnt := 0 + +retryConnection: + for { + // Backs off if the connection has failed in some way without receiving a message in the previous retry. + if tryCnt > 0 && !backoffFunc(ctx, tryCnt-1) { + return nil + } + tryCnt++ + + if ctx.Err() != nil { + return nil + } + setConnectivityState(connectivity.Connecting, nil) + rawS, err := newStream(healthCheckMethod) + if err != nil { + continue retryConnection + } + + s, ok := rawS.(grpc.ClientStream) + // Ideally, this should never happen. But if it happens, the server is marked as healthy for LBing purposes. + if !ok { + setConnectivityState(connectivity.Ready, nil) + return fmt.Errorf("newStream returned %v (type %T); want grpc.ClientStream", rawS, rawS) + } + + if err = s.SendMsg(&healthpb.HealthCheckRequest{Service: service}); err != nil && err != io.EOF { + // Stream should have been closed, so we can safely continue to create a new stream. + continue retryConnection + } + s.CloseSend() + + resp := new(healthpb.HealthCheckResponse) + for { + err = s.RecvMsg(resp) + + // Reports healthy for the LBing purposes if health check is not implemented in the server. + if status.Code(err) == codes.Unimplemented { + setConnectivityState(connectivity.Ready, nil) + return err + } + + // Reports unhealthy if server's Watch method gives an error other than UNIMPLEMENTED. + if err != nil { + setConnectivityState(connectivity.TransientFailure, fmt.Errorf("connection active but received health check RPC error: %v", err)) + continue retryConnection + } + + // As a message has been received, removes the need for backoff for the next retry by resetting the try count. + tryCnt = 0 + if resp.Status == healthpb.HealthCheckResponse_SERVING { + setConnectivityState(connectivity.Ready, nil) + } else { + setConnectivityState(connectivity.TransientFailure, fmt.Errorf("connection active but health check failed. status=%s", resp.Status)) + } + } + } +} diff --git a/vendor/google.golang.org/grpc/health/logging.go b/vendor/google.golang.org/grpc/health/logging.go new file mode 100644 index 00000000000..83c6acf55ef --- /dev/null +++ b/vendor/google.golang.org/grpc/health/logging.go @@ -0,0 +1,23 @@ +/* + * + * Copyright 2020 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package health + +import "google.golang.org/grpc/grpclog" + +var logger = grpclog.Component("health_service") diff --git a/vendor/google.golang.org/grpc/health/server.go b/vendor/google.golang.org/grpc/health/server.go new file mode 100644 index 00000000000..cce6312d77f --- /dev/null +++ b/vendor/google.golang.org/grpc/health/server.go @@ -0,0 +1,163 @@ +/* + * + * Copyright 2017 gRPC authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +// Package health provides a service that exposes server's health and it must be +// imported to enable support for client-side health checks. +package health + +import ( + "context" + "sync" + + "google.golang.org/grpc/codes" + healthgrpc "google.golang.org/grpc/health/grpc_health_v1" + healthpb "google.golang.org/grpc/health/grpc_health_v1" + "google.golang.org/grpc/status" +) + +// Server implements `service Health`. +type Server struct { + healthgrpc.UnimplementedHealthServer + mu sync.RWMutex + // If shutdown is true, it's expected all serving status is NOT_SERVING, and + // will stay in NOT_SERVING. + shutdown bool + // statusMap stores the serving status of the services this Server monitors. + statusMap map[string]healthpb.HealthCheckResponse_ServingStatus + updates map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus +} + +// NewServer returns a new Server. +func NewServer() *Server { + return &Server{ + statusMap: map[string]healthpb.HealthCheckResponse_ServingStatus{"": healthpb.HealthCheckResponse_SERVING}, + updates: make(map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus), + } +} + +// Check implements `service Health`. +func (s *Server) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*healthpb.HealthCheckResponse, error) { + s.mu.RLock() + defer s.mu.RUnlock() + if servingStatus, ok := s.statusMap[in.Service]; ok { + return &healthpb.HealthCheckResponse{ + Status: servingStatus, + }, nil + } + return nil, status.Error(codes.NotFound, "unknown service") +} + +// Watch implements `service Health`. +func (s *Server) Watch(in *healthpb.HealthCheckRequest, stream healthgrpc.Health_WatchServer) error { + service := in.Service + // update channel is used for getting service status updates. + update := make(chan healthpb.HealthCheckResponse_ServingStatus, 1) + s.mu.Lock() + // Puts the initial status to the channel. + if servingStatus, ok := s.statusMap[service]; ok { + update <- servingStatus + } else { + update <- healthpb.HealthCheckResponse_SERVICE_UNKNOWN + } + + // Registers the update channel to the correct place in the updates map. + if _, ok := s.updates[service]; !ok { + s.updates[service] = make(map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus) + } + s.updates[service][stream] = update + defer func() { + s.mu.Lock() + delete(s.updates[service], stream) + s.mu.Unlock() + }() + s.mu.Unlock() + + var lastSentStatus healthpb.HealthCheckResponse_ServingStatus = -1 + for { + select { + // Status updated. Sends the up-to-date status to the client. + case servingStatus := <-update: + if lastSentStatus == servingStatus { + continue + } + lastSentStatus = servingStatus + err := stream.Send(&healthpb.HealthCheckResponse{Status: servingStatus}) + if err != nil { + return status.Error(codes.Canceled, "Stream has ended.") + } + // Context done. Removes the update channel from the updates map. + case <-stream.Context().Done(): + return status.Error(codes.Canceled, "Stream has ended.") + } + } +} + +// SetServingStatus is called when need to reset the serving status of a service +// or insert a new service entry into the statusMap. +func (s *Server) SetServingStatus(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) { + s.mu.Lock() + defer s.mu.Unlock() + if s.shutdown { + logger.Infof("health: status changing for %s to %v is ignored because health service is shutdown", service, servingStatus) + return + } + + s.setServingStatusLocked(service, servingStatus) +} + +func (s *Server) setServingStatusLocked(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) { + s.statusMap[service] = servingStatus + for _, update := range s.updates[service] { + // Clears previous updates, that are not sent to the client, from the channel. + // This can happen if the client is not reading and the server gets flow control limited. + select { + case <-update: + default: + } + // Puts the most recent update to the channel. + update <- servingStatus + } +} + +// Shutdown sets all serving status to NOT_SERVING, and configures the server to +// ignore all future status changes. +// +// This changes serving status for all services. To set status for a particular +// services, call SetServingStatus(). +func (s *Server) Shutdown() { + s.mu.Lock() + defer s.mu.Unlock() + s.shutdown = true + for service := range s.statusMap { + s.setServingStatusLocked(service, healthpb.HealthCheckResponse_NOT_SERVING) + } +} + +// Resume sets all serving status to SERVING, and configures the server to +// accept all future status changes. +// +// This changes serving status for all services. To set status for a particular +// services, call SetServingStatus(). +func (s *Server) Resume() { + s.mu.Lock() + defer s.mu.Unlock() + s.shutdown = false + for service := range s.statusMap { + s.setServingStatusLocked(service, healthpb.HealthCheckResponse_SERVING) + } +} diff --git a/vendor/modules.txt b/vendor/modules.txt index be234374e04..1da9fc2f9dd 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -541,8 +541,12 @@ github.com/grpc-ecosystem/go-grpc-middleware # github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 ## explicit; go 1.19 github.com/grpc-ecosystem/go-grpc-middleware/v2 -# github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 -## explicit; go 1.20 +github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors +github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags +github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tracing +github.com/grpc-ecosystem/go-grpc-middleware/v2/util/metautils +# github.com/grpc-ecosystem/grpc-gateway/v2 v2.15.2 +## explicit; go 1.17 github.com/grpc-ecosystem/grpc-gateway/v2/internal/httprule github.com/grpc-ecosystem/grpc-gateway/v2/runtime github.com/grpc-ecosystem/grpc-gateway/v2/utilities @@ -995,6 +999,7 @@ github.com/thanos-io/thanos/pkg/losertree github.com/thanos-io/thanos/pkg/metadata/metadatapb github.com/thanos-io/thanos/pkg/model github.com/thanos-io/thanos/pkg/pool +github.com/thanos-io/thanos/pkg/prober github.com/thanos-io/thanos/pkg/promclient github.com/thanos-io/thanos/pkg/query github.com/thanos-io/thanos/pkg/querysharding @@ -1284,6 +1289,7 @@ golang.org/x/net/context/ctxhttp golang.org/x/net/http/httpguts golang.org/x/net/http/httpproxy golang.org/x/net/http2 +golang.org/x/net/http2/h2c golang.org/x/net/http2/hpack golang.org/x/net/idna golang.org/x/net/internal/iana @@ -1421,6 +1427,7 @@ google.golang.org/grpc/encoding/gzip google.golang.org/grpc/encoding/proto google.golang.org/grpc/experimental google.golang.org/grpc/grpclog +google.golang.org/grpc/health google.golang.org/grpc/health/grpc_health_v1 google.golang.org/grpc/internal google.golang.org/grpc/internal/backoff From a6e8e04f9811878de24e79de2a13e58474eb9d3f Mon Sep 17 00:00:00 2001 From: afayek Date: Tue, 8 Aug 2023 19:54:31 -0700 Subject: [PATCH 12/38] return json response for unsharded querier requests Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 105 +++++++++++++----- .../tripperware/instantquery/instant_query.go | 1 + .../tripperware/queryrange/query_range.go | 1 + pkg/querier/tripperware/roundtrip.go | 13 +++ 4 files changed, 95 insertions(+), 25 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 1b512fcd4ad..0e10c54e4c9 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -15,6 +15,7 @@ import ( "math" "net/http" "strconv" + "strings" "time" "github.com/cortexproject/cortex/pkg/cortexpb" @@ -32,8 +33,12 @@ import ( type status string const ( - statusSuccess status = "success" - statusError status = "error" + statusSuccess status = "success" + statusError status = "error" + contentTypeHeader string = "Content-Type" + acceptHeader string = "Accept" + applicationProtobuf string = "application/x-protobuf" + applicationJson string = "application/json" // Non-standard status code (originally introduced by nginx) for the case when a client closes // the connection while the server is still processing the request. @@ -151,11 +156,28 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return createPrometheusInstantQueryResponse(&queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - }), res.Warnings, nil, qry.Close + accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] + switch accept { + case applicationProtobuf: + data = createPrometheusInstantQueryResponse(&queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + }) + case applicationJson: + data = &queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + } + default: + data = &queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + } + } + return data, res.Warnings, nil, qry.Close } func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { @@ -246,11 +268,29 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, } qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - return createPrometheusResponse(&queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - }), res.Warnings, nil, qry.Close + accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] + switch accept { + case applicationProtobuf: + data = createPrometheusResponse(&queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + }) + case applicationJson: + data = &queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + } + default: + data = &queryData{ + ResultType: res.Value.Type(), + Result: res.Value, + Stats: qs, + } + } + + return data, res.Warnings, nil, qry.Close } func parseDuration(s string) (time.Duration, error) { @@ -309,22 +349,37 @@ func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storag var b []byte var err error switch resp := data.(type) { - case queryrange.PrometheusResponse: - b, err = proto.Marshal(&resp) - case instantquery.PrometheusInstantQueryResponse: - b, err = proto.Marshal(&resp) + case *queryrange.PrometheusResponse: + w.Header().Set(contentTypeHeader, applicationProtobuf) + for h, hv := range w.Header() { + resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) + } + b, err = proto.Marshal(resp) + case *instantquery.PrometheusInstantQueryResponse: + w.Header().Set(contentTypeHeader, applicationProtobuf) + for h, hv := range w.Header() { + resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) + } + b, err = proto.Marshal(resp) + case *queryData: + w.Header().Set(contentTypeHeader, applicationJson) + json := jsoniter.ConfigCompatibleWithStandardLibrary + b, err = json.Marshal(&response{ + Status: statusSuccess, + Data: data, + Warnings: warningStrings, + }) default: level.Error(api.Logger).Log("msg", "error asserting response type") http.Error(w, "error asserting response type", http.StatusInternalServerError) return } if err != nil { - level.Error(api.Logger).Log("msg", "error marshaling protobuf response", "err", err) + level.Error(api.Logger).Log("msg", "error marshaling response", "err", err) http.Error(w, err.Error(), http.StatusInternalServerError) return } - w.Header().Set("Content-Type", "application/protobuf") w.WriteHeader(http.StatusOK) if n, err := w.Write(b); err != nil { level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) @@ -363,16 +418,16 @@ func (api *API) RespondError(w http.ResponseWriter, apiErr *thanos_api.ApiError, code = http.StatusInternalServerError } - w.Header().Set("Content-Type", "application/json") + w.Header().Set(contentTypeHeader, applicationJson) w.WriteHeader(code) if n, err := w.Write(b); err != nil { level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) } } -func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { +func createPrometheusResponse(data *queryData) *queryrange.PrometheusResponse { if data == nil { - return queryrange.PrometheusResponse{ + return &queryrange.PrometheusResponse{ Status: string(statusSuccess), Data: queryrange.PrometheusData{}, ErrorType: "", @@ -389,7 +444,7 @@ func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} } - return queryrange.PrometheusResponse{ + return &queryrange.PrometheusResponse{ Status: string(statusSuccess), Data: queryrange.PrometheusData{ ResultType: string(data.ResultType), @@ -402,9 +457,9 @@ func createPrometheusResponse(data *queryData) queryrange.PrometheusResponse { } } -func createPrometheusInstantQueryResponse(data *queryData) instantquery.PrometheusInstantQueryResponse { +func createPrometheusInstantQueryResponse(data *queryData) *instantquery.PrometheusInstantQueryResponse { if data == nil { - return instantquery.PrometheusInstantQueryResponse{ + return &instantquery.PrometheusInstantQueryResponse{ Status: string(statusSuccess), Data: instantquery.PrometheusInstantQueryData{}, ErrorType: "", @@ -441,7 +496,7 @@ func createPrometheusInstantQueryResponse(data *queryData) instantquery.Promethe stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} } - return instantquery.PrometheusInstantQueryResponse{ + return &instantquery.PrometheusInstantQueryResponse{ Status: string(statusSuccess), Data: instantquery.PrometheusInstantQueryData{ ResultType: string(data.ResultType), diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index e23039156fd..f735db1fff3 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -211,6 +211,7 @@ func (instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } h.Set("Accept-Encoding", "snappy") + h.Set("Accept", "application/x-protobuf") req := &http.Request{ Method: "GET", diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 36a777da947..90ba64c227f 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -245,6 +245,7 @@ func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) } h.Set("Accept-Encoding", "snappy") + h.Set("Accept", "application/x-protobuf") req := &http.Request{ Method: "GET", diff --git a/pkg/querier/tripperware/roundtrip.go b/pkg/querier/tripperware/roundtrip.go index 4edcd51cc9b..1dfc64739d3 100644 --- a/pkg/querier/tripperware/roundtrip.go +++ b/pkg/querier/tripperware/roundtrip.go @@ -17,6 +17,7 @@ package tripperware import ( "context" + "github.com/cortexproject/cortex/pkg/util/validation" "io" "net/http" "strings" @@ -167,6 +168,18 @@ func NewQueryTripperware( if isQueryRange { return queryrange.RoundTrip(r) } else if isQuery { + // If the given query is not shardable, use downstream roundtripper. + query := r.FormValue("query") + + // If vertical sharding is not enabled for the tenant, use downstream roundtripper. + numShards := validation.SmallestPositiveIntPerTenant(tenantIDs, limits.QueryVerticalShardSize) + if numShards <= 1 { + return next.RoundTrip(r) + } + analysis, err := queryAnalyzer.Analyze(query) + if err != nil || !analysis.IsShardable() { + return next.RoundTrip(r) + } return instantQuery.RoundTrip(r) } return next.RoundTrip(r) From 45c85a7984f397ca41ca115fd59c418691160b51 Mon Sep 17 00:00:00 2001 From: afayek Date: Wed, 9 Aug 2023 10:24:39 -0700 Subject: [PATCH 13/38] remove header copying in codec for querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/tripperware/instantquery/instant_query.go | 3 --- pkg/querier/tripperware/queryrange/query_range.go | 3 --- 2 files changed, 6 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index f735db1fff3..61376dd0eeb 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -177,9 +177,6 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } - for h, hv := range r.Header { - resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) - } return &resp, nil } diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 90ba64c227f..dcca03cf819 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -281,9 +281,6 @@ func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ t return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } - for h, hv := range r.Header { - resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) - } return &resp, nil } From 7af52dbc35c8617e7a3a00ea66b8edee5e8b6d37 Mon Sep 17 00:00:00 2001 From: afayek Date: Mon, 21 Aug 2023 16:26:18 -0700 Subject: [PATCH 14/38] change instant query context cancelled unit test to test new querier handler Signed-off-by: Ahmed Hassan --- .../instantquery/instant_query_test.go | 1075 +++++++++++++++-- 1 file changed, 983 insertions(+), 92 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 3a19641b975..7e5e5cda736 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -5,6 +5,9 @@ import ( "compress/gzip" "context" "fmt" + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" + jsoniter "github.com/json-iterator/go" "io" "net/http" "strconv" @@ -14,7 +17,6 @@ import ( "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/user" "github.com/cortexproject/cortex/pkg/cortexpb" @@ -104,42 +106,71 @@ func TestRequest(t *testing.T) { } } -func TestGzippedResponse(t *testing.T) { +func TestCompressedResponse(t *testing.T) { t.Parallel() for _, tc := range []struct { - body string - status int - err error + name string + compression string + promBody PrometheusInstantQueryResponse + status int + err error }{ { - body: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, + name: "successful response", + compression: "gzip", + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + {Name: "Content-Encoding", Values: []string{"snappy"}}, + }, + }, status: 200, }, { - body: `error generic 400`, - status: 400, - err: httpgrpc.Errorf(400, "error generic 400"), - }, - { - status: 400, - err: httpgrpc.Errorf(400, ""), + name: "successful response", + compression: "snappy", + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + {Name: "Content-Encoding", Values: []string{"snappy"}}, + }, + }, + status: 200, }, } { for _, c := range []bool{true, false} { c := c - t.Run(fmt.Sprintf("compressed %t [%s]", c, tc.body), func(t *testing.T) { + t.Run(fmt.Sprintf("%s compressed %t [%s]", tc.compression, c, tc.name), func(t *testing.T) { t.Parallel() - + protobuf, err := proto.Marshal(&tc.promBody) + require.NoError(t, err) + responseBody := bytes.NewBuffer(protobuf) h := http.Header{ - "Content-Type": []string{"application/json"}, + "Content-Type": []string{"application/x-protobuf"}, } - responseBody := bytes.NewBuffer([]byte(tc.body)) - if c { + var buf bytes.Buffer + if c && tc.compression == "gzip" { h.Set("Content-Encoding", "gzip") - var buf bytes.Buffer w := gzip.NewWriter(&buf) - _, err := w.Write([]byte(tc.body)) + _, err := w.Write(protobuf) + require.NoError(t, err) + w.Close() + responseBody = &buf + } else if c && tc.compression == "snappy" { + h.Set("Content-Encoding", "snappy") + w := snappy.NewBufferedWriter(&buf) + _, err := w.Write(protobuf) require.NoError(t, err) w.Close() responseBody = &buf @@ -154,10 +185,9 @@ func TestGzippedResponse(t *testing.T) { require.Equal(t, tc.err, err) if err == nil { - resp, err := json.Marshal(r) + resp, err := proto.Marshal(r) require.NoError(t, err) - - require.Equal(t, tc.body, string(resp)) + require.Equal(t, protobuf, resp) } }) } @@ -167,25 +197,138 @@ func TestGzippedResponse(t *testing.T) { func TestResponse(t *testing.T) { t.Parallel() for i, tc := range []struct { - body string + expectedResp string + promBody PrometheusInstantQueryResponse }{ { - body: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, + expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { - body: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`)}}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { - body: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"foo", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 137, TimestampMs: 1000}, + {Value: 137, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 5, TimestampMs: 1536673680000}, + {Value: 5, TimestampMs: 1536673780000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { - body: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"foo", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 137, TimestampMs: 1000}, + {Value: 137, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { - body: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, + expectedResp: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"scalar","result":[1,"13"]}`)}}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { - body: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, + promBody: PrometheusInstantQueryResponse{ + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{}, + Sample: cortexpb.Sample{Value: 1266464.0146205237, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, { body: testHistogramResponse, @@ -195,10 +338,18 @@ func TestResponse(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() + json = jsoniter.Config{ + EscapeHTML: false, // No HTML in our responses. + SortMapKeys: true, + ValidateJsonRawMessage: false, + }.Froze() + protobuf, err := proto.Marshal(&tc.promBody) + require.NoError(t, err) + response := &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), } resp, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) require.NoError(t, err) @@ -207,8 +358,8 @@ func TestResponse(t *testing.T) { response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), - ContentLength: int64(len(tc.body)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResp))), + ContentLength: int64(len(tc.expectedResp)), } resp2, err := InstantQueryCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -225,7 +376,7 @@ func TestMergeResponse(t *testing.T) { for _, tc := range []struct { name string req tripperware.Request - resps []string + resps []*PrometheusInstantQueryResponse expectedResp string expectedErr error cancelBeforeDecode bool @@ -233,35 +384,175 @@ func TestMergeResponse(t *testing.T) { cancelBeforeMerge bool }{ { - name: "empty response", - req: defaultReq, - resps: []string{`{"status":"success","data":{"resultType":"vector","result":[]}}`}, + name: "empty response", + req: defaultReq, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: make([]*Sample, 0), + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[]}}`, }, { - name: "empty response with stats", - req: defaultReq, - resps: []string{`{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[]}}}}`}, + name: "empty response with stats", + req: defaultReq, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{}, + TotalQueryableSamples: 0, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[]}}}}`, }, { - name: "single response", - req: defaultReq, - resps: []string{`{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`}, + name: "single response", + req: defaultReq, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, }, { - name: "single response with stats", - req: defaultReq, - resps: []string{`{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`}, + name: "single response with stats", + req: defaultReq, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, }, { name: "duplicated response", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, }, @@ -274,18 +565,124 @@ func TestMergeResponse(t *testing.T) { { name: "duplicated response with stats", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "up"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, }, { name: "merge two responses", req: defaultReq, - 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":[2,"2"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, }, @@ -301,9 +698,55 @@ func TestMergeResponse(t *testing.T) { { name: "merge two responses with sort", req: &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"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, }, @@ -319,9 +762,55 @@ func TestMergeResponse(t *testing.T) { { name: "merge two responses with sort_desc", req: &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"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, }, @@ -337,9 +826,55 @@ func TestMergeResponse(t *testing.T) { { name: "merge two responses with topk", req: &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"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, }, @@ -364,62 +899,371 @@ func TestMergeResponse(t *testing.T) { { name: "merge two responses with stats", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, - `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, }, { name: "responses don't contain vector, should return an error", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"string","result":[1662682521.409,"foo"]}}`, - `{"status":"success","data":{"resultType":"string","result":[1662682521.409,"foo"]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"string","result":[1662682521.409,"foo"]}`), + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValString.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"string","result":[1662682521.409,"foo"]}`), + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedErr: fmt.Errorf("unexpected result type on instant query: %s", "string"), }, { name: "single matrix response", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up"},"values":[[1,"1"],[2,"2"]]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up"},"values":[[1,"1"],[2,"2"]]}]}}`, }, { name: "multiple matrix responses without duplicated series", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"foo"},"values":[[3,"3"],[4,"4"]]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "foo"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 3, TimestampMs: 3000}, + {Value: 4, TimestampMs: 4000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]},{"metric":{"__name__":"foo"},"values":[[3,"3"],[4,"4"]]}]}}`, }, { name: "multiple matrix responses with duplicated series, but not same samples", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[3,"3"]]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, }, { name: "multiple matrix responses with duplicated series and same samples", req: defaultReq, - resps: []string{ - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, - `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, }, { name: "context cancelled before decoding response", req: defaultReq, - 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":[2,"2"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedDecodeErr: context.Canceled, cancelBeforeDecode: true, @@ -427,9 +1271,55 @@ func TestMergeResponse(t *testing.T) { { name: "context cancelled before merging response", req: defaultReq, - 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":[2,"2"]}]}}`, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, expectedErr: context.Canceled, cancelBeforeMerge: true, @@ -442,10 +1332,11 @@ func TestMergeResponse(t *testing.T) { var resps []tripperware.Response for _, r := range tc.resps { + protobuf, err := proto.Marshal(r) hr := &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(r))), + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), } if tc.cancelBeforeDecode { @@ -473,7 +1364,7 @@ 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() }) } @@ -589,7 +1480,7 @@ func Benchmark_Decode(b *testing.B) { }, } - body, err := json.Marshal(r) + body, err := proto.Marshal(&r) require.NoError(b, err) b.ResetTimer() From ad7e03ca52536d9c119cb0b0d041e7f017973ea2 Mon Sep 17 00:00:00 2001 From: afayek Date: Mon, 14 Aug 2023 16:03:49 -0700 Subject: [PATCH 15/38] handle empty response errors Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 42 ++++++++----------- .../tripperware/instantquery/instant_query.go | 6 +++ 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 0e10c54e4c9..67a4dbe03ee 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -159,7 +159,7 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] switch accept { case applicationProtobuf: - data = createPrometheusInstantQueryResponse(&queryData{ + data, err = createPrometheusInstantQueryResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, @@ -177,6 +177,9 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro Stats: qs, } } + if err != nil { + return nil, res.Warnings, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, qry.Close + } return data, res.Warnings, nil, qry.Close } @@ -271,7 +274,7 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] switch accept { case applicationProtobuf: - data = createPrometheusResponse(&queryData{ + data, err = createPrometheusResponse(&queryData{ ResultType: res.Value.Type(), Result: res.Value, Stats: qs, @@ -290,6 +293,9 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, } } + if err != nil { + return nil, res.Warnings, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, qry.Close + } return data, res.Warnings, nil, qry.Close } @@ -425,15 +431,9 @@ func (api *API) RespondError(w http.ResponseWriter, apiErr *thanos_api.ApiError, } } -func createPrometheusResponse(data *queryData) *queryrange.PrometheusResponse { +func createPrometheusResponse(data *queryData) (*queryrange.PrometheusResponse, error) { if data == nil { - return &queryrange.PrometheusResponse{ - Status: string(statusSuccess), - Data: queryrange.PrometheusData{}, - ErrorType: "", - Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, - } + return nil, errors.New("no query response data") } sampleStreams := getSampleStreams(data) @@ -454,29 +454,23 @@ func createPrometheusResponse(data *queryData) *queryrange.PrometheusResponse { ErrorType: "", Error: "", Headers: []*tripperware.PrometheusResponseHeader{}, - } + }, nil } -func createPrometheusInstantQueryResponse(data *queryData) *instantquery.PrometheusInstantQueryResponse { +func createPrometheusInstantQueryResponse(data *queryData) (*instantquery.PrometheusInstantQueryResponse, error) { if data == nil { - return &instantquery.PrometheusInstantQueryResponse{ - Status: string(statusSuccess), - Data: instantquery.PrometheusInstantQueryData{}, - ErrorType: "", - Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, - } + return nil, errors.New("no query response data") } var instantQueryResult instantquery.PrometheusInstantQueryResult - switch data.Result.Type() { - case parser.ValueTypeMatrix: + switch string(data.ResultType) { + case model.ValMatrix.String(): instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ Matrix: &instantquery.Matrix{ SampleStreams: *getSampleStreams(data), }, } - case parser.ValueTypeVector: + case model.ValVector.String(): instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ Vector: &instantquery.Vector{ Samples: *getSamples(data), @@ -485,7 +479,7 @@ func createPrometheusInstantQueryResponse(data *queryData) *instantquery.Prometh default: rawBytes, err := jsoniter.Marshal(data) if err != nil { - // TODO: handler error + return nil, err } instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_RawBytes{RawBytes: rawBytes} } @@ -506,7 +500,7 @@ func createPrometheusInstantQueryResponse(data *queryData) *instantquery.Prometh ErrorType: "", Error: "", Headers: []*tripperware.PrometheusResponseHeader{}, - } + }, nil } func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 61376dd0eeb..a1330551e8c 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -665,6 +665,9 @@ func (s *PrometheusInstantQueryData) MarshalJSON() ([]byte, error) { Data: s.Result.GetVector().Samples, Stats: s.Stats, } + if res.Data == nil { + res.Data = []*Sample{} + } return json.Marshal(res) case model.ValMatrix.String(): res := struct { @@ -676,6 +679,9 @@ func (s *PrometheusInstantQueryData) MarshalJSON() ([]byte, error) { Data: s.Result.GetMatrix().SampleStreams, Stats: s.Stats, } + if res.Data == nil { + res.Data = []tripperware.SampleStream{} + } return json.Marshal(res) default: return s.Result.GetRawBytes(), nil From 5d7ada297496f315563c18dcb49dfa8392d1b126 Mon Sep 17 00:00:00 2001 From: afayek Date: Wed, 16 Aug 2023 16:44:27 -0700 Subject: [PATCH 16/38] add config to specify compression type for querier handler response Signed-off-by: Ahmed Hassan --- pkg/cortex/modules.go | 6 +++--- pkg/querier/querier.go | 4 ++++ .../tripperware/instantquery/instant_query.go | 18 ++++++++++++------ .../tripperware/queryrange/query_range.go | 16 +++++++++++----- 4 files changed, 30 insertions(+), 14 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index de1f15d2604..4da65f5b5b7 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -442,9 +442,9 @@ func (t *Cortex) initFlusher() (serv services.Service, err error) { func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err error) { queryAnalyzer := querysharding.NewQueryAnalyzer() // PrometheusCodec is a codec to encode and decode Prometheus query range requests and responses. - prometheusCodec := queryrange.NewPrometheusCodec(false) + prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression) // ShardedPrometheusCodec is same as PrometheusCodec but to be used on the sharded queries (it sum up the stats) - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true) + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression) queryRangeMiddlewares, cache, err := queryrange.Middlewares( t.Cfg.QueryRange, @@ -472,7 +472,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro queryRangeMiddlewares, instantQueryMiddlewares, prometheusCodec, - instantquery.InstantQueryCodec, + instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression), t.Overrides, queryAnalyzer, t.Cfg.Querier.DefaultEvaluationInterval, diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index ac01c5e9ce6..e34338e0838 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -50,6 +50,9 @@ type Config struct { AtModifierEnabled bool `yaml:"at_modifier_enabled" doc:"hidden"` EnablePerStepStats bool `yaml:"per_step_stats_enabled"` + // Use compression when returning promql response. Supported values 'gzip', 'snappy', and '' (disable compression) + PrometheusCodecCompression string `yaml:"prometheus_codec_compression"` + // QueryStoreAfter the time after which queries should also be sent to the store and not just ingesters. QueryStoreAfter time.Duration `yaml:"query_store_after"` MaxQueryIntoFuture time.Duration `yaml:"max_query_into_future"` @@ -112,6 +115,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxSamples, "querier.max-samples", 50e6, "Maximum number of samples a single query can load into memory.") f.DurationVar(&cfg.QueryIngestersWithin, "querier.query-ingesters-within", 0, "Maximum lookback beyond which queries are not sent to ingester. 0 means all queries are sent to ingester.") f.BoolVar(&cfg.EnablePerStepStats, "querier.per-step-stats-enabled", false, "Enable returning samples stats per steps in query response.") + f.StringVar(&cfg.PrometheusCodecCompression, "querier.prometheus_codec_compression", "", "Use compression when returning promql response. Supported values 'gzip', 'snappy', and '' (disable compression)") f.DurationVar(&cfg.MaxQueryIntoFuture, "querier.max-query-into-future", 10*time.Minute, "Maximum duration into the future you can query. 0 to disable.") f.DurationVar(&cfg.DefaultEvaluationInterval, "querier.default-evaluation-interval", time.Minute, "The default evaluation interval or step size for subqueries.") f.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 0, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. When running the blocks storage, if this option is enabled, the time range of the query sent to the store will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index a1330551e8c..f1f1c7231dd 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -32,7 +32,7 @@ import ( ) var ( - InstantQueryCodec tripperware.Codec = newInstantQueryCodec() + InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("") json = jsoniter.Config{ EscapeHTML: false, // No HTML in our responses. @@ -109,11 +109,15 @@ func (r *PrometheusRequest) WithStats(stats string) tripperware.Request { type instantQueryCodec struct { tripperware.Codec - now func() time.Time + compression string + now func() time.Time } -func newInstantQueryCodec() instantQueryCodec { - return instantQueryCodec{now: time.Now} +func NewInstantQueryCodec(compression string) instantQueryCodec { + return instantQueryCodec{ + compression: compression, + now: time.Now, + } } func (resp *PrometheusInstantQueryResponse) HTTPHeaders() map[string][]string { @@ -180,7 +184,7 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ return &resp, nil } -func (instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { +func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { promReq, ok := r.(*PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") @@ -207,7 +211,9 @@ func (instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } - h.Set("Accept-Encoding", "snappy") + if c.compression == "snappy" || c.compression == "gzip" { + h.Set("Accept-Encoding", c.compression) + } h.Set("Accept", "application/x-protobuf") req := &http.Request{ diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index dcca03cf819..e08b44462fb 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -46,11 +46,15 @@ var ( ) type prometheusCodec struct { - sharded bool + sharded bool + compression string } -func NewPrometheusCodec(sharded bool) *prometheusCodec { //nolint:revive - return &prometheusCodec{sharded: sharded} +func NewPrometheusCodec(sharded bool, compression string) *prometheusCodec { //nolint:revive + return &prometheusCodec{ + sharded: sharded, + compression: compression, + } } // WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. @@ -220,7 +224,7 @@ func (c prometheusCodec) DecodeRequest(_ context.Context, r *http.Request, forwa return &result, nil } -func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { +func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { promReq, ok := r.(*PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") @@ -244,7 +248,9 @@ func (prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) } } - h.Set("Accept-Encoding", "snappy") + if c.compression == "snappy" || c.compression == "gzip" { + h.Set("Accept-Encoding", c.compression) + } h.Set("Accept", "application/x-protobuf") req := &http.Request{ From 661b0780854521b6f596c9d5e580e9cbb5c72856 Mon Sep 17 00:00:00 2001 From: afayek Date: Mon, 21 Aug 2023 16:58:58 -0700 Subject: [PATCH 17/38] change unit tests to work with new querier handler Signed-off-by: Ahmed Hassan --- .../instantquery/instant_query_test.go | 88 +- .../instantquery/shard_by_query_test.go | 872 +++++++++++++++++- .../tripperware/queryrange/marshaling_test.go | 3 +- .../query_range_middlewares_test.go | 11 +- .../queryrange/query_range_test.go | 181 +++- .../queryrange/results_cache_test.go | 2 +- .../queryrange/split_by_interval_test.go | 7 +- pkg/querier/tripperware/roundtrip_test.go | 6 +- 8 files changed, 1082 insertions(+), 88 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 7e5e5cda736..f50361192e8 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -7,7 +7,7 @@ import ( "fmt" "github.com/gogo/protobuf/proto" "github.com/golang/snappy" - jsoniter "github.com/json-iterator/go" + "github.com/weaveworks/common/httpgrpc" "io" "net/http" "strconv" @@ -111,14 +111,15 @@ func TestCompressedResponse(t *testing.T) { for _, tc := range []struct { name string compression string - promBody PrometheusInstantQueryResponse + jsonBody string + promBody *PrometheusInstantQueryResponse status int err error }{ { name: "successful response", compression: "gzip", - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), @@ -126,7 +127,7 @@ func TestCompressedResponse(t *testing.T) { }, Headers: []*tripperware.PrometheusResponseHeader{ {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - {Name: "Content-Encoding", Values: []string{"snappy"}}, + {Name: "Content-Encoding", Values: []string{"gzip"}}, }, }, status: 200, @@ -134,7 +135,7 @@ func TestCompressedResponse(t *testing.T) { { name: "successful response", compression: "snappy", - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), @@ -147,30 +148,62 @@ func TestCompressedResponse(t *testing.T) { }, status: 200, }, + { + name: `400 error`, + compression: `gzip`, + jsonBody: `error generic 400`, + status: 400, + err: httpgrpc.Errorf(400, `error generic 400`), + }, + { + name: `400 error`, + compression: `snappy`, + jsonBody: `error generic 400`, + status: 400, + err: httpgrpc.Errorf(400, `error generic 400`), + }, + { + name: `400 error empty body`, + compression: `gzip`, + status: 400, + err: httpgrpc.Errorf(400, ""), + }, + { + name: `400 error empty body`, + compression: `snappy`, + status: 400, + err: httpgrpc.Errorf(400, ""), + }, } { for _, c := range []bool{true, false} { c := c t.Run(fmt.Sprintf("%s compressed %t [%s]", tc.compression, c, tc.name), func(t *testing.T) { t.Parallel() - protobuf, err := proto.Marshal(&tc.promBody) - require.NoError(t, err) - responseBody := bytes.NewBuffer(protobuf) - h := http.Header{ - "Content-Type": []string{"application/x-protobuf"}, + h := http.Header{} + var b []byte + if tc.promBody != nil { + protobuf, err := proto.Marshal(tc.promBody) + b = protobuf + require.NoError(t, err) + h.Set("Content-Type", "application/x-protobuf") + } else { + b = []byte(tc.jsonBody) + h.Set("Content-Type", "application/json") } + responseBody := bytes.NewBuffer(b) var buf bytes.Buffer if c && tc.compression == "gzip" { h.Set("Content-Encoding", "gzip") w := gzip.NewWriter(&buf) - _, err := w.Write(protobuf) + _, err := w.Write(b) require.NoError(t, err) w.Close() responseBody = &buf } else if c && tc.compression == "snappy" { h.Set("Content-Encoding", "snappy") w := snappy.NewBufferedWriter(&buf) - _, err := w.Write(protobuf) + _, err := w.Write(b) require.NoError(t, err) w.Close() responseBody = &buf @@ -181,13 +214,12 @@ func TestCompressedResponse(t *testing.T) { Header: h, Body: io.NopCloser(responseBody), } - r, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) + resp, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) require.Equal(t, tc.err, err) if err == nil { - resp, err := proto.Marshal(r) require.NoError(t, err) - require.Equal(t, protobuf, resp) + require.Equal(t, tc.promBody, resp) } }) } @@ -198,11 +230,11 @@ func TestResponse(t *testing.T) { t.Parallel() for i, tc := range []struct { expectedResp string - promBody PrometheusInstantQueryResponse + promBody *PrometheusInstantQueryResponse }{ { expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), @@ -215,7 +247,7 @@ func TestResponse(t *testing.T) { }, { expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), @@ -228,7 +260,7 @@ func TestResponse(t *testing.T) { }, { expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValMatrix.String(), @@ -266,7 +298,7 @@ func TestResponse(t *testing.T) { }, { expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValMatrix.String(), @@ -295,7 +327,7 @@ func TestResponse(t *testing.T) { }, { expectedResp: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), @@ -308,7 +340,7 @@ func TestResponse(t *testing.T) { }, { expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, - promBody: PrometheusInstantQueryResponse{ + promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValVector.String(), @@ -337,13 +369,7 @@ func TestResponse(t *testing.T) { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - - json = jsoniter.Config{ - EscapeHTML: false, // No HTML in our responses. - SortMapKeys: true, - ValidateJsonRawMessage: false, - }.Froze() - protobuf, err := proto.Marshal(&tc.promBody) + protobuf, err := proto.Marshal(tc.promBody) require.NoError(t, err) response := &http.Response{ @@ -1049,7 +1075,7 @@ func TestMergeResponse(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "foo"}, + {"__name__", "bar"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1075,7 +1101,7 @@ func TestMergeResponse(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {"__name__", "foo"}, }, Samples: []cortexpb.Sample{ {Value: 3, TimestampMs: 3000}, diff --git a/pkg/querier/tripperware/instantquery/shard_by_query_test.go b/pkg/querier/tripperware/instantquery/shard_by_query_test.go index 0d4dfc41a95..f74adf66b16 100644 --- a/pkg/querier/tripperware/instantquery/shard_by_query_test.go +++ b/pkg/querier/tripperware/instantquery/shard_by_query_test.go @@ -1,13 +1,879 @@ package instantquery import ( + "context" + "encoding/base64" + "fmt" + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/querysharding" + "github.com/go-kit/log" + "github.com/gogo/protobuf/proto" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/promql/parser" + "github.com/stretchr/testify/require" + thanosquerysharding "github.com/thanos-io/thanos/pkg/querysharding" + "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/weaveworks/common/user" + "io" + "net/http" + "net/http/httptest" + "net/url" + "sort" "testing" - "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" ) func Test_shardQuery(t *testing.T) { - t.Parallel() - tripperware.TestQueryShardQuery(t, InstantQueryCodec, queryrange.NewPrometheusCodec(true)) + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, "") + instantQueryCodec := InstantQueryCodec + + type queries struct { + name string + expression string + shardingLabels []string + } + + nonShardable := []queries{ + { + name: "aggregation", + expression: "sum(http_requests_total)", + }, + { + name: "outer aggregation with no grouping", + expression: "count(sum by (pod) (http_requests_total))", + }, + { + name: "outer aggregation with without grouping", + expression: "count(sum without (pod) (http_requests_total))", + }, + { + name: "binary expression with constant", + expression: `http_requests_total{code="400"} / 4`, + }, + { + name: "binary expression with empty vector matching", + expression: `http_requests_total{code="400"} / on () http_requests_total`, + }, + { + name: "binary aggregation with different grouping labels", + expression: `sum by (pod) (http_requests_total{code="400"}) / sum by (cluster) (http_requests_total)`, + }, + { + name: "multiple binary expressions with empty vector matchers", + expression: ` +(http_requests_total{code="400"} + on (cluster, pod) http_requests_total{code="500"}) +/ on () +http_requests_total`, + }, + { + name: "problematic query", + expression: `sum(a by(lanel)`, + }, + { + name: "aggregate by expression with label_replace, sharding label is dynamic", + expression: `sum by (dst_label) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, + }, + { + name: "aggregate by expression with label_join, sharding label is dynamic", + expression: `sum by (dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, + }, + } + + shardableByLabels := []queries{ + { + name: "aggregation with grouping", + expression: "sum by (pod) (http_requests_total)", + shardingLabels: []string{"pod"}, + }, + { + name: "aggregation with comparison", + expression: "avg by (Roles,type) (rss_service_message_handling) > 0.5", + shardingLabels: []string{"Roles", "type"}, + }, + { + name: "multiple aggregations with grouping", + expression: "max by (pod) (sum by (pod, cluster) (http_requests_total))", + shardingLabels: []string{"pod"}, + }, + { + name: "binary expression with vector matching", + expression: `http_requests_total{code="400"} / on (pod) http_requests_total`, + shardingLabels: []string{"pod"}, + }, + { + name: "binary aggregation with same grouping labels", + expression: `sum by (pod) (http_requests_total{code="400"}) / sum by (pod) (http_requests_total)`, + shardingLabels: []string{"pod"}, + }, + { + name: "binary expression with vector matching and grouping", + expression: `sum by (cluster, pod) (http_requests_total{code="400"}) / on (pod) sum by (cluster, pod) (http_requests_total)`, + shardingLabels: []string{"pod"}, + }, + { + name: "multiple binary expressions with vector matchers", + expression: ` +(http_requests_total{code="400"} + on (cluster, pod) http_requests_total{code="500"}) +/ on (pod) +http_requests_total`, + shardingLabels: []string{"pod"}, + }, + { + name: "multiple binary expressions with grouping", + expression: ` +sum by (container) ( + (http_requests_total{code="400"} + on (cluster, pod, container) http_requests_total{code="500"}) + / on (pod, container) + http_requests_total +)`, + shardingLabels: []string{"container"}, + }, + { + name: "multiple binary expressions with grouping", + expression: `(http_requests_total{code="400"} + on (pod) http_requests_total{code="500"}) / on (cluster, pod) http_requests_total`, + shardingLabels: []string{"pod"}, + }, + { + name: "histogram quantile", + expression: "histogram_quantile(0.95, sum(rate(metric[1m])) by (le, cluster))", + shardingLabels: []string{"cluster"}, + }, + { + name: "aggregate by expression with label_replace, sharding label is not dynamic", + expression: `sum by (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, + shardingLabels: []string{"pod"}, + }, + { + name: "aggregate by expression with label_join, sharding label is not dynamic", + expression: `sum by (pod) (label_join(metric, "dst_label", ",", "src_label"))`, + shardingLabels: []string{"pod"}, + }, + { + name: "label_join and aggregation on multiple labels. Can be sharded by the static one", + expression: `sum by (pod, dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, + shardingLabels: []string{"pod"}, + }, + { + name: "binary expression with vector matching and label_replace", + expression: `http_requests_total{code="400"} / on (pod) label_replace(metric, "dst_label", "$1", "src_label", "re")`, + shardingLabels: []string{"pod"}, + }, + { + name: "nested label joins", + expression: `label_join(sum by (pod) (label_join(metric, "dst_label", ",", "src_label")), "dst_label1", ",", "dst_label")`, + shardingLabels: []string{"pod"}, + }, + { + name: "complex query with label_replace, binary expr and aggregations on dynamic label", + expression: `sum(sum_over_time(container_memory_working_set_bytes{container_name!="POD",container_name!="",namespace="kube-system"}[1d:5m])) by (instance, cluster) / avg(label_replace(sum(sum_over_time(kube_node_status_capacity_memory_bytes[1d:5m])) by (node, cluster), "instance", "$1", "node", "(.*)")) by (instance, cluster)`, + shardingLabels: []string{"cluster"}, + }, + { + name: "complex query with label_replace and nested aggregations", + expression: `avg(label_replace(label_replace(avg(count_over_time(kube_pod_container_resource_requests{resource="memory", unit="byte", container!="",container!="POD", node!="", }[1h] )*avg_over_time(kube_pod_container_resource_requests{resource="memory", unit="byte", container!="",container!="POD", node!="", }[1h] )) by (namespace,container,pod,node,cluster_id) , "container_name","$1","container","(.+)"), "pod_name","$1","pod","(.+)")) by (namespace,container_name,pod_name,node,cluster_id)`, + shardingLabels: []string{"namespace", "node", "cluster_id"}, + }, + { + name: "complex query with label_replace, nested aggregations and binary expressions", + expression: `sort_desc(avg(label_replace(label_replace(label_replace(count_over_time(container_memory_working_set_bytes{container!="", container!="POD", instance!="", }[1h] ), "node", "$1", "instance", "(.+)"), "container_name", "$1", "container", "(.+)"), "pod_name", "$1", "pod", "(.+)")*label_replace(label_replace(label_replace(avg_over_time(container_memory_working_set_bytes{container!="", container!="POD", instance!="", }[1h] ), "node", "$1", "instance", "(.+)"), "container_name", "$1", "container", "(.+)"), "pod_name", "$1", "pod", "(.+)")) by (namespace, container_name, pod_name, node, cluster_id))`, + shardingLabels: []string{"namespace", "cluster_id"}, + }, + { + name: "aggregate expression with label_replace", + expression: `sum by (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, + shardingLabels: []string{"pod"}, + }, + } + + // Shardable by labels instant queries with matrix response + shardableByLabelsMatrix := []queries{ + { + name: "subquery", + expression: "sum(http_requests_total) by (pod, cluster) [1h:1m]", + shardingLabels: []string{"cluster", "pod"}, + }, + { + name: "subquery with function", + expression: "increase(sum(http_requests_total) by (pod, cluster) [1h:1m])", + shardingLabels: []string{"cluster", "pod"}, + }, + { + name: "ignore vector matching with 2 aggregations", + expression: `sum(rate(node_cpu_seconds_total[3h])) by (cluster_id, mode) / ignoring(mode) group_left sum(rate(node_cpu_seconds_total[3h])) by (cluster_id)`, + shardingLabels: []string{"cluster_id"}, + }, + } + + shardableWithoutLabels := []queries{ + { + name: "aggregation without grouping", + expression: "sum without (pod) (http_requests_total)", + shardingLabels: []string{"pod"}, + }, + { + name: "multiple aggregations with without grouping", + expression: "max without (pod) (sum without (pod, cluster) (http_requests_total))", + shardingLabels: []string{"pod", "cluster"}, + }, + { + name: "binary expression with without vector matching and grouping", + expression: `sum without (cluster, pod) (http_requests_total{code="400"}) / ignoring (pod) sum without (cluster, pod) (http_requests_total)`, + shardingLabels: []string{"pod", "cluster", model.MetricNameLabel}, + }, + { + name: "binary expression with outer without grouping", + expression: `sum(http_requests_total{code="400"} * http_requests_total) without (pod)`, + shardingLabels: []string{model.MetricNameLabel, "pod"}, + }, + { + name: "binary expression with vector matching and outer without grouping", + expression: `sum(http_requests_total{code="400"} * ignoring(cluster) http_requests_total) without ()`, + shardingLabels: []string{"__name__", "cluster"}, + }, + { + name: "multiple binary expressions with without grouping", + expression: `(http_requests_total{code="400"} + ignoring (pod) http_requests_total{code="500"}) / ignoring (cluster, pod) http_requests_total`, + shardingLabels: []string{"cluster", "pod", model.MetricNameLabel}, + }, + { + name: "multiple binary expressions with without vector matchers", + expression: ` +(http_requests_total{code="400"} + ignoring (cluster, pod) http_requests_total{code="500"}) +/ ignoring (pod) +http_requests_total`, + shardingLabels: []string{"cluster", "pod", model.MetricNameLabel}, + }, + { + name: "aggregate without expression with label_replace, sharding label is not dynamic", + expression: `sum without (dst_label) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, + shardingLabels: []string{"dst_label"}, + }, + { + name: "aggregate without expression with label_join, sharding label is not dynamic", + expression: `sum without (dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, + shardingLabels: []string{"dst_label"}, + }, + { + name: "aggregate without expression with label_replace", + expression: `sum without (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, + shardingLabels: []string{"pod", "dst_label"}, + }, + { + name: "binary expression", + expression: `http_requests_total{code="400"} / http_requests_total`, + shardingLabels: []string{model.MetricNameLabel}, + }, + { + name: "binary expression among vector and scalar", + expression: `aaaa - bbb > 1000`, + shardingLabels: []string{model.MetricNameLabel}, + }, + { + name: "binary expression with set operation", + expression: `aaaa and bbb`, + shardingLabels: []string{model.MetricNameLabel}, + }, + { + name: "multiple binary expressions", + expression: `(http_requests_total{code="400"} + http_requests_total{code="500"}) / http_requests_total`, + shardingLabels: []string{model.MetricNameLabel}, + }, + } + + type testCase struct { + name string + path string + isShardable bool + shardSize int + codec tripperware.Codec + instantQueryResponses []*PrometheusInstantQueryResponse + queryRangeResponses []*queryrange.PrometheusResponse + response string + shardingLabels []string + } + tests := []testCase{ + { + name: "should shard range query when query is shardable", + path: `/api/v1/query_range?end=1&start=0&step=120&query=sum(metric) by (pod,cluster_name)`, + isShardable: true, + codec: shardedPrometheusCodec, + shardingLabels: []string{"pod", "cluster_name"}, + shardSize: 2, + queryRangeResponses: []*queryrange.PrometheusResponse{ + { + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "metric"}, + {Name: "__job__", Value: "a"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + TotalQueryableSamples: 6, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "metric"}, + {Name: "__job__", Value: "b"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + TotalQueryableSamples: 6, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]},{"metric":{"__job__":"b","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":12,"totalQueryableSamplesPerStep":[[1,2],[2,4],[3,6]]}}}}`, + }, + { + name: "should shard instant query when query is shardable", + path: `/api/v1/query?time=120&query=sum(metric) by (pod,cluster_name)`, + codec: instantQueryCodec, + shardSize: 2, + shardingLabels: []string{"pod", "cluster_name"}, + isShardable: true, + instantQueryResponses: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + }, + { + name: "should not shard if shard size is 1", + path: `/api/v1/query?time=120&query=sum(metric) by (pod,cluster_name)`, + codec: instantQueryCodec, + shardSize: 1, + isShardable: false, + instantQueryResponses: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + }, + } + for _, query := range nonShardable { + tests = append(tests, testCase{ + name: fmt.Sprintf("non shardable query: %s", query.name), + path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), + codec: instantQueryCodec, + isShardable: false, + instantQueryResponses: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + }) + tests = append(tests, testCase{ + name: fmt.Sprintf("non shardable query_range: %s", query.name), + path: fmt.Sprintf(`/api/v1/query_range?start=1&end=2&step=1&query=%s`, url.QueryEscape(query.expression)), + codec: shardedPrometheusCodec, + isShardable: false, + queryRangeResponses: []*queryrange.PrometheusResponse{ + { + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__job__", Value: "a"}, + {Name: "__name__", Value: "metric"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + TotalQueryableSamples: 6, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":6,"totalQueryableSamplesPerStep":[[1,1],[2,2],[3,3]]}}}}`, + }) + } + + for _, query := range append(shardableWithoutLabels, shardableByLabels...) { + tests = append(tests, testCase{ + name: fmt.Sprintf("shardable query: %s", query.name), + path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), + codec: instantQueryCodec, + isShardable: true, + shardSize: 2, + shardingLabels: query.shardingLabels, + instantQueryResponses: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + }) + tests = append(tests, testCase{ + name: fmt.Sprintf("shardable query_range: %s", query.name), + path: fmt.Sprintf(`/api/v1/query_range?start=1&end=2&step=1&query=%s`, url.QueryEscape(query.expression)), + codec: shardedPrometheusCodec, + isShardable: true, + shardSize: 2, + shardingLabels: query.shardingLabels, + queryRangeResponses: []*queryrange.PrometheusResponse{ + { + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "metric"}, + {Name: "__job__", Value: "a"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + TotalQueryableSamples: 6, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "metric"}, + {Name: "__job__", Value: "b"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + TotalQueryableSamples: 6, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]},{"metric":{"__job__":"b","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":12,"totalQueryableSamplesPerStep":[[1,2],[2,4],[3,6]]}}}}`, + }) + } + + for _, query := range shardableByLabelsMatrix { + tests = append(tests, testCase{ + name: fmt.Sprintf("shardable query: %s", query.name), + path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), + codec: instantQueryCodec, + isShardable: true, + shardSize: 2, + shardingLabels: query.shardingLabels, + instantQueryResponses: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 2, TimestampMs: 2000}, + }, + }, + }, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 10, TimestampMs: 1000}, + }, + TotalQueryableSamples: 10, + }, + }, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + }, + response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up","job":"bar"},"values":[[2,"2"]]},{"metric":{"__name__":"up","job":"foo"},"values":[[1,"1"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + }) + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + //parallel testing causes data race + sort.Strings(tt.shardingLabels) + s := httptest.NewServer( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + q := r.FormValue("query") + expr, _ := parser.ParseExpr(q) + shardIndex := int64(0) + + parser.Inspect(expr, func(n parser.Node, _ []parser.Node) error { + if selector, ok := n.(*parser.VectorSelector); ok { + for _, matcher := range selector.LabelMatchers { + if matcher.Name == querysharding.CortexShardByLabel { + + decoded, _ := base64.StdEncoding.DecodeString(matcher.Value) + shardInfo := storepb.ShardInfo{} + err := shardInfo.Unmarshal(decoded) + require.NoError(t, err) + sort.Strings(shardInfo.Labels) + require.Equal(t, tt.shardingLabels, shardInfo.Labels) + require.Equal(t, tt.isShardable, shardInfo.TotalShards > 0) + shardIndex = shardInfo.ShardIndex + } + } + } + return nil + }) + if tt.instantQueryResponses != nil { + protobuf, err := proto.Marshal(tt.instantQueryResponses[shardIndex]) + require.NoError(t, err) + _, _ = w.Write(protobuf) + } else { + protobuf, err := proto.Marshal(tt.queryRangeResponses[shardIndex]) + require.NoError(t, err) + _, _ = w.Write(protobuf) + } + }), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := tripperware.SingleHostRoundTripper{ + Host: u.Host, + Next: http.DefaultTransport, + } + + qa := thanosquerysharding.NewQueryAnalyzer() + roundtripper := tripperware.NewRoundTripper(downstream, tt.codec, nil, tripperware.ShardByMiddleware(log.NewNopLogger(), tripperware.MockLimits{ShardSize: tt.shardSize}, tt.codec, qa)) + + ctx := user.InjectOrgID(context.Background(), "1") + + req, err := http.NewRequest("GET", tt.path, http.NoBody) + req = req.WithContext(ctx) + + require.NoError(t, err) + resp, err := roundtripper.RoundTrip(req) + + require.NoError(t, err) + require.NotNil(t, resp) + + contents, err := io.ReadAll(resp.Body) + require.NoError(t, err) + require.Equal(t, tt.response, string(contents)) + }) + } + } diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index 47ac83d9c5c..7d5e57250af 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -3,6 +3,7 @@ package queryrange import ( "bytes" "context" + "github.com/gogo/protobuf/proto" io "io" "math/rand" "net/http" @@ -22,7 +23,7 @@ func BenchmarkPrometheusCodec_DecodeResponse(b *testing.B) { // Generate a mocked response and marshal it. res := mockPrometheusResponse(numSeries, numSamplesPerSeries) - encodedRes, err := json.Marshal(res) + encodedRes, err := proto.Marshal(res) require.NoError(b, err) b.Log("test prometheus response size:", len(encodedRes)) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 526295ac9a7..c72dd3a8155 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -2,6 +2,7 @@ package queryrange import ( "context" + "github.com/gogo/protobuf/proto" "io" "net/http" "net/http/httptest" @@ -20,8 +21,8 @@ import ( ) var ( - PrometheusCodec = NewPrometheusCodec(false) - ShardedPrometheusCodec = NewPrometheusCodec(false) + PrometheusCodec = NewPrometheusCodec(false, "") + ShardedPrometheusCodec = NewPrometheusCodec(false, "") ) func TestRoundTrip(t *testing.T) { @@ -31,7 +32,11 @@ func TestRoundTrip(t *testing.T) { http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { var err error if r.RequestURI == query { - _, err = w.Write([]byte(responseBody)) + resp := parsedResponse + resp.Headers = respHeaders + protobuf, err := proto.Marshal(resp) + require.NoError(t, err) + _, err = w.Write(protobuf) } else if r.RequestURI == queryWithWarnings { _, err = w.Write([]byte(responseBodyWithWarnings)) } else { diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 0da8489188e..0b5d76c598a 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -5,6 +5,8 @@ import ( "compress/gzip" "context" "fmt" + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" "io" "net/http" "strconv" @@ -95,21 +97,17 @@ func TestResponse(t *testing.T) { r.Headers = respHeaders rWithWarnings.Headers = respHeaders for i, tc := range []struct { - body string - expected *PrometheusResponse + promBody *PrometheusResponse + expectedResponse string expectedDecodeErr error cancelCtxBeforeDecode bool }{ { - body: responseBody, - expected: &r, + promBody: &r, + expectedResponse: responseBody, }, { - body: responseBodyWithWarnings, - expected: &rWithWarnings, - }, - { - body: responseBody, + promBody: &r, cancelCtxBeforeDecode: true, expectedDecodeErr: context.Canceled, }, @@ -117,11 +115,13 @@ func TestResponse(t *testing.T) { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() + protobuf, err := proto.Marshal(tc.promBody) + require.NoError(t, err) ctx, cancelCtx := context.WithCancel(context.Background()) response := &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), } if tc.cancelCtxBeforeDecode { cancelCtx() @@ -132,14 +132,13 @@ func TestResponse(t *testing.T) { cancelCtx() return } - assert.Equal(t, tc.expected, resp) // Reset response, as the above call will have consumed the body reader. response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), - ContentLength: int64(len(tc.body)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResponse))), + ContentLength: int64(len(tc.expectedResponse)), } resp2, err := PrometheusCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -152,12 +151,12 @@ func TestResponse(t *testing.T) { func TestResponseWithStats(t *testing.T) { t.Parallel() for i, tc := range []struct { - body string - expected *PrometheusResponse + promBody *PrometheusResponse + expectedResponse string }{ { - 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{ + expectedResponse: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + promBody: &PrometheusResponse{ Status: "success", Data: PrometheusData{ ResultType: model.ValMatrix.String(), @@ -188,22 +187,23 @@ func TestResponseWithStats(t *testing.T) { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - tc.expected.Headers = respHeaders + tc.promBody.Headers = respHeaders + protobuf, err := proto.Marshal(tc.promBody) + require.NoError(t, err) response := &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), } resp, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) require.NoError(t, err) - assert.Equal(t, tc.expected, resp) // Reset response, as the above call will have consumed the body reader. response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.body))), - ContentLength: int64(len(tc.body)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResponse))), + ContentLength: int64(len(tc.expectedResponse)), } resp2, err := PrometheusCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -755,41 +755,134 @@ func TestMergeAPIResponses(t *testing.T) { } } -func TestGzippedResponse(t *testing.T) { +func TestCompressedResponse(t *testing.T) { t.Parallel() for _, tc := range []struct { - body string - status int - err error + name string + compression string + jsonBody string + promBody *PrometheusResponse + status int + err error }{ { - body: `{"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]]}}}}`, + name: `successful response`, + compression: `gzip`, + promBody: &PrometheusResponse{ + Status: StatusSuccess, + Data: 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}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: 20, + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + {Name: "Content-Encoding", Values: []string{"gzip"}}, + }, + }, + status: 200, + }, + { + name: `successful response`, + compression: `snappy`, + promBody: &PrometheusResponse{ + Status: StatusSuccess, + Data: 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}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: 20, + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }}, + }, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + {Name: "Content-Encoding", Values: []string{"snappy"}}, + }, + }, status: 200, }, { - body: `error generic 400`, - status: 400, - err: httpgrpc.Errorf(400, `error generic 400`), + name: `400 error`, + compression: `gzip`, + jsonBody: `error generic 400`, + status: 400, + err: httpgrpc.Errorf(400, `error generic 400`), + }, + { + name: `400 error`, + compression: `snappy`, + jsonBody: `error generic 400`, + status: 400, + err: httpgrpc.Errorf(400, `error generic 400`), }, { - status: 400, - err: httpgrpc.Errorf(400, ""), + name: `400 error empty body`, + compression: `gzip`, + status: 400, + err: httpgrpc.Errorf(400, ""), + }, + { + name: `400 error empty body`, + compression: `snappy`, + status: 400, + err: httpgrpc.Errorf(400, ""), }, } { for _, c := range []bool{true, false} { c := c - t.Run(fmt.Sprintf("compressed %t [%s]", c, tc.body), func(t *testing.T) { + t.Run(fmt.Sprintf("%s compressed %t [%s]", tc.compression, c, tc.name), func(t *testing.T) { t.Parallel() - h := http.Header{ - "Content-Type": []string{"application/json"}, + h := http.Header{} + var b []byte + if tc.promBody != nil { + protobuf, err := proto.Marshal(tc.promBody) + b = protobuf + require.NoError(t, err) + h.Set("Content-Type", "application/x-protobuf") + } else { + b = []byte(tc.jsonBody) + h.Set("Content-Type", "application/json") } + responseBody := bytes.NewBuffer(b) - responseBody := bytes.NewBuffer([]byte(tc.body)) - if c { + var buf bytes.Buffer + if c && tc.compression == "gzip" { h.Set("Content-Encoding", "gzip") - var buf bytes.Buffer w := gzip.NewWriter(&buf) - _, err := w.Write([]byte(tc.body)) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf + } else if c && tc.compression == "snappy" { + h.Set("Content-Encoding", "snappy") + w := snappy.NewBufferedWriter(&buf) + _, err := w.Write(b) require.NoError(t, err) w.Close() responseBody = &buf @@ -800,14 +893,12 @@ func TestGzippedResponse(t *testing.T) { Header: h, Body: io.NopCloser(responseBody), } - r, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) + resp, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) require.Equal(t, tc.err, err) if err == nil { - resp, err := json.Marshal(r) require.NoError(t, err) - - require.Equal(t, tc.body, string(resp)) + require.Equal(t, tc.promBody, resp) } }) } diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index abc216c4bd0..a1f90633cc9 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -62,7 +62,7 @@ var ( respHeaders = []*tripperware.PrometheusResponseHeader{ { Name: "Content-Type", - Values: []string{"application/json"}, + Values: []string{"application/x-protobuf"}, }, } parsedResponse = &PrometheusResponse{ diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 5004c60a800..730f00da1ec 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -2,6 +2,7 @@ package queryrange import ( "context" + "github.com/gogo/protobuf/proto" io "io" "net/http" "net/http/httptest" @@ -296,7 +297,11 @@ func TestSplitByDay(t *testing.T) { middleware.AuthenticateUser.Wrap( http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { actualCount.Inc() - _, _ = w.Write([]byte(responseBody)) + resp := parsedResponse + resp.Headers = respHeaders + protobuf, err := proto.Marshal(resp) + require.NoError(t, err) + _, _ = w.Write(protobuf) }), ), ) diff --git a/pkg/querier/tripperware/roundtrip_test.go b/pkg/querier/tripperware/roundtrip_test.go index 7e6cf474a0d..5ca4a69fb32 100644 --- a/pkg/querier/tripperware/roundtrip_test.go +++ b/pkg/querier/tripperware/roundtrip_test.go @@ -94,9 +94,9 @@ func TestRoundTrip(t *testing.T) { u, err := url.Parse(s.URL) require.NoError(t, err) - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, + downstream := SingleHostRoundTripper{ + Host: u.Host, + Next: http.DefaultTransport, } instantMiddlewares := []Middleware{ From 78b12ab473cb83ba35e7d1b419afa4e0e238fd1c Mon Sep 17 00:00:00 2001 From: afayek Date: Tue, 22 Aug 2023 17:59:47 -0700 Subject: [PATCH 18/38] fix time parsing in querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler.go | 90 ++++++++++++++++++++++++---------- 1 file changed, 65 insertions(+), 25 deletions(-) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 67a4dbe03ee..135956f424b 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" - "github.com/cortexproject/cortex/pkg/util" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gogo/protobuf/proto" @@ -30,6 +29,8 @@ import ( thanos_api "github.com/thanos-io/thanos/pkg/api" ) +type apiFunc func(r *http.Request) (interface{}, []error, *thanos_api.ApiError, func()) + type status string const ( @@ -107,8 +108,7 @@ func invalidParamError(err error, parameter string) (data interface{}, warnings } func (api *API) Query(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { - tms, err := instantquery.ParseTimeParam(r, "time", api.Now().Unix()) - ts := time.Unix(tms/1000, (tms%1000)*10e6).UTC() + ts, err := parseTimeParam(r, "time", api.Now()) if err != nil { return invalidParamError(err, "time") } @@ -188,7 +188,7 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { EnablePerStepStats: r.FormValue("stats") == "all", } if strDuration := r.FormValue("lookback_delta"); strDuration != "" { - duration, err := time.ParseDuration(strDuration) + duration, err := parseDuration(strDuration) if err != nil { return nil, fmt.Errorf("error parsing lookback delta duration: %w", err) } @@ -198,13 +198,11 @@ func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { } func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { - startMs, err := util.ParseTime(r.FormValue("start")) - start := time.Unix(startMs/1000, (startMs%1000)*10e6).UTC() + start, err := parseTime(r.FormValue("start")) if err != nil { return invalidParamError(err, "start") } - endMs, err := util.ParseTime(r.FormValue("end")) - end := time.Unix(endMs/1000, (endMs%1000)*10e6).UTC() + end, err := parseTime(r.FormValue("end")) if err != nil { return invalidParamError(err, "end") } @@ -299,6 +297,41 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, return data, res.Warnings, nil, qry.Close } +func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) (time.Time, error) { + val := r.FormValue(paramName) + if val == "" { + return defaultValue, nil + } + result, err := parseTime(val) + if err != nil { + return time.Time{}, errors.Wrapf(err, "Invalid time value for '%s'", paramName) + } + return result, nil +} + +func parseTime(s string) (time.Time, error) { + if t, err := strconv.ParseFloat(s, 64); err == nil { + s, ns := math.Modf(t) + ns = math.Round(ns*1000) / 1000 + return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil + } + if t, err := time.Parse(time.RFC3339Nano, s); err == nil { + return t, nil + } + + // Stdlib's time parser can only handle 4 digit years. As a workaround until + // that is fixed we want to at least support our own boundary times. + // Context: https://github.com/prometheus/client_golang/issues/614 + // Upstream issue: https://github.com/golang/go/issues/20555 + switch s { + case minTimeFormatted: + return minTime, nil + case maxTimeFormatted: + return maxTime, nil + } + return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) +} + func parseDuration(s string) (time.Duration, error) { if d, err := strconv.ParseFloat(s, 64); err == nil { ts := d * float64(time.Second) @@ -453,7 +486,6 @@ func createPrometheusResponse(data *queryData) (*queryrange.PrometheusResponse, }, ErrorType: "", Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, }, nil } @@ -499,7 +531,6 @@ func createPrometheusInstantQueryResponse(data *queryData) (*instantquery.Promet }, ErrorType: "", Error: "", - Headers: []*tripperware.PrometheusResponseHeader{}, }, nil } @@ -527,20 +558,26 @@ func getSampleStreams(data *queryData) *[]tripperware.SampleStream { for i := 0; i < sampleStreamsLen; i++ { labelsLen := len(data.Result.(promql.Matrix)[i].Metric) - labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Matrix)[i].Metric[j].Name, - Value: data.Result.(promql.Matrix)[i].Metric[j].Value, + var labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter + if labelsLen > 0 { + labels = make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[i].Metric[j].Name, + Value: data.Result.(promql.Matrix)[i].Metric[j].Value, + } } } samplesLen := len(data.Result.(promql.Matrix)[i].Floats) - samples := make([]cortexpb.Sample, samplesLen) - for j := 0; j < samplesLen; j++ { - samples[j] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[i].Floats[j].F, - TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, + var samples []cortexpb.Sample + if samplesLen > 0 { + samples = make([]cortexpb.Sample, samplesLen) + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, + } } } sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} @@ -554,11 +591,14 @@ func getSamples(data *queryData) *[]*instantquery.Sample { for i := 0; i < vectorSamplesLen; i++ { labelsLen := len(data.Result.(promql.Vector)[i].Metric) - labels := make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ - Name: data.Result.(promql.Vector)[i].Metric[j].Name, - Value: data.Result.(promql.Vector)[i].Metric[j].Value, + var labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter + if labelsLen > 0 { + labels = make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + Name: data.Result.(promql.Vector)[i].Metric[j].Name, + Value: data.Result.(promql.Vector)[i].Metric[j].Value, + } } } From a55d23a6cf15b30ee6f366046992c5aa1160f759 Mon Sep 17 00:00:00 2001 From: afayek Date: Tue, 22 Aug 2023 18:02:44 -0700 Subject: [PATCH 19/38] add unit tests for querier handler Signed-off-by: Ahmed Hassan --- pkg/querier/handler/handler_test.go | 1276 +++++++++++++++++ .../prometheus/common/promlog/log.go | 189 +++ vendor/modules.txt | 1 + 3 files changed, 1466 insertions(+) create mode 100644 pkg/querier/handler/handler_test.go create mode 100644 vendor/github.com/prometheus/common/promlog/log.go diff --git a/pkg/querier/handler/handler_test.go b/pkg/querier/handler/handler_test.go new file mode 100644 index 00000000000..a7ca1256e2b --- /dev/null +++ b/pkg/querier/handler/handler_test.go @@ -0,0 +1,1276 @@ +package handler + +import ( + "context" + "encoding/json" + "fmt" + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" + "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" + "github.com/gogo/protobuf/proto" + "github.com/prometheus/client_golang/prometheus" + config_util "github.com/prometheus/common/config" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote" + v1 "github.com/prometheus/prometheus/web/api/v1" + thanos_api "github.com/thanos-io/thanos/pkg/api" + "io" + "net/http" + "net/http/httptest" + "net/url" + "reflect" + "runtime" + "strconv" + "strings" + "testing" + "time" + + "github.com/pkg/errors" + "github.com/prometheus/prometheus/util/stats" + "github.com/stretchr/testify/require" + + "github.com/prometheus/common/promlog" + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" +) + +var ( + testResponse = &queryrange.PrometheusResponse{ + Status: "success", + Data: queryrange.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}, + }, + }, + }, + }, + } +) + +func TestEndpoints(t *testing.T) { + suite, err := promql.NewTest(t, ``) + + require.NoError(t, err) + defer suite.Close() + + require.NoError(t, suite.Run()) + + now := time.Now() + t.Run("local", func(t *testing.T) { + api := &API{ + Queryable: suite.Storage(), + QueryEngine: suite.QueryEngine(), + Now: func() time.Time { return now }, + } + testEndpoints(t, api) + }) + + // Run all the API tests against a API that is wired to forward queries via + // the remote read client to a test server, which in turn sends them to the + // data from the test suite. + t.Run("remote", func(t *testing.T) { + server := setupRemote(suite.Storage()) + defer server.Close() + + u, err := url.Parse(server.URL) + require.NoError(t, err) + + al := promlog.AllowedLevel{} + require.NoError(t, al.Set("debug")) + + af := promlog.AllowedFormat{} + require.NoError(t, af.Set("logfmt")) + + promlogConfig := promlog.Config{ + Level: &al, + Format: &af, + } + + dbDir := t.TempDir() + + remote := remote.NewStorage(promlog.New(&promlogConfig), prometheus.DefaultRegisterer, func() (int64, error) { + return 0, nil + }, dbDir, 1*time.Second, nil) + + err = remote.ApplyConfig(&config.Config{ + RemoteReadConfigs: []*config.RemoteReadConfig{ + { + URL: &config_util.URL{URL: u}, + RemoteTimeout: model.Duration(1 * time.Second), + ReadRecent: true, + }, + }, + }) + require.NoError(t, err) + + api := &API{ + Queryable: remote, + QueryEngine: suite.QueryEngine(), + Now: func() time.Time { return now }, + } + testEndpoints(t, api) + }) +} + +type testStats struct { + Custom string `json:"custom"` +} + +func (testStats) Builtin() (_ stats.BuiltinStats) { + return +} + +func TestStats(t *testing.T) { + suite, err := promql.NewTest(t, ``) + require.NoError(t, err) + defer suite.Close() + require.NoError(t, suite.Run()) + + api := &API{ + Queryable: suite.Storage(), + QueryEngine: suite.QueryEngine(), + Now: func() time.Time { + return time.Unix(123, 0) + }, + } + request := func(method, param string) (*http.Request, error) { + u, err := url.Parse("http://example.com") + require.NoError(t, err) + q := u.Query() + q.Add("stats", param) + q.Add("query", "up") + q.Add("start", "0") + q.Add("end", "100") + q.Add("step", "10") + u.RawQuery = q.Encode() + + r, err := http.NewRequest(method, u.String(), nil) + if method == http.MethodPost { + r.Header.Set("Content-Type", "application/x-www-form-urlencoded") + } + return r, err + } + + for _, tc := range []struct { + name string + renderer v1.StatsRenderer + param string + expected func(*testing.T, interface{}) + }{ + { + name: "stats is blank", + param: "", + expected: func(t *testing.T, i interface{}) { + require.IsType(t, i, &queryData{}) + qd := i.(*queryData) + require.Nil(t, qd.Stats) + }, + }, + { + name: "stats is true", + param: "true", + expected: func(t *testing.T, i interface{}) { + require.IsType(t, i, &queryData{}) + qd := i.(*queryData) + require.NotNil(t, qd.Stats) + qs := qd.Stats.Builtin() + require.NotNil(t, qs.Timings) + require.Greater(t, qs.Timings.EvalTotalTime, float64(0)) + require.NotNil(t, qs.Samples) + require.NotNil(t, qs.Samples.TotalQueryableSamples) + require.Nil(t, qs.Samples.TotalQueryableSamplesPerStep) + }, + }, + { + name: "stats is all", + param: "all", + expected: func(t *testing.T, i interface{}) { + require.IsType(t, i, &queryData{}) + qd := i.(*queryData) + require.NotNil(t, qd.Stats) + qs := qd.Stats.Builtin() + require.NotNil(t, qs.Timings) + require.Greater(t, qs.Timings.EvalTotalTime, float64(0)) + require.NotNil(t, qs.Samples) + require.NotNil(t, qs.Samples.TotalQueryableSamples) + require.NotNil(t, qs.Samples.TotalQueryableSamplesPerStep) + }, + }, + { + name: "custom handler with known value", + renderer: func(ctx context.Context, s *stats.Statistics, p string) stats.QueryStats { + if p == "known" { + return testStats{"Custom Value"} + } + return nil + }, + param: "known", + expected: func(t *testing.T, i interface{}) { + require.IsType(t, i, &queryData{}) + qd := i.(*queryData) + require.NotNil(t, qd.Stats) + j, err := json.Marshal(qd.Stats) + require.NoError(t, err) + require.JSONEq(t, string(j), `{"custom":"Custom Value"}`) + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + before := api.StatsRenderer + defer func() { api.StatsRenderer = before }() + api.StatsRenderer = tc.renderer + + for _, method := range []string{http.MethodGet, http.MethodPost} { + ctx := context.Background() + req, err := request(method, tc.param) + require.NoError(t, err) + data, _, error, _ := api.Query(req.WithContext(ctx)) + assertAPIError(t, error, "") + tc.expected(t, data) + + data, _, error, _ = api.QueryRange(req.WithContext(ctx)) + assertAPIError(t, error, "") + tc.expected(t, data) + } + }) + } +} + +func setupRemote(s storage.Storage) *httptest.Server { + handler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + req, err := remote.DecodeReadRequest(r) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + resp := prompb.ReadResponse{ + Results: make([]*prompb.QueryResult, len(req.Queries)), + } + for i, query := range req.Queries { + matchers, err := remote.FromLabelMatchers(query.Matchers) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + var hints *storage.SelectHints + if query.Hints != nil { + hints = &storage.SelectHints{ + Start: query.Hints.StartMs, + End: query.Hints.EndMs, + Step: query.Hints.StepMs, + Func: query.Hints.Func, + } + } + + querier, err := s.Querier(r.Context(), query.StartTimestampMs, query.EndTimestampMs) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + defer querier.Close() + + set := querier.Select(false, hints, matchers...) + resp.Results[i], _, err = remote.ToQueryResult(set, 1e6) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + } + + if err := remote.EncodeReadResponse(&resp, w); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + }) + + return httptest.NewServer(handler) +} + +func testEndpoints(t *testing.T, api *API) { + start := time.Unix(0, 0) + + type test struct { + endpoint apiFunc + query url.Values + response interface{} + errType thanos_api.ErrorType + } + + tests := []test{ + { + endpoint: api.Query, + query: url.Values{ + "query": []string{"2"}, + "time": []string{"123.4"}, + }, + response: &queryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 2, + T: timestamp.FromTime(start.Add(123*time.Second + 400*time.Millisecond)), + }, + }, + }, + { + endpoint: api.Query, + query: url.Values{ + "query": []string{"0.333"}, + "time": []string{"1970-01-01T00:02:03Z"}, + }, + response: &queryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 0.333, + T: timestamp.FromTime(start.Add(123 * time.Second)), + }, + }, + }, + { + endpoint: api.Query, + query: url.Values{ + "query": []string{"0.333"}, + "time": []string{"1970-01-01T01:02:03+01:00"}, + }, + response: &queryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 0.333, + T: timestamp.FromTime(start.Add(123 * time.Second)), + }, + }, + }, + { + endpoint: api.Query, + query: url.Values{ + "query": []string{"0.333"}, + }, + response: &queryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 0.333, + T: timestamp.FromTime(api.Now()), + }, + }, + }, + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"0"}, + "end": []string{"2"}, + "step": []string{"1"}, + }, + response: &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{ + {F: 0, T: timestamp.FromTime(start)}, + {F: 1, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 2, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + // No Metric returned - use zero value for comparison. + }, + }, + }, + }, + // Missing query params in range queries. + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "end": []string{"2"}, + "step": []string{"1"}, + }, + errType: thanos_api.ErrorBadData, + }, + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"0"}, + "step": []string{"1"}, + }, + errType: thanos_api.ErrorBadData, + }, + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"0"}, + "end": []string{"2"}, + }, + errType: thanos_api.ErrorBadData, + }, + // Bad query expression. + { + endpoint: api.Query, + query: url.Values{ + "query": []string{"invalid][query"}, + "time": []string{"1970-01-01T01:02:03+01:00"}, + }, + errType: thanos_api.ErrorBadData, + }, + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"invalid][query"}, + "start": []string{"0"}, + "end": []string{"100"}, + "step": []string{"1"}, + }, + errType: thanos_api.ErrorBadData, + }, + // Invalid step. + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"1"}, + "end": []string{"2"}, + "step": []string{"0"}, + }, + errType: thanos_api.ErrorBadData, + }, + // Start after end. + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"2"}, + "end": []string{"1"}, + "step": []string{"1"}, + }, + errType: thanos_api.ErrorBadData, + }, + // Start overflows int64 internally. + { + endpoint: api.QueryRange, + query: url.Values{ + "query": []string{"time()"}, + "start": []string{"148966367200.372"}, + "end": []string{"1489667272.372"}, + "step": []string{"1"}, + }, + errType: thanos_api.ErrorBadData, + }, + } + + request := func(m string, q url.Values) (*http.Request, error) { + if m == http.MethodPost { + r, err := http.NewRequest(m, "http://example.com", strings.NewReader(q.Encode())) + r.Header.Set("Content-Type", "application/x-www-form-urlencoded") + r.RemoteAddr = "127.0.0.1:20201" + return r, err + } + r, err := http.NewRequest(m, fmt.Sprintf("http://example.com?%s", q.Encode()), nil) + r.RemoteAddr = "127.0.0.1:20201" + return r, err + } + + for i, test := range tests { + t.Run(fmt.Sprintf("run %d %s %q", i, describeAPIFunc(test.endpoint), test.query.Encode()), func(t *testing.T) { + for _, method := range []string{http.MethodGet, http.MethodPost} { + t.Run(method, func(t *testing.T) { + // Build a context with the correct request params. + ctx := context.Background() + + req, err := request(method, test.query) + if err != nil { + t.Fatal(err) + } + + data, _, error, _ := test.endpoint(req.WithContext(ctx)) + assertAPIError(t, error, test.errType) + assertAPIResponse(t, data, test.response) + }) + } + }) + } +} + +func describeAPIFunc(f apiFunc) string { + name := runtime.FuncForPC(reflect.ValueOf(f).Pointer()).Name() + return strings.Split(name[strings.LastIndex(name, ".")+1:], "-")[0] +} + +func assertAPIError(t *testing.T, got *thanos_api.ApiError, exp thanos_api.ErrorType) { + t.Helper() + + if got != nil { + if exp == thanos_api.ErrorNone { + t.Fatalf("Unexpected error: %s", got) + } + if exp != got.Typ { + t.Fatalf("Expected error of type %q but got type %q (%q)", exp, got.Typ, got) + } + return + } + if exp != thanos_api.ErrorNone { + t.Fatalf("Expected error of type %q but got none", exp) + } +} + +func assertAPIResponse(t *testing.T, got, exp interface{}) { + t.Helper() + + require.Equal(t, exp, got) +} + +func TestRespondSuccess(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + api := API{} + api.Respond(w, testResponse, nil) + })) + defer s.Close() + + resp, err := http.Get(s.URL) + if err != nil { + t.Fatalf("Error on test request: %s", err) + } + body, err := io.ReadAll(resp.Body) + defer resp.Body.Close() + if err != nil { + t.Fatalf("Error reading response body: %s", err) + } + + if resp.StatusCode != http.StatusOK { + t.Fatalf("Return code %d expected in success response but got %d", 200, resp.StatusCode) + } + if h := resp.Header.Get("Content-Type"); h != "application/x-protobuf" { + t.Fatalf("Expected Content-Type %q but got %q", "application/x-protobuf", h) + } + + var res queryrange.PrometheusResponse + if err = proto.Unmarshal(body, &res); err != nil { + t.Fatalf("Error unmarshaling response body: %s", err) + } + + require.Equal(t, testResponse, &res) +} + +func TestRespondError(t *testing.T) { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + api := API{} + api.RespondError(w, &thanos_api.ApiError{thanos_api.ErrorTimeout, errors.New("message")}, "test") + })) + defer s.Close() + + resp, err := http.Get(s.URL) + if err != nil { + t.Fatalf("Error on test request: %s", err) + } + body, err := io.ReadAll(resp.Body) + defer resp.Body.Close() + if err != nil { + t.Fatalf("Error reading response body: %s", err) + } + + if want, have := http.StatusServiceUnavailable, resp.StatusCode; want != have { + t.Fatalf("Return code %d expected in error response but got %d", want, have) + } + if h := resp.Header.Get("Content-Type"); h != "application/json" { + t.Fatalf("Expected Content-Type %q but got %q", "application/json", h) + } + + var res response + if err = json.Unmarshal(body, &res); err != nil { + t.Fatalf("Error unmarshaling JSON body: %s", err) + } + + exp := &response{ + Status: statusError, + Data: "test", + ErrorType: thanos_api.ErrorTimeout, + Error: "message", + } + require.Equal(t, exp, &res) +} + +func TestParseTimeParam(t *testing.T) { + type resultType struct { + asTime time.Time + asError func() error + } + + ts, err := parseTime("1582468023986") + require.NoError(t, err) + + tests := []struct { + paramName string + paramValue string + defaultValue time.Time + result resultType + }{ + { // When data is valid. + paramName: "start", + paramValue: "1582468023986", + defaultValue: minTime, + result: resultType{ + asTime: ts, + asError: nil, + }, + }, + { // When data is empty string. + paramName: "end", + paramValue: "", + defaultValue: maxTime, + result: resultType{ + asTime: maxTime, + asError: nil, + }, + }, + { // When data is not valid. + paramName: "foo", + paramValue: "baz", + defaultValue: maxTime, + result: resultType{ + asTime: time.Time{}, + asError: func() error { + _, err := parseTime("baz") + return errors.Wrapf(err, "Invalid time value for '%s'", "foo") + }, + }, + }, + } + + for _, test := range tests { + req, err := http.NewRequest("GET", "localhost:42/foo?"+test.paramName+"="+test.paramValue, nil) + require.NoError(t, err) + + result := test.result + asTime, err := parseTimeParam(req, test.paramName, test.defaultValue) + + if err != nil { + require.EqualError(t, err, result.asError().Error()) + } else { + require.True(t, asTime.Equal(result.asTime), "time as return value: %s not parsed correctly. Expected %s. Actual %s", test.paramValue, result.asTime, asTime) + } + } +} + +func TestParseTime(t *testing.T) { + ts, err := time.Parse(time.RFC3339Nano, "2015-06-03T13:21:58.555Z") + if err != nil { + panic(err) + } + + tests := []struct { + input string + fail bool + result time.Time + }{ + { + input: "", + fail: true, + }, + { + input: "abc", + fail: true, + }, + { + input: "30s", + fail: true, + }, + { + input: "123", + result: time.Unix(123, 0), + }, + { + input: "123.123", + result: time.Unix(123, 123000000), + }, + { + input: "2015-06-03T13:21:58.555Z", + result: ts, + }, + { + input: "2015-06-03T14:21:58.555+01:00", + result: ts, + }, + { + // Test float rounding. + input: "1543578564.705", + result: time.Unix(1543578564, 705*1e6), + }, + { + input: minTime.Format(time.RFC3339Nano), + result: minTime, + }, + { + input: maxTime.Format(time.RFC3339Nano), + result: maxTime, + }, + } + + for _, test := range tests { + ts, err := parseTime(test.input) + + if err != nil && !test.fail { + t.Errorf("Unexpected error for %q: %s", test.input, err) + continue + } + if err == nil && test.fail { + t.Errorf("Expected error for %q but got none", test.input) + continue + } + if !test.fail && !ts.Equal(test.result) { + t.Errorf("Expected time %v for input %q but got %v", test.result, test.input, ts) + } + } +} + +func TestParseDuration(t *testing.T) { + tests := []struct { + input string + fail bool + result time.Duration + }{ + { + input: "", + fail: true, + }, { + input: "abc", + fail: true, + }, { + input: "2015-06-03T13:21:58.555Z", + fail: true, + }, { + // Internal int64 overflow. + input: "-148966367200.372", + fail: true, + }, { + // Internal int64 overflow. + input: "148966367200.372", + fail: true, + }, { + input: "123", + result: 123 * time.Second, + }, { + input: "123.333", + result: 123*time.Second + 333*time.Millisecond, + }, { + input: "15s", + result: 15 * time.Second, + }, { + input: "5m", + result: 5 * time.Minute, + }, + } + + for _, test := range tests { + d, err := parseDuration(test.input) + if err != nil && !test.fail { + t.Errorf("Unexpected error for %q: %s", test.input, err) + continue + } + if err == nil && test.fail { + t.Errorf("Expected error for %q but got none", test.input) + continue + } + if !test.fail && d != test.result { + t.Errorf("Expected duration %v for input %q but got %v", test.result, test.input, d) + } + } +} + +func TestRespond(t *testing.T) { + cases := []struct { + response interface{} + }{ + { + response: &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: 1000}}, + Metric: labels.FromStrings("__name__", "foo"), + }, + }, + }, + }, + { + response: &queryrange.PrometheusResponse{ + Status: "success", + Data: queryrange.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}, + }, + }, + }, + }, + }, + }, + { + response: &instantquery.PrometheusInstantQueryResponse{ + Status: "success", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: instantquery.PrometheusInstantQueryResult{ + Result: &instantquery.PrometheusInstantQueryResult_Vector{ + Vector: &instantquery.Vector{ + Samples: []*instantquery.Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + }, + }, + } + + for _, c := range cases { + s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + api := API{} + api.Respond(w, c.response, nil) + })) + defer s.Close() + + resp, err := http.Get(s.URL) + if err != nil { + t.Fatalf("Error on test request: %s", err) + } + body, err := io.ReadAll(resp.Body) + defer resp.Body.Close() + if err != nil { + t.Fatalf("Error reading response body: %s", err) + } + + var res interface{} + switch dest := c.response.(type) { + case *queryrange.PrometheusResponse: + err = proto.Unmarshal(body, dest) + res = dest + case *instantquery.PrometheusInstantQueryResponse: + err = proto.Unmarshal(body, dest) + res = dest + case *queryData: + err = json.Unmarshal(body, dest) + res = dest + } + + if err != nil { + t.Fatalf("Error unmarshaling response body: %s", err) + } + + require.Equal(t, c.response, res) + } +} + +func TestReturnAPIError(t *testing.T) { + cases := []struct { + err error + expected thanos_api.ErrorType + }{ + { + err: promql.ErrStorage{Err: errors.New("storage error")}, + expected: thanos_api.ErrorInternal, + }, { + err: fmt.Errorf("wrapped: %w", promql.ErrStorage{Err: errors.New("storage error")}), + expected: thanos_api.ErrorInternal, + }, { + err: promql.ErrQueryTimeout("timeout error"), + expected: thanos_api.ErrorTimeout, + }, { + err: fmt.Errorf("wrapped: %w", promql.ErrQueryTimeout("timeout error")), + expected: thanos_api.ErrorTimeout, + }, { + err: promql.ErrQueryCanceled("canceled error"), + expected: thanos_api.ErrorCanceled, + }, { + err: fmt.Errorf("wrapped: %w", promql.ErrQueryCanceled("canceled error")), + expected: thanos_api.ErrorCanceled, + }, { + err: errors.New("exec error"), + expected: thanos_api.ErrorExec, + }, + } + + for ix, c := range cases { + actual := returnAPIError(c.err) + require.Error(t, actual, ix) + require.Equal(t, c.expected, actual.Typ, ix) + } +} + +// This is a global to avoid the benchmark being optimized away. +var testResponseWriter = httptest.ResponseRecorder{} + +func BenchmarkRespondQueryData(b *testing.B) { + b.ReportAllocs() + points := []promql.FPoint{} + for i := 0; i < 10000; i++ { + points = append(points, promql.FPoint{F: float64(i * 1000000), T: int64(i)}) + } + response := &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Floats: points, + Metric: labels.EmptyLabels(), + }, + }, + } + b.ResetTimer() + api := API{} + for n := 0; n < b.N; n++ { + api.Respond(&testResponseWriter, response, nil) + } +} + +func BenchmarkRespondPrometheusResponse(b *testing.B) { + b.ReportAllocs() + points := []cortexpb.Sample{} + for i := 0; i < 10000; i++ { + points = append(points, cortexpb.Sample{Value: float64(i * 1000000), TimestampMs: int64(i)}) + } + response := &queryrange.PrometheusResponse{ + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: points, + }, + }, + }, + } + b.ResetTimer() + api := API{} + for n := 0; n < b.N; n++ { + api.Respond(&testResponseWriter, response, nil) + } +} + +func BenchmarkRespondPrometheusInstantQueryResponse(b *testing.B) { + b.ReportAllocs() + points := []cortexpb.Sample{} + for i := 0; i < 10000; i++ { + points = append(points, cortexpb.Sample{Value: float64(i * 1000000), TimestampMs: int64(i)}) + } + response := &instantquery.PrometheusInstantQueryResponse{ + Status: "success", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: instantquery.PrometheusInstantQueryResult{ + Result: &instantquery.PrometheusInstantQueryResult_Matrix{ + Matrix: &instantquery.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{}, + Samples: points, + }, + }, + }, + }, + }, + }, + } + b.ResetTimer() + api := API{} + for n := 0; n < b.N; n++ { + api.Respond(&testResponseWriter, response, nil) + } +} + +func TestExtractQueryOpts(t *testing.T) { + tests := []struct { + name string + form url.Values + expect *promql.QueryOpts + err error + }{ + { + name: "with stats all", + form: url.Values{ + "stats": []string{"all"}, + }, + expect: &promql.QueryOpts{ + EnablePerStepStats: true, + }, + err: nil, + }, + { + name: "with stats none", + form: url.Values{ + "stats": []string{"none"}, + }, + expect: &promql.QueryOpts{ + EnablePerStepStats: false, + }, + err: nil, + }, + { + name: "with lookback delta", + form: url.Values{ + "stats": []string{"all"}, + "lookback_delta": []string{"30s"}, + }, + expect: &promql.QueryOpts{ + EnablePerStepStats: true, + LookbackDelta: 30 * time.Second, + }, + err: nil, + }, + { + name: "with invalid lookback delta", + form: url.Values{ + "lookback_delta": []string{"invalid"}, + }, + expect: nil, + err: errors.New(`error parsing lookback delta duration: cannot parse "invalid" to a valid duration`), + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + req := &http.Request{Form: test.form} + opts, err := extractQueryOpts(req) + require.Equal(t, test.expect, opts) + if test.err == nil { + require.NoError(t, err) + } else { + require.Equal(t, test.err.Error(), err.Error()) + } + }) + } +} + +func TestCreatePrometheusResponse(t *testing.T) { + tests := []struct { + data *queryData + response *queryrange.PrometheusResponse + err error + }{ + { + data: &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{ + {F: 1, T: 1000}, + {F: 2, T: 2000}, + {F: 3, T: 3000}, + }, + }, + }, + }, + response: &queryrange.PrometheusResponse{ + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Samples: []cortexpb.Sample{ + {Value: 1, TimestampMs: 1000}, + {Value: 2, TimestampMs: 2000}, + {Value: 3, TimestampMs: 3000}, + }, + }, + }, + }, + }, + }, + { + data: &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Metric: labels.Labels{ + {Name: "__name__", Value: "foo"}, + {Name: "__job__", Value: "bar"}, + }, + Floats: []promql.FPoint{ + {F: 0.14, T: 18555000}, + {F: 2.9, T: 18556000}, + {F: 30, T: 18557000}, + }, + }, + }, + }, + response: &queryrange.PrometheusResponse{ + Status: "success", + Data: queryrange.PrometheusData{ + ResultType: model.ValMatrix.String(), + Result: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {Name: "__name__", Value: "foo"}, + {Name: "__job__", Value: "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 0.14, TimestampMs: 18555000}, + {Value: 2.9, TimestampMs: 18556000}, + {Value: 30, TimestampMs: 18557000}, + }, + }, + }, + }, + }, + }, + { + data: nil, + err: errors.New("no query response data"), + }, + } + for i, test := range tests { + t.Run(strconv.Itoa(i), func(t *testing.T) { + res, err := createPrometheusResponse(test.data) + require.Equal(t, test.response, res) + if test.err == nil { + require.NoError(t, err) + } else { + require.Equal(t, test.err.Error(), err.Error()) + } + }) + } +} + +func TestCreatePrometheusInstantQueryResponse(t *testing.T) { + tests := []struct { + data *queryData + response *instantquery.PrometheusInstantQueryResponse + err error + }{ + { + data: &queryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{T: 1000, V: 1}, + }, + response: &instantquery.PrometheusInstantQueryResponse{ + Status: "success", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: model.ValScalar.String(), + Result: instantquery.PrometheusInstantQueryResult{ + Result: &instantquery.PrometheusInstantQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"scalar","result":[1,"1"]}`), + }, + }, + }, + }, + }, + { + data: &queryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Metric: labels.Labels{ + {Name: "__name__", Value: "foo"}, + {Name: "__job__", Value: "bar"}, + }, + Floats: []promql.FPoint{ + {F: 0.14, T: 18555000}, + {F: 2.9, T: 18556000}, + {F: 30, T: 18557000}, + }, + }, + }, + }, + response: &instantquery.PrometheusInstantQueryResponse{ + Status: "success", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: instantquery.PrometheusInstantQueryResult{ + Result: &instantquery.PrometheusInstantQueryResult_Matrix{ + Matrix: &instantquery.Matrix{ + SampleStreams: []tripperware.SampleStream{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "foo"}, + {"__job__", "bar"}, + }, + Samples: []cortexpb.Sample{ + {Value: 0.14, TimestampMs: 18555000}, + {Value: 2.9, TimestampMs: 18556000}, + {Value: 30, TimestampMs: 18557000}, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + data: &queryData{ + ResultType: parser.ValueTypeVector, + Result: promql.Vector{ + { + F: 1, + T: 1000, + Metric: labels.Labels{ + {"__name__", "up"}, + {"job", "foo"}, + }, + }, + }, + }, + response: &instantquery.PrometheusInstantQueryResponse{ + Status: "success", + Data: instantquery.PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: instantquery.PrometheusInstantQueryResult{ + Result: &instantquery.PrometheusInstantQueryResult_Vector{ + Vector: &instantquery.Vector{ + Samples: []*instantquery.Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + }, + }, + { + data: nil, + err: errors.New("no query response data"), + }, + } + for i, test := range tests { + t.Run(strconv.Itoa(i), func(t *testing.T) { + res, err := createPrometheusInstantQueryResponse(test.data) + require.Equal(t, test.response, res) + if test.err == nil { + require.NoError(t, err) + } else { + require.Equal(t, test.err.Error(), err.Error()) + } + }) + } +} diff --git a/vendor/github.com/prometheus/common/promlog/log.go b/vendor/github.com/prometheus/common/promlog/log.go new file mode 100644 index 00000000000..3ac7b3fdf12 --- /dev/null +++ b/vendor/github.com/prometheus/common/promlog/log.go @@ -0,0 +1,189 @@ +// Copyright 2017 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package promlog defines standardised ways to initialize Go kit loggers +// across Prometheus components. +// It should typically only ever be imported by main packages. +package promlog + +import ( + "fmt" + "os" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" +) + +var ( + // This timestamp format differs from RFC3339Nano by using .000 instead + // of .999999999 which changes the timestamp from 9 variable to 3 fixed + // decimals (.130 instead of .130987456). + timestampFormat = log.TimestampFormat( + func() time.Time { return time.Now().UTC() }, + "2006-01-02T15:04:05.000Z07:00", + ) +) + +// AllowedLevel is a settable identifier for the minimum level a log entry +// must be have. +type AllowedLevel struct { + s string + o level.Option +} + +func (l *AllowedLevel) UnmarshalYAML(unmarshal func(interface{}) error) error { + var s string + type plain string + if err := unmarshal((*plain)(&s)); err != nil { + return err + } + if s == "" { + return nil + } + lo := &AllowedLevel{} + if err := lo.Set(s); err != nil { + return err + } + *l = *lo + return nil +} + +func (l *AllowedLevel) String() string { + return l.s +} + +// Set updates the value of the allowed level. +func (l *AllowedLevel) Set(s string) error { + switch s { + case "debug": + l.o = level.AllowDebug() + case "info": + l.o = level.AllowInfo() + case "warn": + l.o = level.AllowWarn() + case "error": + l.o = level.AllowError() + default: + return fmt.Errorf("unrecognized log level %q", s) + } + l.s = s + return nil +} + +// AllowedFormat is a settable identifier for the output format that the logger can have. +type AllowedFormat struct { + s string +} + +func (f *AllowedFormat) String() string { + return f.s +} + +// Set updates the value of the allowed format. +func (f *AllowedFormat) Set(s string) error { + switch s { + case "logfmt", "json": + f.s = s + default: + return fmt.Errorf("unrecognized log format %q", s) + } + return nil +} + +// Config is a struct containing configurable settings for the logger +type Config struct { + Level *AllowedLevel + Format *AllowedFormat +} + +// New returns a new leveled oklog logger. Each logged line will be annotated +// with a timestamp. The output always goes to stderr. +func New(config *Config) log.Logger { + if config.Format != nil && config.Format.s == "json" { + return NewWithLogger(log.NewJSONLogger(log.NewSyncWriter(os.Stderr)), config) + } + + return NewWithLogger(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), config) +} + +// NewWithLogger returns a new leveled oklog logger with a custom log.Logger. +// Each logged line will be annotated with a timestamp. +func NewWithLogger(l log.Logger, config *Config) log.Logger { + if config.Level != nil { + l = log.With(l, "ts", timestampFormat, "caller", log.Caller(5)) + l = level.NewFilter(l, config.Level.o) + } else { + l = log.With(l, "ts", timestampFormat, "caller", log.DefaultCaller) + } + return l +} + +// NewDynamic returns a new leveled logger. Each logged line will be annotated +// with a timestamp. The output always goes to stderr. Some properties can be +// changed, like the level. +func NewDynamic(config *Config) *logger { + if config.Format != nil && config.Format.s == "json" { + return NewDynamicWithLogger(log.NewJSONLogger(log.NewSyncWriter(os.Stderr)), config) + } + + return NewDynamicWithLogger(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), config) +} + +// NewDynamicWithLogger returns a new leveled logger with a custom io.Writer. +// Each logged line will be annotated with a timestamp. +// Some properties can be changed, like the level. +func NewDynamicWithLogger(l log.Logger, config *Config) *logger { + lo := &logger{ + base: l, + leveled: l, + } + + if config.Level != nil { + lo.SetLevel(config.Level) + } + + return lo +} + +type logger struct { + base log.Logger + leveled log.Logger + currentLevel *AllowedLevel + mtx sync.Mutex +} + +// Log implements logger.Log. +func (l *logger) Log(keyvals ...interface{}) error { + l.mtx.Lock() + defer l.mtx.Unlock() + return l.leveled.Log(keyvals...) +} + +// SetLevel changes the log level. +func (l *logger) SetLevel(lvl *AllowedLevel) { + l.mtx.Lock() + defer l.mtx.Unlock() + if lvl == nil { + l.leveled = log.With(l.base, "ts", timestampFormat, "caller", log.DefaultCaller) + l.currentLevel = nil + return + } + + if l.currentLevel != nil && l.currentLevel.s != lvl.s { + _ = l.base.Log("msg", "Log level changed", "prev", l.currentLevel, "current", lvl) + } + l.currentLevel = lvl + l.leveled = level.NewFilter(log.With(l.base, "ts", timestampFormat, "caller", log.Caller(5)), lvl.o) +} diff --git a/vendor/modules.txt b/vendor/modules.txt index 1da9fc2f9dd..878b1345885 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -815,6 +815,7 @@ github.com/prometheus/common/config github.com/prometheus/common/expfmt github.com/prometheus/common/helpers/templates github.com/prometheus/common/model +github.com/prometheus/common/promlog github.com/prometheus/common/route github.com/prometheus/common/version # github.com/prometheus/common/sigv4 v0.1.0 From 3d35da3f0f958ceb7334c92715203b14aa676d4b Mon Sep 17 00:00:00 2001 From: afayek Date: Thu, 24 Aug 2023 11:58:51 -0700 Subject: [PATCH 20/38] add feature flag for querier handler Signed-off-by: afayek Signed-off-by: Ahmed Hassan --- pkg/api/api.go | 3 ++ pkg/api/handlers.go | 20 ++++++--- pkg/cortex/modules.go | 6 +-- .../tripperware/instantquery/instant_query.go | 30 +++++++++---- .../instantquery/shard_by_query_test.go | 2 +- .../tripperware/queryrange/query_range.go | 45 ++++++++++++++----- .../query_range_middlewares_test.go | 4 +- 7 files changed, 80 insertions(+), 30 deletions(-) diff --git a/pkg/api/api.go b/pkg/api/api.go index 97c7c44095c..5d1733641f0 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -73,6 +73,8 @@ type Config struct { corsRegexString string `yaml:"cors_origin"` buildInfoEnabled bool `yaml:"build_info_enabled"` + + ProtobufQuerierHandler bool `yaml:"protobuf_querier_handler"` } // RegisterFlags adds the flags required to config this to the given FlagSet. @@ -80,6 +82,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.ResponseCompression, "api.response-compression-enabled", false, "Use GZIP compression for API responses. Some endpoints serve large YAML or JSON blobs which can benefit from compression.") f.Var(&cfg.HTTPRequestHeadersToLog, "api.http-request-headers-to-log", "Which HTTP Request headers to add to logs") f.BoolVar(&cfg.buildInfoEnabled, "api.build-info-enabled", false, "If enabled, build Info API will be served by query frontend or querier.") + f.BoolVar(&cfg.ProtobufQuerierHandler, "api.protobuf_querier_handler", false, "Enable using querier handler with protobuf response serialization for query range and unsharded instant query requests") cfg.RegisterFlagsWithPrefix("", f) } diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index b76f5f48b20..c7b0a1bff28 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -9,7 +9,6 @@ import ( "path" "sync" - qapi "github.com/cortexproject/cortex/pkg/querier/handler" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gorilla/mux" @@ -29,6 +28,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier" "github.com/cortexproject/cortex/pkg/querier/handler" + qapi "github.com/cortexproject/cortex/pkg/querier/handler" "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/util" ) @@ -300,8 +300,13 @@ func NewQuerierHandler( router.Path(path.Join(prefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(prefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(prefix, "/api/v1/read")).Methods("POST").Handler(promRouter) - router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) - router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) + if cfg.ProtobufQuerierHandler { + router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) + router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) + } else { + router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(promRouter) + router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(promRouter) + } router.Path(path.Join(prefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/labels")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(promRouter) @@ -313,8 +318,13 @@ func NewQuerierHandler( router.Path(path.Join(legacyPrefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Methods("POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) + if cfg.ProtobufQuerierHandler { + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) + } else { + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) + } router.Path(path.Join(legacyPrefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/labels")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(legacyPromRouter) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 4da65f5b5b7..8319f0c26c3 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -442,9 +442,9 @@ func (t *Cortex) initFlusher() (serv services.Service, err error) { func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err error) { queryAnalyzer := querysharding.NewQueryAnalyzer() // PrometheusCodec is a codec to encode and decode Prometheus query range requests and responses. - prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression) + prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler) // ShardedPrometheusCodec is same as PrometheusCodec but to be used on the sharded queries (it sum up the stats) - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression) + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler) queryRangeMiddlewares, cache, err := queryrange.Middlewares( t.Cfg.QueryRange, @@ -472,7 +472,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro queryRangeMiddlewares, instantQueryMiddlewares, prometheusCodec, - instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression), + instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler), t.Overrides, queryAnalyzer, t.Cfg.Querier.DefaultEvaluationInterval, diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index f1f1c7231dd..0a643dfb0cf 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -32,7 +32,7 @@ import ( ) var ( - InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("") + InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("", true) json = jsoniter.Config{ EscapeHTML: false, // No HTML in our responses. @@ -109,14 +109,22 @@ func (r *PrometheusRequest) WithStats(stats string) tripperware.Request { type instantQueryCodec struct { tripperware.Codec - compression string - now func() time.Time + compression queryrange.Compression + enableProtobuf bool + now func() time.Time } -func NewInstantQueryCodec(compression string) instantQueryCodec { +func NewInstantQueryCodec(c string, enableProtobuf bool) instantQueryCodec { + var compression queryrange.Compression + if c == "gzip" || c == "snappy" { + compression = queryrange.Compression(c) + } else { + compression = queryrange.DisableCompression + } return instantQueryCodec{ - compression: compression, - now: time.Now, + compression: compression, + enableProtobuf: enableProtobuf, + now: time.Now, } } @@ -211,10 +219,14 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } } - if c.compression == "snappy" || c.compression == "gzip" { - h.Set("Accept-Encoding", c.compression) + if c.compression == queryrange.SnappyCompression || c.compression == queryrange.GzipCompression { + h.Set("Accept-Encoding", string(c.compression)) + } + if c.enableProtobuf { + h.Set("Accept", "application/x-protobuf") + } else { + h.Set("Accept", "application/json") } - h.Set("Accept", "application/x-protobuf") req := &http.Request{ Method: "GET", diff --git a/pkg/querier/tripperware/instantquery/shard_by_query_test.go b/pkg/querier/tripperware/instantquery/shard_by_query_test.go index f74adf66b16..22e793edbb3 100644 --- a/pkg/querier/tripperware/instantquery/shard_by_query_test.go +++ b/pkg/querier/tripperware/instantquery/shard_by_query_test.go @@ -26,7 +26,7 @@ import ( ) func Test_shardQuery(t *testing.T) { - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, "") + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, "", true) instantQueryCodec := InstantQueryCodec type queries struct { diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index e08b44462fb..0ed817f365c 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -30,6 +30,14 @@ import ( // StatusSuccess Prometheus success result. const StatusSuccess = "success" +type Compression string + +const ( + DisableCompression Compression = "" + GzipCompression Compression = "gzip" + SnappyCompression Compression = "snappy" +) + var ( matrix = model.ValMatrix.String() json = jsoniter.Config{ @@ -46,14 +54,22 @@ var ( ) type prometheusCodec struct { - sharded bool - compression string + sharded bool + compression Compression + enableProtobuf bool } -func NewPrometheusCodec(sharded bool, compression string) *prometheusCodec { //nolint:revive +func NewPrometheusCodec(sharded bool, c string, enableProtobuf bool) *prometheusCodec { //nolint:revive + var compression Compression + if c == "gzip" || c == "snappy" { + compression = Compression(c) + } else { + compression = DisableCompression + } return &prometheusCodec{ - sharded: sharded, - compression: compression, + sharded: sharded, + compression: compression, + enableProtobuf: enableProtobuf, } } @@ -248,10 +264,14 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } - if c.compression == "snappy" || c.compression == "gzip" { - h.Set("Accept-Encoding", c.compression) + if c.compression == SnappyCompression || c.compression == GzipCompression { + h.Set("Accept-Encoding", string(c.compression)) + } + if c.enableProtobuf { + h.Set("Accept", "application/x-protobuf") + } else { + h.Set("Accept", "application/json") } - h.Set("Accept", "application/x-protobuf") req := &http.Request{ Method: "GET", @@ -264,7 +284,7 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques return req.WithContext(ctx), nil } -func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ tripperware.Request) (tripperware.Response, error) { +func (c prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ tripperware.Request) (tripperware.Response, error) { log, ctx := spanlogger.New(ctx, "ParseQueryRangeResponse") //nolint:ineffassign,staticcheck defer log.Finish() @@ -283,7 +303,12 @@ func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ t log.LogFields(otlog.Int("bytes", len(buf))) var resp PrometheusResponse - if err := proto.Unmarshal(buf, &resp); err != nil { + if c.enableProtobuf { + err = proto.Unmarshal(buf, &resp) + } else { + err = json.Unmarshal(buf, &resp) + } + if err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index c72dd3a8155..bb48ca6cad2 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -21,8 +21,8 @@ import ( ) var ( - PrometheusCodec = NewPrometheusCodec(false, "") - ShardedPrometheusCodec = NewPrometheusCodec(false, "") + PrometheusCodec = NewPrometheusCodec(false, "", true) + ShardedPrometheusCodec = NewPrometheusCodec(false, "", true) ) func TestRoundTrip(t *testing.T) { From 9f77e36c4a74ad950af4a5102ce85f53cd580576 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 6 Aug 2024 05:51:28 -0700 Subject: [PATCH 21/38] refactor querier unit tests Signed-off-by: Ahmed Hassan Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 20 +- pkg/querier/handler/handler.go | 102 +- pkg/querier/handler/handler_test.go | 1276 ----------------- .../tripperware/instantquery/instant_query.go | 15 +- .../instantquery/instant_query_test.go | 679 ++++++--- .../instantquery/shard_by_query_test.go | 874 +---------- pkg/querier/tripperware/query.go | 15 +- .../tripperware/queryrange/marshaling_test.go | 1 + .../tripperware/queryrange/query_range.go | 17 +- .../query_range_middlewares_test.go | 7 +- .../queryrange/query_range_test.go | 345 +++-- .../queryrange/results_cache_test.go | 12 +- .../queryrange/split_by_interval_test.go | 7 +- pkg/querier/tripperware/roundtrip.go | 13 - pkg/querier/tripperware/roundtrip_test.go | 6 +- 15 files changed, 881 insertions(+), 2508 deletions(-) delete mode 100644 pkg/querier/handler/handler_test.go diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index c7b0a1bff28..52a9238d45d 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -3,12 +3,13 @@ package api import ( "context" "encoding/json" - thanos_api "github.com/thanos-io/thanos/pkg/api" "html/template" "net/http" "path" "sync" + thanos_api "github.com/thanos-io/thanos/pkg/api" + "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gorilla/mux" @@ -22,6 +23,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" v1api "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" @@ -233,16 +235,6 @@ func NewQuerierHandler( false, ) - queryapi := qapi.NewAPI( - engine, - querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. - func(f http.HandlerFunc) http.HandlerFunc { return f }, - logger, - false, - regexp.MustCompile(".*"), - nil, - ) - queryapi := qapi.NewAPI( engine, querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. @@ -273,9 +265,9 @@ func NewQuerierHandler( legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) v1api.Register(legacyPromRouter) - wrap := func(f thanos_api.ApiFunc) http.HandlerFunc { + wrap := func(f func(r *http.Request) (interface{}, *thanos_api.ApiError, annotations.Annotations, func())) http.HandlerFunc { hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - data, warnings, err, finalizer := f(r) + data, err, warnings, finalizer := f(r) if finalizer != nil { defer finalizer() } @@ -285,7 +277,7 @@ func NewQuerierHandler( } if data != nil { - queryapi.Respond(w, data, warnings) + queryapi.Respond(w, r, data, warnings, r.FormValue("query")) return } w.WriteHeader(http.StatusNoContent) diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go index 135956f424b..e957134b11f 100644 --- a/pkg/querier/handler/handler.go +++ b/pkg/querier/handler/handler.go @@ -3,6 +3,12 @@ package handler import ( "context" "fmt" + "math" + "net/http" + "strconv" + "strings" + "time" + "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -11,19 +17,15 @@ import ( "github.com/pkg/errors" "github.com/prometheus/common/model" v1 "github.com/prometheus/prometheus/web/api/v1" - "math" - "net/http" - "strconv" - "strings" - "time" "github.com/cortexproject/cortex/pkg/cortexpb" - github_com_cortexproject_cortex_pkg_cortexpb "github.com/cortexproject/cortex/pkg/cortexpb" + cortex_pb "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" "github.com/prometheus/prometheus/util/httputil" "github.com/prometheus/prometheus/util/stats" thanos_api "github.com/thanos-io/thanos/pkg/api" @@ -67,7 +69,7 @@ type response struct { type API struct { Queryable storage.SampleAndChunkQueryable - QueryEngine v1.QueryEngine + QueryEngine promql.QueryEngine Now func() time.Time Logger log.Logger StatsRenderer v1.StatsRenderer @@ -75,7 +77,7 @@ type API struct { // NewAPI returns an initialized API type. func NewAPI( - qe v1.QueryEngine, + qe promql.QueryEngine, q storage.SampleAndChunkQueryable, logger log.Logger, statsRenderer v1.StatsRenderer, @@ -101,13 +103,13 @@ type queryData struct { Stats stats.QueryStats `json:"stats,omitempty"` } -func invalidParamError(err error, parameter string) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { - return nil, nil, &thanos_api.ApiError{ +func invalidParamError(err error, parameter string) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { + return nil, &thanos_api.ApiError{ thanos_api.ErrorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), - }, nil + }, nil, nil } -func (api *API) Query(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { +func (api *API) Query(r *http.Request) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { ts, err := parseTimeParam(r, "time", api.Now()) if err != nil { return invalidParamError(err, "time") @@ -120,13 +122,13 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro return invalidParamError(err, "timeout") } - ctx, cancel = context.WithTimeout(ctx, timeout) + ctx, cancel = context.WithDeadline(ctx, api.Now().Add(timeout)) defer cancel() } opts, err := extractQueryOpts(r) if err != nil { - return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil + return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil } qry, err := api.QueryEngine.NewInstantQuery(ctx, api.Queryable, opts, r.FormValue("query"), ts) if err != nil { @@ -146,7 +148,7 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro res := qry.Exec(ctx) if res.Err != nil { - return nil, res.Warnings, returnAPIError(res.Err), qry.Close + return nil, returnAPIError(res.Err), res.Warnings, qry.Close } // Optional stats field in response if parameter "stats" is not empty. @@ -178,26 +180,26 @@ func (api *API) Query(r *http.Request) (data interface{}, warnings []error, erro } } if err != nil { - return nil, res.Warnings, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, qry.Close + return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, res.Warnings, qry.Close } - return data, res.Warnings, nil, qry.Close + return data, nil, res.Warnings, qry.Close } -func extractQueryOpts(r *http.Request) (*promql.QueryOpts, error) { - opts := &promql.QueryOpts{ - EnablePerStepStats: r.FormValue("stats") == "all", - } +func extractQueryOpts(r *http.Request) (promql.QueryOpts, error) { + var duration time.Duration + if strDuration := r.FormValue("lookback_delta"); strDuration != "" { - duration, err := parseDuration(strDuration) + parsedDuration, err := parseDuration(strDuration) if err != nil { return nil, fmt.Errorf("error parsing lookback delta duration: %w", err) } - opts.LookbackDelta = duration + duration = parsedDuration } - return opts, nil + + return promql.NewPrometheusQueryOpts(r.FormValue("stats") == "all", duration), nil } -func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, error *thanos_api.ApiError, finalizer func()) { +func (api *API) QueryRange(r *http.Request) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { start, err := parseTime(r.FormValue("start")) if err != nil { return invalidParamError(err, "start") @@ -223,7 +225,7 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, // This is sufficient for 60s resolution for a week or 1h resolution for a year. if end.Sub(start)/step > 11000 { err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") - return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil + return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil } ctx := r.Context() @@ -240,7 +242,7 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, opts, err := extractQueryOpts(r) if err != nil { - return nil, nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil + return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil } qry, err := api.QueryEngine.NewRangeQuery(ctx, api.Queryable, opts, r.FormValue("query"), start, end, step) if err != nil { @@ -259,7 +261,7 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, res := qry.Exec(ctx) if res.Err != nil { - return nil, res.Warnings, returnAPIError(res.Err), qry.Close + return nil, returnAPIError(res.Err), res.Warnings, qry.Close } // Optional stats field in response if parameter "stats" is not empty. @@ -292,9 +294,9 @@ func (api *API) QueryRange(r *http.Request) (data interface{}, warnings []error, } if err != nil { - return nil, res.Warnings, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, qry.Close + return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, res.Warnings, qry.Close } - return data, res.Warnings, nil, qry.Close + return data, nil, res.Warnings, qry.Close } func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) (time.Time, error) { @@ -325,9 +327,9 @@ func parseTime(s string) (time.Time, error) { // Upstream issue: https://github.com/golang/go/issues/20555 switch s { case minTimeFormatted: - return minTime, nil + return v1.MinTime, nil case maxTimeFormatted: - return maxTime, nil + return v1.MaxTime, nil } return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) } @@ -373,18 +375,13 @@ func returnAPIError(err error) *thanos_api.ApiError { } var ( - minTime = time.Unix(math.MinInt64/1000+62135596801, 0).UTC() - maxTime = time.Unix(math.MaxInt64/1000-62135596801, 999999999).UTC() - - minTimeFormatted = minTime.Format(time.RFC3339Nano) - maxTimeFormatted = maxTime.Format(time.RFC3339Nano) + minTimeFormatted = v1.MinTime.Format(time.RFC3339Nano) + maxTimeFormatted = v1.MaxTime.Format(time.RFC3339Nano) ) -func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storage.Warnings) { - var warningStrings []string - for _, warning := range warnings { - warningStrings = append(warningStrings, warning.Error()) - } +func (api *API) Respond(w http.ResponseWriter, req *http.Request, data interface{}, warnings annotations.Annotations, query string) { + statusMessage := statusSuccess + var b []byte var err error switch resp := data.(type) { @@ -393,26 +390,29 @@ func (api *API) Respond(w http.ResponseWriter, data interface{}, warnings storag for h, hv := range w.Header() { resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) } + resp.Warnings = warnings.AsStrings(query, 10) b, err = proto.Marshal(resp) case *instantquery.PrometheusInstantQueryResponse: w.Header().Set(contentTypeHeader, applicationProtobuf) for h, hv := range w.Header() { resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) } + resp.Warnings = warnings.AsStrings(query, 10) b, err = proto.Marshal(resp) case *queryData: w.Header().Set(contentTypeHeader, applicationJson) json := jsoniter.ConfigCompatibleWithStandardLibrary b, err = json.Marshal(&response{ - Status: statusSuccess, + Status: statusMessage, Data: data, - Warnings: warningStrings, + Warnings: warnings.AsStrings(query, 10), }) default: level.Error(api.Logger).Log("msg", "error asserting response type") http.Error(w, "error asserting response type", http.StatusInternalServerError) return } + if err != nil { level.Error(api.Logger).Log("msg", "error marshaling response", "err", err) http.Error(w, err.Error(), http.StatusInternalServerError) @@ -558,11 +558,11 @@ func getSampleStreams(data *queryData) *[]tripperware.SampleStream { for i := 0; i < sampleStreamsLen; i++ { labelsLen := len(data.Result.(promql.Matrix)[i].Metric) - var labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter + var labels []cortex_pb.LabelAdapter if labelsLen > 0 { - labels = make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + labels = make([]cortex_pb.LabelAdapter, labelsLen) for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + labels[j] = cortex_pb.LabelAdapter{ Name: data.Result.(promql.Matrix)[i].Metric[j].Name, Value: data.Result.(promql.Matrix)[i].Metric[j].Value, } @@ -591,11 +591,11 @@ func getSamples(data *queryData) *[]*instantquery.Sample { for i := 0; i < vectorSamplesLen; i++ { labelsLen := len(data.Result.(promql.Vector)[i].Metric) - var labels []github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter + var labels []cortex_pb.LabelAdapter if labelsLen > 0 { - labels = make([]github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter, labelsLen) + labels = make([]cortex_pb.LabelAdapter, labelsLen) for j := 0; j < labelsLen; j++ { - labels[j] = github_com_cortexproject_cortex_pkg_cortexpb.LabelAdapter{ + labels[j] = cortex_pb.LabelAdapter{ Name: data.Result.(promql.Vector)[i].Metric[j].Name, Value: data.Result.(promql.Vector)[i].Metric[j].Value, } @@ -603,7 +603,7 @@ func getSamples(data *queryData) *[]*instantquery.Sample { } vectorSamples[i] = &instantquery.Sample{Labels: labels, - Sample: cortexpb.Sample{ + Sample: &cortexpb.Sample{ TimestampMs: data.Result.(promql.Vector)[i].T, Value: data.Result.(promql.Vector)[i].F, }, diff --git a/pkg/querier/handler/handler_test.go b/pkg/querier/handler/handler_test.go deleted file mode 100644 index a7ca1256e2b..00000000000 --- a/pkg/querier/handler/handler_test.go +++ /dev/null @@ -1,1276 +0,0 @@ -package handler - -import ( - "context" - "encoding/json" - "fmt" - "github.com/cortexproject/cortex/pkg/cortexpb" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" - "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" - "github.com/gogo/protobuf/proto" - "github.com/prometheus/client_golang/prometheus" - config_util "github.com/prometheus/common/config" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/prompb" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/remote" - v1 "github.com/prometheus/prometheus/web/api/v1" - thanos_api "github.com/thanos-io/thanos/pkg/api" - "io" - "net/http" - "net/http/httptest" - "net/url" - "reflect" - "runtime" - "strconv" - "strings" - "testing" - "time" - - "github.com/pkg/errors" - "github.com/prometheus/prometheus/util/stats" - "github.com/stretchr/testify/require" - - "github.com/prometheus/common/promlog" - "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/timestamp" - "github.com/prometheus/prometheus/promql" - "github.com/prometheus/prometheus/promql/parser" -) - -var ( - testResponse = &queryrange.PrometheusResponse{ - Status: "success", - Data: queryrange.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}, - }, - }, - }, - }, - } -) - -func TestEndpoints(t *testing.T) { - suite, err := promql.NewTest(t, ``) - - require.NoError(t, err) - defer suite.Close() - - require.NoError(t, suite.Run()) - - now := time.Now() - t.Run("local", func(t *testing.T) { - api := &API{ - Queryable: suite.Storage(), - QueryEngine: suite.QueryEngine(), - Now: func() time.Time { return now }, - } - testEndpoints(t, api) - }) - - // Run all the API tests against a API that is wired to forward queries via - // the remote read client to a test server, which in turn sends them to the - // data from the test suite. - t.Run("remote", func(t *testing.T) { - server := setupRemote(suite.Storage()) - defer server.Close() - - u, err := url.Parse(server.URL) - require.NoError(t, err) - - al := promlog.AllowedLevel{} - require.NoError(t, al.Set("debug")) - - af := promlog.AllowedFormat{} - require.NoError(t, af.Set("logfmt")) - - promlogConfig := promlog.Config{ - Level: &al, - Format: &af, - } - - dbDir := t.TempDir() - - remote := remote.NewStorage(promlog.New(&promlogConfig), prometheus.DefaultRegisterer, func() (int64, error) { - return 0, nil - }, dbDir, 1*time.Second, nil) - - err = remote.ApplyConfig(&config.Config{ - RemoteReadConfigs: []*config.RemoteReadConfig{ - { - URL: &config_util.URL{URL: u}, - RemoteTimeout: model.Duration(1 * time.Second), - ReadRecent: true, - }, - }, - }) - require.NoError(t, err) - - api := &API{ - Queryable: remote, - QueryEngine: suite.QueryEngine(), - Now: func() time.Time { return now }, - } - testEndpoints(t, api) - }) -} - -type testStats struct { - Custom string `json:"custom"` -} - -func (testStats) Builtin() (_ stats.BuiltinStats) { - return -} - -func TestStats(t *testing.T) { - suite, err := promql.NewTest(t, ``) - require.NoError(t, err) - defer suite.Close() - require.NoError(t, suite.Run()) - - api := &API{ - Queryable: suite.Storage(), - QueryEngine: suite.QueryEngine(), - Now: func() time.Time { - return time.Unix(123, 0) - }, - } - request := func(method, param string) (*http.Request, error) { - u, err := url.Parse("http://example.com") - require.NoError(t, err) - q := u.Query() - q.Add("stats", param) - q.Add("query", "up") - q.Add("start", "0") - q.Add("end", "100") - q.Add("step", "10") - u.RawQuery = q.Encode() - - r, err := http.NewRequest(method, u.String(), nil) - if method == http.MethodPost { - r.Header.Set("Content-Type", "application/x-www-form-urlencoded") - } - return r, err - } - - for _, tc := range []struct { - name string - renderer v1.StatsRenderer - param string - expected func(*testing.T, interface{}) - }{ - { - name: "stats is blank", - param: "", - expected: func(t *testing.T, i interface{}) { - require.IsType(t, i, &queryData{}) - qd := i.(*queryData) - require.Nil(t, qd.Stats) - }, - }, - { - name: "stats is true", - param: "true", - expected: func(t *testing.T, i interface{}) { - require.IsType(t, i, &queryData{}) - qd := i.(*queryData) - require.NotNil(t, qd.Stats) - qs := qd.Stats.Builtin() - require.NotNil(t, qs.Timings) - require.Greater(t, qs.Timings.EvalTotalTime, float64(0)) - require.NotNil(t, qs.Samples) - require.NotNil(t, qs.Samples.TotalQueryableSamples) - require.Nil(t, qs.Samples.TotalQueryableSamplesPerStep) - }, - }, - { - name: "stats is all", - param: "all", - expected: func(t *testing.T, i interface{}) { - require.IsType(t, i, &queryData{}) - qd := i.(*queryData) - require.NotNil(t, qd.Stats) - qs := qd.Stats.Builtin() - require.NotNil(t, qs.Timings) - require.Greater(t, qs.Timings.EvalTotalTime, float64(0)) - require.NotNil(t, qs.Samples) - require.NotNil(t, qs.Samples.TotalQueryableSamples) - require.NotNil(t, qs.Samples.TotalQueryableSamplesPerStep) - }, - }, - { - name: "custom handler with known value", - renderer: func(ctx context.Context, s *stats.Statistics, p string) stats.QueryStats { - if p == "known" { - return testStats{"Custom Value"} - } - return nil - }, - param: "known", - expected: func(t *testing.T, i interface{}) { - require.IsType(t, i, &queryData{}) - qd := i.(*queryData) - require.NotNil(t, qd.Stats) - j, err := json.Marshal(qd.Stats) - require.NoError(t, err) - require.JSONEq(t, string(j), `{"custom":"Custom Value"}`) - }, - }, - } { - t.Run(tc.name, func(t *testing.T) { - before := api.StatsRenderer - defer func() { api.StatsRenderer = before }() - api.StatsRenderer = tc.renderer - - for _, method := range []string{http.MethodGet, http.MethodPost} { - ctx := context.Background() - req, err := request(method, tc.param) - require.NoError(t, err) - data, _, error, _ := api.Query(req.WithContext(ctx)) - assertAPIError(t, error, "") - tc.expected(t, data) - - data, _, error, _ = api.QueryRange(req.WithContext(ctx)) - assertAPIError(t, error, "") - tc.expected(t, data) - } - }) - } -} - -func setupRemote(s storage.Storage) *httptest.Server { - handler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - req, err := remote.DecodeReadRequest(r) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - resp := prompb.ReadResponse{ - Results: make([]*prompb.QueryResult, len(req.Queries)), - } - for i, query := range req.Queries { - matchers, err := remote.FromLabelMatchers(query.Matchers) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - var hints *storage.SelectHints - if query.Hints != nil { - hints = &storage.SelectHints{ - Start: query.Hints.StartMs, - End: query.Hints.EndMs, - Step: query.Hints.StepMs, - Func: query.Hints.Func, - } - } - - querier, err := s.Querier(r.Context(), query.StartTimestampMs, query.EndTimestampMs) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - defer querier.Close() - - set := querier.Select(false, hints, matchers...) - resp.Results[i], _, err = remote.ToQueryResult(set, 1e6) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - } - - if err := remote.EncodeReadResponse(&resp, w); err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - }) - - return httptest.NewServer(handler) -} - -func testEndpoints(t *testing.T, api *API) { - start := time.Unix(0, 0) - - type test struct { - endpoint apiFunc - query url.Values - response interface{} - errType thanos_api.ErrorType - } - - tests := []test{ - { - endpoint: api.Query, - query: url.Values{ - "query": []string{"2"}, - "time": []string{"123.4"}, - }, - response: &queryData{ - ResultType: parser.ValueTypeScalar, - Result: promql.Scalar{ - V: 2, - T: timestamp.FromTime(start.Add(123*time.Second + 400*time.Millisecond)), - }, - }, - }, - { - endpoint: api.Query, - query: url.Values{ - "query": []string{"0.333"}, - "time": []string{"1970-01-01T00:02:03Z"}, - }, - response: &queryData{ - ResultType: parser.ValueTypeScalar, - Result: promql.Scalar{ - V: 0.333, - T: timestamp.FromTime(start.Add(123 * time.Second)), - }, - }, - }, - { - endpoint: api.Query, - query: url.Values{ - "query": []string{"0.333"}, - "time": []string{"1970-01-01T01:02:03+01:00"}, - }, - response: &queryData{ - ResultType: parser.ValueTypeScalar, - Result: promql.Scalar{ - V: 0.333, - T: timestamp.FromTime(start.Add(123 * time.Second)), - }, - }, - }, - { - endpoint: api.Query, - query: url.Values{ - "query": []string{"0.333"}, - }, - response: &queryData{ - ResultType: parser.ValueTypeScalar, - Result: promql.Scalar{ - V: 0.333, - T: timestamp.FromTime(api.Now()), - }, - }, - }, - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"0"}, - "end": []string{"2"}, - "step": []string{"1"}, - }, - response: &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Floats: []promql.FPoint{ - {F: 0, T: timestamp.FromTime(start)}, - {F: 1, T: timestamp.FromTime(start.Add(1 * time.Second))}, - {F: 2, T: timestamp.FromTime(start.Add(2 * time.Second))}, - }, - // No Metric returned - use zero value for comparison. - }, - }, - }, - }, - // Missing query params in range queries. - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "end": []string{"2"}, - "step": []string{"1"}, - }, - errType: thanos_api.ErrorBadData, - }, - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"0"}, - "step": []string{"1"}, - }, - errType: thanos_api.ErrorBadData, - }, - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"0"}, - "end": []string{"2"}, - }, - errType: thanos_api.ErrorBadData, - }, - // Bad query expression. - { - endpoint: api.Query, - query: url.Values{ - "query": []string{"invalid][query"}, - "time": []string{"1970-01-01T01:02:03+01:00"}, - }, - errType: thanos_api.ErrorBadData, - }, - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"invalid][query"}, - "start": []string{"0"}, - "end": []string{"100"}, - "step": []string{"1"}, - }, - errType: thanos_api.ErrorBadData, - }, - // Invalid step. - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"1"}, - "end": []string{"2"}, - "step": []string{"0"}, - }, - errType: thanos_api.ErrorBadData, - }, - // Start after end. - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"2"}, - "end": []string{"1"}, - "step": []string{"1"}, - }, - errType: thanos_api.ErrorBadData, - }, - // Start overflows int64 internally. - { - endpoint: api.QueryRange, - query: url.Values{ - "query": []string{"time()"}, - "start": []string{"148966367200.372"}, - "end": []string{"1489667272.372"}, - "step": []string{"1"}, - }, - errType: thanos_api.ErrorBadData, - }, - } - - request := func(m string, q url.Values) (*http.Request, error) { - if m == http.MethodPost { - r, err := http.NewRequest(m, "http://example.com", strings.NewReader(q.Encode())) - r.Header.Set("Content-Type", "application/x-www-form-urlencoded") - r.RemoteAddr = "127.0.0.1:20201" - return r, err - } - r, err := http.NewRequest(m, fmt.Sprintf("http://example.com?%s", q.Encode()), nil) - r.RemoteAddr = "127.0.0.1:20201" - return r, err - } - - for i, test := range tests { - t.Run(fmt.Sprintf("run %d %s %q", i, describeAPIFunc(test.endpoint), test.query.Encode()), func(t *testing.T) { - for _, method := range []string{http.MethodGet, http.MethodPost} { - t.Run(method, func(t *testing.T) { - // Build a context with the correct request params. - ctx := context.Background() - - req, err := request(method, test.query) - if err != nil { - t.Fatal(err) - } - - data, _, error, _ := test.endpoint(req.WithContext(ctx)) - assertAPIError(t, error, test.errType) - assertAPIResponse(t, data, test.response) - }) - } - }) - } -} - -func describeAPIFunc(f apiFunc) string { - name := runtime.FuncForPC(reflect.ValueOf(f).Pointer()).Name() - return strings.Split(name[strings.LastIndex(name, ".")+1:], "-")[0] -} - -func assertAPIError(t *testing.T, got *thanos_api.ApiError, exp thanos_api.ErrorType) { - t.Helper() - - if got != nil { - if exp == thanos_api.ErrorNone { - t.Fatalf("Unexpected error: %s", got) - } - if exp != got.Typ { - t.Fatalf("Expected error of type %q but got type %q (%q)", exp, got.Typ, got) - } - return - } - if exp != thanos_api.ErrorNone { - t.Fatalf("Expected error of type %q but got none", exp) - } -} - -func assertAPIResponse(t *testing.T, got, exp interface{}) { - t.Helper() - - require.Equal(t, exp, got) -} - -func TestRespondSuccess(t *testing.T) { - s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - api := API{} - api.Respond(w, testResponse, nil) - })) - defer s.Close() - - resp, err := http.Get(s.URL) - if err != nil { - t.Fatalf("Error on test request: %s", err) - } - body, err := io.ReadAll(resp.Body) - defer resp.Body.Close() - if err != nil { - t.Fatalf("Error reading response body: %s", err) - } - - if resp.StatusCode != http.StatusOK { - t.Fatalf("Return code %d expected in success response but got %d", 200, resp.StatusCode) - } - if h := resp.Header.Get("Content-Type"); h != "application/x-protobuf" { - t.Fatalf("Expected Content-Type %q but got %q", "application/x-protobuf", h) - } - - var res queryrange.PrometheusResponse - if err = proto.Unmarshal(body, &res); err != nil { - t.Fatalf("Error unmarshaling response body: %s", err) - } - - require.Equal(t, testResponse, &res) -} - -func TestRespondError(t *testing.T) { - s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - api := API{} - api.RespondError(w, &thanos_api.ApiError{thanos_api.ErrorTimeout, errors.New("message")}, "test") - })) - defer s.Close() - - resp, err := http.Get(s.URL) - if err != nil { - t.Fatalf("Error on test request: %s", err) - } - body, err := io.ReadAll(resp.Body) - defer resp.Body.Close() - if err != nil { - t.Fatalf("Error reading response body: %s", err) - } - - if want, have := http.StatusServiceUnavailable, resp.StatusCode; want != have { - t.Fatalf("Return code %d expected in error response but got %d", want, have) - } - if h := resp.Header.Get("Content-Type"); h != "application/json" { - t.Fatalf("Expected Content-Type %q but got %q", "application/json", h) - } - - var res response - if err = json.Unmarshal(body, &res); err != nil { - t.Fatalf("Error unmarshaling JSON body: %s", err) - } - - exp := &response{ - Status: statusError, - Data: "test", - ErrorType: thanos_api.ErrorTimeout, - Error: "message", - } - require.Equal(t, exp, &res) -} - -func TestParseTimeParam(t *testing.T) { - type resultType struct { - asTime time.Time - asError func() error - } - - ts, err := parseTime("1582468023986") - require.NoError(t, err) - - tests := []struct { - paramName string - paramValue string - defaultValue time.Time - result resultType - }{ - { // When data is valid. - paramName: "start", - paramValue: "1582468023986", - defaultValue: minTime, - result: resultType{ - asTime: ts, - asError: nil, - }, - }, - { // When data is empty string. - paramName: "end", - paramValue: "", - defaultValue: maxTime, - result: resultType{ - asTime: maxTime, - asError: nil, - }, - }, - { // When data is not valid. - paramName: "foo", - paramValue: "baz", - defaultValue: maxTime, - result: resultType{ - asTime: time.Time{}, - asError: func() error { - _, err := parseTime("baz") - return errors.Wrapf(err, "Invalid time value for '%s'", "foo") - }, - }, - }, - } - - for _, test := range tests { - req, err := http.NewRequest("GET", "localhost:42/foo?"+test.paramName+"="+test.paramValue, nil) - require.NoError(t, err) - - result := test.result - asTime, err := parseTimeParam(req, test.paramName, test.defaultValue) - - if err != nil { - require.EqualError(t, err, result.asError().Error()) - } else { - require.True(t, asTime.Equal(result.asTime), "time as return value: %s not parsed correctly. Expected %s. Actual %s", test.paramValue, result.asTime, asTime) - } - } -} - -func TestParseTime(t *testing.T) { - ts, err := time.Parse(time.RFC3339Nano, "2015-06-03T13:21:58.555Z") - if err != nil { - panic(err) - } - - tests := []struct { - input string - fail bool - result time.Time - }{ - { - input: "", - fail: true, - }, - { - input: "abc", - fail: true, - }, - { - input: "30s", - fail: true, - }, - { - input: "123", - result: time.Unix(123, 0), - }, - { - input: "123.123", - result: time.Unix(123, 123000000), - }, - { - input: "2015-06-03T13:21:58.555Z", - result: ts, - }, - { - input: "2015-06-03T14:21:58.555+01:00", - result: ts, - }, - { - // Test float rounding. - input: "1543578564.705", - result: time.Unix(1543578564, 705*1e6), - }, - { - input: minTime.Format(time.RFC3339Nano), - result: minTime, - }, - { - input: maxTime.Format(time.RFC3339Nano), - result: maxTime, - }, - } - - for _, test := range tests { - ts, err := parseTime(test.input) - - if err != nil && !test.fail { - t.Errorf("Unexpected error for %q: %s", test.input, err) - continue - } - if err == nil && test.fail { - t.Errorf("Expected error for %q but got none", test.input) - continue - } - if !test.fail && !ts.Equal(test.result) { - t.Errorf("Expected time %v for input %q but got %v", test.result, test.input, ts) - } - } -} - -func TestParseDuration(t *testing.T) { - tests := []struct { - input string - fail bool - result time.Duration - }{ - { - input: "", - fail: true, - }, { - input: "abc", - fail: true, - }, { - input: "2015-06-03T13:21:58.555Z", - fail: true, - }, { - // Internal int64 overflow. - input: "-148966367200.372", - fail: true, - }, { - // Internal int64 overflow. - input: "148966367200.372", - fail: true, - }, { - input: "123", - result: 123 * time.Second, - }, { - input: "123.333", - result: 123*time.Second + 333*time.Millisecond, - }, { - input: "15s", - result: 15 * time.Second, - }, { - input: "5m", - result: 5 * time.Minute, - }, - } - - for _, test := range tests { - d, err := parseDuration(test.input) - if err != nil && !test.fail { - t.Errorf("Unexpected error for %q: %s", test.input, err) - continue - } - if err == nil && test.fail { - t.Errorf("Expected error for %q but got none", test.input) - continue - } - if !test.fail && d != test.result { - t.Errorf("Expected duration %v for input %q but got %v", test.result, test.input, d) - } - } -} - -func TestRespond(t *testing.T) { - cases := []struct { - response interface{} - }{ - { - response: &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Floats: []promql.FPoint{{F: 1, T: 1000}}, - Metric: labels.FromStrings("__name__", "foo"), - }, - }, - }, - }, - { - response: &queryrange.PrometheusResponse{ - Status: "success", - Data: queryrange.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}, - }, - }, - }, - }, - }, - }, - { - response: &instantquery.PrometheusInstantQueryResponse{ - Status: "success", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: instantquery.PrometheusInstantQueryResult{ - Result: &instantquery.PrometheusInstantQueryResult_Vector{ - Vector: &instantquery.Vector{ - Samples: []*instantquery.Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - }, - }, - }, - } - - for _, c := range cases { - s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - api := API{} - api.Respond(w, c.response, nil) - })) - defer s.Close() - - resp, err := http.Get(s.URL) - if err != nil { - t.Fatalf("Error on test request: %s", err) - } - body, err := io.ReadAll(resp.Body) - defer resp.Body.Close() - if err != nil { - t.Fatalf("Error reading response body: %s", err) - } - - var res interface{} - switch dest := c.response.(type) { - case *queryrange.PrometheusResponse: - err = proto.Unmarshal(body, dest) - res = dest - case *instantquery.PrometheusInstantQueryResponse: - err = proto.Unmarshal(body, dest) - res = dest - case *queryData: - err = json.Unmarshal(body, dest) - res = dest - } - - if err != nil { - t.Fatalf("Error unmarshaling response body: %s", err) - } - - require.Equal(t, c.response, res) - } -} - -func TestReturnAPIError(t *testing.T) { - cases := []struct { - err error - expected thanos_api.ErrorType - }{ - { - err: promql.ErrStorage{Err: errors.New("storage error")}, - expected: thanos_api.ErrorInternal, - }, { - err: fmt.Errorf("wrapped: %w", promql.ErrStorage{Err: errors.New("storage error")}), - expected: thanos_api.ErrorInternal, - }, { - err: promql.ErrQueryTimeout("timeout error"), - expected: thanos_api.ErrorTimeout, - }, { - err: fmt.Errorf("wrapped: %w", promql.ErrQueryTimeout("timeout error")), - expected: thanos_api.ErrorTimeout, - }, { - err: promql.ErrQueryCanceled("canceled error"), - expected: thanos_api.ErrorCanceled, - }, { - err: fmt.Errorf("wrapped: %w", promql.ErrQueryCanceled("canceled error")), - expected: thanos_api.ErrorCanceled, - }, { - err: errors.New("exec error"), - expected: thanos_api.ErrorExec, - }, - } - - for ix, c := range cases { - actual := returnAPIError(c.err) - require.Error(t, actual, ix) - require.Equal(t, c.expected, actual.Typ, ix) - } -} - -// This is a global to avoid the benchmark being optimized away. -var testResponseWriter = httptest.ResponseRecorder{} - -func BenchmarkRespondQueryData(b *testing.B) { - b.ReportAllocs() - points := []promql.FPoint{} - for i := 0; i < 10000; i++ { - points = append(points, promql.FPoint{F: float64(i * 1000000), T: int64(i)}) - } - response := &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Floats: points, - Metric: labels.EmptyLabels(), - }, - }, - } - b.ResetTimer() - api := API{} - for n := 0; n < b.N; n++ { - api.Respond(&testResponseWriter, response, nil) - } -} - -func BenchmarkRespondPrometheusResponse(b *testing.B) { - b.ReportAllocs() - points := []cortexpb.Sample{} - for i := 0; i < 10000; i++ { - points = append(points, cortexpb.Sample{Value: float64(i * 1000000), TimestampMs: int64(i)}) - } - response := &queryrange.PrometheusResponse{ - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: points, - }, - }, - }, - } - b.ResetTimer() - api := API{} - for n := 0; n < b.N; n++ { - api.Respond(&testResponseWriter, response, nil) - } -} - -func BenchmarkRespondPrometheusInstantQueryResponse(b *testing.B) { - b.ReportAllocs() - points := []cortexpb.Sample{} - for i := 0; i < 10000; i++ { - points = append(points, cortexpb.Sample{Value: float64(i * 1000000), TimestampMs: int64(i)}) - } - response := &instantquery.PrometheusInstantQueryResponse{ - Status: "success", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: model.ValMatrix.String(), - Result: instantquery.PrometheusInstantQueryResult{ - Result: &instantquery.PrometheusInstantQueryResult_Matrix{ - Matrix: &instantquery.Matrix{ - SampleStreams: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{}, - Samples: points, - }, - }, - }, - }, - }, - }, - } - b.ResetTimer() - api := API{} - for n := 0; n < b.N; n++ { - api.Respond(&testResponseWriter, response, nil) - } -} - -func TestExtractQueryOpts(t *testing.T) { - tests := []struct { - name string - form url.Values - expect *promql.QueryOpts - err error - }{ - { - name: "with stats all", - form: url.Values{ - "stats": []string{"all"}, - }, - expect: &promql.QueryOpts{ - EnablePerStepStats: true, - }, - err: nil, - }, - { - name: "with stats none", - form: url.Values{ - "stats": []string{"none"}, - }, - expect: &promql.QueryOpts{ - EnablePerStepStats: false, - }, - err: nil, - }, - { - name: "with lookback delta", - form: url.Values{ - "stats": []string{"all"}, - "lookback_delta": []string{"30s"}, - }, - expect: &promql.QueryOpts{ - EnablePerStepStats: true, - LookbackDelta: 30 * time.Second, - }, - err: nil, - }, - { - name: "with invalid lookback delta", - form: url.Values{ - "lookback_delta": []string{"invalid"}, - }, - expect: nil, - err: errors.New(`error parsing lookback delta duration: cannot parse "invalid" to a valid duration`), - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - req := &http.Request{Form: test.form} - opts, err := extractQueryOpts(req) - require.Equal(t, test.expect, opts) - if test.err == nil { - require.NoError(t, err) - } else { - require.Equal(t, test.err.Error(), err.Error()) - } - }) - } -} - -func TestCreatePrometheusResponse(t *testing.T) { - tests := []struct { - data *queryData - response *queryrange.PrometheusResponse - err error - }{ - { - data: &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Floats: []promql.FPoint{ - {F: 1, T: 1000}, - {F: 2, T: 2000}, - {F: 3, T: 3000}, - }, - }, - }, - }, - response: &queryrange.PrometheusResponse{ - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - }, - }, - }, - { - data: &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Metric: labels.Labels{ - {Name: "__name__", Value: "foo"}, - {Name: "__job__", Value: "bar"}, - }, - Floats: []promql.FPoint{ - {F: 0.14, T: 18555000}, - {F: 2.9, T: 18556000}, - {F: 30, T: 18557000}, - }, - }, - }, - }, - response: &queryrange.PrometheusResponse{ - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__name__", Value: "foo"}, - {Name: "__job__", Value: "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 0.14, TimestampMs: 18555000}, - {Value: 2.9, TimestampMs: 18556000}, - {Value: 30, TimestampMs: 18557000}, - }, - }, - }, - }, - }, - }, - { - data: nil, - err: errors.New("no query response data"), - }, - } - for i, test := range tests { - t.Run(strconv.Itoa(i), func(t *testing.T) { - res, err := createPrometheusResponse(test.data) - require.Equal(t, test.response, res) - if test.err == nil { - require.NoError(t, err) - } else { - require.Equal(t, test.err.Error(), err.Error()) - } - }) - } -} - -func TestCreatePrometheusInstantQueryResponse(t *testing.T) { - tests := []struct { - data *queryData - response *instantquery.PrometheusInstantQueryResponse - err error - }{ - { - data: &queryData{ - ResultType: parser.ValueTypeScalar, - Result: promql.Scalar{T: 1000, V: 1}, - }, - response: &instantquery.PrometheusInstantQueryResponse{ - Status: "success", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: model.ValScalar.String(), - Result: instantquery.PrometheusInstantQueryResult{ - Result: &instantquery.PrometheusInstantQueryResult_RawBytes{ - RawBytes: []byte(`{"resultType":"scalar","result":[1,"1"]}`), - }, - }, - }, - }, - }, - { - data: &queryData{ - ResultType: parser.ValueTypeMatrix, - Result: promql.Matrix{ - promql.Series{ - Metric: labels.Labels{ - {Name: "__name__", Value: "foo"}, - {Name: "__job__", Value: "bar"}, - }, - Floats: []promql.FPoint{ - {F: 0.14, T: 18555000}, - {F: 2.9, T: 18556000}, - {F: 30, T: 18557000}, - }, - }, - }, - }, - response: &instantquery.PrometheusInstantQueryResponse{ - Status: "success", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: model.ValMatrix.String(), - Result: instantquery.PrometheusInstantQueryResult{ - Result: &instantquery.PrometheusInstantQueryResult_Matrix{ - Matrix: &instantquery.Matrix{ - SampleStreams: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "foo"}, - {"__job__", "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 0.14, TimestampMs: 18555000}, - {Value: 2.9, TimestampMs: 18556000}, - {Value: 30, TimestampMs: 18557000}, - }, - }, - }, - }, - }, - }, - }, - }, - }, - { - data: &queryData{ - ResultType: parser.ValueTypeVector, - Result: promql.Vector{ - { - F: 1, - T: 1000, - Metric: labels.Labels{ - {"__name__", "up"}, - {"job", "foo"}, - }, - }, - }, - }, - response: &instantquery.PrometheusInstantQueryResponse{ - Status: "success", - Data: instantquery.PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: instantquery.PrometheusInstantQueryResult{ - Result: &instantquery.PrometheusInstantQueryResult_Vector{ - Vector: &instantquery.Vector{ - Samples: []*instantquery.Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - }, - }, - }, - { - data: nil, - err: errors.New("no query response data"), - }, - } - for i, test := range tests { - t.Run(strconv.Itoa(i), func(t *testing.T) { - res, err := createPrometheusInstantQueryResponse(test.data) - require.Equal(t, test.response, res) - if test.err == nil { - require.NoError(t, err) - } else { - require.Equal(t, test.err.Error(), err.Error()) - } - }) - } -} diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 0a643dfb0cf..130dc3e4c9f 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "fmt" - "github.com/gogo/protobuf/proto" "io" "net/http" "net/url" @@ -13,6 +12,8 @@ import ( "time" "unsafe" + "github.com/gogo/protobuf/proto" + jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" @@ -185,10 +186,20 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ } var resp PrometheusInstantQueryResponse - if err := proto.Unmarshal(buf, &resp); err != nil { + if r.Header != nil && r.Header.Get("Content-Type") == "application/x-protobuf" { + err = proto.Unmarshal(buf, &resp) + } else { + err = json.Unmarshal(buf, &resp) + } + + if err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } + for h, hv := range r.Header { + resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) + } + return &resp, nil } diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index f50361192e8..36b5456a180 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -5,15 +5,16 @@ import ( "compress/gzip" "context" "fmt" - "github.com/gogo/protobuf/proto" - "github.com/golang/snappy" - "github.com/weaveworks/common/httpgrpc" "io" "net/http" "strconv" "testing" "time" + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" + "github.com/weaveworks/common/httpgrpc" + "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -108,8 +109,7 @@ func TestRequest(t *testing.T) { func TestCompressedResponse(t *testing.T) { t.Parallel() - for _, tc := range []struct { - name string + for i, tc := range []struct { compression string jsonBody string promBody *PrometheusInstantQueryResponse @@ -117,7 +117,6 @@ func TestCompressedResponse(t *testing.T) { err error }{ { - name: "successful response", compression: "gzip", promBody: &PrometheusInstantQueryResponse{ Status: "success", @@ -125,15 +124,11 @@ func TestCompressedResponse(t *testing.T) { ResultType: model.ValString.String(), Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - {Name: "Content-Encoding", Values: []string{"gzip"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, status: 200, }, { - name: "successful response", compression: "snappy", promBody: &PrometheusInstantQueryResponse{ Status: "success", @@ -141,125 +136,135 @@ func TestCompressedResponse(t *testing.T) { ResultType: model.ValString.String(), Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - {Name: "Content-Encoding", Values: []string{"snappy"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, status: 200, }, { - name: `400 error`, compression: `gzip`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, { - name: `400 error`, compression: `snappy`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, { - name: `400 error empty body`, compression: `gzip`, status: 400, err: httpgrpc.Errorf(400, ""), }, { - name: `400 error empty body`, compression: `snappy`, status: 400, err: httpgrpc.Errorf(400, ""), }, } { - for _, c := range []bool{true, false} { - c := c - t.Run(fmt.Sprintf("%s compressed %t [%s]", tc.compression, c, tc.name), func(t *testing.T) { - t.Parallel() - h := http.Header{} - var b []byte - if tc.promBody != nil { - protobuf, err := proto.Marshal(tc.promBody) - b = protobuf - require.NoError(t, err) - h.Set("Content-Type", "application/x-protobuf") - } else { - b = []byte(tc.jsonBody) - h.Set("Content-Type", "application/json") - } - responseBody := bytes.NewBuffer(b) + t.Run(strconv.Itoa(i), func(t *testing.T) { + t.Parallel() + h := http.Header{} + var b []byte + if tc.promBody != nil { + protobuf, err := proto.Marshal(tc.promBody) + b = protobuf + require.NoError(t, err) + h.Set("Content-Type", "application/x-protobuf") + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) + } else { + b = []byte(tc.jsonBody) + h.Set("Content-Type", "application/json") + } + responseBody := bytes.NewBuffer(b) - var buf bytes.Buffer - if c && tc.compression == "gzip" { - h.Set("Content-Encoding", "gzip") - w := gzip.NewWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } else if c && tc.compression == "snappy" { - h.Set("Content-Encoding", "snappy") - w := snappy.NewBufferedWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } + var buf bytes.Buffer + if tc.compression == "gzip" { + h.Set("Content-Encoding", "gzip") + if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} + w := gzip.NewWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf + } else if tc.compression == "snappy" { + h.Set("Content-Encoding", "snappy") + if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}})} + w := snappy.NewBufferedWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf + } - response := &http.Response{ - StatusCode: tc.status, - Header: h, - Body: io.NopCloser(responseBody), - } - resp, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) - require.Equal(t, tc.err, err) + response := &http.Response{ + StatusCode: tc.status, + Header: h, + Body: io.NopCloser(responseBody), + } + resp, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) + require.Equal(t, tc.err, err) - if err == nil { - require.NoError(t, err) - require.Equal(t, tc.promBody, resp) - } - }) - } + if err == nil { + require.NoError(t, err) + require.Equal(t, tc.promBody, resp) + } + }) } } func TestResponse(t *testing.T) { t.Parallel() for i, tc := range []struct { - expectedResp string - promBody *PrometheusInstantQueryResponse + jsonBody string + promBody *PrometheusInstantQueryResponse }{ { - expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, + jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, + }, + { + jsonBody: testHistogramResponse, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { - expectedResp: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`)}}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { - expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ @@ -291,13 +296,11 @@ func TestResponse(t *testing.T) { }, }, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { - expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ @@ -320,26 +323,22 @@ func TestResponse(t *testing.T) { }, }, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { - expectedResp: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, + jsonBody: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ ResultType: model.ValString.String(), Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"scalar","result":[1,"13"]}`)}}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, + jsonBody: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, promBody: &PrometheusInstantQueryResponse{ Status: "success", Data: PrometheusInstantQueryData{ @@ -350,33 +349,39 @@ func TestResponse(t *testing.T) { Samples: []*Sample{ { Labels: []cortexpb.LabelAdapter{}, - Sample: cortexpb.Sample{Value: 1266464.0146205237, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1266464.0146205237, TimestampMs: 1000}, }, }, }, }, }, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, }, - { - body: testHistogramResponse, - }, } { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - protobuf, err := proto.Marshal(tc.promBody) - require.NoError(t, err) - - response := &http.Response{ - StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, - Body: io.NopCloser(bytes.NewBuffer(protobuf)), + + var response *http.Response + if tc.promBody != nil { + protobuf, err := proto.Marshal(tc.promBody) + require.NoError(t, err) + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), + } + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) + } else { + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{"application/json"}}, + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + } } + resp, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) require.NoError(t, err) @@ -384,8 +389,8 @@ func TestResponse(t *testing.T) { response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResp))), - ContentLength: int64(len(tc.expectedResp)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + ContentLength: int64(len(tc.jsonBody)), } resp2, err := InstantQueryCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -395,6 +400,268 @@ func TestResponse(t *testing.T) { } func TestMergeResponse(t *testing.T) { + t.Parallel() + defaultReq := &PrometheusRequest{ + Query: "sum(up)", + } + for _, tc := range []struct { + name string + req tripperware.Request + resps []string + expectedResp string + expectedErr error + cancelBeforeDecode bool + expectedDecodeErr error + cancelBeforeMerge bool + }{ + { + name: "empty response", + req: defaultReq, + resps: []string{`{"status":"success","data":{"resultType":"vector","result":[]}}`}, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[]}}`, + }, + { + name: "empty response with stats", + req: defaultReq, + resps: []string{`{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[]}}}}`}, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[]}}}}`, + }, + { + name: "single response", + req: defaultReq, + resps: []string{`{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`}, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, + }, + { + name: "single response with stats", + req: defaultReq, + resps: []string{`{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`}, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + }, + { + name: "duplicated response", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, + }, + { + name: "duplicated histogram responses", + req: defaultReq, + resps: []string{testHistogramResponse, testHistogramResponse}, + expectedResp: testHistogramResponse, + }, + { + name: "duplicated response with stats", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + }, + { + name: "merge two responses", + req: defaultReq, + 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":[2,"2"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, + }, + { + name: "merge two histogram responses", + req: defaultReq, + 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":[1719528800,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528800,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]},{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, + }, + { + name: "merge two responses with sort", + req: &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"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, + }, + { + name: "merge two histogram responses with sort", + req: &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"]]}]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]},{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, + }, + { + name: "merge two responses with sort_desc", + req: &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"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, + }, + { + name: "merge two histogram responses with sort_desc", + req: &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"]]}]}]}}`, + }, + expectedResp: `{"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"]]}]},{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, + }, + { + name: "merge two responses with topk", + req: &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"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, + }, + { + name: "merge two histogram responses with topk", + req: &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"]]}]}]}}`, + }, + expectedResp: `{"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"]]}]},{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, + }, + { + name: "merge with warnings.", + req: &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"]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]},"warnings":["warning1","warning2","warning3"]}`, + }, + { + name: "merge two responses with stats", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + }, + { + name: "responses don't contain vector, should return an error", + req: defaultReq, + resps: []string{ + `{"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"), + }, + { + name: "single matrix response", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up"},"values":[[1,"1"],[2,"2"]]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up"},"values":[[1,"1"],[2,"2"]]}]}}`, + }, + { + name: "multiple matrix responses without duplicated series", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"foo"},"values":[[3,"3"],[4,"4"]]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]},{"metric":{"__name__":"foo"},"values":[[3,"3"],[4,"4"]]}]}}`, + }, + { + name: "multiple matrix responses with duplicated series, but not same samples", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[3,"3"]]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, + }, + { + name: "multiple matrix responses with duplicated series and same samples", + req: defaultReq, + resps: []string{ + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"]]}]}}`, + `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, + }, + expectedResp: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"bar"},"values":[[1,"1"],[2,"2"],[3,"3"]]}]}}`, + }, + { + name: "context cancelled before decoding response", + req: defaultReq, + 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":[2,"2"]}]}}`, + }, + expectedDecodeErr: context.Canceled, + cancelBeforeDecode: true, + }, + { + name: "context cancelled before merging response", + req: defaultReq, + 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":[2,"2"]}]}}`, + }, + expectedErr: context.Canceled, + cancelBeforeMerge: true, + }, + } { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + ctx, cancelCtx := context.WithCancel(context.Background()) + + var resps []tripperware.Response + for _, r := range tc.resps { + hr := &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{"application/json"}}, + Body: io.NopCloser(bytes.NewBuffer([]byte(r))), + } + + if tc.cancelBeforeDecode { + cancelCtx() + } + dr, err := InstantQueryCodec.DecodeResponse(ctx, hr, nil) + assert.Equal(t, tc.expectedDecodeErr, err) + if err != nil { + cancelCtx() + return + } + resps = append(resps, dr) + } + + if tc.cancelBeforeMerge { + cancelCtx() + } + resp, err := InstantQueryCodec.MergeResponse(ctx, tc.req, resps...) + assert.Equal(t, tc.expectedErr, err) + if err != nil { + cancelCtx() + return + } + dr, err := InstantQueryCodec.EncodeResponse(ctx, resp) + 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) + cancelCtx() + }) + } +} + +func TestMergeResponseProtobuf(t *testing.T) { t.Parallel() defaultReq := &PrometheusRequest{ Query: "sum(up)", @@ -477,7 +744,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -507,7 +774,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -545,7 +812,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -568,7 +835,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -582,12 +849,6 @@ func TestMergeResponse(t *testing.T) { }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}]}}`, }, - { - name: "duplicated histogram responses", - req: defaultReq, - resps: []string{testHistogramResponse, testHistogramResponse}, - expectedResp: testHistogramResponse, - }, { name: "duplicated response with stats", req: defaultReq, @@ -604,7 +865,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -635,7 +896,7 @@ func TestMergeResponse(t *testing.T) { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -674,7 +935,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -698,7 +959,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 2000}, }, }, }, @@ -712,15 +973,6 @@ func TestMergeResponse(t *testing.T) { }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, }, - { - name: "merge two histogram responses", - req: defaultReq, - 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":[1719528800,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, - }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528800,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]},{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, - }, { name: "merge two responses with sort", req: &PrometheusRequest{Query: "sort(sum by (job) (up))"}, @@ -738,7 +990,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -762,7 +1014,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, }, }, @@ -776,15 +1028,6 @@ func TestMergeResponse(t *testing.T) { }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, }, - { - name: "merge two histogram responses with sort", - req: &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"]]}]}]}}`, - }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]},{"metric":{"__name__":"up","job":"foo"},"histogram":[1719528871.898,{"count":"6342","sum":"43.31319875499995","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, - }, { name: "merge two responses with sort_desc", req: &PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, @@ -802,7 +1045,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -826,7 +1069,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, }, }, @@ -840,15 +1083,6 @@ func TestMergeResponse(t *testing.T) { }, expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}]}}`, }, - { - name: "merge two histogram responses with sort_desc", - req: &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"]]}]}]}}`, - }, - expectedResp: `{"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"]]}]},{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, - }, { name: "merge two responses with topk", req: &PrometheusRequest{Query: "topk(10, up) by(job)"}, @@ -866,7 +1100,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -890,7 +1124,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, }, }, @@ -905,22 +1139,61 @@ func TestMergeResponse(t *testing.T) { expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]}}`, }, { - name: "merge two histogram responses with topk", - req: &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"]]}]}]}}`, - }, - expectedResp: `{"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"]]}]},{"metric":{"__name__":"up","job":"bar"},"histogram":[1719528880,{"count":"1","sum":"0","buckets":[[0,"0.0013810679320049755","0.0015060652591874421","1"]]}]}]}}`, - }, - { - name: "merge with warnings.", - req: &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"]}]}}`, + name: "merge with warnings", + req: defaultReq, + resps: []*PrometheusInstantQueryResponse{ + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "foo"}, + }, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Warnings: []string{"warning1","warning2"}, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, + { + Status: "success", + Data: PrometheusInstantQueryData{ + ResultType: model.ValVector.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Vector{ + Vector: &Vector{ + Samples: []*Sample{ + { + Labels: []cortexpb.LabelAdapter{ + {"__name__", "up"}, + {"job", "bar"}, + }, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, + }, + }, + }, + }, + }, + }, + Warnings: []string{"warning1","warning3"}, + Headers: []*tripperware.PrometheusResponseHeader{ + {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, + }, + }, }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"bar"},"value":[1,"2"]}]},"warnings":["warning1","warning2","warning3"]}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[1,"1"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"2"]}]},"warnings":["warning1","warning2","warning3"]}`, }, { name: "merge two responses with stats", @@ -939,7 +1212,7 @@ func TestMergeResponse(t *testing.T) { {Name: "__name__", Value: "up"}, {Name: "job", Value: "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -971,7 +1244,7 @@ func TestMergeResponse(t *testing.T) { {Name: "__name__", Value: "up"}, {Name: "job", Value: "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 2000}, }, }, }, @@ -1255,7 +1528,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -1279,7 +1552,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, }, }, @@ -1311,7 +1584,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "foo"}, }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, }, }, @@ -1335,7 +1608,7 @@ func TestMergeResponse(t *testing.T) { {"__name__", "up"}, {"job", "bar"}, }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 1000}, + Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, }, }, @@ -1473,6 +1746,69 @@ func Benchmark_Decode(b *testing.B) { samples[i].Samples = append(samples[i].Samples, cortexpb.Sample{TimestampMs: int64(i), Value: float64(i)}) } + for name, tc := range map[string]struct { + sampleStream []tripperware.SampleStream + }{ + "100 samples": { + sampleStream: samples[:100], + }, + "1000 samples": { + sampleStream: samples[:1000], + }, + "10000 samples": { + sampleStream: samples[:10000], + }, + "100000 samples": { + sampleStream: samples[:100000], + }, + "1000000 samples": { + sampleStream: samples[:1000000], + }, + } { + b.Run(name, func(b *testing.B) { + r := PrometheusInstantQueryResponse{ + Data: PrometheusInstantQueryData{ + ResultType: model.ValMatrix.String(), + Result: PrometheusInstantQueryResult{ + Result: &PrometheusInstantQueryResult_Matrix{ + Matrix: &Matrix{ + SampleStreams: tc.sampleStream, + }, + }, + }, + }, + } + + body, err := json.Marshal(r) + require.NoError(b, err) + + b.ResetTimer() + b.ReportAllocs() + + for i := 0; i < b.N; i++ { + response := &http.Response{ + StatusCode: 200, + Body: io.NopCloser(bytes.NewBuffer(body)), + } + _, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) + require.NoError(b, err) + } + }) + } + +} + +func Benchmark_Decode_Protobuf(b *testing.B) { + maxSamplesCount := 1000000 + samples := make([]tripperware.SampleStream, maxSamplesCount) + + for i := 0; i < maxSamplesCount; i++ { + samples[i].Labels = append(samples[i].Labels, cortexpb.LabelAdapter{Name: fmt.Sprintf("Sample%v", i), Value: fmt.Sprintf("Value%v", i)}) + samples[i].Labels = append(samples[i].Labels, cortexpb.LabelAdapter{Name: fmt.Sprintf("Sample2%v", i), Value: fmt.Sprintf("Value2%v", i)}) + samples[i].Labels = append(samples[i].Labels, cortexpb.LabelAdapter{Name: fmt.Sprintf("Sample3%v", i), Value: fmt.Sprintf("Value3%v", i)}) + samples[i].Samples = append(samples[i].Samples, cortexpb.Sample{TimestampMs: int64(i), Value: float64(i)}) + } + for name, tc := range map[string]struct { sampleStream []tripperware.SampleStream }{ @@ -1515,6 +1851,7 @@ func Benchmark_Decode(b *testing.B) { for i := 0; i < b.N; i++ { response := &http.Response{ StatusCode: 200, + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, Body: io.NopCloser(bytes.NewBuffer(body)), } _, err := InstantQueryCodec.DecodeResponse(context.Background(), response, nil) diff --git a/pkg/querier/tripperware/instantquery/shard_by_query_test.go b/pkg/querier/tripperware/instantquery/shard_by_query_test.go index 22e793edbb3..26980cb8dd2 100644 --- a/pkg/querier/tripperware/instantquery/shard_by_query_test.go +++ b/pkg/querier/tripperware/instantquery/shard_by_query_test.go @@ -1,879 +1,13 @@ package instantquery import ( - "context" - "encoding/base64" - "fmt" - "github.com/cortexproject/cortex/pkg/cortexpb" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/querysharding" - "github.com/go-kit/log" - "github.com/gogo/protobuf/proto" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/promql/parser" - "github.com/stretchr/testify/require" - thanosquerysharding "github.com/thanos-io/thanos/pkg/querysharding" - "github.com/thanos-io/thanos/pkg/store/storepb" - "github.com/weaveworks/common/user" - "io" - "net/http" - "net/http/httptest" - "net/url" - "sort" "testing" + "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" ) func Test_shardQuery(t *testing.T) { - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, "", true) - instantQueryCodec := InstantQueryCodec - - type queries struct { - name string - expression string - shardingLabels []string - } - - nonShardable := []queries{ - { - name: "aggregation", - expression: "sum(http_requests_total)", - }, - { - name: "outer aggregation with no grouping", - expression: "count(sum by (pod) (http_requests_total))", - }, - { - name: "outer aggregation with without grouping", - expression: "count(sum without (pod) (http_requests_total))", - }, - { - name: "binary expression with constant", - expression: `http_requests_total{code="400"} / 4`, - }, - { - name: "binary expression with empty vector matching", - expression: `http_requests_total{code="400"} / on () http_requests_total`, - }, - { - name: "binary aggregation with different grouping labels", - expression: `sum by (pod) (http_requests_total{code="400"}) / sum by (cluster) (http_requests_total)`, - }, - { - name: "multiple binary expressions with empty vector matchers", - expression: ` -(http_requests_total{code="400"} + on (cluster, pod) http_requests_total{code="500"}) -/ on () -http_requests_total`, - }, - { - name: "problematic query", - expression: `sum(a by(lanel)`, - }, - { - name: "aggregate by expression with label_replace, sharding label is dynamic", - expression: `sum by (dst_label) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, - }, - { - name: "aggregate by expression with label_join, sharding label is dynamic", - expression: `sum by (dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, - }, - } - - shardableByLabels := []queries{ - { - name: "aggregation with grouping", - expression: "sum by (pod) (http_requests_total)", - shardingLabels: []string{"pod"}, - }, - { - name: "aggregation with comparison", - expression: "avg by (Roles,type) (rss_service_message_handling) > 0.5", - shardingLabels: []string{"Roles", "type"}, - }, - { - name: "multiple aggregations with grouping", - expression: "max by (pod) (sum by (pod, cluster) (http_requests_total))", - shardingLabels: []string{"pod"}, - }, - { - name: "binary expression with vector matching", - expression: `http_requests_total{code="400"} / on (pod) http_requests_total`, - shardingLabels: []string{"pod"}, - }, - { - name: "binary aggregation with same grouping labels", - expression: `sum by (pod) (http_requests_total{code="400"}) / sum by (pod) (http_requests_total)`, - shardingLabels: []string{"pod"}, - }, - { - name: "binary expression with vector matching and grouping", - expression: `sum by (cluster, pod) (http_requests_total{code="400"}) / on (pod) sum by (cluster, pod) (http_requests_total)`, - shardingLabels: []string{"pod"}, - }, - { - name: "multiple binary expressions with vector matchers", - expression: ` -(http_requests_total{code="400"} + on (cluster, pod) http_requests_total{code="500"}) -/ on (pod) -http_requests_total`, - shardingLabels: []string{"pod"}, - }, - { - name: "multiple binary expressions with grouping", - expression: ` -sum by (container) ( - (http_requests_total{code="400"} + on (cluster, pod, container) http_requests_total{code="500"}) - / on (pod, container) - http_requests_total -)`, - shardingLabels: []string{"container"}, - }, - { - name: "multiple binary expressions with grouping", - expression: `(http_requests_total{code="400"} + on (pod) http_requests_total{code="500"}) / on (cluster, pod) http_requests_total`, - shardingLabels: []string{"pod"}, - }, - { - name: "histogram quantile", - expression: "histogram_quantile(0.95, sum(rate(metric[1m])) by (le, cluster))", - shardingLabels: []string{"cluster"}, - }, - { - name: "aggregate by expression with label_replace, sharding label is not dynamic", - expression: `sum by (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, - shardingLabels: []string{"pod"}, - }, - { - name: "aggregate by expression with label_join, sharding label is not dynamic", - expression: `sum by (pod) (label_join(metric, "dst_label", ",", "src_label"))`, - shardingLabels: []string{"pod"}, - }, - { - name: "label_join and aggregation on multiple labels. Can be sharded by the static one", - expression: `sum by (pod, dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, - shardingLabels: []string{"pod"}, - }, - { - name: "binary expression with vector matching and label_replace", - expression: `http_requests_total{code="400"} / on (pod) label_replace(metric, "dst_label", "$1", "src_label", "re")`, - shardingLabels: []string{"pod"}, - }, - { - name: "nested label joins", - expression: `label_join(sum by (pod) (label_join(metric, "dst_label", ",", "src_label")), "dst_label1", ",", "dst_label")`, - shardingLabels: []string{"pod"}, - }, - { - name: "complex query with label_replace, binary expr and aggregations on dynamic label", - expression: `sum(sum_over_time(container_memory_working_set_bytes{container_name!="POD",container_name!="",namespace="kube-system"}[1d:5m])) by (instance, cluster) / avg(label_replace(sum(sum_over_time(kube_node_status_capacity_memory_bytes[1d:5m])) by (node, cluster), "instance", "$1", "node", "(.*)")) by (instance, cluster)`, - shardingLabels: []string{"cluster"}, - }, - { - name: "complex query with label_replace and nested aggregations", - expression: `avg(label_replace(label_replace(avg(count_over_time(kube_pod_container_resource_requests{resource="memory", unit="byte", container!="",container!="POD", node!="", }[1h] )*avg_over_time(kube_pod_container_resource_requests{resource="memory", unit="byte", container!="",container!="POD", node!="", }[1h] )) by (namespace,container,pod,node,cluster_id) , "container_name","$1","container","(.+)"), "pod_name","$1","pod","(.+)")) by (namespace,container_name,pod_name,node,cluster_id)`, - shardingLabels: []string{"namespace", "node", "cluster_id"}, - }, - { - name: "complex query with label_replace, nested aggregations and binary expressions", - expression: `sort_desc(avg(label_replace(label_replace(label_replace(count_over_time(container_memory_working_set_bytes{container!="", container!="POD", instance!="", }[1h] ), "node", "$1", "instance", "(.+)"), "container_name", "$1", "container", "(.+)"), "pod_name", "$1", "pod", "(.+)")*label_replace(label_replace(label_replace(avg_over_time(container_memory_working_set_bytes{container!="", container!="POD", instance!="", }[1h] ), "node", "$1", "instance", "(.+)"), "container_name", "$1", "container", "(.+)"), "pod_name", "$1", "pod", "(.+)")) by (namespace, container_name, pod_name, node, cluster_id))`, - shardingLabels: []string{"namespace", "cluster_id"}, - }, - { - name: "aggregate expression with label_replace", - expression: `sum by (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, - shardingLabels: []string{"pod"}, - }, - } - - // Shardable by labels instant queries with matrix response - shardableByLabelsMatrix := []queries{ - { - name: "subquery", - expression: "sum(http_requests_total) by (pod, cluster) [1h:1m]", - shardingLabels: []string{"cluster", "pod"}, - }, - { - name: "subquery with function", - expression: "increase(sum(http_requests_total) by (pod, cluster) [1h:1m])", - shardingLabels: []string{"cluster", "pod"}, - }, - { - name: "ignore vector matching with 2 aggregations", - expression: `sum(rate(node_cpu_seconds_total[3h])) by (cluster_id, mode) / ignoring(mode) group_left sum(rate(node_cpu_seconds_total[3h])) by (cluster_id)`, - shardingLabels: []string{"cluster_id"}, - }, - } - - shardableWithoutLabels := []queries{ - { - name: "aggregation without grouping", - expression: "sum without (pod) (http_requests_total)", - shardingLabels: []string{"pod"}, - }, - { - name: "multiple aggregations with without grouping", - expression: "max without (pod) (sum without (pod, cluster) (http_requests_total))", - shardingLabels: []string{"pod", "cluster"}, - }, - { - name: "binary expression with without vector matching and grouping", - expression: `sum without (cluster, pod) (http_requests_total{code="400"}) / ignoring (pod) sum without (cluster, pod) (http_requests_total)`, - shardingLabels: []string{"pod", "cluster", model.MetricNameLabel}, - }, - { - name: "binary expression with outer without grouping", - expression: `sum(http_requests_total{code="400"} * http_requests_total) without (pod)`, - shardingLabels: []string{model.MetricNameLabel, "pod"}, - }, - { - name: "binary expression with vector matching and outer without grouping", - expression: `sum(http_requests_total{code="400"} * ignoring(cluster) http_requests_total) without ()`, - shardingLabels: []string{"__name__", "cluster"}, - }, - { - name: "multiple binary expressions with without grouping", - expression: `(http_requests_total{code="400"} + ignoring (pod) http_requests_total{code="500"}) / ignoring (cluster, pod) http_requests_total`, - shardingLabels: []string{"cluster", "pod", model.MetricNameLabel}, - }, - { - name: "multiple binary expressions with without vector matchers", - expression: ` -(http_requests_total{code="400"} + ignoring (cluster, pod) http_requests_total{code="500"}) -/ ignoring (pod) -http_requests_total`, - shardingLabels: []string{"cluster", "pod", model.MetricNameLabel}, - }, - { - name: "aggregate without expression with label_replace, sharding label is not dynamic", - expression: `sum without (dst_label) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, - shardingLabels: []string{"dst_label"}, - }, - { - name: "aggregate without expression with label_join, sharding label is not dynamic", - expression: `sum without (dst_label) (label_join(metric, "dst_label", ",", "src_label"))`, - shardingLabels: []string{"dst_label"}, - }, - { - name: "aggregate without expression with label_replace", - expression: `sum without (pod) (label_replace(metric, "dst_label", "$1", "src_label", "re"))`, - shardingLabels: []string{"pod", "dst_label"}, - }, - { - name: "binary expression", - expression: `http_requests_total{code="400"} / http_requests_total`, - shardingLabels: []string{model.MetricNameLabel}, - }, - { - name: "binary expression among vector and scalar", - expression: `aaaa - bbb > 1000`, - shardingLabels: []string{model.MetricNameLabel}, - }, - { - name: "binary expression with set operation", - expression: `aaaa and bbb`, - shardingLabels: []string{model.MetricNameLabel}, - }, - { - name: "multiple binary expressions", - expression: `(http_requests_total{code="400"} + http_requests_total{code="500"}) / http_requests_total`, - shardingLabels: []string{model.MetricNameLabel}, - }, - } - - type testCase struct { - name string - path string - isShardable bool - shardSize int - codec tripperware.Codec - instantQueryResponses []*PrometheusInstantQueryResponse - queryRangeResponses []*queryrange.PrometheusResponse - response string - shardingLabels []string - } - tests := []testCase{ - { - name: "should shard range query when query is shardable", - path: `/api/v1/query_range?end=1&start=0&step=120&query=sum(metric) by (pod,cluster_name)`, - isShardable: true, - codec: shardedPrometheusCodec, - shardingLabels: []string{"pod", "cluster_name"}, - shardSize: 2, - queryRangeResponses: []*queryrange.PrometheusResponse{ - { - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__name__", Value: "metric"}, - {Name: "__job__", Value: "a"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - TotalQueryableSamples: 6, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - { - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__name__", Value: "metric"}, - {Name: "__job__", Value: "b"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - TotalQueryableSamples: 6, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]},{"metric":{"__job__":"b","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":12,"totalQueryableSamplesPerStep":[[1,2],[2,4],[3,6]]}}}}`, - }, - { - name: "should shard instant query when query is shardable", - path: `/api/v1/query?time=120&query=sum(metric) by (pod,cluster_name)`, - codec: instantQueryCodec, - shardSize: 2, - shardingLabels: []string{"pod", "cluster_name"}, - isShardable: true, - instantQueryResponses: []*PrometheusInstantQueryResponse{ - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, - }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, - }, - { - name: "should not shard if shard size is 1", - path: `/api/v1/query?time=120&query=sum(metric) by (pod,cluster_name)`, - codec: instantQueryCodec, - shardSize: 1, - isShardable: false, - instantQueryResponses: []*PrometheusInstantQueryResponse{ - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, - }, - } - for _, query := range nonShardable { - tests = append(tests, testCase{ - name: fmt.Sprintf("non shardable query: %s", query.name), - path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), - codec: instantQueryCodec, - isShardable: false, - instantQueryResponses: []*PrometheusInstantQueryResponse{ - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, - }) - tests = append(tests, testCase{ - name: fmt.Sprintf("non shardable query_range: %s", query.name), - path: fmt.Sprintf(`/api/v1/query_range?start=1&end=2&step=1&query=%s`, url.QueryEscape(query.expression)), - codec: shardedPrometheusCodec, - isShardable: false, - queryRangeResponses: []*queryrange.PrometheusResponse{ - { - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__job__", Value: "a"}, - {Name: "__name__", Value: "metric"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - TotalQueryableSamples: 6, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":6,"totalQueryableSamplesPerStep":[[1,1],[2,2],[3,3]]}}}}`, - }) - } - - for _, query := range append(shardableWithoutLabels, shardableByLabels...) { - tests = append(tests, testCase{ - name: fmt.Sprintf("shardable query: %s", query.name), - path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), - codec: instantQueryCodec, - isShardable: true, - shardSize: 2, - shardingLabels: query.shardingLabels, - instantQueryResponses: []*PrometheusInstantQueryResponse{ - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Sample: cortexpb.Sample{Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, - }, - Sample: cortexpb.Sample{Value: 2, TimestampMs: 2000}, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, - }) - tests = append(tests, testCase{ - name: fmt.Sprintf("shardable query_range: %s", query.name), - path: fmt.Sprintf(`/api/v1/query_range?start=1&end=2&step=1&query=%s`, url.QueryEscape(query.expression)), - codec: shardedPrometheusCodec, - isShardable: true, - shardSize: 2, - shardingLabels: query.shardingLabels, - queryRangeResponses: []*queryrange.PrometheusResponse{ - { - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__name__", Value: "metric"}, - {Name: "__job__", Value: "a"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - TotalQueryableSamples: 6, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - { - Status: "success", - Data: queryrange.PrometheusData{ - ResultType: model.ValMatrix.String(), - Result: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {Name: "__name__", Value: "metric"}, - {Name: "__job__", Value: "b"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 1, TimestampMs: 1000}, - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - TotalQueryableSamples: 6, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__job__":"a","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]},{"metric":{"__job__":"b","__name__":"metric"},"values":[[1,"1"],[2,"2"],[3,"3"]]}],"stats":{"samples":{"totalQueryableSamples":12,"totalQueryableSamplesPerStep":[[1,2],[2,4],[3,6]]}}}}`, - }) - } - - for _, query := range shardableByLabelsMatrix { - tests = append(tests, testCase{ - name: fmt.Sprintf("shardable query: %s", query.name), - path: fmt.Sprintf(`/api/v1/query?time=120&query=%s`, url.QueryEscape(query.expression)), - codec: instantQueryCodec, - isShardable: true, - shardSize: 2, - shardingLabels: query.shardingLabels, - instantQueryResponses: []*PrometheusInstantQueryResponse{ - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValMatrix.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Matrix{ - Matrix: &Matrix{ - SampleStreams: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, - }, - Samples: []cortexpb.Sample{ - {Value: 1, TimestampMs: 1000}, - }, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - { - Status: "success", - Data: PrometheusInstantQueryData{ - ResultType: model.ValMatrix.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Matrix{ - Matrix: &Matrix{ - SampleStreams: []tripperware.SampleStream{ - { - Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, - }, - Samples: []cortexpb.Sample{ - {Value: 2, TimestampMs: 2000}, - }, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{ - Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 10, TimestampMs: 1000}, - }, - TotalQueryableSamples: 10, - }, - }, - }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - }, - }, - }, - response: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"__name__":"up","job":"bar"},"values":[[2,"2"]]},{"metric":{"__name__":"up","job":"foo"},"values":[[1,"1"]]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, - }) - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - //parallel testing causes data race - sort.Strings(tt.shardingLabels) - s := httptest.NewServer( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - q := r.FormValue("query") - expr, _ := parser.ParseExpr(q) - shardIndex := int64(0) - - parser.Inspect(expr, func(n parser.Node, _ []parser.Node) error { - if selector, ok := n.(*parser.VectorSelector); ok { - for _, matcher := range selector.LabelMatchers { - if matcher.Name == querysharding.CortexShardByLabel { - - decoded, _ := base64.StdEncoding.DecodeString(matcher.Value) - shardInfo := storepb.ShardInfo{} - err := shardInfo.Unmarshal(decoded) - require.NoError(t, err) - sort.Strings(shardInfo.Labels) - require.Equal(t, tt.shardingLabels, shardInfo.Labels) - require.Equal(t, tt.isShardable, shardInfo.TotalShards > 0) - shardIndex = shardInfo.ShardIndex - } - } - } - return nil - }) - if tt.instantQueryResponses != nil { - protobuf, err := proto.Marshal(tt.instantQueryResponses[shardIndex]) - require.NoError(t, err) - _, _ = w.Write(protobuf) - } else { - protobuf, err := proto.Marshal(tt.queryRangeResponses[shardIndex]) - require.NoError(t, err) - _, _ = w.Write(protobuf) - } - }), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := tripperware.SingleHostRoundTripper{ - Host: u.Host, - Next: http.DefaultTransport, - } - - qa := thanosquerysharding.NewQueryAnalyzer() - roundtripper := tripperware.NewRoundTripper(downstream, tt.codec, nil, tripperware.ShardByMiddleware(log.NewNopLogger(), tripperware.MockLimits{ShardSize: tt.shardSize}, tt.codec, qa)) - - ctx := user.InjectOrgID(context.Background(), "1") - - req, err := http.NewRequest("GET", tt.path, http.NoBody) - req = req.WithContext(ctx) - - require.NoError(t, err) - resp, err := roundtripper.RoundTrip(req) - - require.NoError(t, err) - require.NotNil(t, resp) - - contents, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, tt.response, string(contents)) - }) - } - -} + t.Parallel() + tripperware.TestQueryShardQuery(t, InstantQueryCodec, queryrange.NewPrometheusCodec(true, "", true)) +} \ No newline at end of file diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index 80fb5ff96d9..784de900b51 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -5,7 +5,6 @@ import ( "compress/gzip" "context" "fmt" - "github.com/golang/snappy" "io" "net/http" "sort" @@ -14,6 +13,8 @@ import ( "time" "unsafe" + "github.com/golang/snappy" + "github.com/go-kit/log" "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" @@ -454,3 +455,15 @@ func marshalHistogramBucket(b HistogramBucket, stream *jsoniter.Stream) { jsonutil.MarshalFloat(b.Count, stream) stream.WriteArrayEnd() } + +func (s *PrometheusResponseStats) MarshalJSON() ([]byte, error) { + stats := struct { + Samples *PrometheusResponseSamplesStats `json:"samples"` + }{ + Samples: s.Samples, + } + if s.Samples.TotalQueryableSamplesPerStep == nil { + s.Samples.TotalQueryableSamplesPerStep = []*PrometheusResponseQueryableSamplesStatsPerStep{} + } + return json.Marshal(stats) +} diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index 7d5e57250af..981f4ce3d71 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -33,6 +33,7 @@ func BenchmarkPrometheusCodec_DecodeResponse(b *testing.B) { for n := 0; n < b.N; n++ { _, err := PrometheusCodec.DecodeResponse(context.Background(), &http.Response{ StatusCode: 200, + Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, Body: io.NopCloser(bytes.NewReader(encodedRes)), ContentLength: int64(len(encodedRes)), }, nil) diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 0ed817f365c..51afddfca40 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "fmt" - "github.com/gogo/protobuf/proto" "io" "net/http" "net/url" @@ -13,6 +12,8 @@ import ( "strings" "time" + "github.com/gogo/protobuf/proto" + "github.com/gogo/status" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -36,6 +37,8 @@ const ( DisableCompression Compression = "" GzipCompression Compression = "gzip" SnappyCompression Compression = "snappy" + applicationProtobuf string = "application/x-protobuf" + applicationJson string = "application/json" ) var ( @@ -268,9 +271,9 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques h.Set("Accept-Encoding", string(c.compression)) } if c.enableProtobuf { - h.Set("Accept", "application/x-protobuf") + h.Set("Accept", applicationProtobuf) } else { - h.Set("Accept", "application/json") + h.Set("Accept", applicationJson) } req := &http.Request{ @@ -303,15 +306,19 @@ func (c prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ log.LogFields(otlog.Int("bytes", len(buf))) var resp PrometheusResponse - if c.enableProtobuf { + if r.Header != nil && r.Header.Get("Content-Type") == applicationProtobuf { err = proto.Unmarshal(buf, &resp) } else { err = json.Unmarshal(buf, &resp) } + if err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } + for h, hv := range r.Header { + resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) + } return &resp, nil } @@ -335,7 +342,7 @@ func (prometheusCodec) EncodeResponse(ctx context.Context, res tripperware.Respo resp := http.Response{ Header: http.Header{ - "Content-Type": []string{"application/json"}, + "Content-Type": []string{applicationJson}, }, Body: io.NopCloser(bytes.NewBuffer(b)), StatusCode: http.StatusOK, diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index bb48ca6cad2..0dbbebce13e 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -2,7 +2,6 @@ package queryrange import ( "context" - "github.com/gogo/protobuf/proto" "io" "net/http" "net/http/httptest" @@ -32,11 +31,7 @@ func TestRoundTrip(t *testing.T) { http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { var err error if r.RequestURI == query { - resp := parsedResponse - resp.Headers = respHeaders - protobuf, err := proto.Marshal(resp) - require.NoError(t, err) - _, err = w.Write(protobuf) + _, err = w.Write([]byte(responseBody)) } else if r.RequestURI == queryWithWarnings { _, err = w.Write([]byte(responseBodyWithWarnings)) } else { diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 0b5d76c598a..9f0f3500de5 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -4,7 +4,6 @@ import ( "bytes" "compress/gzip" "context" - "fmt" "github.com/gogo/protobuf/proto" "github.com/golang/snappy" "io" @@ -92,37 +91,169 @@ func TestRequest(t *testing.T) { func TestResponse(t *testing.T) { t.Parallel() - r := *parsedResponse - rWithWarnings := *parsedResponseWithWarnings - r.Headers = respHeaders - rWithWarnings.Headers = respHeaders - for i, tc := range []struct { - promBody *PrometheusResponse - expectedResponse string + testCases := []struct { + promBody *PrometheusResponse + jsonBody string expectedDecodeErr error cancelCtxBeforeDecode bool + isProtobuf bool }{ { - promBody: &r, - expectedResponse: responseBody, + promBody: &PrometheusResponse{ + Status: "success", + Data: 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}, + }, + }, + }, + }, + }, + jsonBody: responseBody, + isProtobuf: true, }, { - promBody: &r, + promBody: &PrometheusResponse{ + Status: "success", + Data: 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}, + }, + }, + }, + }, + }, cancelCtxBeforeDecode: true, expectedDecodeErr: context.Canceled, + isProtobuf: true, }, - } { + { + promBody: &PrometheusResponse{ + Status: "success", + Data: 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}, + }, + }, + }, + }, + }, + jsonBody: responseBody, + isProtobuf: false, + }, + { + promBody: &PrometheusResponse{ + Status: "success", + Data: 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}, + }, + }, + }, + }, + }, + cancelCtxBeforeDecode: true, + expectedDecodeErr: context.Canceled, + isProtobuf: false, + }, + { + promBody: &PrometheusResponse{ + Status: "success", + Warnings: []string{"test-warn"}, + Data: 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}, + }, + }, + }, + }, + }, + jsonBody: responseBodyWithWarnings, + isProtobuf: true, + }, + { + promBody: &PrometheusResponse{ + Status: "success", + Warnings: []string{"test-warn"}, + Data: 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}, + }, + }, + }, + }, + }, + jsonBody: responseBodyWithWarnings, + isProtobuf: false, + }, + } + for i, tc := range testCases { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() protobuf, err := proto.Marshal(tc.promBody) require.NoError(t, err) ctx, cancelCtx := context.WithCancel(context.Background()) - response := &http.Response{ - StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, - Body: io.NopCloser(bytes.NewBuffer(protobuf)), + + var response *http.Response + if tc.isProtobuf { + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{applicationProtobuf}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), + } + tc.promBody.Headers = respHeadersProtobuf + } else { + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{applicationJson}}, + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + } + tc.promBody.Headers = respHeadersJson } + if tc.cancelCtxBeforeDecode { cancelCtx() } @@ -133,12 +264,14 @@ func TestResponse(t *testing.T) { return } + assert.Equal(t, tc.promBody, resp) + // Reset response, as the above call will have consumed the body reader. response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResponse))), - ContentLength: int64(len(tc.expectedResponse)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + ContentLength: int64(len(tc.jsonBody)), } resp2, err := PrometheusCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -152,10 +285,11 @@ func TestResponseWithStats(t *testing.T) { t.Parallel() for i, tc := range []struct { promBody *PrometheusResponse - expectedResponse string + jsonBody string + isProtobuf bool }{ { - expectedResponse: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, promBody: &PrometheusResponse{ Status: "success", Data: PrometheusData{ @@ -182,28 +316,72 @@ func TestResponseWithStats(t *testing.T) { }, }, }, + isProtobuf: true, + }, + { + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + promBody: &PrometheusResponse{ + Status: "success", + Data: 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}, + }, + }, + }, + Stats: &tripperware.PrometheusResponseStats{ + Samples: &tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: 10, + TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + {Value: 5, TimestampMs: 1536673680000}, + {Value: 5, TimestampMs: 1536673780000}, + }, + }, + }, + }, + }, + isProtobuf: false, }, } { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - tc.promBody.Headers = respHeaders protobuf, err := proto.Marshal(tc.promBody) require.NoError(t, err) - response := &http.Response{ - StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, - Body: io.NopCloser(bytes.NewBuffer(protobuf)), + + var response *http.Response + if tc.isProtobuf { + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{applicationProtobuf}}, + Body: io.NopCloser(bytes.NewBuffer(protobuf)), + } + tc.promBody.Headers = respHeadersProtobuf + } else { + response = &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{applicationJson}}, + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + } + tc.promBody.Headers = respHeadersJson } + resp, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) require.NoError(t, err) + assert.Equal(t, tc.promBody, resp) // Reset response, as the above call will have consumed the body reader. response = &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/json"}}, - Body: io.NopCloser(bytes.NewBuffer([]byte(tc.expectedResponse))), - ContentLength: int64(len(tc.expectedResponse)), + Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), + ContentLength: int64(len(tc.jsonBody)), } resp2, err := PrometheusCodec.EncodeResponse(context.Background(), resp) require.NoError(t, err) @@ -757,8 +935,7 @@ func TestMergeAPIResponses(t *testing.T) { func TestCompressedResponse(t *testing.T) { t.Parallel() - for _, tc := range []struct { - name string + for i, tc := range []struct { compression string jsonBody string promBody *PrometheusResponse @@ -766,7 +943,6 @@ func TestCompressedResponse(t *testing.T) { err error }{ { - name: `successful response`, compression: `gzip`, promBody: &PrometheusResponse{ Status: StatusSuccess, @@ -789,15 +965,12 @@ func TestCompressedResponse(t *testing.T) { }, }}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - {Name: "Content-Encoding", Values: []string{"gzip"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, + jsonBody:`{"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]]}}}}`, status: 200, }, { - name: `successful response`, compression: `snappy`, promBody: &PrometheusResponse{ Status: StatusSuccess, @@ -820,88 +993,84 @@ func TestCompressedResponse(t *testing.T) { }, }}, }, - Headers: []*tripperware.PrometheusResponseHeader{ - {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, - {Name: "Content-Encoding", Values: []string{"snappy"}}, - }, + Headers: []*tripperware.PrometheusResponseHeader{}, }, + jsonBody:`{"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]]}}}}`, status: 200, }, { - name: `400 error`, compression: `gzip`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, { - name: `400 error`, compression: `snappy`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, { - name: `400 error empty body`, compression: `gzip`, status: 400, err: httpgrpc.Errorf(400, ""), }, { - name: `400 error empty body`, compression: `snappy`, status: 400, err: httpgrpc.Errorf(400, ""), }, } { - for _, c := range []bool{true, false} { - c := c - t.Run(fmt.Sprintf("%s compressed %t [%s]", tc.compression, c, tc.name), func(t *testing.T) { - t.Parallel() - h := http.Header{} - var b []byte - if tc.promBody != nil { - protobuf, err := proto.Marshal(tc.promBody) - b = protobuf - require.NoError(t, err) - h.Set("Content-Type", "application/x-protobuf") - } else { - b = []byte(tc.jsonBody) - h.Set("Content-Type", "application/json") - } - responseBody := bytes.NewBuffer(b) + t.Run(strconv.Itoa(i), func(t *testing.T) { + t.Parallel() + h := http.Header{} + var b []byte + if tc.promBody != nil { + protobuf, err := proto.Marshal(tc.promBody) + b = protobuf + require.NoError(t, err) + h.Set("Content-Type", "application/x-protobuf") + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) - var buf bytes.Buffer - if c && tc.compression == "gzip" { - h.Set("Content-Encoding", "gzip") - w := gzip.NewWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } else if c && tc.compression == "snappy" { - h.Set("Content-Encoding", "snappy") - w := snappy.NewBufferedWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } + } else { + b = []byte(tc.jsonBody) + h.Set("Content-Type", "application/json") + } - response := &http.Response{ - StatusCode: tc.status, - Header: h, - Body: io.NopCloser(responseBody), - } - resp, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) - require.Equal(t, tc.err, err) + responseBody := bytes.NewBuffer(b) - if err == nil { - require.NoError(t, err) - require.Equal(t, tc.promBody, resp) - } - }) - } + var buf bytes.Buffer + if tc.compression == "gzip" { + h.Set("Content-Encoding", "gzip") + if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} + w := gzip.NewWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf + } else if tc.compression == "snappy" { + h.Set("Content-Encoding", "snappy") + if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}})} + w := snappy.NewBufferedWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf + } + + response := &http.Response{ + StatusCode: tc.status, + Header: h, + Body: io.NopCloser(responseBody), + } + resp, err := PrometheusCodec.DecodeResponse(context.Background(), response, nil) + require.Equal(t, tc.err, err) + + if err == nil { + require.NoError(t, err) + require.Equal(t, tc.promBody.Data, resp.(*PrometheusResponse).Data) + } + }) } } diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index a1f90633cc9..b2de6891316 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -59,12 +59,20 @@ var ( Query: "sum(container_memory_rss) by (namespace)", CachingOptions: CachingOptions{Disabled: true}, } - respHeaders = []*tripperware.PrometheusResponseHeader{ + respHeadersJson = []*tripperware.PrometheusResponseHeader{ { Name: "Content-Type", - Values: []string{"application/x-protobuf"}, + Values: []string{applicationJson}, }, } + + respHeadersProtobuf = []*tripperware.PrometheusResponseHeader{ + { + Name: "Content-Type", + Values: []string{applicationProtobuf}, + }, + } + parsedResponse = &PrometheusResponse{ Status: "success", Data: PrometheusData{ diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 730f00da1ec..5004c60a800 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -2,7 +2,6 @@ package queryrange import ( "context" - "github.com/gogo/protobuf/proto" io "io" "net/http" "net/http/httptest" @@ -297,11 +296,7 @@ func TestSplitByDay(t *testing.T) { middleware.AuthenticateUser.Wrap( http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { actualCount.Inc() - resp := parsedResponse - resp.Headers = respHeaders - protobuf, err := proto.Marshal(resp) - require.NoError(t, err) - _, _ = w.Write(protobuf) + _, _ = w.Write([]byte(responseBody)) }), ), ) diff --git a/pkg/querier/tripperware/roundtrip.go b/pkg/querier/tripperware/roundtrip.go index 1dfc64739d3..4edcd51cc9b 100644 --- a/pkg/querier/tripperware/roundtrip.go +++ b/pkg/querier/tripperware/roundtrip.go @@ -17,7 +17,6 @@ package tripperware import ( "context" - "github.com/cortexproject/cortex/pkg/util/validation" "io" "net/http" "strings" @@ -168,18 +167,6 @@ func NewQueryTripperware( if isQueryRange { return queryrange.RoundTrip(r) } else if isQuery { - // If the given query is not shardable, use downstream roundtripper. - query := r.FormValue("query") - - // If vertical sharding is not enabled for the tenant, use downstream roundtripper. - numShards := validation.SmallestPositiveIntPerTenant(tenantIDs, limits.QueryVerticalShardSize) - if numShards <= 1 { - return next.RoundTrip(r) - } - analysis, err := queryAnalyzer.Analyze(query) - if err != nil || !analysis.IsShardable() { - return next.RoundTrip(r) - } return instantQuery.RoundTrip(r) } return next.RoundTrip(r) diff --git a/pkg/querier/tripperware/roundtrip_test.go b/pkg/querier/tripperware/roundtrip_test.go index 5ca4a69fb32..7e6cf474a0d 100644 --- a/pkg/querier/tripperware/roundtrip_test.go +++ b/pkg/querier/tripperware/roundtrip_test.go @@ -94,9 +94,9 @@ func TestRoundTrip(t *testing.T) { u, err := url.Parse(s.URL) require.NoError(t, err) - downstream := SingleHostRoundTripper{ - Host: u.Host, - Next: http.DefaultTransport, + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, } instantMiddlewares := []Middleware{ From 1c7397e42bff9cb6759016f5bd35d0e484f71832 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 9 Aug 2024 10:42:44 -0700 Subject: [PATCH 22/38] add protobuf codec to reuse prometheus handler for querier Signed-off-by: Ahmed Hassan --- pkg/api/api.go | 2 +- pkg/api/handlers.go | 82 ++++-------- pkg/querier/codec/instant_query_protobuf.go | 80 ++++++++++++ pkg/querier/codec/query_range_protobuf.go | 136 ++++++++++++++++++++ 4 files changed, 239 insertions(+), 61 deletions(-) create mode 100644 pkg/querier/codec/instant_query_protobuf.go create mode 100644 pkg/querier/codec/query_range_protobuf.go diff --git a/pkg/api/api.go b/pkg/api/api.go index 5d1733641f0..2ad90a09631 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -82,7 +82,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.ResponseCompression, "api.response-compression-enabled", false, "Use GZIP compression for API responses. Some endpoints serve large YAML or JSON blobs which can benefit from compression.") f.Var(&cfg.HTTPRequestHeadersToLog, "api.http-request-headers-to-log", "Which HTTP Request headers to add to logs") f.BoolVar(&cfg.buildInfoEnabled, "api.build-info-enabled", false, "If enabled, build Info API will be served by query frontend or querier.") - f.BoolVar(&cfg.ProtobufQuerierHandler, "api.protobuf_querier_handler", false, "Enable using querier handler with protobuf response serialization for query range and unsharded instant query requests") + f.BoolVar(&cfg.ProtobufQuerierHandler, "api.protobuf_querier_handler", false, "Enable protobuf serialization for range query and unsharded instant query responses from querier") cfg.RegisterFlagsWithPrefix("", f) } diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index b30aad0855e..9fd3c238846 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -8,8 +8,6 @@ import ( "path" "sync" - thanos_api "github.com/thanos-io/thanos/pkg/api" - "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/gorilla/mux" @@ -23,14 +21,12 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/annotations" - v1api "github.com/prometheus/prometheus/web/api/v1" + api "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" "github.com/cortexproject/cortex/pkg/querier" - "github.com/cortexproject/cortex/pkg/querier/handler" - qapi "github.com/cortexproject/cortex/pkg/querier/handler" + "github.com/cortexproject/cortex/pkg/querier/codec" "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/util" ) @@ -198,28 +194,28 @@ func NewQuerierHandler( Help: "Current number of inflight requests to the querier.", }, []string{"method", "route"}) - v1api := v1api.NewAPI( + api := api.NewAPI( engine, querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. nil, // No remote write support. exemplarQueryable, - func(ctx context.Context) v1api.ScrapePoolsRetriever { return nil }, - func(context.Context) v1api.TargetRetriever { return &querier.DummyTargetRetriever{} }, - func(context.Context) v1api.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, + func(ctx context.Context) api.ScrapePoolsRetriever { return nil }, + func(context.Context) api.TargetRetriever { return &querier.DummyTargetRetriever{} }, + func(context.Context) api.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, func() config.Config { return config.Config{} }, map[string]string{}, // TODO: include configuration flags - v1api.GlobalURLOptions{}, + api.GlobalURLOptions{}, func(f http.HandlerFunc) http.HandlerFunc { return f }, nil, // Only needed for admin APIs. "", // This is for snapshots, which is disabled when admin APIs are disabled. Hence empty. false, // Disable admin APIs. logger, - func(context.Context) v1api.RulesRetriever { return &querier.DummyRulesRetriever{} }, + func(context.Context) api.RulesRetriever { return &querier.DummyRulesRetriever{} }, 0, 0, 0, // Remote read samples and concurrency limit. false, regexp.MustCompile(".*"), - func() (v1api.RuntimeInfo, error) { return v1api.RuntimeInfo{}, errors.New("not implemented") }, - &v1api.PrometheusVersion{ + func() (api.RuntimeInfo, error) { return api.RuntimeInfo{}, errors.New("not implemented") }, + &api.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, @@ -236,12 +232,10 @@ func NewQuerierHandler( false, ) - queryapi := qapi.NewAPI( - engine, - querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. - logger, - nil, - ) + if cfg.ProtobufQuerierHandler { + api.InstallCodec(codec.InstantQueryProtobufCodec{}) + api.InstallCodec(codec.QueryRangeProtobufCodec{}) + } router := mux.NewRouter() @@ -261,45 +255,18 @@ func NewQuerierHandler( legacyPrefix := path.Join(cfg.ServerPrefix, cfg.LegacyHTTPPrefix) promRouter := route.New().WithPrefix(path.Join(prefix, "/api/v1")) - v1api.Register(promRouter) + api.Register(promRouter) legacyPromRouter := route.New().WithPrefix(path.Join(legacyPrefix, "/api/v1")) - v1api.Register(legacyPromRouter) - - wrap := func(f func(r *http.Request) (interface{}, *thanos_api.ApiError, annotations.Annotations, func())) http.HandlerFunc { - hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - data, err, warnings, finalizer := f(r) - if finalizer != nil { - defer finalizer() - } - if err != nil { - queryapi.RespondError(w, err, data) - return - } - - if data != nil { - queryapi.Respond(w, r, data, warnings, r.FormValue("query")) - return - } - w.WriteHeader(http.StatusNoContent) - }) - return handler.CompressionHandler{ - Handler: hf, - }.ServeHTTP - } + api.Register(legacyPromRouter) // TODO(gotjosh): This custom handler is temporary until we're able to vendor the changes in: // https://github.com/prometheus/prometheus/pull/7125/files router.Path(path.Join(prefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(prefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(prefix, "/api/v1/read")).Methods("POST").Handler(promRouter) - if cfg.ProtobufQuerierHandler { - router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) - router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) - } else { - router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(promRouter) - router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(promRouter) - } + router.Path(path.Join(prefix, "/api/v1/query")).Methods("GET", "POST").Handler(promRouter) + router.Path(path.Join(prefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/labels")).Methods("GET", "POST").Handler(promRouter) router.Path(path.Join(prefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(promRouter) @@ -311,13 +278,8 @@ func NewQuerierHandler( router.Path(path.Join(legacyPrefix, "/api/v1/metadata")).Handler(querier.MetadataHandler(distributor)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Handler(querier.RemoteReadHandler(queryable, logger)) router.Path(path.Join(legacyPrefix, "/api/v1/read")).Methods("POST").Handler(legacyPromRouter) - if cfg.ProtobufQuerierHandler { - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(wrap(queryapi.Query)) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(wrap(queryapi.QueryRange)) - } else { - router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) - router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) - } + router.Path(path.Join(legacyPrefix, "/api/v1/query")).Methods("GET", "POST").Handler(legacyPromRouter) + router.Path(path.Join(legacyPrefix, "/api/v1/query_range")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/query_exemplars")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/labels")).Methods("GET", "POST").Handler(legacyPromRouter) router.Path(path.Join(legacyPrefix, "/api/v1/label/{name}/values")).Methods("GET").Handler(legacyPromRouter) @@ -339,13 +301,13 @@ type buildInfoHandler struct { type buildInfoResponse struct { Status string `json:"status"` - Data *v1api.PrometheusVersion `json:"data"` + Data *api.PrometheusVersion `json:"data"` } func (h *buildInfoHandler) ServeHTTP(writer http.ResponseWriter, _ *http.Request) { infoResponse := buildInfoResponse{ Status: "success", - Data: &v1api.PrometheusVersion{ + Data: &api.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, diff --git a/pkg/querier/codec/instant_query_protobuf.go b/pkg/querier/codec/instant_query_protobuf.go new file mode 100644 index 00000000000..ed56d208c9d --- /dev/null +++ b/pkg/querier/codec/instant_query_protobuf.go @@ -0,0 +1,80 @@ +package codec + +import ( + "net/http" + "strings" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" + jsoniter "github.com/json-iterator/go" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/web/api/v1" + "github.com/gogo/protobuf/proto" +) +type InstantQueryProtobufCodec struct{} + +func (p InstantQueryProtobufCodec) ContentType() v1.MIMEType { + return v1.MIMEType{Type: "application", SubType: "x-protobuf"} +} + +func (p InstantQueryProtobufCodec) CanEncode(req *http.Request, resp *v1.Response) bool { + if resp.Error != "" || resp.Data == nil { + return false + } + return strings.HasSuffix(req.URL.Path, "/query") +} + +func (p InstantQueryProtobufCodec) Encode(_ *http.Request, resp *v1.Response) ([]byte, error) { + prometheusInstantQueryResponse, err := createPrometheusInstantQueryResponse(resp) + if err != nil { + return []byte{}, err + } + b, err := proto.Marshal(prometheusInstantQueryResponse) + return b, err +} + + +func createPrometheusInstantQueryResponse(resp *v1.Response) (*instantquery.PrometheusInstantQueryResponse, error) { + var data = resp.Data.(*v1.QueryData) + + var instantQueryResult instantquery.PrometheusInstantQueryResult + switch string(data.ResultType) { + case model.ValMatrix.String(): + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ + Matrix: &instantquery.Matrix{ + SampleStreams: *getSampleStreams(data), + }, + } + case model.ValVector.String(): + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ + Vector: &instantquery.Vector{ + Samples: *getSamples(data), + }, + } + default: + json := jsoniter.ConfigCompatibleWithStandardLibrary + rawBytes, err := json.Marshal(data) + if err != nil { + return nil, err + } + instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_RawBytes{RawBytes: rawBytes} + } + + var stats *tripperware.PrometheusResponseStats + if data.Stats != nil { + builtin := data.Stats.Builtin() + stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} + } + + return &instantquery.PrometheusInstantQueryResponse{ + Status: string(resp.Status), + Data: instantquery.PrometheusInstantQueryData{ + ResultType: string(data.ResultType), + Result: instantQueryResult, + Stats: stats, + }, + ErrorType: string(resp.ErrorType), + Error: resp.Error, + Warnings: resp.Warnings, + }, nil +} diff --git a/pkg/querier/codec/query_range_protobuf.go b/pkg/querier/codec/query_range_protobuf.go new file mode 100644 index 00000000000..a792ee6f242 --- /dev/null +++ b/pkg/querier/codec/query_range_protobuf.go @@ -0,0 +1,136 @@ +package codec + +import ( + "net/http" + "strings" + + "github.com/prometheus/prometheus/util/stats" + "github.com/prometheus/prometheus/promql" + "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/querier/tripperware/instantquery" + "github.com/prometheus/prometheus/web/api/v1" + "github.com/gogo/protobuf/proto" +) + +type QueryRangeProtobufCodec struct{} + +func (p QueryRangeProtobufCodec) ContentType() v1.MIMEType { + return v1.MIMEType{Type: "application", SubType: "x-protobuf"} +} + +func (p QueryRangeProtobufCodec) CanEncode(req *http.Request, resp *v1.Response) bool { + if resp.Error != "" || resp.Data == nil { + return false + } + return strings.HasSuffix(req.URL.Path, "/query_range") +} + +func (p QueryRangeProtobufCodec) Encode(_ *http.Request, resp *v1.Response) ([]byte, error) { + promtheusResponse := createPrometheusResponse(resp) + b, err := proto.Marshal(promtheusResponse) + return b, err +} + +func createPrometheusResponse(resp *v1.Response) (*queryrange.PrometheusResponse) { + data := resp.Data.(*v1.QueryData) + + sampleStreams := getSampleStreams(data) + + var stats *tripperware.PrometheusResponseStats + if data.Stats != nil { + builtin := data.Stats.Builtin() + stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} + } + + return &queryrange.PrometheusResponse{ + Status: string(resp.Status), + Data: queryrange.PrometheusData{ + ResultType: string(data.ResultType), + Result: *sampleStreams, + Stats: stats, + }, + ErrorType: string(resp.ErrorType), + Error: resp.Error, + Warnings: resp.Warnings, + } +} + +func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { + queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) + queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) + for i := 0; i < queryableSamplesStatsPerStepLen; i++ { + queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, + TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, + } + } + + statSamples := tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, + TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + } + + return &statSamples +} + +func getSampleStreams(data *v1.QueryData) *[]tripperware.SampleStream { + sampleStreamsLen := len(data.Result.(promql.Matrix)) + sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) + + for i := 0; i < sampleStreamsLen; i++ { + labelsLen := len(data.Result.(promql.Matrix)[i].Metric) + var labels []cortexpb.LabelAdapter + if labelsLen > 0 { + labels = make([]cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = cortexpb.LabelAdapter{ + Name: data.Result.(promql.Matrix)[i].Metric[j].Name, + Value: data.Result.(promql.Matrix)[i].Metric[j].Value, + } + } + } + + samplesLen := len(data.Result.(promql.Matrix)[i].Floats) + var samples []cortexpb.Sample + if samplesLen > 0 { + samples = make([]cortexpb.Sample, samplesLen) + for j := 0; j < samplesLen; j++ { + samples[j] = cortexpb.Sample{ + Value: data.Result.(promql.Matrix)[i].Floats[j].F, + TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, + } + } + } + sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} + } + return &sampleStreams +} + +func getSamples(data *v1.QueryData) *[]*instantquery.Sample { + vectorSamplesLen := len(data.Result.(promql.Vector)) + vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) + + for i := 0; i < vectorSamplesLen; i++ { + labelsLen := len(data.Result.(promql.Vector)[i].Metric) + var labels []cortexpb.LabelAdapter + if labelsLen > 0 { + labels = make([]cortexpb.LabelAdapter, labelsLen) + for j := 0; j < labelsLen; j++ { + labels[j] = cortexpb.LabelAdapter{ + Name: data.Result.(promql.Vector)[i].Metric[j].Name, + Value: data.Result.(promql.Vector)[i].Metric[j].Value, + } + } + } + + vectorSamples[i] = &instantquery.Sample{Labels: labels, + Sample: &cortexpb.Sample{ + TimestampMs: data.Result.(promql.Vector)[i].T, + Value: data.Result.(promql.Vector)[i].F, + }, + } + } + return &vectorSamples +} From a466d86b6e506e6204a64abf012a8afe5145ab77 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 11 Sep 2024 11:36:24 -0700 Subject: [PATCH 23/38] refactor to use unified protobuf struct for both range and instant queries Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 25 +- pkg/querier/codec/instant_query_protobuf.go | 80 --- ...ry_range_protobuf.go => protobuf_codec.go} | 110 ++-- pkg/querier/handler/compression.go | 93 --- pkg/querier/handler/handler.go | 613 ------------------ pkg/querier/querier.go | 2 +- .../tripperware/instantquery/instant_query.go | 6 +- .../instantquery/instant_query_test.go | 514 +++++++-------- pkg/querier/tripperware/merge.go | 11 +- pkg/querier/tripperware/query.go | 4 +- pkg/querier/tripperware/query.pb.go | 168 ++--- pkg/querier/tripperware/query.proto | 2 +- .../tripperware/queryrange/marshaling_test.go | 1 + .../tripperware/queryrange/query_range.go | 6 +- .../v2/interceptors/logging/doc.go | 36 - .../v2/interceptors/logging/interceptors.go | 135 ---- .../v2/interceptors/logging/logging.go | 139 ---- .../v2/interceptors/logging/options.go | 94 --- .../v2/interceptors/logging/payload.go | 152 ----- .../prometheus/common/promlog/log.go | 189 ------ .../thanos-io/thanos/pkg/api/api.go | 278 -------- .../thanos-io/thanos/pkg/logging/grpc.go | 207 ------ .../thanos-io/thanos/pkg/logging/http.go | 195 ------ .../thanos-io/thanos/pkg/logging/logger.go | 59 -- .../thanos-io/thanos/pkg/logging/options.go | 245 ------- .../thanos/pkg/logging/yaml_parser.go | 40 -- .../thanos-io/thanos/pkg/prober/combiner.go | 56 -- .../thanos-io/thanos/pkg/prober/grpc.go | 47 -- .../thanos-io/thanos/pkg/prober/http.go | 81 --- .../thanos/pkg/prober/intrumentation.go | 82 --- .../thanos-io/thanos/pkg/prober/prober.go | 25 - .../thanos-io/thanos/pkg/server/http/http.go | 145 ----- .../thanos/pkg/server/http/option.go | 63 -- .../thanos-io/thanos/pkg/server/http/utils.go | 32 - vendor/golang.org/x/net/http2/h2c/h2c.go | 240 ------- .../google.golang.org/grpc/health/client.go | 117 ---- .../google.golang.org/grpc/health/logging.go | 23 - .../google.golang.org/grpc/health/server.go | 163 ----- vendor/modules.txt | 4 - 39 files changed, 404 insertions(+), 4078 deletions(-) delete mode 100644 pkg/querier/codec/instant_query_protobuf.go rename pkg/querier/codec/{query_range_protobuf.go => protobuf_codec.go} (65%) delete mode 100644 pkg/querier/handler/compression.go delete mode 100644 pkg/querier/handler/handler.go delete mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go delete mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go delete mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go delete mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go delete mode 100644 vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go delete mode 100644 vendor/github.com/prometheus/common/promlog/log.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/api/api.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/http.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/logger.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/options.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/http.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/prober/prober.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/http.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/option.go delete mode 100644 vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go delete mode 100644 vendor/golang.org/x/net/http2/h2c/h2c.go delete mode 100644 vendor/google.golang.org/grpc/health/client.go delete mode 100644 vendor/google.golang.org/grpc/health/logging.go delete mode 100644 vendor/google.golang.org/grpc/health/server.go diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 9fd3c238846..1f30ae402e4 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -21,7 +21,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" - api "github.com/prometheus/prometheus/web/api/v1" + v1 "github.com/prometheus/prometheus/web/api/v1" "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/middleware" @@ -194,28 +194,28 @@ func NewQuerierHandler( Help: "Current number of inflight requests to the querier.", }, []string{"method", "route"}) - api := api.NewAPI( + api := v1.NewAPI( engine, querier.NewErrorTranslateSampleAndChunkQueryable(queryable), // Translate errors to errors expected by API. nil, // No remote write support. exemplarQueryable, - func(ctx context.Context) api.ScrapePoolsRetriever { return nil }, - func(context.Context) api.TargetRetriever { return &querier.DummyTargetRetriever{} }, - func(context.Context) api.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, + func(ctx context.Context) v1.ScrapePoolsRetriever { return nil }, + func(context.Context) v1.TargetRetriever { return &querier.DummyTargetRetriever{} }, + func(context.Context) v1.AlertmanagerRetriever { return &querier.DummyAlertmanagerRetriever{} }, func() config.Config { return config.Config{} }, map[string]string{}, // TODO: include configuration flags - api.GlobalURLOptions{}, + v1.GlobalURLOptions{}, func(f http.HandlerFunc) http.HandlerFunc { return f }, nil, // Only needed for admin APIs. "", // This is for snapshots, which is disabled when admin APIs are disabled. Hence empty. false, // Disable admin APIs. logger, - func(context.Context) api.RulesRetriever { return &querier.DummyRulesRetriever{} }, + func(context.Context) v1.RulesRetriever { return &querier.DummyRulesRetriever{} }, 0, 0, 0, // Remote read samples and concurrency limit. false, regexp.MustCompile(".*"), - func() (api.RuntimeInfo, error) { return api.RuntimeInfo{}, errors.New("not implemented") }, - &api.PrometheusVersion{ + func() (v1.RuntimeInfo, error) { return v1.RuntimeInfo{}, errors.New("not implemented") }, + &v1.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, @@ -233,8 +233,7 @@ func NewQuerierHandler( ) if cfg.ProtobufQuerierHandler { - api.InstallCodec(codec.InstantQueryProtobufCodec{}) - api.InstallCodec(codec.QueryRangeProtobufCodec{}) + api.InstallCodec(codec.ProtobufCodec{}) } router := mux.NewRouter() @@ -301,13 +300,13 @@ type buildInfoHandler struct { type buildInfoResponse struct { Status string `json:"status"` - Data *api.PrometheusVersion `json:"data"` + Data *v1.PrometheusVersion `json:"data"` } func (h *buildInfoHandler) ServeHTTP(writer http.ResponseWriter, _ *http.Request) { infoResponse := buildInfoResponse{ Status: "success", - Data: &api.PrometheusVersion{ + Data: &v1.PrometheusVersion{ Version: version.Version, Branch: version.Branch, Revision: version.Revision, diff --git a/pkg/querier/codec/instant_query_protobuf.go b/pkg/querier/codec/instant_query_protobuf.go deleted file mode 100644 index ed56d208c9d..00000000000 --- a/pkg/querier/codec/instant_query_protobuf.go +++ /dev/null @@ -1,80 +0,0 @@ -package codec - -import ( - "net/http" - "strings" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" - jsoniter "github.com/json-iterator/go" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/web/api/v1" - "github.com/gogo/protobuf/proto" -) -type InstantQueryProtobufCodec struct{} - -func (p InstantQueryProtobufCodec) ContentType() v1.MIMEType { - return v1.MIMEType{Type: "application", SubType: "x-protobuf"} -} - -func (p InstantQueryProtobufCodec) CanEncode(req *http.Request, resp *v1.Response) bool { - if resp.Error != "" || resp.Data == nil { - return false - } - return strings.HasSuffix(req.URL.Path, "/query") -} - -func (p InstantQueryProtobufCodec) Encode(_ *http.Request, resp *v1.Response) ([]byte, error) { - prometheusInstantQueryResponse, err := createPrometheusInstantQueryResponse(resp) - if err != nil { - return []byte{}, err - } - b, err := proto.Marshal(prometheusInstantQueryResponse) - return b, err -} - - -func createPrometheusInstantQueryResponse(resp *v1.Response) (*instantquery.PrometheusInstantQueryResponse, error) { - var data = resp.Data.(*v1.QueryData) - - var instantQueryResult instantquery.PrometheusInstantQueryResult - switch string(data.ResultType) { - case model.ValMatrix.String(): - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ - Matrix: &instantquery.Matrix{ - SampleStreams: *getSampleStreams(data), - }, - } - case model.ValVector.String(): - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ - Vector: &instantquery.Vector{ - Samples: *getSamples(data), - }, - } - default: - json := jsoniter.ConfigCompatibleWithStandardLibrary - rawBytes, err := json.Marshal(data) - if err != nil { - return nil, err - } - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_RawBytes{RawBytes: rawBytes} - } - - var stats *tripperware.PrometheusResponseStats - if data.Stats != nil { - builtin := data.Stats.Builtin() - stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} - } - - return &instantquery.PrometheusInstantQueryResponse{ - Status: string(resp.Status), - Data: instantquery.PrometheusInstantQueryData{ - ResultType: string(data.ResultType), - Result: instantQueryResult, - Stats: stats, - }, - ErrorType: string(resp.ErrorType), - Error: resp.Error, - Warnings: resp.Warnings, - }, nil -} diff --git a/pkg/querier/codec/query_range_protobuf.go b/pkg/querier/codec/protobuf_codec.go similarity index 65% rename from pkg/querier/codec/query_range_protobuf.go rename to pkg/querier/codec/protobuf_codec.go index a792ee6f242..28a4221f97d 100644 --- a/pkg/querier/codec/query_range_protobuf.go +++ b/pkg/querier/codec/protobuf_codec.go @@ -1,42 +1,63 @@ package codec import ( - "net/http" - "strings" - - "github.com/prometheus/prometheus/util/stats" + "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/prometheus/prometheus/promql" "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/querier/tripperware/instantquery" + "github.com/prometheus/prometheus/util/stats" + jsoniter "github.com/json-iterator/go" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/web/api/v1" "github.com/gogo/protobuf/proto" ) +type ProtobufCodec struct{} -type QueryRangeProtobufCodec struct{} - -func (p QueryRangeProtobufCodec) ContentType() v1.MIMEType { +func (p ProtobufCodec) ContentType() v1.MIMEType { return v1.MIMEType{Type: "application", SubType: "x-protobuf"} } -func (p QueryRangeProtobufCodec) CanEncode(req *http.Request, resp *v1.Response) bool { +func (p ProtobufCodec) CanEncode(resp *v1.Response) bool { + // Errors are parsed by default json codec if resp.Error != "" || resp.Data == nil { return false } - return strings.HasSuffix(req.URL.Path, "/query_range") + return true } -func (p QueryRangeProtobufCodec) Encode(_ *http.Request, resp *v1.Response) ([]byte, error) { - promtheusResponse := createPrometheusResponse(resp) - b, err := proto.Marshal(promtheusResponse) +func (p ProtobufCodec) Encode(resp *v1.Response) ([]byte, error) { + prometheusInstantQueryResponse, err := createPrometheusInstantQueryResponse(resp) + if err != nil { + return []byte{}, err + } + b, err := proto.Marshal(prometheusInstantQueryResponse) return b, err } -func createPrometheusResponse(resp *v1.Response) (*queryrange.PrometheusResponse) { - data := resp.Data.(*v1.QueryData) +func createPrometheusInstantQueryResponse(resp *v1.Response) (*tripperware.PrometheusResponse, error) { + var data = resp.Data.(*v1.QueryData) - sampleStreams := getSampleStreams(data) + var instantQueryResult tripperware.PrometheusQueryResult + switch string(data.ResultType) { + case model.ValMatrix.String(): + instantQueryResult.Result = &tripperware.PrometheusQueryResult_Matrix{ + Matrix: &tripperware.Matrix{ + SampleStreams: *getMatrixSampleStreams(data), + }, + } + case model.ValVector.String(): + instantQueryResult.Result = &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: *getVectorSamples(data), + }, + } + default: + json := jsoniter.ConfigCompatibleWithStandardLibrary + rawBytes, err := json.Marshal(data) + if err != nil { + return nil, err + } + instantQueryResult.Result = &tripperware.PrometheusQueryResult_RawBytes{RawBytes: rawBytes} + } var stats *tripperware.PrometheusResponseStats if data.Stats != nil { @@ -44,38 +65,20 @@ func createPrometheusResponse(resp *v1.Response) (*queryrange.PrometheusResponse stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} } - return &queryrange.PrometheusResponse{ + return &tripperware.PrometheusResponse{ Status: string(resp.Status), - Data: queryrange.PrometheusData{ + Data: tripperware.PrometheusData{ ResultType: string(data.ResultType), - Result: *sampleStreams, + Result: instantQueryResult, Stats: stats, }, ErrorType: string(resp.ErrorType), Error: resp.Error, Warnings: resp.Warnings, - } -} - -func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { - queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) - queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) - for i := 0; i < queryableSamplesStatsPerStepLen; i++ { - queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, - TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, - } - } - - statSamples := tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, - TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, - } - - return &statSamples + }, nil } -func getSampleStreams(data *v1.QueryData) *[]tripperware.SampleStream { +func getMatrixSampleStreams(data *v1.QueryData) *[]tripperware.SampleStream { sampleStreamsLen := len(data.Result.(promql.Matrix)) sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) @@ -108,9 +111,9 @@ func getSampleStreams(data *v1.QueryData) *[]tripperware.SampleStream { return &sampleStreams } -func getSamples(data *v1.QueryData) *[]*instantquery.Sample { +func getVectorSamples(data *v1.QueryData) *[]tripperware.Sample { vectorSamplesLen := len(data.Result.(promql.Vector)) - vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) + vectorSamples := make([]tripperware.Sample, vectorSamplesLen) for i := 0; i < vectorSamplesLen; i++ { labelsLen := len(data.Result.(promql.Vector)[i].Metric) @@ -125,7 +128,8 @@ func getSamples(data *v1.QueryData) *[]*instantquery.Sample { } } - vectorSamples[i] = &instantquery.Sample{Labels: labels, + vectorSamples[i] = tripperware.Sample{ + Labels: labels, Sample: &cortexpb.Sample{ TimestampMs: data.Result.(promql.Vector)[i].T, Value: data.Result.(promql.Vector)[i].F, @@ -134,3 +138,21 @@ func getSamples(data *v1.QueryData) *[]*instantquery.Sample { } return &vectorSamples } + +func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { + queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) + queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) + for i := 0; i < queryableSamplesStatsPerStepLen; i++ { + queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ + Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, + TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, + } + } + + statSamples := tripperware.PrometheusResponseSamplesStats{ + TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, + TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + } + + return &statSamples +} diff --git a/pkg/querier/handler/compression.go b/pkg/querier/handler/compression.go deleted file mode 100644 index a4db3191923..00000000000 --- a/pkg/querier/handler/compression.go +++ /dev/null @@ -1,93 +0,0 @@ -package handler - -import ( - "net/http" - "strings" - - "github.com/golang/snappy" - "github.com/klauspost/compress/gzhttp" - "github.com/klauspost/compress/zlib" -) - -const ( - acceptEncodingHeader = "Accept-Encoding" - contentEncodingHeader = "Content-Encoding" - snappyEncoding = "snappy" - gzipEncoding = "gzip" - deflateEncoding = "deflate" -) - -// Wrapper around http.ResponseWriter which adds deflate compression -type deflatedResponseWriter struct { - http.ResponseWriter - writer *zlib.Writer -} - -func (c *deflatedResponseWriter) Write(p []byte) (int, error) { - return c.writer.Write(p) -} - -func (c *deflatedResponseWriter) Close() { - c.writer.Close() -} - -func newDeflateResponseWriter(writer http.ResponseWriter) *deflatedResponseWriter { - return &deflatedResponseWriter{ - ResponseWriter: writer, - writer: zlib.NewWriter(writer), - } -} - -// Wrapper around http.ResponseWriter which adds deflate compression -type snappyResponseWriter struct { - http.ResponseWriter - writer *snappy.Writer -} - -func (c *snappyResponseWriter) Write(p []byte) (int, error) { - return c.writer.Write(p) -} - -func (c *snappyResponseWriter) Close() { - c.writer.Close() -} - -func newSnappyResponseWriter(writer http.ResponseWriter) *snappyResponseWriter { - return &snappyResponseWriter{ - ResponseWriter: writer, - writer: snappy.NewBufferedWriter(writer), - } -} - -// CompressionHandler is a wrapper around http.Handler which adds suitable -// response compression based on the client's Accept-Encoding headers. -type CompressionHandler struct { - Handler http.Handler -} - -// ServeHTTP adds compression to the original http.Handler's ServeHTTP() method. -func (c CompressionHandler) ServeHTTP(writer http.ResponseWriter, req *http.Request) { - encodings := strings.Split(req.Header.Get(acceptEncodingHeader), ",") - for _, encoding := range encodings { - switch strings.TrimSpace(encoding) { - case gzipEncoding: - gzhttp.GzipHandler(c.Handler).ServeHTTP(writer, req) - return - case snappyEncoding: - compWriter := newSnappyResponseWriter(writer) - writer.Header().Set(contentEncodingHeader, snappyEncoding) - c.Handler.ServeHTTP(compWriter, req) - compWriter.Close() - return - case deflateEncoding: - compWriter := newDeflateResponseWriter(writer) - writer.Header().Set(contentEncodingHeader, deflateEncoding) - c.Handler.ServeHTTP(compWriter, req) - compWriter.Close() - return - default: - c.Handler.ServeHTTP(writer, req) - return - } - } -} diff --git a/pkg/querier/handler/handler.go b/pkg/querier/handler/handler.go deleted file mode 100644 index e957134b11f..00000000000 --- a/pkg/querier/handler/handler.go +++ /dev/null @@ -1,613 +0,0 @@ -package handler - -import ( - "context" - "fmt" - "math" - "net/http" - "strconv" - "strings" - "time" - - "github.com/cortexproject/cortex/pkg/querier/tripperware/instantquery" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/gogo/protobuf/proto" - jsoniter "github.com/json-iterator/go" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - v1 "github.com/prometheus/prometheus/web/api/v1" - - "github.com/cortexproject/cortex/pkg/cortexpb" - cortex_pb "github.com/cortexproject/cortex/pkg/cortexpb" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/querier/tripperware/queryrange" - "github.com/prometheus/prometheus/promql" - "github.com/prometheus/prometheus/promql/parser" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/annotations" - "github.com/prometheus/prometheus/util/httputil" - "github.com/prometheus/prometheus/util/stats" - thanos_api "github.com/thanos-io/thanos/pkg/api" -) - -type apiFunc func(r *http.Request) (interface{}, []error, *thanos_api.ApiError, func()) - -type status string - -const ( - statusSuccess status = "success" - statusError status = "error" - contentTypeHeader string = "Content-Type" - acceptHeader string = "Accept" - applicationProtobuf string = "application/x-protobuf" - applicationJson string = "application/json" - - // Non-standard status code (originally introduced by nginx) for the case when a client closes - // the connection while the server is still processing the request. - statusClientClosedConnection = 499 -) - -const ( - errorNotFound thanos_api.ErrorType = "not_found" -) - -func defaultStatsRenderer(_ context.Context, s *stats.Statistics, param string) stats.QueryStats { - if param != "" { - return stats.NewQueryStats(s) - } - return nil -} - -type response struct { - Status status `json:"status"` - Data interface{} `json:"data,omitempty"` - ErrorType thanos_api.ErrorType `json:"errorType,omitempty"` - Error string `json:"error,omitempty"` - Warnings []string `json:"warnings,omitempty"` -} - -type API struct { - Queryable storage.SampleAndChunkQueryable - QueryEngine promql.QueryEngine - Now func() time.Time - Logger log.Logger - StatsRenderer v1.StatsRenderer -} - -// NewAPI returns an initialized API type. -func NewAPI( - qe promql.QueryEngine, - q storage.SampleAndChunkQueryable, - logger log.Logger, - statsRenderer v1.StatsRenderer, -) *API { - a := &API{ - QueryEngine: qe, - Queryable: q, - Now: time.Now, - Logger: logger, - StatsRenderer: defaultStatsRenderer, - } - - if statsRenderer != nil { - a.StatsRenderer = statsRenderer - } - - return a -} - -type queryData struct { - ResultType parser.ValueType `json:"resultType"` - Result parser.Value `json:"result"` - Stats stats.QueryStats `json:"stats,omitempty"` -} - -func invalidParamError(err error, parameter string) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { - return nil, &thanos_api.ApiError{ - thanos_api.ErrorBadData, errors.Wrapf(err, "invalid parameter %q", parameter), - }, nil, nil -} - -func (api *API) Query(r *http.Request) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { - ts, err := parseTimeParam(r, "time", api.Now()) - if err != nil { - return invalidParamError(err, "time") - } - ctx := r.Context() - if to := r.FormValue("timeout"); to != "" { - var cancel context.CancelFunc - timeout, err := parseDuration(to) - if err != nil { - return invalidParamError(err, "timeout") - } - - ctx, cancel = context.WithDeadline(ctx, api.Now().Add(timeout)) - defer cancel() - } - - opts, err := extractQueryOpts(r) - if err != nil { - return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil - } - qry, err := api.QueryEngine.NewInstantQuery(ctx, api.Queryable, opts, r.FormValue("query"), ts) - if err != nil { - return invalidParamError(err, "query") - } - - // From now on, we must only return with a Finalizer in the result (to - // be called by the caller) or call qry.Close ourselves (which is - // required in the case of a panic). - defer func() { - if finalizer == nil { - qry.Close() - } - }() - - ctx = httputil.ContextFromRequest(ctx, r) - - res := qry.Exec(ctx) - if res.Err != nil { - return nil, returnAPIError(res.Err), res.Warnings, qry.Close - } - - // Optional stats field in response if parameter "stats" is not empty. - sr := api.StatsRenderer - if sr == nil { - sr = defaultStatsRenderer - } - qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - - accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] - switch accept { - case applicationProtobuf: - data, err = createPrometheusInstantQueryResponse(&queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - }) - case applicationJson: - data = &queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - } - default: - data = &queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - } - } - if err != nil { - return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, res.Warnings, qry.Close - } - return data, nil, res.Warnings, qry.Close -} - -func extractQueryOpts(r *http.Request) (promql.QueryOpts, error) { - var duration time.Duration - - if strDuration := r.FormValue("lookback_delta"); strDuration != "" { - parsedDuration, err := parseDuration(strDuration) - if err != nil { - return nil, fmt.Errorf("error parsing lookback delta duration: %w", err) - } - duration = parsedDuration - } - - return promql.NewPrometheusQueryOpts(r.FormValue("stats") == "all", duration), nil -} - -func (api *API) QueryRange(r *http.Request) (data interface{}, error *thanos_api.ApiError, warnings annotations.Annotations, finalizer func()) { - start, err := parseTime(r.FormValue("start")) - if err != nil { - return invalidParamError(err, "start") - } - end, err := parseTime(r.FormValue("end")) - if err != nil { - return invalidParamError(err, "end") - } - if end.Before(start) { - return invalidParamError(errors.New("end timestamp must not be before start time"), "end") - } - - step, err := parseDuration(r.FormValue("step")) - if err != nil { - return invalidParamError(err, "step") - } - - if step <= 0 { - return invalidParamError(errors.New("zero or negative query resolution step widths are not accepted. Try a positive integer"), "step") - } - - // For safety, limit the number of returned points per timeseries. - // This is sufficient for 60s resolution for a week or 1h resolution for a year. - if end.Sub(start)/step > 11000 { - err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") - return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil - } - - ctx := r.Context() - if to := r.FormValue("timeout"); to != "" { - var cancel context.CancelFunc - timeout, err := parseDuration(to) - if err != nil { - return invalidParamError(err, "timeout") - } - - ctx, cancel = context.WithTimeout(ctx, timeout) - defer cancel() - } - - opts, err := extractQueryOpts(r) - if err != nil { - return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, nil, nil - } - qry, err := api.QueryEngine.NewRangeQuery(ctx, api.Queryable, opts, r.FormValue("query"), start, end, step) - if err != nil { - return invalidParamError(err, "query") - } - // From now on, we must only return with a Finalizer in the result (to - // be called by the caller) or call qry.Close ourselves (which is - // required in the case of a panic). - defer func() { - if finalizer == nil { - qry.Close() - } - }() - - ctx = httputil.ContextFromRequest(ctx, r) - - res := qry.Exec(ctx) - if res.Err != nil { - return nil, returnAPIError(res.Err), res.Warnings, qry.Close - } - - // Optional stats field in response if parameter "stats" is not empty. - sr := api.StatsRenderer - if sr == nil { - sr = defaultStatsRenderer - } - qs := sr(ctx, qry.Stats(), r.FormValue("stats")) - - accept := strings.Split(r.Header.Get(acceptHeader), ",")[0] - switch accept { - case applicationProtobuf: - data, err = createPrometheusResponse(&queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - }) - case applicationJson: - data = &queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - } - default: - data = &queryData{ - ResultType: res.Value.Type(), - Result: res.Value, - Stats: qs, - } - } - - if err != nil { - return nil, &thanos_api.ApiError{thanos_api.ErrorBadData, err}, res.Warnings, qry.Close - } - return data, nil, res.Warnings, qry.Close -} - -func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) (time.Time, error) { - val := r.FormValue(paramName) - if val == "" { - return defaultValue, nil - } - result, err := parseTime(val) - if err != nil { - return time.Time{}, errors.Wrapf(err, "Invalid time value for '%s'", paramName) - } - return result, nil -} - -func parseTime(s string) (time.Time, error) { - if t, err := strconv.ParseFloat(s, 64); err == nil { - s, ns := math.Modf(t) - ns = math.Round(ns*1000) / 1000 - return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil - } - if t, err := time.Parse(time.RFC3339Nano, s); err == nil { - return t, nil - } - - // Stdlib's time parser can only handle 4 digit years. As a workaround until - // that is fixed we want to at least support our own boundary times. - // Context: https://github.com/prometheus/client_golang/issues/614 - // Upstream issue: https://github.com/golang/go/issues/20555 - switch s { - case minTimeFormatted: - return v1.MinTime, nil - case maxTimeFormatted: - return v1.MaxTime, nil - } - return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) -} - -func parseDuration(s string) (time.Duration, error) { - if d, err := strconv.ParseFloat(s, 64); err == nil { - ts := d * float64(time.Second) - if ts > float64(math.MaxInt64) || ts < float64(math.MinInt64) { - return 0, errors.Errorf("cannot parse %q to a valid duration. It overflows int64", s) - } - return time.Duration(ts), nil - } - if d, err := model.ParseDuration(s); err == nil { - return time.Duration(d), nil - } - return 0, errors.Errorf("cannot parse %q to a valid duration", s) -} - -func returnAPIError(err error) *thanos_api.ApiError { - if err == nil { - return nil - } - - cause := errors.Unwrap(err) - if cause == nil { - cause = err - } - - switch cause.(type) { - case promql.ErrQueryCanceled: - return &thanos_api.ApiError{thanos_api.ErrorCanceled, err} - case promql.ErrQueryTimeout: - return &thanos_api.ApiError{thanos_api.ErrorTimeout, err} - case promql.ErrStorage: - return &thanos_api.ApiError{thanos_api.ErrorInternal, err} - } - - if errors.Is(err, context.Canceled) { - return &thanos_api.ApiError{thanos_api.ErrorCanceled, err} - } - - return &thanos_api.ApiError{thanos_api.ErrorExec, err} -} - -var ( - minTimeFormatted = v1.MinTime.Format(time.RFC3339Nano) - maxTimeFormatted = v1.MaxTime.Format(time.RFC3339Nano) -) - -func (api *API) Respond(w http.ResponseWriter, req *http.Request, data interface{}, warnings annotations.Annotations, query string) { - statusMessage := statusSuccess - - var b []byte - var err error - switch resp := data.(type) { - case *queryrange.PrometheusResponse: - w.Header().Set(contentTypeHeader, applicationProtobuf) - for h, hv := range w.Header() { - resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) - } - resp.Warnings = warnings.AsStrings(query, 10) - b, err = proto.Marshal(resp) - case *instantquery.PrometheusInstantQueryResponse: - w.Header().Set(contentTypeHeader, applicationProtobuf) - for h, hv := range w.Header() { - resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) - } - resp.Warnings = warnings.AsStrings(query, 10) - b, err = proto.Marshal(resp) - case *queryData: - w.Header().Set(contentTypeHeader, applicationJson) - json := jsoniter.ConfigCompatibleWithStandardLibrary - b, err = json.Marshal(&response{ - Status: statusMessage, - Data: data, - Warnings: warnings.AsStrings(query, 10), - }) - default: - level.Error(api.Logger).Log("msg", "error asserting response type") - http.Error(w, "error asserting response type", http.StatusInternalServerError) - return - } - - if err != nil { - level.Error(api.Logger).Log("msg", "error marshaling response", "err", err) - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - w.WriteHeader(http.StatusOK) - if n, err := w.Write(b); err != nil { - level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) - } -} - -func (api *API) RespondError(w http.ResponseWriter, apiErr *thanos_api.ApiError, data interface{}) { - json := jsoniter.ConfigCompatibleWithStandardLibrary - b, err := json.Marshal(&response{ - Status: statusError, - ErrorType: apiErr.Typ, - Error: apiErr.Err.Error(), - Data: data, - }) - if err != nil { - level.Error(api.Logger).Log("msg", "error marshaling json response", "err", err) - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - var code int - switch apiErr.Typ { - case thanos_api.ErrorBadData: - code = http.StatusBadRequest - case thanos_api.ErrorExec: - code = http.StatusUnprocessableEntity - case thanos_api.ErrorCanceled: - code = statusClientClosedConnection - case thanos_api.ErrorTimeout: - code = http.StatusServiceUnavailable - case thanos_api.ErrorInternal: - code = http.StatusInternalServerError - case errorNotFound: - code = http.StatusNotFound - default: - code = http.StatusInternalServerError - } - - w.Header().Set(contentTypeHeader, applicationJson) - w.WriteHeader(code) - if n, err := w.Write(b); err != nil { - level.Error(api.Logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) - } -} - -func createPrometheusResponse(data *queryData) (*queryrange.PrometheusResponse, error) { - if data == nil { - return nil, errors.New("no query response data") - } - - sampleStreams := getSampleStreams(data) - - var stats *tripperware.PrometheusResponseStats - if data.Stats != nil { - builtin := data.Stats.Builtin() - stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} - } - - return &queryrange.PrometheusResponse{ - Status: string(statusSuccess), - Data: queryrange.PrometheusData{ - ResultType: string(data.ResultType), - Result: *sampleStreams, - Stats: stats, - }, - ErrorType: "", - Error: "", - }, nil -} - -func createPrometheusInstantQueryResponse(data *queryData) (*instantquery.PrometheusInstantQueryResponse, error) { - if data == nil { - return nil, errors.New("no query response data") - } - - var instantQueryResult instantquery.PrometheusInstantQueryResult - switch string(data.ResultType) { - case model.ValMatrix.String(): - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Matrix{ - Matrix: &instantquery.Matrix{ - SampleStreams: *getSampleStreams(data), - }, - } - case model.ValVector.String(): - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_Vector{ - Vector: &instantquery.Vector{ - Samples: *getSamples(data), - }, - } - default: - rawBytes, err := jsoniter.Marshal(data) - if err != nil { - return nil, err - } - instantQueryResult.Result = &instantquery.PrometheusInstantQueryResult_RawBytes{RawBytes: rawBytes} - } - - var stats *tripperware.PrometheusResponseStats - if data.Stats != nil { - builtin := data.Stats.Builtin() - stats = &tripperware.PrometheusResponseStats{Samples: getStats(&builtin)} - } - - return &instantquery.PrometheusInstantQueryResponse{ - Status: string(statusSuccess), - Data: instantquery.PrometheusInstantQueryData{ - ResultType: string(data.ResultType), - Result: instantQueryResult, - Stats: stats, - }, - ErrorType: "", - Error: "", - }, nil -} - -func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSamplesStats { - queryableSamplesStatsPerStepLen := len(builtin.Samples.TotalQueryableSamplesPerStep) - queryableSamplesStatsPerStep := make([]*tripperware.PrometheusResponseQueryableSamplesStatsPerStep, queryableSamplesStatsPerStepLen) - for i := 0; i < queryableSamplesStatsPerStepLen; i++ { - queryableSamplesStatsPerStep[i] = &tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - Value: builtin.Samples.TotalQueryableSamplesPerStep[i].V, - TimestampMs: builtin.Samples.TotalQueryableSamplesPerStep[i].T, - } - } - - statSamples := tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, - TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, - } - - return &statSamples -} - -func getSampleStreams(data *queryData) *[]tripperware.SampleStream { - sampleStreamsLen := len(data.Result.(promql.Matrix)) - sampleStreams := make([]tripperware.SampleStream, sampleStreamsLen) - - for i := 0; i < sampleStreamsLen; i++ { - labelsLen := len(data.Result.(promql.Matrix)[i].Metric) - var labels []cortex_pb.LabelAdapter - if labelsLen > 0 { - labels = make([]cortex_pb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = cortex_pb.LabelAdapter{ - Name: data.Result.(promql.Matrix)[i].Metric[j].Name, - Value: data.Result.(promql.Matrix)[i].Metric[j].Value, - } - } - } - - samplesLen := len(data.Result.(promql.Matrix)[i].Floats) - var samples []cortexpb.Sample - if samplesLen > 0 { - samples = make([]cortexpb.Sample, samplesLen) - for j := 0; j < samplesLen; j++ { - samples[j] = cortexpb.Sample{ - Value: data.Result.(promql.Matrix)[i].Floats[j].F, - TimestampMs: data.Result.(promql.Matrix)[i].Floats[j].T, - } - } - } - sampleStreams[i] = tripperware.SampleStream{Labels: labels, Samples: samples} - } - return &sampleStreams -} - -func getSamples(data *queryData) *[]*instantquery.Sample { - vectorSamplesLen := len(data.Result.(promql.Vector)) - vectorSamples := make([]*instantquery.Sample, vectorSamplesLen) - - for i := 0; i < vectorSamplesLen; i++ { - labelsLen := len(data.Result.(promql.Vector)[i].Metric) - var labels []cortex_pb.LabelAdapter - if labelsLen > 0 { - labels = make([]cortex_pb.LabelAdapter, labelsLen) - for j := 0; j < labelsLen; j++ { - labels[j] = cortex_pb.LabelAdapter{ - Name: data.Result.(promql.Vector)[i].Metric[j].Name, - Value: data.Result.(promql.Vector)[i].Metric[j].Value, - } - } - } - - vectorSamples[i] = &instantquery.Sample{Labels: labels, - Sample: &cortexpb.Sample{ - TimestampMs: data.Result.(promql.Vector)[i].T, - Value: data.Result.(promql.Vector)[i].F, - }, - } - } - return &vectorSamples -} diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 15b19114690..6f10900c138 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -49,7 +49,7 @@ type Config struct { QueryIngestersWithin time.Duration `yaml:"query_ingesters_within"` EnablePerStepStats bool `yaml:"per_step_stats_enabled"` - // Use compression when returning promql response. Supported values 'gzip', 'snappy', and '' (disable compression) + // Use compression when returning promql response. Supported values are 'gzip' or '' (disable compression) PrometheusCodecCompression string `yaml:"prometheus_codec_compression"` // QueryStoreAfter the time after which queries should also be sent to the store and not just ingesters. diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index e39617b5f49..ba35a665109 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -9,8 +9,6 @@ import ( "net/url" "strings" "time" - "unsafe" - "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" @@ -26,7 +24,7 @@ import ( ) var ( - InstantQueryCodec tripperware.Codec = newInstantQueryCodec("", true) + InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("", true) json = jsoniter.Config{ EscapeHTML: false, // No HTML in our responses. @@ -42,7 +40,7 @@ type instantQueryCodec struct { now func() time.Time } -func newInstantQueryCodec(compressionStr string, enableProtobuf bool) instantQueryCodec { +func NewInstantQueryCodec(compressionStr string, enableProtobuf bool) instantQueryCodec { var compression queryrange.Compression if compressionStr == "gzip" { compression = queryrange.Compression(compressionStr) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index fb7da2b3d55..d210d8e3abe 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -113,17 +113,17 @@ func TestCompressedResponse(t *testing.T) { for i, tc := range []struct { compression string jsonBody string - promBody *PrometheusInstantQueryResponse + promBody *tripperware.PrometheusResponse status int err error }{ { compression: "gzip", - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -131,11 +131,11 @@ func TestCompressedResponse(t *testing.T) { }, { compression: "snappy", - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -183,7 +183,9 @@ func TestCompressedResponse(t *testing.T) { var buf bytes.Buffer if tc.compression == "gzip" { h.Set("Content-Encoding", "gzip") - if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} + if tc.promBody != nil { + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}}) + } w := gzip.NewWriter(&buf) _, err := w.Write(b) require.NoError(t, err) @@ -191,7 +193,9 @@ func TestCompressedResponse(t *testing.T) { responseBody = &buf } else if tc.compression == "snappy" { h.Set("Content-Encoding", "snappy") - if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}})} + if tc.promBody != nil { + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}}) + } w := snappy.NewBufferedWriter(&buf) _, err := w.Write(b) require.NoError(t, err) @@ -218,8 +222,8 @@ func TestCompressedResponse(t *testing.T) { func TestResponse(t *testing.T) { t.Parallel() for i, tc := range []struct { - jsonBody string - promBody *PrometheusInstantQueryResponse + jsonBody string + promBody *tripperware.PrometheusResponse }{ { jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, @@ -244,35 +248,35 @@ func TestResponse(t *testing.T) { }, { jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"]}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { jsonBody: `{"status":"success","data":{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`)}}, + Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`)}}, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -302,13 +306,13 @@ func TestResponse(t *testing.T) { }, { jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}]}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -329,25 +333,25 @@ func TestResponse(t *testing.T) { }, { jsonBody: `{"status":"success","data":{"resultType":"scalar","result":[1,"13"]}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{Result: &PrometheusInstantQueryResult_RawBytes{[]byte(`{"resultType":"scalar","result":[1,"13"]}`)}}, + Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"scalar","result":[1,"13"]}`)}}, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, }, { jsonBody: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{},"value":[1,"1266464.0146205237"]}]}}`, - promBody: &PrometheusInstantQueryResponse{ + promBody: &tripperware.PrometheusResponse{ Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{}, Sample: &cortexpb.Sample{Value: 1266464.0146205237, TimestampMs: 1000}, @@ -364,7 +368,7 @@ func TestResponse(t *testing.T) { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - + var response *http.Response if tc.promBody != nil { protobuf, err := proto.Marshal(tc.promBody) @@ -664,13 +668,13 @@ func TestMergeResponse(t *testing.T) { func TestMergeResponseProtobuf(t *testing.T) { t.Parallel() - defaultReq := &PrometheusRequest{ + defaultReq := &tripperware.PrometheusRequest{ Query: "sum(up)", } for _, tc := range []struct { name string req tripperware.Request - resps []*PrometheusInstantQueryResponse + resps []*tripperware.PrometheusResponse expectedResp string expectedErr error cancelBeforeDecode bool @@ -680,15 +684,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "empty response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: make([]*Sample, 0), + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: make([]*/tripperware.Sample, 0), }, }, }, @@ -703,15 +707,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "empty response with stats", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{}, }, }, }, @@ -732,15 +736,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "single response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -762,15 +766,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "single response with stats", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -800,15 +804,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "duplicated response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -826,12 +830,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -853,15 +857,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "duplicated response with stats", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -887,12 +891,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -922,15 +926,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "merge two responses", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -949,12 +953,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -976,16 +980,16 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { name: "merge two responses with sort", - req: &PrometheusRequest{Query: "sort(sum by (job) (up))"}, - resps: []*PrometheusInstantQueryResponse{ + req: &tripperware.PrometheusRequest{Query: "sort(sum by (job) (up))"}, + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1004,12 +1008,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1031,16 +1035,16 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { name: "merge two responses with sort_desc", - req: &PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, - resps: []*PrometheusInstantQueryResponse{ + req: &tripperware.PrometheusRequest{Query: "sort_desc(sum by (job) (up))"}, + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1059,12 +1063,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1086,16 +1090,16 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { name: "merge two responses with topk", - req: &PrometheusRequest{Query: "topk(10, up) by(job)"}, - resps: []*PrometheusInstantQueryResponse{ + req: &tripperware.PrometheusRequest{Query: "topk(10, up) by(job)"}, + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1114,12 +1118,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1142,15 +1146,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "merge with warnings", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1163,19 +1167,19 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - Warnings: []string{"warning1","warning2"}, + Warnings: []string{"warning1", "warning2"}, Headers: []*tripperware.PrometheusResponseHeader{ {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, }, }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1188,7 +1192,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - Warnings: []string{"warning1","warning3"}, + Warnings: []string{"warning1", "warning3"}, Headers: []*tripperware.PrometheusResponseHeader{ {Name: "Content-Type", Values: []string{"application/x-protobuf"}}, }, @@ -1199,15 +1203,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "merge two responses with stats", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -1234,12 +1238,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -1270,13 +1274,13 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "responses don't contain vector, should return an error", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_RawBytes{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ RawBytes: []byte(`{"resultType":"string","result":[1662682521.409,"foo"]}`), }, }, @@ -1287,10 +1291,10 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_RawBytes{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ RawBytes: []byte(`{"resultType":"string","result":[1662682521.409,"foo"]}`), }, }, @@ -1305,14 +1309,14 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "single matrix response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1338,14 +1342,14 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "multiple matrix responses without duplicated series", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1367,11 +1371,11 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1397,14 +1401,14 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "multiple matrix responses with duplicated series, but not same samples", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1426,11 +1430,11 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1455,14 +1459,14 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "multiple matrix responses with duplicated series and same samples", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1484,11 +1488,11 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + 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: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ @@ -1515,15 +1519,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "context cancelled before decoding response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1542,12 +1546,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1571,15 +1575,15 @@ func TestMergeResponseProtobuf(t *testing.T) { { name: "context cancelled before merging response", req: defaultReq, - resps: []*PrometheusInstantQueryResponse{ + resps: []*tripperware.PrometheusResponse{ { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1598,12 +1602,12 @@ func TestMergeResponseProtobuf(t *testing.T) { }, { Status: "success", - Data: PrometheusInstantQueryData{ + Data: tripperware.PrometheusData{ ResultType: model.ValVector.String(), - Result: PrometheusInstantQueryResult{ - Result: &PrometheusInstantQueryResult_Vector{ - Vector: &Vector{ - Samples: []*Sample{ + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_Vector{ + Vector: &tripperware.Vector{ + Samples: []*/tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1670,72 +1674,6 @@ func TestMergeResponseProtobuf(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) - } - }) - } -} - func Benchmark_Decode(b *testing.B) { maxSamplesCount := 1000000 samples := make([]tripperware.SampleStream, maxSamplesCount) @@ -1830,12 +1768,12 @@ func Benchmark_Decode_Protobuf(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/merge.go b/pkg/querier/tripperware/merge.go index 6266946b4a2..87a49448ea1 100644 --- a/pkg/querier/tripperware/merge.go +++ b/pkg/querier/tripperware/merge.go @@ -130,7 +130,7 @@ func matrixMerge(ctx context.Context, resps []*PrometheusResponse) ([]SampleStre } func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) (*Vector, error) { - output := map[string]*Sample{} + output := map[string]Sample{} metrics := []string{} // Used to preserve the order for topk and bottomk. sortPlan, err := sortPlanForQuery(req.GetQuery()) if err != nil { @@ -151,9 +151,6 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) } 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 @@ -166,7 +163,7 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) } result := &Vector{ - Samples: make([]*Sample, 0, len(output)), + Samples: make([]Sample, 0, len(output)), } if len(output) == 0 { @@ -184,7 +181,7 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) for k, v := range output { samples = append(samples, &pair{ metric: k, - s: v, + s: &v, }) } @@ -202,7 +199,7 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) }) for _, p := range samples { - result.Samples = append(result.Samples, p.s) + result.Samples = append(result.Samples, *p.s) } return result, nil } diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index b985f6b3ddb..e77fa676248 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -499,7 +499,7 @@ func (s *PrometheusData) UnmarshalJSON(data []byte) error { switch s.ResultType { case model.ValVector.String(): var result struct { - Samples []*Sample `json:"result"` + Samples []Sample `json:"result"` } if err := json.Unmarshal(data, &result); err != nil { return err @@ -535,7 +535,7 @@ func (s *PrometheusData) MarshalJSON() ([]byte, error) { case model.ValVector.String(): res := struct { ResultType string `json:"resultType"` - Data []*Sample `json:"result"` + Data []Sample `json:"result"` Stats *PrometheusResponseStats `json:"stats,omitempty"` }{ ResultType: s.ResultType, diff --git a/pkg/querier/tripperware/query.pb.go b/pkg/querier/tripperware/query.pb.go index 84c1c406bd3..bbb539643c0 100644 --- a/pkg/querier/tripperware/query.pb.go +++ b/pkg/querier/tripperware/query.pb.go @@ -819,7 +819,7 @@ func (*PrometheusQueryResult) XXX_OneofWrappers() []interface{} { } type Vector struct { - Samples []*Sample `protobuf:"bytes,1,rep,name=samples,proto3" json:"samples,omitempty"` + Samples []Sample `protobuf:"bytes,1,rep,name=samples,proto3" json:"samples"` } func (m *Vector) Reset() { *m = Vector{} } @@ -854,7 +854,7 @@ func (m *Vector) XXX_DiscardUnknown() { var xxx_messageInfo_Vector proto.InternalMessageInfo -func (m *Vector) GetSamples() []*Sample { +func (m *Vector) GetSamples() []Sample { if m != nil { return m.Samples } @@ -978,81 +978,81 @@ func init() { 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, + // 1174 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0xcd, 0x6f, 0x1b, 0x45, + 0x14, 0xf7, 0xfa, 0x63, 0xe3, 0x3c, 0xa7, 0x49, 0x99, 0xf4, 0xc3, 0x29, 0x65, 0xd7, 0xac, 0x40, + 0x0a, 0x82, 0x3a, 0x52, 0x2a, 0x40, 0x80, 0xa8, 0xc8, 0x42, 0x20, 0x14, 0x4a, 0xd2, 0x49, 0x54, + 0x24, 0x2e, 0xd5, 0xd8, 0x1e, 0x9c, 0x25, 0x5e, 0xef, 0x76, 0x76, 0xb6, 0x89, 0x39, 0xf1, 0x17, + 0x20, 0x2e, 0x5c, 0x90, 0x38, 0x70, 0xe3, 0xc0, 0x1f, 0x92, 0x63, 0x8e, 0x15, 0x12, 0x0b, 0x71, + 0x2e, 0x68, 0x4f, 0xfd, 0x13, 0xd0, 0x7c, 0xac, 0xbd, 0x4e, 0x9c, 0x44, 0x3d, 0x71, 0x71, 0x76, + 0xde, 0xfb, 0xbd, 0xaf, 0xdf, 0xbc, 0xf7, 0x26, 0x50, 0x7b, 0x12, 0x53, 0x36, 0x68, 0x86, 0x2c, + 0xe0, 0x01, 0xaa, 0x71, 0xe6, 0x85, 0x21, 0x65, 0xfb, 0x84, 0xd1, 0x5b, 0xd7, 0xba, 0x41, 0x37, + 0x90, 0xf2, 0x15, 0xf1, 0xa5, 0x20, 0xb7, 0xac, 0x6e, 0x10, 0x74, 0x7b, 0x74, 0x45, 0x9e, 0x5a, + 0xf1, 0xb7, 0x2b, 0x9d, 0x98, 0x11, 0xee, 0x05, 0x7d, 0xad, 0x5f, 0x3a, 0xad, 0x27, 0x7d, 0xed, + 0xfd, 0xd6, 0x7b, 0x5d, 0x8f, 0xef, 0xc6, 0xad, 0x66, 0x3b, 0xf0, 0x57, 0xda, 0x01, 0xe3, 0xf4, + 0x20, 0x64, 0xc1, 0x77, 0xb4, 0xcd, 0xf5, 0x69, 0x25, 0xdc, 0xeb, 0x66, 0x8a, 0x96, 0xfe, 0x50, + 0xa6, 0xce, 0xdf, 0x45, 0x40, 0x5b, 0x2c, 0xf0, 0x29, 0xdf, 0xa5, 0x71, 0x84, 0x69, 0x14, 0x06, + 0xfd, 0x88, 0x22, 0x07, 0xcc, 0x6d, 0x4e, 0x78, 0x1c, 0xd5, 0x8d, 0x86, 0xb1, 0x3c, 0xeb, 0x42, + 0x9a, 0xd8, 0x66, 0x24, 0x25, 0x58, 0x6b, 0xd0, 0x67, 0x50, 0xfe, 0x84, 0x70, 0x52, 0x2f, 0x36, + 0x8c, 0xe5, 0xda, 0xea, 0xcb, 0xcd, 0x5c, 0x89, 0xcd, 0xb1, 0x4b, 0x01, 0x71, 0x6f, 0x1c, 0x26, + 0x76, 0x21, 0x4d, 0xec, 0xf9, 0x0e, 0xe1, 0xe4, 0xad, 0xc0, 0xf7, 0x38, 0xf5, 0x43, 0x3e, 0xc0, + 0xd2, 0x01, 0x7a, 0x1b, 0x66, 0xd7, 0x19, 0x0b, 0xd8, 0xce, 0x20, 0xa4, 0xf5, 0x92, 0x8c, 0x77, + 0x33, 0x4d, 0xec, 0x45, 0x9a, 0x09, 0x73, 0x16, 0x63, 0x24, 0x7a, 0x03, 0x2a, 0xf2, 0x50, 0x2f, + 0x4b, 0x93, 0xc5, 0x34, 0xb1, 0x17, 0xa4, 0x49, 0x0e, 0xae, 0x10, 0xe8, 0x53, 0x98, 0xd9, 0xa0, + 0xa4, 0x43, 0x59, 0x54, 0xaf, 0x34, 0x4a, 0xcb, 0xb5, 0xd5, 0xd7, 0xcf, 0xc9, 0x36, 0x23, 0x40, + 0xa1, 0xdd, 0x4a, 0x9a, 0xd8, 0xc6, 0x1d, 0x9c, 0x19, 0xa3, 0x55, 0xa8, 0x7e, 0x4d, 0x58, 0xdf, + 0xeb, 0x77, 0xa3, 0xba, 0xd9, 0x28, 0x2d, 0xcf, 0xba, 0x37, 0xd2, 0xc4, 0x46, 0xfb, 0x5a, 0x96, + 0x0b, 0x3c, 0xc2, 0x39, 0x7f, 0x19, 0x30, 0x3f, 0x49, 0x07, 0x6a, 0x02, 0x60, 0x1a, 0xc5, 0x3d, + 0x2e, 0x2b, 0x56, 0x0c, 0xcf, 0xa7, 0x89, 0x0d, 0x6c, 0x24, 0xc5, 0x39, 0x04, 0xba, 0x0f, 0xa6, + 0x3a, 0x69, 0xae, 0x9d, 0x73, 0xb2, 0x7f, 0x28, 0x3a, 0x4e, 0x21, 0xdd, 0x79, 0x4d, 0xb9, 0xa9, + 0x7c, 0x62, 0xed, 0x01, 0x6d, 0x42, 0x45, 0xdc, 0x63, 0x24, 0x89, 0xae, 0xad, 0xbe, 0x76, 0x09, + 0x11, 0xe2, 0xae, 0x23, 0xc5, 0xad, 0x34, 0xcb, 0x73, 0x2b, 0x05, 0xce, 0x1e, 0xcc, 0x7f, 0x4c, + 0xda, 0xbb, 0xb4, 0x33, 0x6a, 0x9e, 0x25, 0x28, 0xed, 0xd1, 0x81, 0xae, 0x6b, 0x26, 0x4d, 0x6c, + 0x71, 0xc4, 0xe2, 0x07, 0xdd, 0x83, 0x19, 0x7a, 0xc0, 0x69, 0x9f, 0x47, 0xf5, 0xa2, 0xbc, 0x88, + 0xc5, 0x89, 0xf8, 0xeb, 0x52, 0xe7, 0x2e, 0xe8, 0xdc, 0x33, 0x2c, 0xce, 0x3e, 0x9c, 0x3f, 0x0c, + 0x30, 0x15, 0x08, 0xd9, 0xb2, 0x10, 0xc6, 0x65, 0x9c, 0x92, 0x3b, 0x9b, 0x26, 0xb6, 0x12, 0x60, + 0xf5, 0x47, 0xa4, 0x41, 0xfb, 0x1d, 0x49, 0x59, 0x49, 0xa5, 0x41, 0xfb, 0x1d, 0x2c, 0x7e, 0x50, + 0x03, 0xaa, 0x9c, 0x91, 0x36, 0x7d, 0xec, 0x75, 0x74, 0xf7, 0x64, 0x37, 0x2d, 0xc5, 0x9f, 0x77, + 0xd0, 0x3d, 0xa8, 0x32, 0x5d, 0x4f, 0xbd, 0x22, 0x99, 0xba, 0xd6, 0x54, 0x03, 0xd8, 0xcc, 0x06, + 0xb0, 0xb9, 0xd6, 0x1f, 0xb8, 0x73, 0x69, 0x62, 0x8f, 0x90, 0x78, 0xf4, 0x75, 0xbf, 0x5c, 0x2d, + 0x5d, 0x2d, 0x3b, 0xbf, 0x14, 0x61, 0x6e, 0x9b, 0xf8, 0x61, 0x8f, 0x6e, 0x73, 0x46, 0x89, 0x8f, + 0x0e, 0xc0, 0xec, 0x91, 0x16, 0xed, 0x89, 0xb9, 0x52, 0xe5, 0x67, 0x63, 0xd9, 0xfc, 0x52, 0xc8, + 0xb7, 0x88, 0xc7, 0xdc, 0x2f, 0x44, 0xf9, 0x7f, 0x26, 0xf6, 0x0b, 0x8d, 0xb5, 0xb2, 0x5f, 0xeb, + 0x90, 0x90, 0x53, 0x26, 0xee, 0xdd, 0xa7, 0x9c, 0x79, 0x6d, 0xac, 0xe3, 0xa1, 0xf7, 0x61, 0x26, + 0x92, 0x99, 0x64, 0xcc, 0x5f, 0x1d, 0x87, 0x56, 0x29, 0x8e, 0x5b, 0xe6, 0x29, 0xe9, 0xc5, 0x34, + 0xc2, 0x99, 0x01, 0xda, 0x01, 0xd8, 0xf5, 0x22, 0x1e, 0x74, 0x19, 0xf1, 0x45, 0xe3, 0x08, 0xf3, + 0xc6, 0xc4, 0xc5, 0x29, 0x0f, 0x1b, 0x19, 0x48, 0x96, 0x81, 0xb4, 0xbb, 0x9c, 0x2d, 0xce, 0x7d, + 0x3b, 0xdf, 0xc3, 0xe2, 0x14, 0x33, 0xf4, 0x2a, 0xcc, 0x71, 0xcf, 0xa7, 0x11, 0x27, 0x7e, 0xf8, + 0xd8, 0x57, 0x0b, 0xa8, 0x84, 0x6b, 0x23, 0xd9, 0x83, 0x08, 0x7d, 0x04, 0xb3, 0x23, 0x3f, 0x7a, + 0x24, 0x6e, 0x5f, 0x94, 0x8e, 0x5b, 0x16, 0xa9, 0xe0, 0xb1, 0x91, 0xf3, 0x04, 0x16, 0x4e, 0x61, + 0xd0, 0x35, 0xa8, 0xb4, 0x83, 0xb8, 0xaf, 0xfa, 0xc9, 0xc0, 0xea, 0x80, 0xae, 0x42, 0x29, 0x8a, + 0x55, 0x10, 0x03, 0x8b, 0x4f, 0xf4, 0x0e, 0xcc, 0xb4, 0xe2, 0xf6, 0x1e, 0xe5, 0x19, 0x13, 0x93, + 0xa1, 0xc7, 0x41, 0x25, 0x08, 0x67, 0x60, 0x27, 0x82, 0x85, 0x53, 0x3a, 0x64, 0x01, 0xb4, 0x82, + 0xb8, 0xdf, 0x21, 0xcc, 0xa3, 0xaa, 0xd0, 0x0a, 0xce, 0x49, 0x44, 0x4a, 0xbd, 0x60, 0x9f, 0x32, + 0x1d, 0x5e, 0x1d, 0x84, 0x34, 0x16, 0xe1, 0xe4, 0x04, 0x1b, 0x58, 0x1d, 0xc6, 0xe9, 0x97, 0x73, + 0xe9, 0x3b, 0x3e, 0xdc, 0x3c, 0x67, 0xa6, 0x11, 0x1e, 0x37, 0x84, 0x21, 0x29, 0x7c, 0xf3, 0xb2, + 0x55, 0xa0, 0xd0, 0x6a, 0x23, 0xd4, 0xc4, 0x78, 0x6a, 0xfb, 0x51, 0xa3, 0x38, 0x3f, 0x17, 0xc1, + 0xba, 0xd8, 0x10, 0x6d, 0xc2, 0x75, 0x1e, 0x70, 0xd2, 0x93, 0xbb, 0x8a, 0xb4, 0x7a, 0x99, 0x56, + 0x8f, 0xf1, 0x52, 0x9a, 0xd8, 0xd3, 0x01, 0x78, 0xba, 0x18, 0xfd, 0x66, 0xc0, 0xed, 0xa9, 0x9a, + 0x2d, 0xca, 0xb6, 0x39, 0x0d, 0x75, 0xbb, 0x7f, 0x70, 0x49, 0x75, 0xa7, 0xad, 0x65, 0xb6, 0xda, + 0x85, 0xdb, 0x48, 0x13, 0xfb, 0xc2, 0x20, 0xf8, 0x42, 0xad, 0xe3, 0xc1, 0x0b, 0x46, 0x14, 0xd7, + 0x29, 0xa7, 0x50, 0xb7, 0xbf, 0x3a, 0x9c, 0x99, 0x8d, 0xe2, 0x99, 0xd9, 0x70, 0x76, 0xa0, 0x7e, + 0xde, 0x73, 0x86, 0x96, 0xa0, 0xfc, 0x15, 0xf1, 0xb3, 0x17, 0x47, 0xaf, 0x3c, 0x29, 0x42, 0xaf, + 0x80, 0xf9, 0x48, 0x4e, 0xbd, 0xa4, 0x6b, 0xa4, 0xd4, 0x42, 0xe7, 0x57, 0x03, 0xae, 0x4f, 0x7d, + 0x67, 0xd0, 0x1d, 0x30, 0x9f, 0xd2, 0x36, 0x0f, 0x98, 0xee, 0xa2, 0xc9, 0x85, 0xfe, 0x48, 0xaa, + 0x36, 0x0a, 0x58, 0x83, 0xd0, 0x6d, 0xa8, 0x32, 0xb2, 0xef, 0x0e, 0x38, 0x55, 0xd9, 0xcf, 0x6d, + 0x14, 0xf0, 0x48, 0x22, 0x9c, 0xf9, 0x84, 0x33, 0xef, 0x40, 0xbf, 0x4e, 0x93, 0xce, 0x1e, 0x48, + 0x95, 0x70, 0xa6, 0x40, 0x6e, 0x15, 0xf4, 0xeb, 0xe6, 0x7c, 0x08, 0xa6, 0x0a, 0x85, 0xee, 0xe6, + 0xdb, 0xfa, 0xec, 0x0b, 0xa3, 0x57, 0x9d, 0x5a, 0x08, 0xa3, 0xbe, 0xfd, 0xb1, 0x08, 0xa6, 0xd2, + 0xfc, 0x8f, 0x1b, 0xfa, 0x5d, 0x30, 0x55, 0x3e, 0x7a, 0xa5, 0x9d, 0x5d, 0xd0, 0x57, 0x0e, 0x13, + 0xdb, 0x10, 0xef, 0x9c, 0xec, 0x06, 0xac, 0xe1, 0xe8, 0x61, 0x7e, 0x1d, 0x2a, 0xe2, 0x2e, 0xdf, + 0xce, 0x2f, 0x69, 0x5f, 0x63, 0xd3, 0xfc, 0x7e, 0xdc, 0x04, 0x53, 0xb1, 0x8d, 0xd6, 0xe1, 0x4a, + 0x94, 0x7b, 0xc1, 0x32, 0x5a, 0x96, 0xa6, 0x04, 0x50, 0x08, 0xcd, 0xed, 0xa4, 0x95, 0xbb, 0x76, + 0x74, 0x6c, 0x15, 0x9e, 0x1d, 0x5b, 0x85, 0xe7, 0xc7, 0x96, 0xf1, 0xc3, 0xd0, 0x32, 0x7e, 0x1f, + 0x5a, 0xc6, 0xe1, 0xd0, 0x32, 0x8e, 0x86, 0x96, 0xf1, 0xcf, 0xd0, 0x32, 0xfe, 0x1d, 0x5a, 0x85, + 0xe7, 0x43, 0xcb, 0xf8, 0xe9, 0xc4, 0x2a, 0x1c, 0x9d, 0x58, 0x85, 0x67, 0x27, 0x56, 0xe1, 0x9b, + 0xfc, 0xbf, 0xcd, 0x2d, 0x53, 0x3e, 0xbc, 0x77, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x26, 0xdf, + 0x3a, 0xdc, 0x59, 0x0b, 0x00, 0x00, } func (this *PrometheusResponse) Equal(that interface{}) bool { @@ -1579,7 +1579,7 @@ func (this *Vector) Equal(that interface{}) bool { return false } for i := range this.Samples { - if !this.Samples[i].Equal(that1.Samples[i]) { + if !this.Samples[i].Equal(&that1.Samples[i]) { return false } } @@ -1864,7 +1864,11 @@ func (this *Vector) GoString() string { s := make([]string, 0, 5) s = append(s, "&tripperware.Vector{") if this.Samples != nil { - s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") + vs := make([]*Sample, len(this.Samples)) + for i := range vs { + vs[i] = &this.Samples[i] + } + s = append(s, "Samples: "+fmt.Sprintf("%#v", vs)+",\n") } s = append(s, "}") return strings.Join(s, "") @@ -3281,9 +3285,9 @@ func (this *Vector) String() string { if this == nil { return "nil" } - repeatedStringForSamples := "[]*Sample{" + repeatedStringForSamples := "[]Sample{" for _, f := range this.Samples { - repeatedStringForSamples += strings.Replace(f.String(), "Sample", "Sample", 1) + "," + repeatedStringForSamples += strings.Replace(strings.Replace(f.String(), "Sample", "Sample", 1), `&`, ``, 1) + "," } repeatedStringForSamples += "}" s := strings.Join([]string{`&Vector{`, @@ -5098,7 +5102,7 @@ func (m *Vector) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Samples = append(m.Samples, &Sample{}) + m.Samples = append(m.Samples, Sample{}) if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } diff --git a/pkg/querier/tripperware/query.proto b/pkg/querier/tripperware/query.proto index c2eb585cb3f..908c1380fdf 100644 --- a/pkg/querier/tripperware/query.proto +++ b/pkg/querier/tripperware/query.proto @@ -95,7 +95,7 @@ message PrometheusQueryResult { } message Vector { - repeated Sample samples = 1; + repeated Sample samples = 1 [(gogoproto.nullable) = false]; } message Sample { diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index e126952c98c..dc669371be9 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -8,6 +8,7 @@ import ( "net/http" "testing" + "github.com/gogo/protobuf/proto" "github.com/stretchr/testify/require" "github.com/cortexproject/cortex/pkg/cortexpb" diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 939532e415e..f0e9b9275f3 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -58,10 +58,10 @@ type prometheusCodec struct { enableProtobuf bool } -func NewPrometheusCodec(sharded bool, c string, enableProtobuf bool) *prometheusCodec { //nolint:revive +func NewPrometheusCodec(sharded bool, compressionStr string, enableProtobuf bool) *prometheusCodec { //nolint:revive var compression Compression - if c == "gzip" { - compression = Compression(c) + if compressionStr == "gzip" || compressionStr == "snappy" { + compression = Compression(compressionStr) } else { compression = DisableCompression } diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go deleted file mode 100644 index 378baed7766..00000000000 --- a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/doc.go +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright 2017 Michal Witkowski. All Rights Reserved. -// See LICENSE for licensing terms. - -// -/* -logging is a "parent" package for gRPC logging middlewares. - -The gRPC logging middleware populates request-scoped data to `grpc_ctxtags.Tags` that relate to the current gRPC call -(e.g. service and method names). - -Once the gRPC logging middleware has added the gRPC specific Tags to the ctx they will then be written with the logs -that are made using the `ctx_logrus` or `ctx_zap` loggers. - -All logging middleware will emit a final log statement. It is based on the error returned by the handler function, -the gRPC status code, an error (if any) and it emit at a level controlled via `WithLevels`. - -This parent package - -This particular package is intended for use by other middleware, logging or otherwise. It contains interfaces that other -logging middlewares *could* share . This allows code to be shared between different implementations. - -Field names - -All field names of loggers follow the OpenTracing semantics definitions, with `grpc.` prefix if needed: -https://github.com/opentracing/specification/blob/master/semantic_conventions.md - -Implementations: - -* providers/logrus -* providers/zap -* providers/kit -* providers/zerolog - -See relevant packages below. -*/ -package logging diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go deleted file mode 100644 index 654223f2afd..00000000000 --- a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/interceptors.go +++ /dev/null @@ -1,135 +0,0 @@ -package logging - -import ( - "context" - "fmt" - "io" - "time" - - "google.golang.org/grpc" - - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" -) - -// extractFields returns all fields from tags. -func extractFields(tags tags.Tags) Fields { - var fields Fields - for k, v := range tags.Values() { - fields = append(fields, k, v) - } - return fields -} - -type reporter struct { - ctx context.Context - typ interceptors.GRPCType - service, method string - startCallLogged bool - opts *options - logger Logger - kind string -} - -func (c *reporter) logMessage(logger Logger, err error, msg string, duration time.Duration) { - code := c.opts.codeFunc(err) - logger = logger.With("grpc.code", code.String()) - if err != nil { - logger = logger.With("grpc.error", fmt.Sprintf("%v", err)) - } - logger = logger.With(extractFields(tags.Extract(c.ctx))...) - logger.With(c.opts.durationFieldFunc(duration)...).Log(c.opts.levelFunc(code), msg) -} - -func (c *reporter) PostCall(err error, duration time.Duration) { - switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { - case LogFinishCall, LogStartAndFinishCall: - if err == io.EOF { - err = nil - } - c.logMessage(c.logger, err, "finished call", duration) - default: - return - } -} - -func (c *reporter) PostMsgSend(_ interface{}, err error, duration time.Duration) { - if c.startCallLogged { - return - } - switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { - case LogStartAndFinishCall: - c.startCallLogged = true - c.logMessage(c.logger, err, "started call", duration) - } -} - -func (c *reporter) PostMsgReceive(_ interface{}, err error, duration time.Duration) { - if c.startCallLogged { - return - } - switch c.opts.shouldLog(interceptors.FullMethod(c.service, c.method), err) { - case LogStartAndFinishCall: - c.startCallLogged = true - c.logMessage(c.logger, err, "started call", duration) - } -} - -type reportable struct { - opts *options - logger Logger -} - -func (r *reportable) ServerReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { - return r.reporter(ctx, typ, service, method, KindServerFieldValue) -} - -func (r *reportable) ClientReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { - return r.reporter(ctx, typ, service, method, KindClientFieldValue) -} - -func (r *reportable) reporter(ctx context.Context, typ interceptors.GRPCType, service string, method string, kind string) (interceptors.Reporter, context.Context) { - fields := commonFields(kind, typ, service, method) - fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) - if d, ok := ctx.Deadline(); ok { - fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) - } - return &reporter{ - ctx: ctx, - typ: typ, - service: service, - method: method, - startCallLogged: false, - opts: r.opts, - logger: r.logger.With(fields...), - kind: kind, - }, ctx -} - -// UnaryClientInterceptor returns a new unary client interceptor that optionally logs the execution of external gRPC calls. -// Logger will use all tags (from tags package) available in current context as fields. -func UnaryClientInterceptor(logger Logger, opts ...Option) grpc.UnaryClientInterceptor { - o := evaluateClientOpt(opts) - return interceptors.UnaryClientInterceptor(&reportable{logger: logger, opts: o}) -} - -// StreamClientInterceptor returns a new streaming client interceptor that optionally logs the execution of external gRPC calls. -// Logger will use all tags (from tags package) available in current context as fields. -func StreamClientInterceptor(logger Logger, opts ...Option) grpc.StreamClientInterceptor { - o := evaluateClientOpt(opts) - return interceptors.StreamClientInterceptor(&reportable{logger: logger, opts: o}) -} - -// UnaryServerInterceptor returns a new unary server interceptors that optionally logs endpoint handling. -// Logger will use all tags (from tags package) available in current context as fields. -func UnaryServerInterceptor(logger Logger, opts ...Option) grpc.UnaryServerInterceptor { - o := evaluateServerOpt(opts) - return interceptors.UnaryServerInterceptor(&reportable{logger: logger, opts: o}) -} - -// StreamServerInterceptor returns a new stream server interceptors that optionally logs endpoint handling. -// Logger will use all tags (from tags package) available in current context as fields. -func StreamServerInterceptor(logger Logger, opts ...Option) grpc.StreamServerInterceptor { - o := evaluateServerOpt(opts) - return interceptors.StreamServerInterceptor(&reportable{logger: logger, opts: o}) -} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go deleted file mode 100644 index 67d6861bda1..00000000000 --- a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/logging.go +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2017 Michal Witkowski. All Rights Reserved. -// See LICENSE for licensing terms. - -package logging - -import ( - "context" - "io" - - "github.com/golang/protobuf/proto" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" - - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" -) - -// Decision defines rules for enabling start and end of logging. -type Decision int - -const ( - // NoLogCall - Logging is disabled. - NoLogCall Decision = iota - // LogFinishCall - Only finish logs of request is enabled. - LogFinishCall - // LogStartAndFinishCall - Logging of start and end of request is enabled. - LogStartAndFinishCall -) - -var ( - // SystemTag is tag representing an event inside gRPC call. - SystemTag = []string{"protocol", "grpc"} - // ComponentFieldKey is a tag representing the client/server that is calling. - ComponentFieldKey = "grpc.component" - KindServerFieldValue = "server" - KindClientFieldValue = "client" - ServiceFieldKey = "grpc.service" - MethodFieldKey = "grpc.method" - MethodTypeFieldKey = "grpc.method_type" -) - -func commonFields(kind string, typ interceptors.GRPCType, service string, method string) Fields { - return Fields{ - SystemTag[0], SystemTag[1], - ComponentFieldKey, kind, - ServiceFieldKey, service, - MethodFieldKey, method, - MethodTypeFieldKey, string(typ), - } -} - -// Fields represents logging fields. It has to have even number of elements (pairs). -type Fields []string - -// ErrorToCode function determines the error code of an error -// This makes using custom errors with grpc middleware easier -type ErrorToCode func(err error) codes.Code - -func DefaultErrorToCode(err error) codes.Code { - return status.Code(err) -} - -// Decider function defines rules for suppressing any interceptor logs -type Decider func(fullMethodName string, err error) Decision - -// DefaultDeciderMethod is the default implementation of decider to see if you should log the call -// by default this if always true so all calls are logged -func DefaultDeciderMethod(_ string, _ error) Decision { - return LogStartAndFinishCall -} - -// ServerPayloadLoggingDecider is a user-provided function for deciding whether to log the server-side -// request/response payloads -type ServerPayloadLoggingDecider func(ctx context.Context, fullMethodName string, servingObject interface{}) bool - -// ClientPayloadLoggingDecider is a user-provided function for deciding whether to log the client-side -// request/response payloads -type ClientPayloadLoggingDecider func(ctx context.Context, fullMethodName string) bool - -// JsonPbMarshaller is a marshaller that serializes protobuf messages. -type JsonPbMarshaler interface { - Marshal(out io.Writer, pb proto.Message) error -} - -// Logger is unified interface that we used for all our interceptors. Official implementations are available under -// provider/ directory as separate modules. -type Logger interface { - // Log logs the fields for given log level. We can assume users (middleware library) will put fields in pairs and - // those will be unique. - Log(Level, string) - // With returns mockLogger with given fields appended. We can assume users (middleware library) will put fields in pairs - // and those will be unique. - With(fields ...string) Logger -} - -// Level represents logging level. -type Level string - -const ( - DEBUG = Level("debug") - INFO = Level("info") - WARNING = Level("warning") - ERROR = Level("error") -) - -// CodeToLevel function defines the mapping between gRPC return codes and interceptor log level. -type CodeToLevel func(code codes.Code) Level - -// DefaultServerCodeToLevel is the helper mapper that maps gRPC return codes to log levels for server side. -func DefaultServerCodeToLevel(code codes.Code) Level { - switch code { - case codes.OK, codes.NotFound, codes.Canceled, codes.AlreadyExists, codes.InvalidArgument, codes.Unauthenticated: - return INFO - - case codes.DeadlineExceeded, codes.PermissionDenied, codes.ResourceExhausted, codes.FailedPrecondition, codes.Aborted, - codes.OutOfRange, codes.Unavailable: - return WARNING - - case codes.Unknown, codes.Unimplemented, codes.Internal, codes.DataLoss: - return ERROR - - default: - return ERROR - } -} - -// DefaultClientCodeToLevel is the helper mapper that maps gRPC return codes to log levels for client side. -func DefaultClientCodeToLevel(code codes.Code) Level { - switch code { - case codes.OK, codes.Canceled, codes.InvalidArgument, codes.NotFound, codes.AlreadyExists, codes.ResourceExhausted, - codes.FailedPrecondition, codes.Aborted, codes.OutOfRange: - return DEBUG - case codes.Unknown, codes.DeadlineExceeded, codes.PermissionDenied, codes.Unauthenticated: - return INFO - case codes.Unimplemented, codes.Internal, codes.Unavailable, codes.DataLoss: - return WARNING - default: - return INFO - } -} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go deleted file mode 100644 index b1aa328cf25..00000000000 --- a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/options.go +++ /dev/null @@ -1,94 +0,0 @@ -package logging - -import ( - "fmt" - "time" -) - -var ( - defaultOptions = &options{ - shouldLog: DefaultDeciderMethod, - codeFunc: DefaultErrorToCode, - durationFieldFunc: DefaultDurationToFields, - // levelFunc depends if it's client or server. - levelFunc: nil, - } -) - -type options struct { - levelFunc CodeToLevel - shouldLog Decider - codeFunc ErrorToCode - durationFieldFunc DurationToFields -} - -type Option func(*options) - -// DurationToFields function defines how to produce duration fields for logging. -type DurationToFields func(duration time.Duration) Fields - -func evaluateServerOpt(opts []Option) *options { - optCopy := &options{} - *optCopy = *defaultOptions - optCopy.levelFunc = DefaultServerCodeToLevel - for _, o := range opts { - o(optCopy) - } - return optCopy -} - -func evaluateClientOpt(opts []Option) *options { - optCopy := &options{} - *optCopy = *defaultOptions - optCopy.levelFunc = DefaultClientCodeToLevel - for _, o := range opts { - o(optCopy) - } - return optCopy -} - -// WithDecider customizes the function for deciding if the gRPC interceptor logs should log. -func WithDecider(f Decider) Option { - return func(o *options) { - o.shouldLog = f - } -} - -// WithLevels customizes the function for mapping gRPC return codes and interceptor log level statements. -func WithLevels(f CodeToLevel) Option { - return func(o *options) { - o.levelFunc = f - } -} - -// WithCodes customizes the function for mapping errors to error codes. -func WithCodes(f ErrorToCode) Option { - return func(o *options) { - o.codeFunc = f - } -} - -// WithDurationField customizes the function for mapping request durations to log fields. -func WithDurationField(f DurationToFields) Option { - return func(o *options) { - o.durationFieldFunc = f - } -} - -// DefaultDurationToFields is the default implementation of converting request duration to a field. -var DefaultDurationToFields = DurationToTimeMillisFields - -// DurationToTimeMillisFields converts the duration to milliseconds and uses the key `grpc.time_ms`. -func DurationToTimeMillisFields(duration time.Duration) Fields { - return Fields{"grpc.time_ms", fmt.Sprintf("%v", durationToMilliseconds(duration))} -} - -// DurationToDurationField uses a Duration field to log the request duration -// and leaves it up to Log's encoder settings to determine how that is output. -func DurationToDurationField(duration time.Duration) Fields { - return Fields{"grpc.duration", duration.String()} -} - -func durationToMilliseconds(duration time.Duration) float32 { - return float32(duration.Nanoseconds()/1000) / 1000 -} diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go deleted file mode 100644 index ea5e2ceecc0..00000000000 --- a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging/payload.go +++ /dev/null @@ -1,152 +0,0 @@ -package logging - -import ( - "bytes" - "context" - "fmt" - "time" - - "github.com/golang/protobuf/jsonpb" - "github.com/golang/protobuf/proto" - "google.golang.org/grpc" - - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" -) - -var ( - // JsonPbMarshaller is the marshaller used for serializing protobuf messages. - // If needed, this variable can be reassigned with a different marshaller with the same Marshal() signature. - JsonPbMarshaller JsonPbMarshaler = &jsonpb.Marshaler{} -) - -type serverPayloadReporter struct { - ctx context.Context - logger Logger -} - -func (c *serverPayloadReporter) PostCall(error, time.Duration) {} - -func (c *serverPayloadReporter) PostMsgSend(req interface{}, err error, duration time.Duration) { - if err != nil { - return - } - logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) - // For server send message is the response. - logProtoMessageAsJson(logger.With("grpc.send.duration", duration.String()), req, "grpc.response.content", "response payload logged as grpc.response.content field") -} - -func (c *serverPayloadReporter) PostMsgReceive(reply interface{}, err error, duration time.Duration) { - if err != nil { - return - } - logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) - // For server recv message is the request. - logProtoMessageAsJson(logger.With("grpc.recv.duration", duration.String()), reply, "grpc.request.content", "request payload logged as grpc.request.content field") -} - -type clientPayloadReporter struct { - ctx context.Context - logger Logger -} - -func (c *clientPayloadReporter) PostCall(error, time.Duration) {} - -func (c *clientPayloadReporter) PostMsgSend(req interface{}, err error, duration time.Duration) { - if err != nil { - return - } - logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) - logProtoMessageAsJson(logger.With("grpc.send.duration", duration.String()), req, "grpc.request.content", "request payload logged as grpc.request.content field") -} - -func (c *clientPayloadReporter) PostMsgReceive(reply interface{}, err error, duration time.Duration) { - if err != nil { - return - } - logger := c.logger.With(extractFields(tags.Extract(c.ctx))...) - logProtoMessageAsJson(logger.With("grpc.recv.duration", duration.String()), reply, "grpc.response.content", "response payload logged as grpc.response.content field") -} - -type payloadReportable struct { - clientDecider ClientPayloadLoggingDecider - serverDecider ServerPayloadLoggingDecider - logger Logger -} - -func (r *payloadReportable) ServerReporter(ctx context.Context, req interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { - if !r.serverDecider(ctx, interceptors.FullMethod(service, method), req) { - return interceptors.NoopReporter{}, ctx - } - fields := commonFields(KindServerFieldValue, typ, service, method) - fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) - if d, ok := ctx.Deadline(); ok { - fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) - } - return &serverPayloadReporter{ - ctx: ctx, - logger: r.logger.With(fields...), - }, ctx -} -func (r *payloadReportable) ClientReporter(ctx context.Context, _ interface{}, typ interceptors.GRPCType, service string, method string) (interceptors.Reporter, context.Context) { - if !r.clientDecider(ctx, interceptors.FullMethod(service, method)) { - return interceptors.NoopReporter{}, ctx - } - fields := commonFields(KindClientFieldValue, typ, service, method) - fields = append(fields, "grpc.start_time", time.Now().Format(time.RFC3339)) - if d, ok := ctx.Deadline(); ok { - fields = append(fields, "grpc.request.deadline", d.Format(time.RFC3339)) - } - return &clientPayloadReporter{ - ctx: ctx, - logger: r.logger.With(fields...), - }, ctx -} - -// PayloadUnaryServerInterceptor returns a new unary server interceptors that logs the payloads of requests on INFO level. -// Logger tags will be used from tags context. -func PayloadUnaryServerInterceptor(logger Logger, decider ServerPayloadLoggingDecider) grpc.UnaryServerInterceptor { - return interceptors.UnaryServerInterceptor(&payloadReportable{logger: logger, serverDecider: decider}) -} - -// PayloadStreamServerInterceptor returns a new server server interceptors that logs the payloads of requests on INFO level. -// Logger tags will be used from tags context. -func PayloadStreamServerInterceptor(logger Logger, decider ServerPayloadLoggingDecider) grpc.StreamServerInterceptor { - return interceptors.StreamServerInterceptor(&payloadReportable{logger: logger, serverDecider: decider}) -} - -// PayloadUnaryClientInterceptor returns a new unary client interceptor that logs the paylods of requests and responses on INFO level. -// Logger tags will be used from tags context. -func PayloadUnaryClientInterceptor(logger Logger, decider ClientPayloadLoggingDecider) grpc.UnaryClientInterceptor { - return interceptors.UnaryClientInterceptor(&payloadReportable{logger: logger, clientDecider: decider}) -} - -// PayloadStreamClientInterceptor returns a new streaming client interceptor that logs the paylods of requests and responses on INFO level. -// Logger tags will be used from tags context. -func PayloadStreamClientInterceptor(logger Logger, decider ClientPayloadLoggingDecider) grpc.StreamClientInterceptor { - return interceptors.StreamClientInterceptor(&payloadReportable{logger: logger, clientDecider: decider}) -} - -func logProtoMessageAsJson(logger Logger, pbMsg interface{}, key string, msg string) { - if p, ok := pbMsg.(proto.Message); ok { - payload, err := (&jsonpbObjectMarshaler{pb: p}).marshalJSON() - if err != nil { - logger = logger.With(key, err.Error()) - } else { - logger = logger.With(key, string(payload)) - } - logger.Log(INFO, msg) - } -} - -type jsonpbObjectMarshaler struct { - pb proto.Message -} - -func (j *jsonpbObjectMarshaler) marshalJSON() ([]byte, error) { - b := &bytes.Buffer{} - if err := JsonPbMarshaller.Marshal(b, j.pb); err != nil { - return nil, fmt.Errorf("jsonpb serializer failed: %v", err) - } - return b.Bytes(), nil -} diff --git a/vendor/github.com/prometheus/common/promlog/log.go b/vendor/github.com/prometheus/common/promlog/log.go deleted file mode 100644 index 3ac7b3fdf12..00000000000 --- a/vendor/github.com/prometheus/common/promlog/log.go +++ /dev/null @@ -1,189 +0,0 @@ -// Copyright 2017 The Prometheus Authors -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package promlog defines standardised ways to initialize Go kit loggers -// across Prometheus components. -// It should typically only ever be imported by main packages. -package promlog - -import ( - "fmt" - "os" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" -) - -var ( - // This timestamp format differs from RFC3339Nano by using .000 instead - // of .999999999 which changes the timestamp from 9 variable to 3 fixed - // decimals (.130 instead of .130987456). - timestampFormat = log.TimestampFormat( - func() time.Time { return time.Now().UTC() }, - "2006-01-02T15:04:05.000Z07:00", - ) -) - -// AllowedLevel is a settable identifier for the minimum level a log entry -// must be have. -type AllowedLevel struct { - s string - o level.Option -} - -func (l *AllowedLevel) UnmarshalYAML(unmarshal func(interface{}) error) error { - var s string - type plain string - if err := unmarshal((*plain)(&s)); err != nil { - return err - } - if s == "" { - return nil - } - lo := &AllowedLevel{} - if err := lo.Set(s); err != nil { - return err - } - *l = *lo - return nil -} - -func (l *AllowedLevel) String() string { - return l.s -} - -// Set updates the value of the allowed level. -func (l *AllowedLevel) Set(s string) error { - switch s { - case "debug": - l.o = level.AllowDebug() - case "info": - l.o = level.AllowInfo() - case "warn": - l.o = level.AllowWarn() - case "error": - l.o = level.AllowError() - default: - return fmt.Errorf("unrecognized log level %q", s) - } - l.s = s - return nil -} - -// AllowedFormat is a settable identifier for the output format that the logger can have. -type AllowedFormat struct { - s string -} - -func (f *AllowedFormat) String() string { - return f.s -} - -// Set updates the value of the allowed format. -func (f *AllowedFormat) Set(s string) error { - switch s { - case "logfmt", "json": - f.s = s - default: - return fmt.Errorf("unrecognized log format %q", s) - } - return nil -} - -// Config is a struct containing configurable settings for the logger -type Config struct { - Level *AllowedLevel - Format *AllowedFormat -} - -// New returns a new leveled oklog logger. Each logged line will be annotated -// with a timestamp. The output always goes to stderr. -func New(config *Config) log.Logger { - if config.Format != nil && config.Format.s == "json" { - return NewWithLogger(log.NewJSONLogger(log.NewSyncWriter(os.Stderr)), config) - } - - return NewWithLogger(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), config) -} - -// NewWithLogger returns a new leveled oklog logger with a custom log.Logger. -// Each logged line will be annotated with a timestamp. -func NewWithLogger(l log.Logger, config *Config) log.Logger { - if config.Level != nil { - l = log.With(l, "ts", timestampFormat, "caller", log.Caller(5)) - l = level.NewFilter(l, config.Level.o) - } else { - l = log.With(l, "ts", timestampFormat, "caller", log.DefaultCaller) - } - return l -} - -// NewDynamic returns a new leveled logger. Each logged line will be annotated -// with a timestamp. The output always goes to stderr. Some properties can be -// changed, like the level. -func NewDynamic(config *Config) *logger { - if config.Format != nil && config.Format.s == "json" { - return NewDynamicWithLogger(log.NewJSONLogger(log.NewSyncWriter(os.Stderr)), config) - } - - return NewDynamicWithLogger(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), config) -} - -// NewDynamicWithLogger returns a new leveled logger with a custom io.Writer. -// Each logged line will be annotated with a timestamp. -// Some properties can be changed, like the level. -func NewDynamicWithLogger(l log.Logger, config *Config) *logger { - lo := &logger{ - base: l, - leveled: l, - } - - if config.Level != nil { - lo.SetLevel(config.Level) - } - - return lo -} - -type logger struct { - base log.Logger - leveled log.Logger - currentLevel *AllowedLevel - mtx sync.Mutex -} - -// Log implements logger.Log. -func (l *logger) Log(keyvals ...interface{}) error { - l.mtx.Lock() - defer l.mtx.Unlock() - return l.leveled.Log(keyvals...) -} - -// SetLevel changes the log level. -func (l *logger) SetLevel(lvl *AllowedLevel) { - l.mtx.Lock() - defer l.mtx.Unlock() - if lvl == nil { - l.leveled = log.With(l.base, "ts", timestampFormat, "caller", log.DefaultCaller) - l.currentLevel = nil - return - } - - if l.currentLevel != nil && l.currentLevel.s != lvl.s { - _ = l.base.Log("msg", "Log level changed", "prev", l.currentLevel, "current", lvl) - } - l.currentLevel = lvl - l.leveled = level.NewFilter(log.With(l.base, "ts", timestampFormat, "caller", log.Caller(5)), lvl.o) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/api/api.go b/vendor/github.com/thanos-io/thanos/pkg/api/api.go deleted file mode 100644 index 6be81c8b158..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/api/api.go +++ /dev/null @@ -1,278 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -// Copyright 2016 The Prometheus Authors -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// This package is a modified copy from -// github.com/prometheus/prometheus/web/api/v1@2121b4628baa7d9d9406aa468712a6a332e77aff. - -package api - -import ( - "encoding/json" - "fmt" - "net/http" - "os" - "runtime" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/klauspost/compress/gzhttp" - "github.com/opentracing/opentracing-go" - "github.com/prometheus/common/route" - "github.com/prometheus/common/version" - - extpromhttp "github.com/thanos-io/thanos/pkg/extprom/http" - "github.com/thanos-io/thanos/pkg/logging" - "github.com/thanos-io/thanos/pkg/server/http/middleware" - "github.com/thanos-io/thanos/pkg/tracing" -) - -type status string - -const ( - StatusSuccess status = "success" - StatusError status = "error" -) - -type ErrorType string - -const ( - ErrorNone ErrorType = "" - ErrorTimeout ErrorType = "timeout" - ErrorCanceled ErrorType = "canceled" - ErrorExec ErrorType = "execution" - ErrorBadData ErrorType = "bad_data" - ErrorInternal ErrorType = "internal" -) - -var corsHeaders = map[string]string{ - "Access-Control-Allow-Headers": "Accept, Accept-Encoding, Authorization, Content-Type, Origin", - "Access-Control-Allow-Methods": "GET, OPTIONS", - "Access-Control-Allow-Origin": "*", - "Access-Control-Expose-Headers": "Date", -} - -// ThanosVersion contains build information about Thanos. -type ThanosVersion struct { - Version string `json:"version"` - Revision string `json:"revision"` - Branch string `json:"branch"` - BuildUser string `json:"buildUser"` - BuildDate string `json:"buildDate"` - GoVersion string `json:"goVersion"` -} - -var BuildInfo = &ThanosVersion{ - Version: version.Version, - Revision: version.Revision, - Branch: version.Branch, - BuildUser: version.BuildUser, - BuildDate: version.BuildDate, - GoVersion: version.GoVersion, -} - -type ApiError struct { - Typ ErrorType - Err error -} - -func (e *ApiError) Error() string { - return fmt.Sprintf("%s: %s", e.Typ, e.Err) -} - -// RuntimeInfo contains runtime information about Thanos. -type RuntimeInfo struct { - StartTime time.Time `json:"startTime"` - CWD string `json:"CWD"` - GoroutineCount int `json:"goroutineCount"` - GOMAXPROCS int `json:"GOMAXPROCS"` - GOGC string `json:"GOGC"` - GODEBUG string `json:"GODEBUG"` -} - -// RuntimeInfoFn returns updated runtime information about Thanos. -type RuntimeInfoFn func() RuntimeInfo - -type response struct { - Status status `json:"status"` - Data interface{} `json:"data,omitempty"` - ErrorType ErrorType `json:"errorType,omitempty"` - Error string `json:"error,omitempty"` - Warnings []string `json:"warnings,omitempty"` -} - -// SetCORS enables cross-site script calls. -func SetCORS(w http.ResponseWriter) { - for h, v := range corsHeaders { - w.Header().Set(h, v) - } -} - -type ApiFunc func(r *http.Request) (interface{}, []error, *ApiError, func()) - -type BaseAPI struct { - logger log.Logger - flagsMap map[string]string - runtimeInfo RuntimeInfoFn - buildInfo *ThanosVersion - Now func() time.Time - disableCORS bool -} - -// NewBaseAPI returns a new initialized BaseAPI type. -func NewBaseAPI(logger log.Logger, disableCORS bool, flagsMap map[string]string) *BaseAPI { - - return &BaseAPI{ - logger: logger, - flagsMap: flagsMap, - runtimeInfo: GetRuntimeInfoFunc(logger), - buildInfo: BuildInfo, - disableCORS: disableCORS, - Now: time.Now, - } -} - -// Register registers the common API endpoints. -func (api *BaseAPI) Register(r *route.Router, tracer opentracing.Tracer, logger log.Logger, ins extpromhttp.InstrumentationMiddleware, logMiddleware *logging.HTTPServerMiddleware) { - instr := GetInstr(tracer, logger, ins, logMiddleware, api.disableCORS) - - r.Options("/*path", instr("options", api.options)) - - r.Get("/status/flags", instr("status_flags", api.flags)) - r.Get("/status/runtimeinfo", instr("status_runtime", api.serveRuntimeInfo)) - r.Get("/status/buildinfo", instr("status_build", api.serveBuildInfo)) -} - -func (api *BaseAPI) options(r *http.Request) (interface{}, []error, *ApiError, func()) { - return nil, nil, nil, func() {} -} - -func (api *BaseAPI) flags(r *http.Request) (interface{}, []error, *ApiError, func()) { - return api.flagsMap, nil, nil, func() {} -} - -func (api *BaseAPI) serveRuntimeInfo(r *http.Request) (interface{}, []error, *ApiError, func()) { - return api.runtimeInfo(), nil, nil, func() {} -} - -func (api *BaseAPI) serveBuildInfo(r *http.Request) (interface{}, []error, *ApiError, func()) { - return api.buildInfo, nil, nil, func() {} -} - -func GetRuntimeInfoFunc(logger log.Logger) RuntimeInfoFn { - CWD, err := os.Getwd() - if err != nil { - CWD = "" - level.Warn(logger).Log("msg", "failed to retrieve current working directory", "err", err) - } - - birth := time.Now() - - return func() RuntimeInfo { - return RuntimeInfo{ - StartTime: birth, - CWD: CWD, - GoroutineCount: runtime.NumGoroutine(), - GOMAXPROCS: runtime.GOMAXPROCS(0), - GOGC: os.Getenv("GOGC"), - GODEBUG: os.Getenv("GODEBUG"), - } - } -} - -type InstrFunc func(name string, f ApiFunc) http.HandlerFunc - -// GetInstr returns a http HandlerFunc with the instrumentation middleware. -func GetInstr( - tracer opentracing.Tracer, - logger log.Logger, - ins extpromhttp.InstrumentationMiddleware, - logMiddleware *logging.HTTPServerMiddleware, - disableCORS bool, -) InstrFunc { - instr := func(name string, f ApiFunc) http.HandlerFunc { - hf := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - if !disableCORS { - SetCORS(w) - } - if data, warnings, err, releaseResources := f(r); err != nil { - RespondError(w, err, data) - releaseResources() - } else if data != nil { - Respond(w, data, warnings) - releaseResources() - } else { - w.WriteHeader(http.StatusNoContent) - releaseResources() - } - }) - - return tracing.HTTPMiddleware(tracer, name, logger, - ins.NewHandler(name, - gzhttp.GzipHandler( - middleware.RequestID( - logMiddleware.HTTPMiddleware(name, hf), - ), - ), - ), - ) - } - return instr -} - -func Respond(w http.ResponseWriter, data interface{}, warnings []error) { - w.Header().Set("Content-Type", "application/json") - if len(warnings) > 0 { - w.Header().Set("Cache-Control", "no-store") - } - w.WriteHeader(http.StatusOK) - - resp := &response{ - Status: StatusSuccess, - Data: data, - } - for _, warn := range warnings { - resp.Warnings = append(resp.Warnings, warn.Error()) - } - _ = json.NewEncoder(w).Encode(resp) -} - -func RespondError(w http.ResponseWriter, apiErr *ApiError, data interface{}) { - w.Header().Set("Content-Type", "application/json") - w.Header().Set("Cache-Control", "no-store") - - var code int - switch apiErr.Typ { - case ErrorBadData: - code = http.StatusBadRequest - case ErrorExec: - code = 422 - case ErrorCanceled, ErrorTimeout: - code = http.StatusServiceUnavailable - case ErrorInternal: - code = http.StatusInternalServerError - default: - code = http.StatusInternalServerError - } - w.WriteHeader(code) - - _ = json.NewEncoder(w).Encode(&response{ - Status: StatusError, - ErrorType: apiErr.Typ, - Error: apiErr.Err.Error(), - Data: data, - }) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go b/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go deleted file mode 100644 index 44387d395f4..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/logging/grpc.go +++ /dev/null @@ -1,207 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package logging - -import ( - "fmt" - "math/rand" - "sort" - "strings" - "time" - - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors" - grpc_logging "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging" - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" - "github.com/oklog/ulid" - "google.golang.org/grpc/status" - "gopkg.in/yaml.v2" -) - -// NewRequestConfig parses the string into a req logging config structure. -// Raise an error if unmarshalling is not possible, or values are not valid. -func NewRequestConfig(configYAML []byte) (*RequestConfig, error) { - reqLogConfig := &RequestConfig{} - if err := yaml.UnmarshalStrict(configYAML, reqLogConfig); err != nil { - return nil, err - } - return reqLogConfig, nil -} - -// checkOptionsConfigEmpty checks if the OptionsConfig struct is empty and valid. -// If invalid combination is present, return an error. -func checkOptionsConfigEmpty(optcfg OptionsConfig) (bool, error) { - if optcfg.Level == "" && !optcfg.Decision.LogEnd && !optcfg.Decision.LogStart { - return true, nil - } - if optcfg.Level == "" && (optcfg.Decision.LogStart || optcfg.Decision.LogEnd) { - return false, fmt.Errorf("level field is empty") - } - return false, nil -} - -// fillGlobalOptionConfig configures all the method to have global config for logging. -func fillGlobalOptionConfig(reqLogConfig *RequestConfig, isgRPC bool) (string, bool, bool, error) { - globalLevel := "ERROR" - globalStart, globalEnd := false, false - - globalOptionConfig := reqLogConfig.Options - isEmpty, err := checkOptionsConfigEmpty(globalOptionConfig) - - // If the decision for logging is enabled with empty level field. - if err != nil { - return "", false, false, err - } - if !isEmpty { - globalLevel = globalOptionConfig.Level - globalStart = globalOptionConfig.Decision.LogStart - globalEnd = globalOptionConfig.Decision.LogEnd - } - - protocolOptionConfig := reqLogConfig.HTTP.Options - if isgRPC { - // gRPC config overrides the global config. - protocolOptionConfig = reqLogConfig.GRPC.Options - } - - isEmpty, err = checkOptionsConfigEmpty(protocolOptionConfig) - // If the decision for logging is enabled with empty level field. - if err != nil { - return "", false, false, err - } - - if !isEmpty { - globalLevel = protocolOptionConfig.Level - globalStart = protocolOptionConfig.Decision.LogStart - globalEnd = protocolOptionConfig.Decision.LogEnd - } - return globalLevel, globalStart, globalEnd, nil -} - -// getGRPCLoggingOption returns the logging ENUM based on logStart and logEnd values. -func getGRPCLoggingOption(logStart, logEnd bool) (grpc_logging.Decision, error) { - if !logStart && !logEnd { - return grpc_logging.NoLogCall, nil - } - if !logStart && logEnd { - return grpc_logging.LogFinishCall, nil - } - if logStart && logEnd { - return grpc_logging.LogStartAndFinishCall, nil - } - return -1, fmt.Errorf("log start call is not supported") -} - -// validateLevel validates the list of level entries. -// Raise an error if empty or log level not in uppercase. -func validateLevel(level string) error { - if level == "" { - return fmt.Errorf("level field in YAML file is empty") - } - if level == "INFO" || level == "DEBUG" || level == "ERROR" || level == "WARNING" { - return nil - } - return fmt.Errorf("the format of level is invalid. Expected INFO/DEBUG/ERROR/WARNING, got this %v", level) -} - -// NewGRPCOption adds in the config options and returns tags for logging middleware. -func NewGRPCOption(configYAML []byte) ([]tags.Option, []grpc_logging.Option, error) { - - // Configure tagOpts and logOpts. - tagOpts := []tags.Option{ - tags.WithFieldExtractor(func(_ string, req interface{}) map[string]string { - tagMap := tags.TagBasedRequestFieldExtractor("request-id")("", req) - // If a request-id exists for a given request. - if tagMap != nil { - if _, ok := tagMap["request-id"]; ok { - return tagMap - } - } - entropy := ulid.Monotonic(rand.New(rand.NewSource(time.Now().UnixNano())), 0) - reqID := ulid.MustNew(ulid.Timestamp(time.Now()), entropy) - tagMap = make(map[string]string) - tagMap["request-id"] = reqID.String() - return tagMap - }), - } - logOpts := []grpc_logging.Option{ - grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { - return grpc_logging.NoLogCall - }), - grpc_logging.WithLevels(DefaultCodeToLevelGRPC), - } - - // Unmarshal YAML. - // if req logging is disabled. - if len(configYAML) == 0 { - return tagOpts, logOpts, nil - } - - reqLogConfig, err := NewRequestConfig(configYAML) - // If unmarshalling is an issue. - if err != nil { - return tagOpts, logOpts, err - } - - globalLevel, globalStart, globalEnd, err := fillGlobalOptionConfig(reqLogConfig, true) - // If global options have invalid entries. - if err != nil { - return tagOpts, logOpts, err - } - - // If the level entry does not matches our entries. - if err := validateLevel(globalLevel); err != nil { - return tagOpts, logOpts, err - } - - // If the combination is valid, use them, otherwise return error. - reqLogDecision, err := getGRPCLoggingOption(globalStart, globalEnd) - if err != nil { - return tagOpts, logOpts, err - } - - if len(reqLogConfig.GRPC.Config) == 0 { - logOpts = []grpc_logging.Option{ - grpc_logging.WithDecider(func(_ string, err error) grpc_logging.Decision { - - runtimeLevel := grpc_logging.DefaultServerCodeToLevel(status.Code(err)) - for _, lvl := range MapAllowedLevels[globalLevel] { - if string(runtimeLevel) == strings.ToLower(lvl) { - return reqLogDecision - } - } - return grpc_logging.NoLogCall - }), - grpc_logging.WithLevels(DefaultCodeToLevelGRPC), - } - return tagOpts, logOpts, nil - } - - logOpts = []grpc_logging.Option{ - grpc_logging.WithLevels(DefaultCodeToLevelGRPC), - } - - methodNameSlice := []string{} - - for _, eachConfig := range reqLogConfig.GRPC.Config { - eachConfigMethodName := interceptors.FullMethod(eachConfig.Service, eachConfig.Method) - methodNameSlice = append(methodNameSlice, eachConfigMethodName) - } - - logOpts = append(logOpts, []grpc_logging.Option{ - grpc_logging.WithDecider(func(runtimeMethodName string, err error) grpc_logging.Decision { - - idx := sort.SearchStrings(methodNameSlice, runtimeMethodName) - if idx < len(methodNameSlice) && methodNameSlice[idx] == runtimeMethodName { - runtimeLevel := grpc_logging.DefaultServerCodeToLevel(status.Code(err)) - for _, lvl := range MapAllowedLevels[globalLevel] { - if string(runtimeLevel) == strings.ToLower(lvl) { - return reqLogDecision - } - } - } - return grpc_logging.NoLogCall - }), - }...) - return tagOpts, logOpts, nil -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/http.go b/vendor/github.com/thanos-io/thanos/pkg/logging/http.go deleted file mode 100644 index 36c4395f763..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/logging/http.go +++ /dev/null @@ -1,195 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package logging - -import ( - "fmt" - "net" - "sort" - "strings" - - "net/http" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - - httputil "github.com/thanos-io/thanos/pkg/server/http" -) - -type HTTPServerMiddleware struct { - opts *options - logger log.Logger -} - -func (m *HTTPServerMiddleware) preCall(name string, start time.Time, r *http.Request) { - logger := m.opts.filterLog(m.logger) - level.Debug(logger).Log("http.start_time", start.String(), "http.method", fmt.Sprintf("%s %s", r.Method, r.URL), "http.request_id", r.Header.Get("X-Request-ID"), "thanos.method_name", name, "msg", "started call") -} - -func (m *HTTPServerMiddleware) postCall(name string, start time.Time, wrapped *httputil.ResponseWriterWithStatus, r *http.Request) { - status := wrapped.Status() - logger := log.With(m.logger, "http.method", fmt.Sprintf("%s %s", r.Method, r.URL), "http.request_id", r.Header.Get("X-Request-ID"), "http.status_code", fmt.Sprintf("%d", status), - "http.time_ms", fmt.Sprintf("%v", durationToMilliseconds(time.Since(start))), "http.remote_addr", r.RemoteAddr, "thanos.method_name", name) - - logger = m.opts.filterLog(logger) - m.opts.levelFunc(logger, status).Log("msg", "finished call") -} - -func (m *HTTPServerMiddleware) HTTPMiddleware(name string, next http.Handler) http.HandlerFunc { - return func(w http.ResponseWriter, r *http.Request) { - wrapped := httputil.WrapResponseWriterWithStatus(w) - start := time.Now() - hostPort := r.Host - if hostPort == "" { - hostPort = r.URL.Host - } - - var port string - var err error - // Try to extract port if there is ':' as part of 'hostPort'. - if strings.Contains(hostPort, ":") { - _, port, err = net.SplitHostPort(hostPort) - if err != nil { - level.Error(m.logger).Log("msg", "failed to parse host port for http log decision", "err", err) - next.ServeHTTP(w, r) - return - } - } - - deciderURL := r.URL.String() - if len(port) > 0 { - deciderURL = net.JoinHostPort(deciderURL, port) - } - decision := m.opts.shouldLog(deciderURL, nil) - - switch decision { - case NoLogCall: - next.ServeHTTP(w, r) - - case LogStartAndFinishCall: - m.preCall(name, start, r) - next.ServeHTTP(wrapped, r) - m.postCall(name, start, wrapped, r) - - case LogFinishCall: - next.ServeHTTP(wrapped, r) - m.postCall(name, start, wrapped, r) - } - } -} - -// NewHTTPServerMiddleware returns an http middleware. -func NewHTTPServerMiddleware(logger log.Logger, opts ...Option) *HTTPServerMiddleware { - o := evaluateOpt(opts) - return &HTTPServerMiddleware{ - logger: log.With(logger, "protocol", "http", "http.component", "server"), - opts: o, - } -} - -// getHTTPLoggingOption returns the logging ENUM based on logStart and logEnd values. -func getHTTPLoggingOption(logStart, logEnd bool) (Decision, error) { - if !logStart && !logEnd { - return NoLogCall, nil - } - if !logStart && logEnd { - return LogFinishCall, nil - } - if logStart && logEnd { - return LogStartAndFinishCall, nil - } - return -1, fmt.Errorf("log start call is not supported") -} - -// getLevel returns the level based logger. -func getLevel(lvl string) level.Option { - switch lvl { - case "INFO": - return level.AllowInfo() - case "DEBUG": - return level.AllowDebug() - case "WARN": - return level.AllowWarn() - case "ERROR": - return level.AllowError() - default: - return level.AllowAll() - } -} - -// NewHTTPOption returns a http config option. -func NewHTTPOption(configYAML []byte) ([]Option, error) { - // Define a black config option. - logOpts := []Option{ - WithDecider(func(_ string, err error) Decision { - return NoLogCall - }), - } - - // If req logging is disabled. - if len(configYAML) == 0 { - return logOpts, nil - } - - reqLogConfig, err := NewRequestConfig(configYAML) - // If unmarshalling is an issue. - if err != nil { - return logOpts, err - } - - globalLevel, globalStart, globalEnd, err := fillGlobalOptionConfig(reqLogConfig, false) - - // If global options have invalid entries. - if err != nil { - return logOpts, err - } - // If the level entry does not matches our entries. - if err := validateLevel(globalLevel); err != nil { - // fmt.Printf("HTTP") - return logOpts, err - } - - // If the combination is valid, use them, otherwise return error. - reqLogDecision, err := getHTTPLoggingOption(globalStart, globalEnd) - if err != nil { - return logOpts, err - } - - logOpts = []Option{ - WithFilter(func(logger log.Logger) log.Logger { - return level.NewFilter(logger, getLevel(globalLevel)) - }), - WithLevels(DefaultCodeToLevel), - } - - if len(reqLogConfig.HTTP.Config) == 0 { - logOpts = append(logOpts, []Option{WithDecider(func(_ string, err error) Decision { - return reqLogDecision - }), - }...) - return logOpts, nil - } - - methodNameSlice := []string{} - - for _, eachConfig := range reqLogConfig.HTTP.Config { - eachConfigName := fmt.Sprintf("%v:%v", eachConfig.Path, eachConfig.Port) - methodNameSlice = append(methodNameSlice, eachConfigName) - } - - sort.Strings(methodNameSlice) - - logOpts = append(logOpts, []Option{ - WithDecider(func(runtimeMethodName string, err error) Decision { - idx := sort.SearchStrings(methodNameSlice, runtimeMethodName) - if idx < len(methodNameSlice) && methodNameSlice[idx] == runtimeMethodName { - return reqLogDecision - } - return NoLogCall - }), - }...) - return logOpts, nil - -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go b/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go deleted file mode 100644 index cca8853da58..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/logging/logger.go +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package logging - -import ( - "os" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" -) - -const ( - LogFormatLogfmt = "logfmt" - LogFormatJSON = "json" -) - -// NewLogger returns a log.Logger that prints in the provided format at the -// provided level with a UTC timestamp and the caller of the log entry. If non -// empty, the debug name is also appended as a field to all log lines. Panics -// if the log level is not error, warn, info or debug. Log level is expected to -// be validated before passed to this function. -func NewLogger(logLevel, logFormat, debugName string) log.Logger { - var ( - logger log.Logger - lvl level.Option - ) - - switch logLevel { - case "error": - lvl = level.AllowError() - case "warn": - lvl = level.AllowWarn() - case "info": - lvl = level.AllowInfo() - case "debug": - lvl = level.AllowDebug() - default: - // This enum is already checked and enforced by flag validations, so - // this should never happen. - panic("unexpected log level") - } - - logger = log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - if logFormat == LogFormatJSON { - logger = log.NewJSONLogger(log.NewSyncWriter(os.Stderr)) - } - - // Sort the logger chain to avoid expensive log.Valuer evaluation for disallowed level. - // Ref: https://github.com/go-kit/log/issues/14#issuecomment-945038252 - logger = log.With(logger, "ts", log.DefaultTimestampUTC, "caller", log.Caller(5)) - logger = level.NewFilter(logger, lvl) - - if debugName != "" { - logger = log.With(logger, "name", debugName) - } - - return logger -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/options.go b/vendor/github.com/thanos-io/thanos/pkg/logging/options.go deleted file mode 100644 index 76b0f8be7d3..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/logging/options.go +++ /dev/null @@ -1,245 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package logging - -import ( - "fmt" - "math/rand" - "time" - - extflag "github.com/efficientgo/tools/extkingpin" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - grpc_logging "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging" - "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/tags" - "github.com/oklog/ulid" - "google.golang.org/grpc/codes" -) - -// Decision defines rules for enabling start and end of logging. -type Decision int - -const ( - // NoLogCall - Logging is disabled. - NoLogCall Decision = iota - // LogFinishCall - Only finish logs of request is enabled. - LogFinishCall - // LogStartAndFinishCall - Logging of start and end of request is enabled. - LogStartAndFinishCall -) - -var defaultOptions = &options{ - shouldLog: DefaultDeciderMethod, - codeFunc: DefaultErrorToCode, - levelFunc: DefaultCodeToLevel, - durationFieldFunc: DurationToTimeMillisFields, - filterLog: DefaultFilterLogging, -} - -func evaluateOpt(opts []Option) *options { - optCopy := &options{} - *optCopy = *defaultOptions - optCopy.levelFunc = DefaultCodeToLevel - for _, o := range opts { - o(optCopy) - } - return optCopy -} - -// WithDecider customizes the function for deciding if the HTTP Middlewares/Tripperwares should log. -func WithDecider(f Decider) Option { - return func(o *options) { - o.shouldLog = f - } -} - -// WithLevels customizes the function for mapping HTTP response codes and interceptor log level statements. -func WithLevels(f CodeToLevel) Option { - return func(o *options) { - o.levelFunc = f - } -} - -// WithFilter customizes the function for deciding which level of logging should be allowed. -// Follows go-kit Allow convention. -func WithFilter(f FilterLogging) Option { - return func(o *options) { - o.filterLog = f - } -} - -// Interface for the additional methods. - -// Types for the Options. -type Option func(*options) - -// Fields represents logging fields. It has to have even number of elements (pairs). -type Fields []string - -// ErrorToCode function determines the error code of the error -// for the http response. -type ErrorToCode func(err error) int - -// DefaultErrorToCode returns an InternalServerError. -func DefaultErrorToCode(_ error) int { - return 500 -} - -// Decider function defines rules for suppressing the logging. -type Decider func(methodName string, err error) Decision - -// DefaultDeciderMethod is the default implementation of decider to see if you should log the call -// by default this is set to LogStartAndFinishCall. -func DefaultDeciderMethod(_ string, _ error) Decision { - return LogStartAndFinishCall -} - -// CodeToLevel function defines the mapping between HTTP Response codes to log levels for server side. -type CodeToLevel func(logger log.Logger, code int) log.Logger - -// DurationToFields function defines how to produce duration fields for logging. -type DurationToFields func(duration time.Duration) Fields - -// FilterLogging makes sure only the logs with level=lvl gets logged, or filtered. -type FilterLogging func(logger log.Logger) log.Logger - -// DefaultFilterLogging allows logs from all levels to be logged in output. -func DefaultFilterLogging(logger log.Logger) log.Logger { - return level.NewFilter(logger, level.AllowAll()) -} - -type options struct { - levelFunc CodeToLevel - shouldLog Decider - codeFunc ErrorToCode - durationFieldFunc DurationToFields - filterLog FilterLogging -} - -// DefaultCodeToLevel is the helper mapper that maps HTTP Response codes to log levels. -func DefaultCodeToLevel(logger log.Logger, code int) log.Logger { - if code >= 200 && code < 500 { - return level.Debug(logger) - } - return level.Error(logger) -} - -// DefaultCodeToLevelGRPC is the helper mapper that maps gRPC Response codes to log levels. -func DefaultCodeToLevelGRPC(c codes.Code) grpc_logging.Level { - switch c { - case codes.Unknown, codes.Unimplemented, codes.Internal, codes.DataLoss: - return grpc_logging.ERROR - default: - return grpc_logging.DEBUG - } -} - -// DurationToTimeMillisFields converts the duration to milliseconds and uses the key `http.time_ms`. -func DurationToTimeMillisFields(duration time.Duration) Fields { - return Fields{"http.time_ms", fmt.Sprintf("%v", durationToMilliseconds(duration))} -} - -func durationToMilliseconds(duration time.Duration) float32 { - return float32(duration.Nanoseconds()/1000) / 1000 -} - -// LogDecision defines mapping of flag options to the logging decision. -var LogDecision = map[string]Decision{ - "NoLogCall": NoLogCall, - "LogFinishCall": LogFinishCall, - "LogStartAndFinishCall": LogStartAndFinishCall, -} - -// MapAllowedLevels allows to map a given level to a list of allowed level. -// Convention taken from go-kit/level v0.10.0 https://godoc.org/github.com/go-kit/log/level#AllowAll. -var MapAllowedLevels = map[string][]string{ - "DEBUG": {"INFO", "DEBUG", "WARN", "ERROR"}, - "ERROR": {"ERROR"}, - "INFO": {"INFO", "WARN", "ERROR"}, - "WARN": {"WARN", "ERROR"}, -} - -// TODO: @yashrsharma44 - To be deprecated in the next release. -func ParseHTTPOptions(flagDecision string, reqLogConfig *extflag.PathOrContent) ([]Option, error) { - // Default Option: No Logging. - logOpts := []Option{WithDecider(func(_ string, _ error) Decision { - return NoLogCall - })} - - // If flag is incorrectly parsed. - configYAML, err := reqLogConfig.Content() - if err != nil { - return logOpts, fmt.Errorf("getting request logging config failed. %v", err) - } - - // Check if the user enables request logging through flags and YAML. - if len(configYAML) != 0 && flagDecision != "" { - return logOpts, fmt.Errorf("both log.request.decision and request.logging have been enabled, please use only one of the flags") - } - // If old flag is enabled. - if len(flagDecision) > 0 { - logOpts := []Option{WithDecider(func(_ string, _ error) Decision { - return LogDecision[flagDecision] - })} - return logOpts, nil - } - return NewHTTPOption(configYAML) -} - -// TODO: @yashrsharma44 - To be deprecated in the next release. -func ParsegRPCOptions(flagDecision string, reqLogConfig *extflag.PathOrContent) ([]tags.Option, []grpc_logging.Option, error) { - // Default Option: No Logging. - logOpts := []grpc_logging.Option{grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { - return grpc_logging.NoLogCall - })} - - configYAML, err := reqLogConfig.Content() - if err != nil { - return []tags.Option{}, logOpts, fmt.Errorf("getting request logging config failed. %v", err) - } - - // Check if the user enables request logging through flags and YAML. - if len(configYAML) != 0 && flagDecision != "" { - return []tags.Option{}, logOpts, fmt.Errorf("both log.request.decision and request.logging-config have been enabled, please use only one of the flags") - } - - // If the old flag is empty, use the new YAML config. - if flagDecision == "" { - tagOpts, logOpts, err := NewGRPCOption(configYAML) - if err != nil { - return []tags.Option{}, logOpts, err - } - return tagOpts, logOpts, nil - } - - tagOpts := []tags.Option{ - tags.WithFieldExtractor(func(_ string, req interface{}) map[string]string { - tagMap := tags.TagBasedRequestFieldExtractor("request-id")("", req) - // If a request-id exists for a given request. - if tagMap != nil { - if _, ok := tagMap["request-id"]; ok { - return tagMap - } - } - entropy := ulid.Monotonic(rand.New(rand.NewSource(time.Now().UnixNano())), 0) - reqID := ulid.MustNew(ulid.Timestamp(time.Now()), entropy) - tagMap = make(map[string]string) - tagMap["request-id"] = reqID.String() - return tagMap - }), - } - logOpts = []grpc_logging.Option{grpc_logging.WithDecider(func(_ string, _ error) grpc_logging.Decision { - switch flagDecision { - case "NoLogCall": - return grpc_logging.NoLogCall - case "LogFinishCall": - return grpc_logging.LogFinishCall - case "LogStartAndFinishCall": - return grpc_logging.LogStartAndFinishCall - default: - return grpc_logging.NoLogCall - } - })} - return tagOpts, logOpts, nil -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go b/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go deleted file mode 100644 index ed3c893ed9f..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/logging/yaml_parser.go +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package logging - -type RequestConfig struct { - HTTP HTTPProtocolConfigs `yaml:"http"` - GRPC GRPCProtocolConfigs `yaml:"grpc"` - Options OptionsConfig `yaml:"options"` -} - -type HTTPProtocolConfigs struct { - Options OptionsConfig `yaml:"options"` - Config []HTTPProtocolConfig `yaml:"config"` -} - -type GRPCProtocolConfigs struct { - Options OptionsConfig `yaml:"options"` - Config []GRPCProtocolConfig `yaml:"config"` -} - -type OptionsConfig struct { - Level string `yaml:"level"` - Decision DecisionConfig `yaml:"decision"` -} - -type DecisionConfig struct { - LogStart bool `yaml:"log_start"` - LogEnd bool `yaml:"log_end"` -} - -type HTTPProtocolConfig struct { - Path string `yaml:"path"` - Port uint64 `yaml:"port"` -} - -type GRPCProtocolConfig struct { - Service string `yaml:"service"` - Method string `yaml:"method"` -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go b/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go deleted file mode 100644 index 348e71cf4cb..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/prober/combiner.go +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package prober - -import "sync" - -type combined struct { - mu sync.Mutex - probes []Probe -} - -// Combine folds given probes into one, reflects their statuses in a thread-safe way. -func Combine(probes ...Probe) Probe { - return &combined{probes: probes} -} - -// Ready sets components status to ready. -func (p *combined) Ready() { - p.mu.Lock() - defer p.mu.Unlock() - - for _, probe := range p.probes { - probe.Ready() - } -} - -// NotReady sets components status to not ready with given error as a cause. -func (p *combined) NotReady(err error) { - p.mu.Lock() - defer p.mu.Unlock() - - for _, probe := range p.probes { - probe.NotReady(err) - } -} - -// Healthy sets components status to healthy. -func (p *combined) Healthy() { - p.mu.Lock() - defer p.mu.Unlock() - - for _, probe := range p.probes { - probe.Healthy() - } -} - -// NotHealthy sets components status to not healthy with given error as a cause. -func (p *combined) NotHealthy(err error) { - p.mu.Lock() - defer p.mu.Unlock() - - for _, probe := range p.probes { - probe.NotHealthy(err) - } -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go b/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go deleted file mode 100644 index 6151e040338..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/prober/grpc.go +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package prober - -import ( - "google.golang.org/grpc/health" - grpc_health "google.golang.org/grpc/health/grpc_health_v1" -) - -// GRPCProbe represents health and readiness status of given component, and provides GRPC integration. -type GRPCProbe struct { - h *health.Server -} - -// NewGRPC creates a Probe that wrapped around grpc/healt.Server which reflects status of server. -func NewGRPC() *GRPCProbe { - h := health.NewServer() - h.SetServingStatus("", grpc_health.HealthCheckResponse_NOT_SERVING) - - return &GRPCProbe{h: h} -} - -// HealthServer returns a gRPC health server which responds readiness and liveness checks. -func (p *GRPCProbe) HealthServer() *health.Server { - return p.h -} - -// Ready sets components status to ready. -func (p *GRPCProbe) Ready() { - p.h.SetServingStatus("", grpc_health.HealthCheckResponse_SERVING) -} - -// NotReady sets components status to not ready with given error as a cause. -func (p *GRPCProbe) NotReady(err error) { - p.h.SetServingStatus("", grpc_health.HealthCheckResponse_NOT_SERVING) -} - -// Healthy sets components status to healthy. -func (p *GRPCProbe) Healthy() { - p.h.Resume() -} - -// NotHealthy sets components status to not healthy with given error as a cause. -func (p *GRPCProbe) NotHealthy(err error) { - p.h.Shutdown() -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/http.go b/vendor/github.com/thanos-io/thanos/pkg/prober/http.go deleted file mode 100644 index 18f9c98af17..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/prober/http.go +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package prober - -import ( - "io" - "net/http" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "go.uber.org/atomic" -) - -type check func() bool - -// HTTPProbe represents health and readiness status of given component, and provides HTTP integration. -type HTTPProbe struct { - ready atomic.Uint32 - healthy atomic.Uint32 -} - -// NewHTTP returns HTTPProbe representing readiness and healthiness of given component. -func NewHTTP() *HTTPProbe { - return &HTTPProbe{} -} - -// HealthyHandler returns a HTTP Handler which responds health checks. -func (p *HTTPProbe) HealthyHandler(logger log.Logger) http.HandlerFunc { - return p.handler(logger, p.isHealthy) -} - -// ReadyHandler returns a HTTP Handler which responds readiness checks. -func (p *HTTPProbe) ReadyHandler(logger log.Logger) http.HandlerFunc { - return p.handler(logger, p.IsReady) -} - -func (p *HTTPProbe) handler(logger log.Logger, c check) http.HandlerFunc { - return func(w http.ResponseWriter, _ *http.Request) { - if !c() { - http.Error(w, "NOT OK", http.StatusServiceUnavailable) - return - } - if _, err := io.WriteString(w, "OK"); err != nil { - level.Error(logger).Log("msg", "failed to write probe response", "err", err) - } - } -} - -// IsReady returns true if component is ready. -func (p *HTTPProbe) IsReady() bool { - ready := p.ready.Load() - return ready > 0 -} - -// isHealthy returns true if component is healthy. -func (p *HTTPProbe) isHealthy() bool { - healthy := p.healthy.Load() - return healthy > 0 -} - -// Ready sets components status to ready. -func (p *HTTPProbe) Ready() { - p.ready.Swap(1) -} - -// NotReady sets components status to not ready with given error as a cause. -func (p *HTTPProbe) NotReady(err error) { - p.ready.Swap(0) - -} - -// Healthy sets components status to healthy. -func (p *HTTPProbe) Healthy() { - p.healthy.Swap(1) -} - -// NotHealthy sets components status to not healthy with given error as a cause. -func (p *HTTPProbe) NotHealthy(err error) { - p.healthy.Swap(0) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go b/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go deleted file mode 100644 index 7c8d78ff85b..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/prober/intrumentation.go +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package prober - -import ( - "sync" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/thanos-io/thanos/pkg/component" -) - -const ( - ready = "ready" - notReady = "not-ready" - healthy = "healthy" -) - -// InstrumentationProbe stores instrumentation state of Probe. -// This is created with an intention to combine with other Probe's using prober.Combine. -type InstrumentationProbe struct { - component component.Component - logger log.Logger - - statusMetric *prometheus.GaugeVec - mu sync.Mutex - statusString string -} - -// NewInstrumentation returns InstrumentationProbe records readiness and healthiness for given component. -func NewInstrumentation(component component.Component, logger log.Logger, reg prometheus.Registerer) *InstrumentationProbe { - p := InstrumentationProbe{ - component: component, - logger: logger, - statusMetric: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Name: "status", - Help: "Represents status (0 indicates failure, 1 indicates success) of the component.", - ConstLabels: map[string]string{"component": component.String()}, - }, - []string{"check"}, - ), - } - return &p -} - -// Ready records the component status when Ready is called, if combined with other Probes. -func (p *InstrumentationProbe) Ready() { - p.statusMetric.WithLabelValues(ready).Set(1) - p.mu.Lock() - defer p.mu.Unlock() - if p.statusString != ready { - level.Info(p.logger).Log("msg", "changing probe status", "status", ready) - p.statusString = ready - } -} - -// NotReady records the component status when NotReady is called, if combined with other Probes. -func (p *InstrumentationProbe) NotReady(err error) { - p.statusMetric.WithLabelValues(ready).Set(0) - p.mu.Lock() - defer p.mu.Unlock() - if p.statusString != notReady { - level.Warn(p.logger).Log("msg", "changing probe status", "status", notReady, "reason", err) - p.statusString = notReady - } -} - -// Healthy records the component status when Healthy is called, if combined with other Probes. -func (p *InstrumentationProbe) Healthy() { - p.statusMetric.WithLabelValues(healthy).Set(1) - level.Info(p.logger).Log("msg", "changing probe status", "status", "healthy") -} - -// NotHealthy records the component status when NotHealthy is called, if combined with other Probes. -func (p *InstrumentationProbe) NotHealthy(err error) { - p.statusMetric.WithLabelValues(healthy).Set(0) - level.Info(p.logger).Log("msg", "changing probe status", "status", "not-healthy", "reason", err) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go b/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go deleted file mode 100644 index db03e6e046f..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/prober/prober.go +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package prober - -// Prober represents health and readiness status of given component. -// -// From Kubernetes documentation https://kubernetes.io/docs/tasks/configure-pod-container/configure-liveness-readiness-probes/ : -// -// liveness: Many applications running for long periods of time eventually transition to broken states, -// (healthy) and cannot recover except by being restarted. -// Kubernetes provides liveness probes to detect and remedy such situations. -// -// readiness: Sometimes, applications are temporarily unable to serve traffic. -// (ready) For example, an application might need to load large data or configuration files during startup, -// or depend on external services after startup. In such cases, you don’t want to kill the application, -// but you don’t want to send it requests either. Kubernetes provides readiness probes to detect -// and mitigate these situations. A pod with containers reporting that they are not ready -// does not receive traffic through Kubernetes Services. -type Probe interface { - Healthy() - NotHealthy(err error) - Ready() - NotReady(err error) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go deleted file mode 100644 index 795d6c4fdb1..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/server/http/http.go +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package http - -import ( - "context" - "net/http" - "net/http/pprof" - - "github.com/felixge/fgprof" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promhttp" - toolkit_web "github.com/prometheus/exporter-toolkit/web" - "golang.org/x/net/http2" - "golang.org/x/net/http2/h2c" - - "github.com/thanos-io/thanos/pkg/component" - "github.com/thanos-io/thanos/pkg/prober" -) - -// A Server defines parameters for serve HTTP requests, a wrapper around http.Server. -type Server struct { - logger log.Logger - comp component.Component - prober *prober.HTTPProbe - - mux *http.ServeMux - srv *http.Server - - opts options -} - -// New creates a new Server. -func New(logger log.Logger, reg *prometheus.Registry, comp component.Component, prober *prober.HTTPProbe, opts ...Option) *Server { - options := options{} - for _, o := range opts { - o.apply(&options) - } - - mux := http.NewServeMux() - if options.mux != nil { - mux = options.mux - } - - registerMetrics(mux, reg) - registerProbes(mux, prober, logger) - registerProfiler(mux) - - var h http.Handler - if options.enableH2C { - h2s := &http2.Server{} - h = h2c.NewHandler(mux, h2s) - } else { - h = mux - } - - return &Server{ - logger: log.With(logger, "service", "http/server", "component", comp.String()), - comp: comp, - prober: prober, - mux: mux, - srv: &http.Server{Addr: options.listen, Handler: h}, - opts: options, - } -} - -// ListenAndServe listens on the TCP network address and handles requests on incoming connections. -func (s *Server) ListenAndServe() error { - level.Info(s.logger).Log("msg", "listening for requests and metrics", "address", s.opts.listen) - err := toolkit_web.Validate(s.opts.tlsConfigPath) - if err != nil { - return errors.Wrap(err, "server could not be started") - } - - flags := &toolkit_web.FlagConfig{ - WebListenAddresses: &([]string{s.opts.listen}), - WebSystemdSocket: ofBool(false), - WebConfigFile: &s.opts.tlsConfigPath, - } - - return errors.Wrap(toolkit_web.ListenAndServe(s.srv, flags, s.logger), "serve HTTP and metrics") -} - -// Shutdown gracefully shuts down the server by waiting, -// for specified amount of time (by gracePeriod) for connections to return to idle and then shut down. -func (s *Server) Shutdown(err error) { - level.Info(s.logger).Log("msg", "internal server is shutting down", "err", err) - if err == http.ErrServerClosed { - level.Warn(s.logger).Log("msg", "internal server closed unexpectedly") - return - } - - if s.opts.gracePeriod == 0 { - s.srv.Close() - level.Info(s.logger).Log("msg", "internal server is shutdown", "err", err) - return - } - - ctx, cancel := context.WithTimeout(context.Background(), s.opts.gracePeriod) - defer cancel() - - if err := s.srv.Shutdown(ctx); err != nil { - level.Error(s.logger).Log("msg", "internal server shut down failed", "err", err) - return - } - level.Info(s.logger).Log("msg", "internal server is shutdown gracefully", "err", err) -} - -// Handle registers the handler for the given pattern. -func (s *Server) Handle(pattern string, handler http.Handler) { - s.mux.Handle(pattern, handler) -} - -func registerProfiler(mux *http.ServeMux) { - mux.HandleFunc("/debug/pprof/", pprof.Index) - mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - mux.HandleFunc("/debug/pprof/profile", pprof.Profile) - mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - mux.Handle("/debug/fgprof", fgprof.Handler()) -} - -func registerMetrics(mux *http.ServeMux, g prometheus.Gatherer) { - if g != nil { - mux.Handle("/metrics", promhttp.HandlerFor(g, promhttp.HandlerOpts{ - EnableOpenMetrics: true, - })) - } -} - -func registerProbes(mux *http.ServeMux, p *prober.HTTPProbe, logger log.Logger) { - if p != nil { - mux.Handle("/-/healthy", p.HealthyHandler(logger)) - mux.Handle("/-/ready", p.ReadyHandler(logger)) - } -} - -// Helper for exporter toolkit FlagConfig. -func ofBool(i bool) *bool { - return &i -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go deleted file mode 100644 index d08699f9dff..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/server/http/option.go +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package http - -import ( - "net/http" - "time" -) - -type options struct { - gracePeriod time.Duration - listen string - tlsConfigPath string - mux *http.ServeMux - enableH2C bool -} - -// Option overrides behavior of Server. -type Option interface { - apply(*options) -} - -type optionFunc func(*options) - -func (f optionFunc) apply(o *options) { - f(o) -} - -// WithGracePeriod sets shutdown grace period for HTTP server. -// Server waits connections to drain for specified amount of time. -func WithGracePeriod(t time.Duration) Option { - return optionFunc(func(o *options) { - o.gracePeriod = t - }) -} - -// WithListen sets address to listen for HTTP server. -// Server accepts incoming TCP connections on given address. -func WithListen(s string) Option { - return optionFunc(func(o *options) { - o.listen = s - }) -} - -func WithTLSConfig(tls string) Option { - return optionFunc(func(o *options) { - o.tlsConfigPath = tls - }) -} - -func WithEnableH2C(enableH2C bool) Option { - return optionFunc(func(o *options) { - o.enableH2C = enableH2C - }) -} - -// WithMux overrides the server's default mux. -func WithMux(mux *http.ServeMux) Option { - return optionFunc(func(o *options) { - o.mux = mux - }) -} diff --git a/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go b/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go deleted file mode 100644 index d8da1dc3c54..00000000000 --- a/vendor/github.com/thanos-io/thanos/pkg/server/http/utils.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright (c) The Thanos Authors. -// Licensed under the Apache License 2.0. - -package http - -import "net/http" - -// ResponseWriterWithStatus wraps around http.ResponseWriter to capture the status code of the response. -type ResponseWriterWithStatus struct { - http.ResponseWriter - status int - isHeaderWritten bool -} - -// WrapResponseWriterWithStatus wraps the http.ResponseWriter for extracting status. -func WrapResponseWriterWithStatus(w http.ResponseWriter) *ResponseWriterWithStatus { - return &ResponseWriterWithStatus{ResponseWriter: w} -} - -// Status returns http response status. -func (r *ResponseWriterWithStatus) Status() int { - return r.status -} - -// WriteHeader writes the header. -func (r *ResponseWriterWithStatus) WriteHeader(code int) { - if !r.isHeaderWritten { - r.status = code - r.ResponseWriter.WriteHeader(code) - r.isHeaderWritten = true - } -} diff --git a/vendor/golang.org/x/net/http2/h2c/h2c.go b/vendor/golang.org/x/net/http2/h2c/h2c.go deleted file mode 100644 index a72bbed1bc3..00000000000 --- a/vendor/golang.org/x/net/http2/h2c/h2c.go +++ /dev/null @@ -1,240 +0,0 @@ -// Copyright 2018 The Go Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -// Package h2c implements the unencrypted "h2c" form of HTTP/2. -// -// The h2c protocol is the non-TLS version of HTTP/2 which is not available from -// net/http or golang.org/x/net/http2. -package h2c - -import ( - "bufio" - "bytes" - "encoding/base64" - "errors" - "fmt" - "io" - "log" - "net" - "net/http" - "net/textproto" - "os" - "strings" - - "golang.org/x/net/http/httpguts" - "golang.org/x/net/http2" -) - -var ( - http2VerboseLogs bool -) - -func init() { - e := os.Getenv("GODEBUG") - if strings.Contains(e, "http2debug=1") || strings.Contains(e, "http2debug=2") { - http2VerboseLogs = true - } -} - -// h2cHandler is a Handler which implements h2c by hijacking the HTTP/1 traffic -// that should be h2c traffic. There are two ways to begin a h2c connection -// (RFC 7540 Section 3.2 and 3.4): (1) Starting with Prior Knowledge - this -// works by starting an h2c connection with a string of bytes that is valid -// HTTP/1, but unlikely to occur in practice and (2) Upgrading from HTTP/1 to -// h2c - this works by using the HTTP/1 Upgrade header to request an upgrade to -// h2c. When either of those situations occur we hijack the HTTP/1 connection, -// convert it to a HTTP/2 connection and pass the net.Conn to http2.ServeConn. -type h2cHandler struct { - Handler http.Handler - s *http2.Server -} - -// NewHandler returns an http.Handler that wraps h, intercepting any h2c -// traffic. If a request is an h2c connection, it's hijacked and redirected to -// s.ServeConn. Otherwise the returned Handler just forwards requests to h. This -// works because h2c is designed to be parseable as valid HTTP/1, but ignored by -// any HTTP server that does not handle h2c. Therefore we leverage the HTTP/1 -// compatible parts of the Go http library to parse and recognize h2c requests. -// Once a request is recognized as h2c, we hijack the connection and convert it -// to an HTTP/2 connection which is understandable to s.ServeConn. (s.ServeConn -// understands HTTP/2 except for the h2c part of it.) -// -// The first request on an h2c connection is read entirely into memory before -// the Handler is called. To limit the memory consumed by this request, wrap -// the result of NewHandler in an http.MaxBytesHandler. -func NewHandler(h http.Handler, s *http2.Server) http.Handler { - return &h2cHandler{ - Handler: h, - s: s, - } -} - -// extractServer extracts existing http.Server instance from http.Request or create an empty http.Server -func extractServer(r *http.Request) *http.Server { - server, ok := r.Context().Value(http.ServerContextKey).(*http.Server) - if ok { - return server - } - return new(http.Server) -} - -// ServeHTTP implement the h2c support that is enabled by h2c.GetH2CHandler. -func (s h2cHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - // Handle h2c with prior knowledge (RFC 7540 Section 3.4) - if r.Method == "PRI" && len(r.Header) == 0 && r.URL.Path == "*" && r.Proto == "HTTP/2.0" { - if http2VerboseLogs { - log.Print("h2c: attempting h2c with prior knowledge.") - } - conn, err := initH2CWithPriorKnowledge(w) - if err != nil { - if http2VerboseLogs { - log.Printf("h2c: error h2c with prior knowledge: %v", err) - } - return - } - defer conn.Close() - s.s.ServeConn(conn, &http2.ServeConnOpts{ - Context: r.Context(), - BaseConfig: extractServer(r), - Handler: s.Handler, - SawClientPreface: true, - }) - return - } - // Handle Upgrade to h2c (RFC 7540 Section 3.2) - if isH2CUpgrade(r.Header) { - conn, settings, err := h2cUpgrade(w, r) - if err != nil { - if http2VerboseLogs { - log.Printf("h2c: error h2c upgrade: %v", err) - } - w.WriteHeader(http.StatusInternalServerError) - return - } - defer conn.Close() - s.s.ServeConn(conn, &http2.ServeConnOpts{ - Context: r.Context(), - BaseConfig: extractServer(r), - Handler: s.Handler, - UpgradeRequest: r, - Settings: settings, - }) - return - } - s.Handler.ServeHTTP(w, r) - return -} - -// initH2CWithPriorKnowledge implements creating a h2c connection with prior -// knowledge (Section 3.4) and creates a net.Conn suitable for http2.ServeConn. -// All we have to do is look for the client preface that is suppose to be part -// of the body, and reforward the client preface on the net.Conn this function -// creates. -func initH2CWithPriorKnowledge(w http.ResponseWriter) (net.Conn, error) { - hijacker, ok := w.(http.Hijacker) - if !ok { - return nil, errors.New("h2c: connection does not support Hijack") - } - conn, rw, err := hijacker.Hijack() - if err != nil { - return nil, err - } - - const expectedBody = "SM\r\n\r\n" - - buf := make([]byte, len(expectedBody)) - n, err := io.ReadFull(rw, buf) - if err != nil { - return nil, fmt.Errorf("h2c: error reading client preface: %s", err) - } - - if string(buf[:n]) == expectedBody { - return newBufConn(conn, rw), nil - } - - conn.Close() - return nil, errors.New("h2c: invalid client preface") -} - -// h2cUpgrade establishes a h2c connection using the HTTP/1 upgrade (Section 3.2). -func h2cUpgrade(w http.ResponseWriter, r *http.Request) (_ net.Conn, settings []byte, err error) { - settings, err = getH2Settings(r.Header) - if err != nil { - return nil, nil, err - } - hijacker, ok := w.(http.Hijacker) - if !ok { - return nil, nil, errors.New("h2c: connection does not support Hijack") - } - - body, err := io.ReadAll(r.Body) - if err != nil { - return nil, nil, err - } - r.Body = io.NopCloser(bytes.NewBuffer(body)) - - conn, rw, err := hijacker.Hijack() - if err != nil { - return nil, nil, err - } - - rw.Write([]byte("HTTP/1.1 101 Switching Protocols\r\n" + - "Connection: Upgrade\r\n" + - "Upgrade: h2c\r\n\r\n")) - return newBufConn(conn, rw), settings, nil -} - -// isH2CUpgrade returns true if the header properly request an upgrade to h2c -// as specified by Section 3.2. -func isH2CUpgrade(h http.Header) bool { - return httpguts.HeaderValuesContainsToken(h[textproto.CanonicalMIMEHeaderKey("Upgrade")], "h2c") && - httpguts.HeaderValuesContainsToken(h[textproto.CanonicalMIMEHeaderKey("Connection")], "HTTP2-Settings") -} - -// getH2Settings returns the settings in the HTTP2-Settings header. -func getH2Settings(h http.Header) ([]byte, error) { - vals, ok := h[textproto.CanonicalMIMEHeaderKey("HTTP2-Settings")] - if !ok { - return nil, errors.New("missing HTTP2-Settings header") - } - if len(vals) != 1 { - return nil, fmt.Errorf("expected 1 HTTP2-Settings. Got: %v", vals) - } - settings, err := base64.RawURLEncoding.DecodeString(vals[0]) - if err != nil { - return nil, err - } - return settings, nil -} - -func newBufConn(conn net.Conn, rw *bufio.ReadWriter) net.Conn { - rw.Flush() - if rw.Reader.Buffered() == 0 { - // If there's no buffered data to be read, - // we can just discard the bufio.ReadWriter. - return conn - } - return &bufConn{conn, rw.Reader} -} - -// bufConn wraps a net.Conn, but reads drain the bufio.Reader first. -type bufConn struct { - net.Conn - *bufio.Reader -} - -func (c *bufConn) Read(p []byte) (int, error) { - if c.Reader == nil { - return c.Conn.Read(p) - } - n := c.Reader.Buffered() - if n == 0 { - c.Reader = nil - return c.Conn.Read(p) - } - if n < len(p) { - p = p[:n] - } - return c.Reader.Read(p) -} diff --git a/vendor/google.golang.org/grpc/health/client.go b/vendor/google.golang.org/grpc/health/client.go deleted file mode 100644 index b5bee483802..00000000000 --- a/vendor/google.golang.org/grpc/health/client.go +++ /dev/null @@ -1,117 +0,0 @@ -/* - * - * Copyright 2018 gRPC authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package health - -import ( - "context" - "fmt" - "io" - "time" - - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/connectivity" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/internal" - "google.golang.org/grpc/internal/backoff" - "google.golang.org/grpc/status" -) - -var ( - backoffStrategy = backoff.DefaultExponential - backoffFunc = func(ctx context.Context, retries int) bool { - d := backoffStrategy.Backoff(retries) - timer := time.NewTimer(d) - select { - case <-timer.C: - return true - case <-ctx.Done(): - timer.Stop() - return false - } - } -) - -func init() { - internal.HealthCheckFunc = clientHealthCheck -} - -const healthCheckMethod = "/grpc.health.v1.Health/Watch" - -// This function implements the protocol defined at: -// https://github.com/grpc/grpc/blob/master/doc/health-checking.md -func clientHealthCheck(ctx context.Context, newStream func(string) (interface{}, error), setConnectivityState func(connectivity.State, error), service string) error { - tryCnt := 0 - -retryConnection: - for { - // Backs off if the connection has failed in some way without receiving a message in the previous retry. - if tryCnt > 0 && !backoffFunc(ctx, tryCnt-1) { - return nil - } - tryCnt++ - - if ctx.Err() != nil { - return nil - } - setConnectivityState(connectivity.Connecting, nil) - rawS, err := newStream(healthCheckMethod) - if err != nil { - continue retryConnection - } - - s, ok := rawS.(grpc.ClientStream) - // Ideally, this should never happen. But if it happens, the server is marked as healthy for LBing purposes. - if !ok { - setConnectivityState(connectivity.Ready, nil) - return fmt.Errorf("newStream returned %v (type %T); want grpc.ClientStream", rawS, rawS) - } - - if err = s.SendMsg(&healthpb.HealthCheckRequest{Service: service}); err != nil && err != io.EOF { - // Stream should have been closed, so we can safely continue to create a new stream. - continue retryConnection - } - s.CloseSend() - - resp := new(healthpb.HealthCheckResponse) - for { - err = s.RecvMsg(resp) - - // Reports healthy for the LBing purposes if health check is not implemented in the server. - if status.Code(err) == codes.Unimplemented { - setConnectivityState(connectivity.Ready, nil) - return err - } - - // Reports unhealthy if server's Watch method gives an error other than UNIMPLEMENTED. - if err != nil { - setConnectivityState(connectivity.TransientFailure, fmt.Errorf("connection active but received health check RPC error: %v", err)) - continue retryConnection - } - - // As a message has been received, removes the need for backoff for the next retry by resetting the try count. - tryCnt = 0 - if resp.Status == healthpb.HealthCheckResponse_SERVING { - setConnectivityState(connectivity.Ready, nil) - } else { - setConnectivityState(connectivity.TransientFailure, fmt.Errorf("connection active but health check failed. status=%s", resp.Status)) - } - } - } -} diff --git a/vendor/google.golang.org/grpc/health/logging.go b/vendor/google.golang.org/grpc/health/logging.go deleted file mode 100644 index 83c6acf55ef..00000000000 --- a/vendor/google.golang.org/grpc/health/logging.go +++ /dev/null @@ -1,23 +0,0 @@ -/* - * - * Copyright 2020 gRPC authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package health - -import "google.golang.org/grpc/grpclog" - -var logger = grpclog.Component("health_service") diff --git a/vendor/google.golang.org/grpc/health/server.go b/vendor/google.golang.org/grpc/health/server.go deleted file mode 100644 index cce6312d77f..00000000000 --- a/vendor/google.golang.org/grpc/health/server.go +++ /dev/null @@ -1,163 +0,0 @@ -/* - * - * Copyright 2017 gRPC authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -// Package health provides a service that exposes server's health and it must be -// imported to enable support for client-side health checks. -package health - -import ( - "context" - "sync" - - "google.golang.org/grpc/codes" - healthgrpc "google.golang.org/grpc/health/grpc_health_v1" - healthpb "google.golang.org/grpc/health/grpc_health_v1" - "google.golang.org/grpc/status" -) - -// Server implements `service Health`. -type Server struct { - healthgrpc.UnimplementedHealthServer - mu sync.RWMutex - // If shutdown is true, it's expected all serving status is NOT_SERVING, and - // will stay in NOT_SERVING. - shutdown bool - // statusMap stores the serving status of the services this Server monitors. - statusMap map[string]healthpb.HealthCheckResponse_ServingStatus - updates map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus -} - -// NewServer returns a new Server. -func NewServer() *Server { - return &Server{ - statusMap: map[string]healthpb.HealthCheckResponse_ServingStatus{"": healthpb.HealthCheckResponse_SERVING}, - updates: make(map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus), - } -} - -// Check implements `service Health`. -func (s *Server) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*healthpb.HealthCheckResponse, error) { - s.mu.RLock() - defer s.mu.RUnlock() - if servingStatus, ok := s.statusMap[in.Service]; ok { - return &healthpb.HealthCheckResponse{ - Status: servingStatus, - }, nil - } - return nil, status.Error(codes.NotFound, "unknown service") -} - -// Watch implements `service Health`. -func (s *Server) Watch(in *healthpb.HealthCheckRequest, stream healthgrpc.Health_WatchServer) error { - service := in.Service - // update channel is used for getting service status updates. - update := make(chan healthpb.HealthCheckResponse_ServingStatus, 1) - s.mu.Lock() - // Puts the initial status to the channel. - if servingStatus, ok := s.statusMap[service]; ok { - update <- servingStatus - } else { - update <- healthpb.HealthCheckResponse_SERVICE_UNKNOWN - } - - // Registers the update channel to the correct place in the updates map. - if _, ok := s.updates[service]; !ok { - s.updates[service] = make(map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus) - } - s.updates[service][stream] = update - defer func() { - s.mu.Lock() - delete(s.updates[service], stream) - s.mu.Unlock() - }() - s.mu.Unlock() - - var lastSentStatus healthpb.HealthCheckResponse_ServingStatus = -1 - for { - select { - // Status updated. Sends the up-to-date status to the client. - case servingStatus := <-update: - if lastSentStatus == servingStatus { - continue - } - lastSentStatus = servingStatus - err := stream.Send(&healthpb.HealthCheckResponse{Status: servingStatus}) - if err != nil { - return status.Error(codes.Canceled, "Stream has ended.") - } - // Context done. Removes the update channel from the updates map. - case <-stream.Context().Done(): - return status.Error(codes.Canceled, "Stream has ended.") - } - } -} - -// SetServingStatus is called when need to reset the serving status of a service -// or insert a new service entry into the statusMap. -func (s *Server) SetServingStatus(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) { - s.mu.Lock() - defer s.mu.Unlock() - if s.shutdown { - logger.Infof("health: status changing for %s to %v is ignored because health service is shutdown", service, servingStatus) - return - } - - s.setServingStatusLocked(service, servingStatus) -} - -func (s *Server) setServingStatusLocked(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) { - s.statusMap[service] = servingStatus - for _, update := range s.updates[service] { - // Clears previous updates, that are not sent to the client, from the channel. - // This can happen if the client is not reading and the server gets flow control limited. - select { - case <-update: - default: - } - // Puts the most recent update to the channel. - update <- servingStatus - } -} - -// Shutdown sets all serving status to NOT_SERVING, and configures the server to -// ignore all future status changes. -// -// This changes serving status for all services. To set status for a particular -// services, call SetServingStatus(). -func (s *Server) Shutdown() { - s.mu.Lock() - defer s.mu.Unlock() - s.shutdown = true - for service := range s.statusMap { - s.setServingStatusLocked(service, healthpb.HealthCheckResponse_NOT_SERVING) - } -} - -// Resume sets all serving status to SERVING, and configures the server to -// accept all future status changes. -// -// This changes serving status for all services. To set status for a particular -// services, call SetServingStatus(). -func (s *Server) Resume() { - s.mu.Lock() - defer s.mu.Unlock() - s.shutdown = false - for service := range s.statusMap { - s.setServingStatusLocked(service, healthpb.HealthCheckResponse_SERVING) - } -} diff --git a/vendor/modules.txt b/vendor/modules.txt index d7da0ad7201..831ac5fbb5b 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -815,7 +815,6 @@ github.com/prometheus/common/config github.com/prometheus/common/expfmt github.com/prometheus/common/helpers/templates github.com/prometheus/common/model -github.com/prometheus/common/promlog github.com/prometheus/common/route github.com/prometheus/common/version # github.com/prometheus/common/sigv4 v0.1.0 @@ -1001,7 +1000,6 @@ github.com/thanos-io/thanos/pkg/losertree github.com/thanos-io/thanos/pkg/metadata/metadatapb github.com/thanos-io/thanos/pkg/model github.com/thanos-io/thanos/pkg/pool -github.com/thanos-io/thanos/pkg/prober github.com/thanos-io/thanos/pkg/promclient github.com/thanos-io/thanos/pkg/query github.com/thanos-io/thanos/pkg/querysharding @@ -1292,7 +1290,6 @@ golang.org/x/net/context/ctxhttp golang.org/x/net/http/httpguts golang.org/x/net/http/httpproxy golang.org/x/net/http2 -golang.org/x/net/http2/h2c golang.org/x/net/http2/hpack golang.org/x/net/idna golang.org/x/net/internal/iana @@ -1429,7 +1426,6 @@ google.golang.org/grpc/encoding/gzip google.golang.org/grpc/encoding/proto google.golang.org/grpc/experimental google.golang.org/grpc/grpclog -google.golang.org/grpc/health google.golang.org/grpc/health/grpc_health_v1 google.golang.org/grpc/internal google.golang.org/grpc/internal/backoff From 018c931eeadb31bf639d9a201a72732140f8f544 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 11 Sep 2024 14:09:15 -0700 Subject: [PATCH 24/38] fix issue with sample pointers Signed-off-by: Ahmed Hassan --- .../instantquery/instant_query_test.go | 48 +++++++++---------- pkg/querier/tripperware/merge.go | 6 +-- 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index d210d8e3abe..2c8ed13358e 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -692,7 +692,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: make([]*/tripperware.Sample, 0), + Samples: make([]tripperware.Sample, 0), }, }, }, @@ -715,7 +715,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{}, + Samples: []tripperware.Sample{}, }, }, }, @@ -744,7 +744,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -774,7 +774,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -812,7 +812,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -835,7 +835,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -865,7 +865,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -896,7 +896,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -934,7 +934,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -958,7 +958,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -989,7 +989,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1013,7 +1013,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1044,7 +1044,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1068,7 +1068,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1099,7 +1099,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1123,7 +1123,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1154,7 +1154,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1179,7 +1179,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1211,7 +1211,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -1243,7 +1243,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {Name: "__name__", Value: "up"}, @@ -1527,7 +1527,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1551,7 +1551,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1583,7 +1583,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, @@ -1607,7 +1607,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ - Samples: []*/tripperware.Sample{ + Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ {"__name__", "up"}, diff --git a/pkg/querier/tripperware/merge.go b/pkg/querier/tripperware/merge.go index 87a49448ea1..3c959366a6a 100644 --- a/pkg/querier/tripperware/merge.go +++ b/pkg/querier/tripperware/merge.go @@ -181,7 +181,7 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) for k, v := range output { samples = append(samples, &pair{ metric: k, - s: &v, + s: v, }) } @@ -199,7 +199,7 @@ func vectorMerge(ctx context.Context, req Request, resps []*PrometheusResponse) }) for _, p := range samples { - result.Samples = append(result.Samples, *p.s) + result.Samples = append(result.Samples, p.s) } return result, nil } @@ -261,7 +261,7 @@ const ( type pair struct { metric string - s *Sample + s Sample } // getSortValueFromPair gets the float value used for sorting from samples. From 0afa97d84c1d06f2627bb15c349096724bf94ba7 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 13 Sep 2024 12:11:40 -0700 Subject: [PATCH 25/38] fix protobuf tests to expect peakSamples Signed-off-by: Ahmed Hassan --- .../instantquery/instant_query_test.go | 19 +++++++++++++------ .../queryrange/query_range_test.go | 5 +++-- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 45071f69cf7..02e8d0b77f0 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -269,7 +269,7 @@ func TestResponse(t *testing.T) { }, }, { - jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1,"137"],[2,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]],"peakSamples":10}}}}`, promBody: &tripperware.PrometheusResponse{ Status: "success", Data: tripperware.PrometheusData{ @@ -298,6 +298,7 @@ func TestResponse(t *testing.T) { {Value: 5, TimestampMs: 1536673780000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -732,6 +733,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: &tripperware.PrometheusResponseSamplesStats{ TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{}, TotalQueryableSamples: 0, + PeakSamples: 10, }, }, }, @@ -740,7 +742,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[],"stats":{"samples":{"totalQueryableSamples":0,"totalQueryableSamplesPerStep":[],"peakSamples":10}}}}`, }, { name: "single response", @@ -800,6 +802,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -808,7 +811,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1,10]],"peakSamples":10}}}}`, }, { name: "duplicated response", @@ -891,6 +894,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -922,6 +926,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -930,7 +935,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]],"peakSamples":10}}}}`, }, { name: "merge two responses", @@ -1238,6 +1243,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -1270,6 +1276,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, + PeakSamples: 10, }, }, }, @@ -1278,7 +1285,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]]}}}}`, + expectedResp: `{"status":"success","data":{"resultType":"vector","result":[{"metric":{"__name__":"up","job":"bar"},"value":[2,"2"]},{"metric":{"__name__":"up","job":"foo"},"value":[1,"1"]}],"stats":{"samples":{"totalQueryableSamples":20,"totalQueryableSamplesPerStep":[[1,20]],"peakSamples":10}}}}`, }, { name: "responses don't contain vector, should return an error", @@ -1313,7 +1320,7 @@ func TestMergeResponseProtobuf(t *testing.T) { }, }, }, - expectedErr: fmt.Errorf("unexpected result type on instant query: %s", "string"), + expectedErr: errors.New("unexpected result type: string"), }, { name: "single matrix response", diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 4945bece08c..8eda6555056 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -362,7 +362,7 @@ func TestResponseWithStats(t *testing.T) { isProtobuf: true, }, { - jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}}`, + jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]],"peakSamples":16}}}}`, promBody: &tripperware.PrometheusResponse{ Status: "success", Data: tripperware.PrometheusData{ @@ -391,6 +391,7 @@ func TestResponseWithStats(t *testing.T) { {Value: 5, TimestampMs: 1536673680000}, {Value: 5, TimestampMs: 1536673780000}, }, + PeakSamples: 16, }, }, }, @@ -1208,7 +1209,7 @@ func TestCompressedResponse(t *testing.T) { {Value: 2, TimestampMs: 2000}, {Value: 3, TimestampMs: 3000}, }, - PeakSamples: 16, + PeakSamples: 10, }}, }, Headers: []*tripperware.PrometheusResponseHeader{}, From 4da5cb0e3aaefcabef453ef5ca3456c2c71c668f Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 16 Sep 2024 19:51:56 -0700 Subject: [PATCH 26/38] refactor protobuf config logic Signed-off-by: Ahmed Hassan --- pkg/api/api.go | 4 +- pkg/api/handlers.go | 5 +-- pkg/cortex/modules.go | 6 +-- pkg/querier/codec/protobuf_codec.go | 1 + .../tripperware/instantquery/instant_query.go | 35 ++++----------- .../instantquery/instant_query_test.go | 8 ++-- .../instantquery/shard_by_query_test.go | 2 +- pkg/querier/tripperware/query.go | 24 +++++++++++ .../tripperware/queryrange/marshaling_test.go | 32 ++++++++++++-- .../tripperware/queryrange/query_range.go | 43 ++++--------------- .../query_range_middlewares_test.go | 4 +- .../queryrange/query_range_test.go | 14 +++--- .../queryrange/results_cache_test.go | 4 +- 13 files changed, 93 insertions(+), 89 deletions(-) diff --git a/pkg/api/api.go b/pkg/api/api.go index 660c7ceb28c..cfcfa050c36 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -74,7 +74,7 @@ type Config struct { buildInfoEnabled bool `yaml:"build_info_enabled"` - ProtobufQuerierHandler bool `yaml:"protobuf_querier_handler"` + QuerierDefaultCodec string `yaml:"querier_default_codec"` } // RegisterFlags adds the flags required to config this to the given FlagSet. @@ -82,7 +82,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.ResponseCompression, "api.response-compression-enabled", false, "Use GZIP compression for API responses. Some endpoints serve large YAML or JSON blobs which can benefit from compression.") f.Var(&cfg.HTTPRequestHeadersToLog, "api.http-request-headers-to-log", "Which HTTP Request headers to add to logs") f.BoolVar(&cfg.buildInfoEnabled, "api.build-info-enabled", false, "If enabled, build Info API will be served by query frontend or querier.") - f.BoolVar(&cfg.ProtobufQuerierHandler, "api.protobuf_querier_handler", false, "Enable protobuf serialization for range query and unsharded instant query responses from querier") + f.StringVar(&cfg.QuerierDefaultCodec, "api.querier-default-codec", "json", "Choose default codec for querier response serialization. Supports 'json' and 'protobuf'.") cfg.RegisterFlagsWithPrefix("", f) } diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 1f30ae402e4..a63a1146f34 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -232,9 +232,8 @@ func NewQuerierHandler( false, ) - if cfg.ProtobufQuerierHandler { - api.InstallCodec(codec.ProtobufCodec{}) - } + // JSON codec is already installed. Install Protobuf codec to give the option for using either. + api.InstallCodec(codec.ProtobufCodec{}) router := mux.NewRouter() diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 2610a0af9fa..0a437af0e16 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -442,9 +442,9 @@ func (t *Cortex) initFlusher() (serv services.Service, err error) { func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err error) { queryAnalyzer := querysharding.NewQueryAnalyzer() // PrometheusCodec is a codec to encode and decode Prometheus query range requests and responses. - prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler) + prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec) // ShardedPrometheusCodec is same as PrometheusCodec but to be used on the sharded queries (it sum up the stats) - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler) + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec) queryRangeMiddlewares, cache, err := queryrange.Middlewares( t.Cfg.QueryRange, @@ -472,7 +472,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro queryRangeMiddlewares, instantQueryMiddlewares, prometheusCodec, - instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.ProtobufQuerierHandler), + instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec), t.Overrides, queryAnalyzer, t.Cfg.Querier.DefaultEvaluationInterval, diff --git a/pkg/querier/codec/protobuf_codec.go b/pkg/querier/codec/protobuf_codec.go index 28a4221f97d..942d783f306 100644 --- a/pkg/querier/codec/protobuf_codec.go +++ b/pkg/querier/codec/protobuf_codec.go @@ -152,6 +152,7 @@ func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSample statSamples := tripperware.PrometheusResponseSamplesStats{ TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, + PeakSamples: int64(builtin.Samples.PeakSamples), } return &statSamples diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index e04bfec8ab0..975e5fe9fd4 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -9,7 +9,6 @@ import ( "net/url" "strings" "time" - "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -18,13 +17,12 @@ import ( "google.golang.org/grpc/status" "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" ) var ( - InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("", true) + InstantQueryCodec tripperware.Codec = NewInstantQueryCodec("", "protobuf") json = jsoniter.Config{ EscapeHTML: false, // No HTML in our responses. @@ -35,21 +33,15 @@ var ( type instantQueryCodec struct { tripperware.Codec - compression queryrange.Compression - enableProtobuf bool + compression string + defaultCodec string now func() time.Time } -func NewInstantQueryCodec(compressionStr string, enableProtobuf bool) instantQueryCodec { - var compression queryrange.Compression - if compressionStr == "gzip" { - compression = queryrange.Compression(compressionStr) - } else { - compression = queryrange.DisableCompression - } +func NewInstantQueryCodec(compression string, defaultCodec string) instantQueryCodec { return instantQueryCodec{ compression: compression, - enableProtobuf: enableProtobuf, + defaultCodec: defaultCodec, now: time.Now, } } @@ -97,11 +89,7 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ } var resp tripperware.PrometheusResponse - if r.Header != nil && r.Header.Get("Content-Type") == "application/x-protobuf" { - err = proto.Unmarshal(buf, &resp) - } else { - err = json.Unmarshal(buf, &resp) - } + err = tripperware.UnmarshalResponse(r, buf, &resp) if err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) @@ -141,14 +129,7 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } } - if c.compression == queryrange.GzipCompression { - h.Set("Accept-Encoding", string(c.compression)) - } - if c.enableProtobuf { - h.Set("Accept", "application/x-protobuf") - } else { - h.Set("Accept", "application/json") - } + tripperware.SetRequestHeaders(h, c.defaultCodec, c.compression) req := &http.Request{ Method: "GET", @@ -179,7 +160,7 @@ func (instantQueryCodec) EncodeResponse(ctx context.Context, res tripperware.Res resp := http.Response{ Header: http.Header{ - "Content-Type": []string{"application/json"}, + "Content-Type": []string{tripperware.ApplicationJson}, }, Body: io.NopCloser(bytes.NewBuffer(b)), StatusCode: http.StatusOK, diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 02e8d0b77f0..be951cf5ac4 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -172,8 +172,8 @@ func TestCompressedResponse(t *testing.T) { protobuf, err := proto.Marshal(tc.promBody) b = protobuf require.NoError(t, err) - h.Set("Content-Type", "application/x-protobuf") - tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) + h.Set("Content-Type", tripperware.ApplicationProtobuf) + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{tripperware.ApplicationProtobuf}}) } else { b = []byte(tc.jsonBody) h.Set("Content-Type", "application/json") @@ -376,10 +376,10 @@ func TestResponse(t *testing.T) { require.NoError(t, err) response = &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, Body: io.NopCloser(bytes.NewBuffer(protobuf)), } - tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{tripperware.ApplicationProtobuf}}) } else { response = &http.Response{ StatusCode: 200, diff --git a/pkg/querier/tripperware/instantquery/shard_by_query_test.go b/pkg/querier/tripperware/instantquery/shard_by_query_test.go index 26980cb8dd2..0f873b5d9a0 100644 --- a/pkg/querier/tripperware/instantquery/shard_by_query_test.go +++ b/pkg/querier/tripperware/instantquery/shard_by_query_test.go @@ -9,5 +9,5 @@ import ( func Test_shardQuery(t *testing.T) { t.Parallel() - tripperware.TestQueryShardQuery(t, InstantQueryCodec, queryrange.NewPrometheusCodec(true, "", true)) + tripperware.TestQueryShardQuery(t, InstantQueryCodec, queryrange.NewPrometheusCodec(true, "", "protobuf")) } \ No newline at end of file diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index e77fa676248..fc0121eff27 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -38,6 +38,12 @@ var ( }.Froze() ) +const ( + GzipCompression string = "gzip" + ApplicationProtobuf string = "application/x-protobuf" + ApplicationJson string = "application/json" +) + // Codec is used to encode/decode query range requests and responses so they can be passed down to middlewares. type Codec interface { Merger @@ -738,3 +744,21 @@ func (s *PrometheusResponseStats) MarshalJSON() ([]byte, error) { } return json.Marshal(stats) } + +func SetRequestHeaders(h http.Header, defaultCodec string, compression string) { + if compression == GzipCompression { + h.Set("Accept-Encoding", GzipCompression) + } + if defaultCodec == "protobuf" { + h.Set("Accept", ApplicationProtobuf) + } + h.Set("Accept", ApplicationJson) +} + +func UnmarshalResponse(r *http.Response, buf []byte, resp *PrometheusResponse) error { + if r.Header != nil && r.Header.Get("Content-Type") == ApplicationProtobuf { + return proto.Unmarshal(buf, resp) + } else { + return json.Unmarshal(buf, resp) + } +} \ No newline at end of file diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index dc669371be9..cdbcce99c4b 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -15,7 +15,33 @@ import ( "github.com/cortexproject/cortex/pkg/querier/tripperware" ) -func BenchmarkPrometheusCodec_DecodeResponse(b *testing.B) { +func BenchmarkPrometheusCodec_DecodeResponse_Json(b *testing.B) { + const ( + numSeries = 1000 + numSamplesPerSeries = 1000 + ) + + // Generate a mocked response and marshal it. + res := mockPrometheusResponse(numSeries, numSamplesPerSeries) + encodedRes, err := json.Marshal(res) + require.NoError(b, err) + b.Log("test prometheus response size:", len(encodedRes)) + + b.ResetTimer() + b.ReportAllocs() + + for n := 0; n < b.N; n++ { + _, err := PrometheusCodec.DecodeResponse(context.Background(), &http.Response{ + StatusCode: 200, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationJson}}, + Body: io.NopCloser(bytes.NewReader(encodedRes)), + ContentLength: int64(len(encodedRes)), + }, nil) + require.NoError(b, err) + } +} + +func BenchmarkPrometheusCodec_DecodeResponse_Protobuf(b *testing.B) { const ( numSeries = 1000 numSamplesPerSeries = 1000 @@ -33,7 +59,7 @@ func BenchmarkPrometheusCodec_DecodeResponse(b *testing.B) { for n := 0; n < b.N; n++ { _, err := PrometheusCodec.DecodeResponse(context.Background(), &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, Body: io.NopCloser(bytes.NewReader(encodedRes)), ContentLength: int64(len(encodedRes)), }, nil) @@ -87,7 +113,7 @@ func mockPrometheusResponse(numSeries, numSamplesPerSeries int) *tripperware.Pro return &tripperware.PrometheusResponse{ Status: "success", Data: tripperware.PrometheusData{ - ResultType: "vector", + ResultType: "matrix", Result: tripperware.PrometheusQueryResult{ Result: &tripperware.PrometheusQueryResult_Matrix{ Matrix: &tripperware.Matrix{ diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index de0b1ca8fb6..7a798a5b350 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -11,8 +11,6 @@ import ( "strings" "time" - "github.com/gogo/protobuf/proto" - "github.com/gogo/status" jsoniter "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -28,15 +26,6 @@ import ( // StatusSuccess Prometheus success result. const StatusSuccess = "success" -type Compression string - -const ( - DisableCompression Compression = "" - GzipCompression Compression = "gzip" - applicationProtobuf string = "application/x-protobuf" - applicationJson string = "application/json" -) - var ( matrix = model.ValMatrix.String() json = jsoniter.Config{ @@ -67,22 +56,17 @@ func (resp *PrometheusResponse) HTTPHeaders() map[string][]string { } type prometheusCodec struct { + tripperware.Codec sharded bool - compression Compression - enableProtobuf bool + compression string + defaultCodec string } -func NewPrometheusCodec(sharded bool, compressionStr string, enableProtobuf bool) *prometheusCodec { //nolint:revive - var compression Compression - if compressionStr == "gzip" || compressionStr == "snappy" { - compression = Compression(compressionStr) - } else { - compression = DisableCompression - } +func NewPrometheusCodec(sharded bool, compression string, defaultCodec string) *prometheusCodec { //nolint:revive return &prometheusCodec{ sharded: sharded, compression: compression, - enableProtobuf: enableProtobuf, + defaultCodec: defaultCodec, } } @@ -182,14 +166,7 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } - if c.compression == GzipCompression { - h.Set("Accept-Encoding", string(c.compression)) - } - if c.enableProtobuf { - h.Set("Accept", applicationProtobuf) - } else { - h.Set("Accept", applicationJson) - } + tripperware.SetRequestHeaders(h, c.defaultCodec, c.compression) req := &http.Request{ Method: "GET", @@ -221,11 +198,7 @@ func (c prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ log.LogFields(otlog.Int("bytes", len(buf))) var resp tripperware.PrometheusResponse - if r.Header != nil && r.Header.Get("Content-Type") == applicationProtobuf { - err = proto.Unmarshal(buf, &resp) - } else { - err = json.Unmarshal(buf, &resp) - } + err = tripperware.UnmarshalResponse(r, buf, &resp) if err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) @@ -260,7 +233,7 @@ func (prometheusCodec) EncodeResponse(ctx context.Context, res tripperware.Respo resp := http.Response{ Header: http.Header{ - "Content-Type": []string{applicationJson}, + "Content-Type": []string{tripperware.ApplicationJson}, }, Body: io.NopCloser(bytes.NewBuffer(b)), StatusCode: http.StatusOK, diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 0dbbebce13e..c4479e01ca3 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -20,8 +20,8 @@ import ( ) var ( - PrometheusCodec = NewPrometheusCodec(false, "", true) - ShardedPrometheusCodec = NewPrometheusCodec(false, "", true) + PrometheusCodec = NewPrometheusCodec(false, "", "protobuf") + ShardedPrometheusCodec = NewPrometheusCodec(false, "", "protobuf") ) func TestRoundTrip(t *testing.T) { diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 8eda6555056..0e5fb686463 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -277,14 +277,14 @@ func TestResponse(t *testing.T) { if tc.isProtobuf { response = &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{applicationProtobuf}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, Body: io.NopCloser(bytes.NewBuffer(protobuf)), } tc.promBody.Headers = respHeadersProtobuf } else { response = &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{applicationJson}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationJson}}, Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), } tc.promBody.Headers = respHeadersJson @@ -409,14 +409,14 @@ func TestResponseWithStats(t *testing.T) { if tc.isProtobuf { response = &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{applicationProtobuf}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, Body: io.NopCloser(bytes.NewBuffer(protobuf)), } tc.promBody.Headers = respHeadersProtobuf } else { response = &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{applicationJson}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationJson}}, Body: io.NopCloser(bytes.NewBuffer([]byte(tc.jsonBody))), } tc.promBody.Headers = respHeadersJson @@ -1282,12 +1282,12 @@ func TestCompressedResponse(t *testing.T) { protobuf, err := proto.Marshal(tc.promBody) b = protobuf require.NoError(t, err) - h.Set("Content-Type", "application/x-protobuf") - tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{"application/x-protobuf"}}) + h.Set("Content-Type", tripperware.ApplicationProtobuf) + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{tripperware.ApplicationProtobuf}}) } else { b = []byte(tc.jsonBody) - h.Set("Content-Type", "application/json") + h.Set("Content-Type", tripperware.ApplicationJson) } responseBody := bytes.NewBuffer(b) diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index d6ddf16b6e5..e3d00e37b5c 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -61,14 +61,14 @@ var ( respHeadersJson = []*tripperware.PrometheusResponseHeader{ { Name: "Content-Type", - Values: []string{applicationJson}, + Values: []string{tripperware.ApplicationJson}, }, } respHeadersProtobuf = []*tripperware.PrometheusResponseHeader{ { Name: "Content-Type", - Values: []string{applicationProtobuf}, + Values: []string{tripperware.ApplicationProtobuf}, }, } parsedResponse = &tripperware.PrometheusResponse{ From a917d28787a2de19d28ead3f539b590018b7ec6d Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 16 Sep 2024 19:55:48 -0700 Subject: [PATCH 27/38] refactor protobuf codec Signed-off-by: Ahmed Hassan --- pkg/querier/codec/protobuf_codec.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/querier/codec/protobuf_codec.go b/pkg/querier/codec/protobuf_codec.go index 942d783f306..abbc3e19146 100644 --- a/pkg/querier/codec/protobuf_codec.go +++ b/pkg/querier/codec/protobuf_codec.go @@ -25,27 +25,27 @@ func (p ProtobufCodec) CanEncode(resp *v1.Response) bool { } func (p ProtobufCodec) Encode(resp *v1.Response) ([]byte, error) { - prometheusInstantQueryResponse, err := createPrometheusInstantQueryResponse(resp) + prometheusQueryResponse, err := createPrometheusQueryResponse(resp) if err != nil { return []byte{}, err } - b, err := proto.Marshal(prometheusInstantQueryResponse) + b, err := proto.Marshal(prometheusQueryResponse) return b, err } -func createPrometheusInstantQueryResponse(resp *v1.Response) (*tripperware.PrometheusResponse, error) { +func createPrometheusQueryResponse(resp *v1.Response) (*tripperware.PrometheusResponse, error) { var data = resp.Data.(*v1.QueryData) - var instantQueryResult tripperware.PrometheusQueryResult + var queryResult tripperware.PrometheusQueryResult switch string(data.ResultType) { case model.ValMatrix.String(): - instantQueryResult.Result = &tripperware.PrometheusQueryResult_Matrix{ + queryResult.Result = &tripperware.PrometheusQueryResult_Matrix{ Matrix: &tripperware.Matrix{ SampleStreams: *getMatrixSampleStreams(data), }, } case model.ValVector.String(): - instantQueryResult.Result = &tripperware.PrometheusQueryResult_Vector{ + queryResult.Result = &tripperware.PrometheusQueryResult_Vector{ Vector: &tripperware.Vector{ Samples: *getVectorSamples(data), }, @@ -56,7 +56,7 @@ func createPrometheusInstantQueryResponse(resp *v1.Response) (*tripperware.Prome if err != nil { return nil, err } - instantQueryResult.Result = &tripperware.PrometheusQueryResult_RawBytes{RawBytes: rawBytes} + queryResult.Result = &tripperware.PrometheusQueryResult_RawBytes{RawBytes: rawBytes} } var stats *tripperware.PrometheusResponseStats @@ -69,7 +69,7 @@ func createPrometheusInstantQueryResponse(resp *v1.Response) (*tripperware.Prome Status: string(resp.Status), Data: tripperware.PrometheusData{ ResultType: string(data.ResultType), - Result: instantQueryResult, + Result: queryResult, Stats: stats, }, ErrorType: string(resp.ErrorType), From 2f1b3159bdf860805c83b877da3f007e0b8b2588 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 17 Sep 2024 19:02:53 -0700 Subject: [PATCH 28/38] add config validation Signed-off-by: Ahmed Hassan --- pkg/api/api.go | 13 ++++++++++ pkg/cortex/cortex.go | 3 +++ pkg/cortex/modules.go | 6 ++--- pkg/querier/querier.go | 11 +++++--- .../tripperware/instantquery/instant_query.go | 26 +++++++++++++------ pkg/querier/tripperware/query.go | 23 ++++++++++------ .../tripperware/queryrange/query_range.go | 26 +++++++++++++------ 7 files changed, 78 insertions(+), 30 deletions(-) diff --git a/pkg/api/api.go b/pkg/api/api.go index cfcfa050c36..92b6e1f3687 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -12,6 +12,7 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/regexp" "github.com/klauspost/compress/gzhttp" + "github.com/pkg/errors" "github.com/prometheus/prometheus/model/relabel" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/httputil" @@ -77,6 +78,10 @@ type Config struct { QuerierDefaultCodec string `yaml:"querier_default_codec"` } +var ( + errUnsupportedDefaultCodec = errors.New("unsupported default codec type. Supported types are 'json' and 'protobuf'") +) + // RegisterFlags adds the flags required to config this to the given FlagSet. func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.ResponseCompression, "api.response-compression-enabled", false, "Use GZIP compression for API responses. Some endpoints serve large YAML or JSON blobs which can benefit from compression.") @@ -93,6 +98,14 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.StringVar(&cfg.corsRegexString, prefix+"server.cors-origin", ".*", `Regex for CORS origin. It is fully anchored. Example: 'https?://(domain1|domain2)\.com'`) } +// validate config +func (cfg *Config) Validate() error { + if cfg.QuerierDefaultCodec != "json" && cfg.QuerierDefaultCodec != "protobuf" { + return errUnsupportedDefaultCodec + } + return nil +} + // Push either wraps the distributor push function as configured or returns the distributor push directly. func (cfg *Config) wrapDistributorPush(d *distributor.Distributor) push.Func { if cfg.DistributorPushWrapper != nil { diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 1607bb525cc..03ceb6e3cc3 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -182,6 +182,9 @@ func (c *Config) Validate(log log.Logger) error { return errInvalidHTTPPrefix } + if err := c.API.Validate(); err != nil { + return errors.Wrap(err, "invalid api config") + } if err := c.Storage.Validate(); err != nil { return errors.Wrap(err, "invalid storage config") } diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 0a437af0e16..e32a7437ad5 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -442,9 +442,9 @@ func (t *Cortex) initFlusher() (serv services.Service, err error) { func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err error) { queryAnalyzer := querysharding.NewQueryAnalyzer() // PrometheusCodec is a codec to encode and decode Prometheus query range requests and responses. - prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec) + prometheusCodec := queryrange.NewPrometheusCodec(false, t.Cfg.Querier.ResponseCompression, t.Cfg.API.QuerierDefaultCodec) // ShardedPrometheusCodec is same as PrometheusCodec but to be used on the sharded queries (it sum up the stats) - shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec) + shardedPrometheusCodec := queryrange.NewPrometheusCodec(true, t.Cfg.Querier.ResponseCompression, t.Cfg.API.QuerierDefaultCodec) queryRangeMiddlewares, cache, err := queryrange.Middlewares( t.Cfg.QueryRange, @@ -472,7 +472,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro queryRangeMiddlewares, instantQueryMiddlewares, prometheusCodec, - instantquery.NewInstantQueryCodec(t.Cfg.Querier.PrometheusCodecCompression, t.Cfg.API.QuerierDefaultCodec), + instantquery.NewInstantQueryCodec(t.Cfg.Querier.ResponseCompression, t.Cfg.API.QuerierDefaultCodec), t.Overrides, queryAnalyzer, t.Cfg.Querier.DefaultEvaluationInterval, diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 6f10900c138..64ab768424c 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -49,8 +49,8 @@ type Config struct { QueryIngestersWithin time.Duration `yaml:"query_ingesters_within"` EnablePerStepStats bool `yaml:"per_step_stats_enabled"` - // Use compression when returning promql response. Supported values are 'gzip' or '' (disable compression) - PrometheusCodecCompression string `yaml:"prometheus_codec_compression"` + // Use compression for metrics query API or instant and range query APIs. + ResponseCompression string `yaml:"response_compression"` // QueryStoreAfter the time after which queries should also be sent to the store and not just ingesters. QueryStoreAfter time.Duration `yaml:"query_store_after"` @@ -92,6 +92,7 @@ var ( errBadLookbackConfigs = errors.New("bad settings, query_store_after >= query_ingesters_within which can result in queries not being sent") errShuffleShardingLookbackLessThanQueryStoreAfter = errors.New("the shuffle-sharding lookback period should be greater or equal than the configured 'query store after'") errEmptyTimeRange = errors.New("empty time range") + errUnsupportedResponseCompression = errors.New("unsupported response compression. Supported compression 'gzip' and '' (disable compression)") ) // RegisterFlags adds the flags required to config this to the given FlagSet. @@ -112,7 +113,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxSamples, "querier.max-samples", 50e6, "Maximum number of samples a single query can load into memory.") f.DurationVar(&cfg.QueryIngestersWithin, "querier.query-ingesters-within", 0, "Maximum lookback beyond which queries are not sent to ingester. 0 means all queries are sent to ingester.") f.BoolVar(&cfg.EnablePerStepStats, "querier.per-step-stats-enabled", false, "Enable returning samples stats per steps in query response.") - f.StringVar(&cfg.PrometheusCodecCompression, "querier.prometheus_codec_compression", "", "Use compression when returning promql response. Supported values 'gzip', 'snappy', and '' (disable compression)") + f.StringVar(&cfg.ResponseCompression, "querier.response-compression", "", "Use compression for metrics query API or instant and range query APIs. Supports 'gzip' and '' (disable compression)") f.DurationVar(&cfg.MaxQueryIntoFuture, "querier.max-query-into-future", 10*time.Minute, "Maximum duration into the future you can query. 0 to disable.") f.DurationVar(&cfg.DefaultEvaluationInterval, "querier.default-evaluation-interval", time.Minute, "The default evaluation interval or step size for subqueries.") f.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 0, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. When running the blocks storage, if this option is enabled, the time range of the query sent to the store will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") @@ -135,6 +136,10 @@ func (cfg *Config) Validate() error { } } + if cfg.ResponseCompression != "" && cfg.ResponseCompression != "gzip" { + return errUnsupportedResponseCompression + } + if cfg.ShuffleShardingIngestersLookbackPeriod > 0 { if cfg.ShuffleShardingIngestersLookbackPeriod < cfg.QueryStoreAfter { return errShuffleShardingLookbackLessThanQueryStoreAfter diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 975e5fe9fd4..23305252eed 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -33,16 +33,26 @@ var ( type instantQueryCodec struct { tripperware.Codec - compression string - defaultCodec string - now func() time.Time + compression tripperware.Compression + defaultCodecType tripperware.CodecType + now func() time.Time } -func NewInstantQueryCodec(compression string, defaultCodec string) instantQueryCodec { +func NewInstantQueryCodec(compressionStr string, defaultCodecTypeStr string) instantQueryCodec { + compression := tripperware.NonCompression // default + if compressionStr == string(tripperware.GzipCompression) { + compression = tripperware.GzipCompression + } + + defaultCodecType := tripperware.JsonCodecType // default + if defaultCodecTypeStr == string(tripperware.ProtobufCodecType) { + defaultCodecType = tripperware.ProtobufCodecType + } + return instantQueryCodec{ - compression: compression, - defaultCodec: defaultCodec, - now: time.Now, + compression: compression, + defaultCodecType: defaultCodecType, + now: time.Now, } } @@ -129,7 +139,7 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } } - tripperware.SetRequestHeaders(h, c.defaultCodec, c.compression) + tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) req := &http.Request{ Method: "GET", diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index fc0121eff27..a07827a96e5 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -38,8 +38,14 @@ var ( }.Froze() ) +type CodecType string +type Compression string + const ( - GzipCompression string = "gzip" + GzipCompression Compression = "gzip" + NonCompression Compression = "" + JsonCodecType CodecType = "json" + ProtobufCodecType CodecType = "protobuf" ApplicationProtobuf string = "application/x-protobuf" ApplicationJson string = "application/json" ) @@ -745,20 +751,21 @@ func (s *PrometheusResponseStats) MarshalJSON() ([]byte, error) { return json.Marshal(stats) } -func SetRequestHeaders(h http.Header, defaultCodec string, compression string) { +func SetRequestHeaders(h http.Header, defaultCodecType CodecType, compression Compression) { if compression == GzipCompression { - h.Set("Accept-Encoding", GzipCompression) + h.Set("Accept-Encoding", string(GzipCompression)) } - if defaultCodec == "protobuf" { - h.Set("Accept", ApplicationProtobuf) + if defaultCodecType == ProtobufCodecType { + h.Set("Accept", ApplicationProtobuf + ", " + ApplicationJson) + } else { + h.Set("Accept", ApplicationJson) } - h.Set("Accept", ApplicationJson) } func UnmarshalResponse(r *http.Response, buf []byte, resp *PrometheusResponse) error { if r.Header != nil && r.Header.Get("Content-Type") == ApplicationProtobuf { return proto.Unmarshal(buf, resp) - } else { + }else { return json.Unmarshal(buf, resp) } -} \ No newline at end of file +} diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 7a798a5b350..7e1d30cc661 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -57,16 +57,26 @@ func (resp *PrometheusResponse) HTTPHeaders() map[string][]string { type prometheusCodec struct { tripperware.Codec - sharded bool - compression string - defaultCodec string + sharded bool + compression tripperware.Compression + defaultCodecType tripperware.CodecType } -func NewPrometheusCodec(sharded bool, compression string, defaultCodec string) *prometheusCodec { //nolint:revive +func NewPrometheusCodec(sharded bool, compressionStr string, defaultCodecTypeStr string) *prometheusCodec { //nolint:revive + compression := tripperware.NonCompression // default + if compressionStr == string(tripperware.GzipCompression) { + compression = tripperware.GzipCompression + } + + defaultCodecType := tripperware.JsonCodecType // default + if defaultCodecTypeStr == string(tripperware.ProtobufCodecType) { + defaultCodecType = tripperware.ProtobufCodecType + } + return &prometheusCodec{ - sharded: sharded, - compression: compression, - defaultCodec: defaultCodec, + sharded: sharded, + compression: compression, + defaultCodecType: defaultCodecType, } } @@ -166,7 +176,7 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } - tripperware.SetRequestHeaders(h, c.defaultCodec, c.compression) + tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) req := &http.Request{ Method: "GET", From 6a457e7aa95cbf97f063245de1e22ca4d5c92a54 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 17 Sep 2024 19:11:38 -0700 Subject: [PATCH 29/38] remove snappy compression tests Signed-off-by: Ahmed Hassan --- .../instantquery/instant_query_test.go | 52 +++----------- .../queryrange/query_range_test.go | 70 ++----------------- 2 files changed, 15 insertions(+), 107 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index be951cf5ac4..374359fe673 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -13,7 +13,6 @@ import ( "time" "github.com/gogo/protobuf/proto" - "github.com/golang/snappy" "github.com/weaveworks/common/httpgrpc" "github.com/prometheus/common/model" @@ -129,40 +128,17 @@ func TestCompressedResponse(t *testing.T) { }, status: 200, }, - { - compression: "snappy", - promBody: &tripperware.PrometheusResponse{ - Status: "success", - Data: tripperware.PrometheusData{ - ResultType: model.ValString.String(), - Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, - }, - Headers: []*tripperware.PrometheusResponseHeader{}, - }, - status: 200, - }, { compression: `gzip`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, - { - compression: `snappy`, - jsonBody: `error generic 400`, - status: 400, - err: httpgrpc.Errorf(400, `error generic 400`), - }, { compression: `gzip`, status: 400, err: httpgrpc.Errorf(400, ""), }, - { - compression: `snappy`, - status: 400, - err: httpgrpc.Errorf(400, ""), - }, } { t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() @@ -181,27 +157,15 @@ func TestCompressedResponse(t *testing.T) { responseBody := bytes.NewBuffer(b) var buf bytes.Buffer - if tc.compression == "gzip" { - h.Set("Content-Encoding", "gzip") - if tc.promBody != nil { - tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}}) - } - w := gzip.NewWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } else if tc.compression == "snappy" { - h.Set("Content-Encoding", "snappy") - if tc.promBody != nil { - tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}}) - } - w := snappy.NewBufferedWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf + h.Set("Content-Encoding", tc.compression) + if tc.promBody != nil { + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}}) } + w := gzip.NewWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf response := &http.Response{ StatusCode: tc.status, diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 0e5fb686463..28bf58f092b 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -5,7 +5,6 @@ import ( "compress/gzip" "context" "github.com/gogo/protobuf/proto" - "github.com/golang/snappy" "io" "net/http" "strconv" @@ -1217,62 +1216,17 @@ func TestCompressedResponse(t *testing.T) { jsonBody:`{"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]],"peakSamples":10}}}}`, status: 200, }, - { - compression: `snappy`, - promBody: &tripperware.PrometheusResponse{ - Status: StatusSuccess, - Data: tripperware.PrometheusData{ - ResultType: matrix, - 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}, - }, - }, - }, - }, - }, - }, - Stats: &tripperware.PrometheusResponseStats{Samples: &tripperware.PrometheusResponseSamplesStats{ - TotalQueryableSamples: 20, - TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{ - {Value: 2, TimestampMs: 2000}, - {Value: 3, TimestampMs: 3000}, - }, - }}, - }, - Headers: []*tripperware.PrometheusResponseHeader{}, - }, - jsonBody:`{"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]]}}}}`, - status: 200, - }, { compression: `gzip`, jsonBody: `error generic 400`, status: 400, err: httpgrpc.Errorf(400, `error generic 400`), }, - { - compression: `snappy`, - jsonBody: `error generic 400`, - status: 400, - err: httpgrpc.Errorf(400, `error generic 400`), - }, { compression: `gzip`, status: 400, err: httpgrpc.Errorf(400, ""), }, - { - compression: `snappy`, - status: 400, - err: httpgrpc.Errorf(400, ""), - }, } { t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() @@ -1293,23 +1247,13 @@ func TestCompressedResponse(t *testing.T) { responseBody := bytes.NewBuffer(b) var buf bytes.Buffer - if tc.compression == "gzip" { - h.Set("Content-Encoding", "gzip") - if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} - w := gzip.NewWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } else if tc.compression == "snappy" { - h.Set("Content-Encoding", "snappy") - if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"snappy"}})} - w := snappy.NewBufferedWriter(&buf) - _, err := w.Write(b) - require.NoError(t, err) - w.Close() - responseBody = &buf - } + h.Set("Content-Encoding", tc.compression) + if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} + w := gzip.NewWriter(&buf) + _, err := w.Write(b) + require.NoError(t, err) + w.Close() + responseBody = &buf response := &http.Response{ StatusCode: tc.status, From 7898191a6c9286078f94ec561d24858b765d1e32 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 18 Sep 2024 16:43:31 -0700 Subject: [PATCH 30/38] fix linter formatting Signed-off-by: Ahmed Hassan --- .../tripperware/instantquery/instant_query.go | 14 +- .../instantquery/instant_query_test.go | 125 ++++++++++-------- .../instantquery/shard_by_query_test.go | 2 +- pkg/querier/tripperware/query.go | 12 +- .../tripperware/queryrange/marshaling_test.go | 4 +- .../tripperware/queryrange/query_range.go | 6 +- .../queryrange/query_range_test.go | 45 +++---- .../queryrange/results_cache_test.go | 25 ---- 8 files changed, 111 insertions(+), 122 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 23305252eed..4fc9064dbc8 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -33,9 +33,9 @@ var ( type instantQueryCodec struct { tripperware.Codec - compression tripperware.Compression - defaultCodecType tripperware.CodecType - now func() time.Time + compression tripperware.Compression + defaultCodecType tripperware.CodecType + now func() time.Time } func NewInstantQueryCodec(compressionStr string, defaultCodecTypeStr string) instantQueryCodec { @@ -48,11 +48,11 @@ func NewInstantQueryCodec(compressionStr string, defaultCodecTypeStr string) ins if defaultCodecTypeStr == string(tripperware.ProtobufCodecType) { defaultCodecType = tripperware.ProtobufCodecType } - + return instantQueryCodec{ - compression: compression, - defaultCodecType: defaultCodecType, - now: time.Now, + compression: compression, + defaultCodecType: defaultCodecType, + now: time.Now, } } diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index 374359fe673..b35e4551024 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -122,7 +122,11 @@ func TestCompressedResponse(t *testing.T) { Status: "success", Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"string","result":[1,"foo"]}`), + }, + }, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -154,18 +158,16 @@ func TestCompressedResponse(t *testing.T) { b = []byte(tc.jsonBody) h.Set("Content-Type", "application/json") } - responseBody := bytes.NewBuffer(b) - var buf bytes.Buffer h.Set("Content-Encoding", tc.compression) if tc.promBody != nil { tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}}) } - w := gzip.NewWriter(&buf) + responseBody := &bytes.Buffer{} + w := gzip.NewWriter(responseBody) _, err := w.Write(b) require.NoError(t, err) w.Close() - responseBody = &buf response := &http.Response{ StatusCode: tc.status, @@ -216,7 +218,11 @@ func TestResponse(t *testing.T) { Status: "success", Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"]}`)}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"string","result":[1,"foo"]}`), + }, + }, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -227,7 +233,11 @@ func TestResponse(t *testing.T) { Status: "success", Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`)}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"string","result":[1,"foo"],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]]}}}`), + }, + }, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -244,7 +254,7 @@ func TestResponse(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"foo", "bar"}, + {Name: "foo", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 137, TimestampMs: 1000}, @@ -262,7 +272,7 @@ func TestResponse(t *testing.T) { {Value: 5, TimestampMs: 1536673780000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -281,7 +291,7 @@ func TestResponse(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"foo", "bar"}, + {Name: "foo", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 137, TimestampMs: 1000}, @@ -302,7 +312,11 @@ func TestResponse(t *testing.T) { Status: "success", Data: tripperware.PrometheusData{ ResultType: model.ValString.String(), - Result: tripperware.PrometheusQueryResult{Result: &tripperware.PrometheusQueryResult_RawBytes{[]byte(`{"resultType":"scalar","result":[1,"13"]}`)}}, + Result: tripperware.PrometheusQueryResult{ + Result: &tripperware.PrometheusQueryResult_RawBytes{ + RawBytes: []byte(`{"resultType":"scalar","result":[1,"13"]}`), + }, + }, }, Headers: []*tripperware.PrometheusResponseHeader{}, }, @@ -697,7 +711,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: &tripperware.PrometheusResponseSamplesStats{ TotalQueryableSamplesPerStep: []*tripperware.PrometheusResponseQueryableSamplesStatsPerStep{}, TotalQueryableSamples: 0, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -722,7 +736,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, + {Name: "__name__", Value: "up"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -766,7 +780,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -791,7 +805,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, + {Name: "__name__", Value: "up"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -814,7 +828,7 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, + {Name: "__name__", Value: "up"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -858,7 +872,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -890,7 +904,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -915,8 +929,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -939,8 +953,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 2000}, }, @@ -970,8 +984,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -994,8 +1008,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1025,8 +1039,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -1049,8 +1063,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1080,8 +1094,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -1104,8 +1118,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1135,8 +1149,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1160,8 +1174,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -1207,7 +1221,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -1240,7 +1254,7 @@ func TestMergeResponseProtobuf(t *testing.T) { {Value: 10, TimestampMs: 1000}, }, TotalQueryableSamples: 10, - PeakSamples: 10, + PeakSamples: 10, }, }, }, @@ -1300,7 +1314,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, + {Name: "__name__", Value: "up"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1333,7 +1347,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {Name: "__name__", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1359,7 +1373,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "foo"}, + {Name: "__name__", Value: "foo"}, }, Samples: []cortexpb.Sample{ {Value: 3, TimestampMs: 3000}, @@ -1392,7 +1406,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {Name: "__name__", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1418,7 +1432,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {Name: "__name__", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 3, TimestampMs: 3000}, @@ -1450,7 +1464,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {Name: "__name__", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1476,7 +1490,7 @@ func TestMergeResponseProtobuf(t *testing.T) { SampleStreams: []tripperware.SampleStream{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "bar"}, + {Name: "__name__", Value: "bar"}, }, Samples: []cortexpb.Sample{ {Value: 1, TimestampMs: 1000}, @@ -1510,8 +1524,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -1534,8 +1548,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1566,8 +1580,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "foo"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "foo"}, }, Sample: &cortexpb.Sample{Value: 1, TimestampMs: 1000}, }, @@ -1590,8 +1604,8 @@ func TestMergeResponseProtobuf(t *testing.T) { Samples: []tripperware.Sample{ { Labels: []cortexpb.LabelAdapter{ - {"__name__", "up"}, - {"job", "bar"}, + {Name: "__name__", Value: "up"}, + {Name: "job", Value: "bar"}, }, Sample: &cortexpb.Sample{Value: 2, TimestampMs: 1000}, }, @@ -1617,6 +1631,7 @@ func TestMergeResponseProtobuf(t *testing.T) { var resps []tripperware.Response for _, r := range tc.resps { protobuf, err := proto.Marshal(r) + require.NoError(t, err) hr := &http.Response{ StatusCode: 200, Header: http.Header{"Content-Type": []string{"application/x-protobuf"}}, diff --git a/pkg/querier/tripperware/instantquery/shard_by_query_test.go b/pkg/querier/tripperware/instantquery/shard_by_query_test.go index 0f873b5d9a0..aac85b2a9a4 100644 --- a/pkg/querier/tripperware/instantquery/shard_by_query_test.go +++ b/pkg/querier/tripperware/instantquery/shard_by_query_test.go @@ -10,4 +10,4 @@ import ( func Test_shardQuery(t *testing.T) { t.Parallel() tripperware.TestQueryShardQuery(t, InstantQueryCodec, queryrange.NewPrometheusCodec(true, "", "protobuf")) -} \ No newline at end of file +} diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index a07827a96e5..28e7c2430b4 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -44,10 +44,10 @@ type Compression string const ( GzipCompression Compression = "gzip" NonCompression Compression = "" - JsonCodecType CodecType = "json" - ProtobufCodecType CodecType = "protobuf" - ApplicationProtobuf string = "application/x-protobuf" - ApplicationJson string = "application/json" + JsonCodecType CodecType = "json" + ProtobufCodecType CodecType = "protobuf" + ApplicationProtobuf string = "application/x-protobuf" + ApplicationJson string = "application/json" ) // Codec is used to encode/decode query range requests and responses so they can be passed down to middlewares. @@ -756,7 +756,7 @@ func SetRequestHeaders(h http.Header, defaultCodecType CodecType, compression Co h.Set("Accept-Encoding", string(GzipCompression)) } if defaultCodecType == ProtobufCodecType { - h.Set("Accept", ApplicationProtobuf + ", " + ApplicationJson) + h.Set("Accept", ApplicationProtobuf+", "+ApplicationJson) } else { h.Set("Accept", ApplicationJson) } @@ -765,7 +765,7 @@ func SetRequestHeaders(h http.Header, defaultCodecType CodecType, compression Co func UnmarshalResponse(r *http.Response, buf []byte, resp *PrometheusResponse) error { if r.Header != nil && r.Header.Get("Content-Type") == ApplicationProtobuf { return proto.Unmarshal(buf, resp) - }else { + } else { return json.Unmarshal(buf, resp) } } diff --git a/pkg/querier/tripperware/queryrange/marshaling_test.go b/pkg/querier/tripperware/queryrange/marshaling_test.go index cdbcce99c4b..4652a0e10bd 100644 --- a/pkg/querier/tripperware/queryrange/marshaling_test.go +++ b/pkg/querier/tripperware/queryrange/marshaling_test.go @@ -33,7 +33,7 @@ func BenchmarkPrometheusCodec_DecodeResponse_Json(b *testing.B) { for n := 0; n < b.N; n++ { _, err := PrometheusCodec.DecodeResponse(context.Background(), &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{tripperware.ApplicationJson}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationJson}}, Body: io.NopCloser(bytes.NewReader(encodedRes)), ContentLength: int64(len(encodedRes)), }, nil) @@ -59,7 +59,7 @@ func BenchmarkPrometheusCodec_DecodeResponse_Protobuf(b *testing.B) { for n := 0; n < b.N; n++ { _, err := PrometheusCodec.DecodeResponse(context.Background(), &http.Response{ StatusCode: 200, - Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, + Header: http.Header{"Content-Type": []string{tripperware.ApplicationProtobuf}}, Body: io.NopCloser(bytes.NewReader(encodedRes)), ContentLength: int64(len(encodedRes)), }, nil) diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 7e1d30cc661..094351bcfe5 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -57,9 +57,9 @@ func (resp *PrometheusResponse) HTTPHeaders() map[string][]string { type prometheusCodec struct { tripperware.Codec - sharded bool - compression tripperware.Compression - defaultCodecType tripperware.CodecType + sharded bool + compression tripperware.Compression + defaultCodecType tripperware.CodecType } func NewPrometheusCodec(sharded bool, compressionStr string, defaultCodecTypeStr string) *prometheusCodec { //nolint:revive diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 28bf58f092b..8e9c73e04da 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -4,12 +4,13 @@ import ( "bytes" "compress/gzip" "context" - "github.com/gogo/protobuf/proto" "io" "net/http" "strconv" "testing" + "github.com/gogo/protobuf/proto" + "github.com/prometheus/common/model" jsoniter "github.com/json-iterator/go" @@ -121,8 +122,8 @@ func TestResponse(t *testing.T) { }, }, }, - jsonBody: responseBody, - isProtobuf: true, + jsonBody: responseBody, + isProtobuf: true, }, { promBody: &tripperware.PrometheusResponse{ @@ -176,8 +177,8 @@ func TestResponse(t *testing.T) { }, }, }, - jsonBody: responseBody, - isProtobuf: false, + jsonBody: responseBody, + isProtobuf: false, }, { promBody: &tripperware.PrometheusResponse{ @@ -232,8 +233,8 @@ func TestResponse(t *testing.T) { }, }, }, - jsonBody: responseBodyWithWarnings, - isProtobuf: true, + jsonBody: responseBodyWithWarnings, + isProtobuf: true, }, { promBody: &tripperware.PrometheusResponse{ @@ -260,10 +261,10 @@ func TestResponse(t *testing.T) { }, }, }, - jsonBody: responseBodyWithWarnings, - isProtobuf: false, + jsonBody: responseBodyWithWarnings, + isProtobuf: false, }, - } + } for i, tc := range testCases { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -271,7 +272,7 @@ func TestResponse(t *testing.T) { protobuf, err := proto.Marshal(tc.promBody) require.NoError(t, err) ctx, cancelCtx := context.WithCancel(context.Background()) - + var response *http.Response if tc.isProtobuf { response = &http.Response{ @@ -319,9 +320,9 @@ func TestResponse(t *testing.T) { func TestResponseWithStats(t *testing.T) { t.Parallel() for i, tc := range []struct { - promBody *tripperware.PrometheusResponse - jsonBody string - isProtobuf bool + promBody *tripperware.PrometheusResponse + jsonBody string + isProtobuf bool }{ { jsonBody: `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}],"stats":{"samples":{"totalQueryableSamples":10,"totalQueryableSamplesPerStep":[[1536673680,5],[1536673780,5]],"peakSamples":16}}}}`, @@ -1213,8 +1214,8 @@ func TestCompressedResponse(t *testing.T) { }, Headers: []*tripperware.PrometheusResponseHeader{}, }, - jsonBody:`{"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]],"peakSamples":10}}}}`, - status: 200, + jsonBody: `{"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]],"peakSamples":10}}}}`, + status: 200, }, { compression: `gzip`, @@ -1238,22 +1239,20 @@ func TestCompressedResponse(t *testing.T) { require.NoError(t, err) h.Set("Content-Type", tripperware.ApplicationProtobuf) tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Type", Values: []string{tripperware.ApplicationProtobuf}}) - } else { b = []byte(tc.jsonBody) h.Set("Content-Type", tripperware.ApplicationJson) } - responseBody := bytes.NewBuffer(b) - - var buf bytes.Buffer h.Set("Content-Encoding", tc.compression) - if tc.promBody != nil {tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}})} - w := gzip.NewWriter(&buf) + if tc.promBody != nil { + tc.promBody.Headers = append(tc.promBody.Headers, &tripperware.PrometheusResponseHeader{Name: "Content-Encoding", Values: []string{"gzip"}}) + } + responseBody := &bytes.Buffer{} + w := gzip.NewWriter(responseBody) _, err := w.Write(b) require.NoError(t, err) w.Close() - responseBody = &buf response := &http.Response{ StatusCode: tc.status, diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index e3d00e37b5c..8325a71f1b0 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -94,31 +94,6 @@ var ( }, }, } - - parsedResponseWithWarnings = &tripperware.PrometheusResponse{ - Status: "success", - Warnings: []string{"test-warn"}, - Data: tripperware.PrometheusData{ - ResultType: model.ValMatrix.String(), - 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}, - }, - }, - }, - }, - }, - }, - }, - } ) func mkAPIResponse(start, end, step int64) tripperware.Response { From 9b6ef8c3d879a219e47eb09f05c5609428f901c2 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Thu, 19 Sep 2024 11:56:17 -0700 Subject: [PATCH 31/38] fix protobuf empty slice decoding Signed-off-by: Ahmed Hassan --- pkg/api/handlers.go | 2 +- pkg/querier/codec/protobuf_codec.go | 18 ++++++++++-------- .../tripperware/instantquery/instant_query.go | 17 +++++++++++++++++ .../tripperware/queryrange/query_range.go | 9 +++++++++ 4 files changed, 37 insertions(+), 9 deletions(-) diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index a63a1146f34..8eff8bd30e0 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -298,7 +298,7 @@ type buildInfoHandler struct { } type buildInfoResponse struct { - Status string `json:"status"` + Status string `json:"status"` Data *v1.PrometheusVersion `json:"data"` } diff --git a/pkg/querier/codec/protobuf_codec.go b/pkg/querier/codec/protobuf_codec.go index abbc3e19146..b835d573e8a 100644 --- a/pkg/querier/codec/protobuf_codec.go +++ b/pkg/querier/codec/protobuf_codec.go @@ -1,15 +1,17 @@ package codec import ( - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/prometheus/prometheus/promql" - "github.com/cortexproject/cortex/pkg/cortexpb" - "github.com/prometheus/prometheus/util/stats" + "github.com/gogo/protobuf/proto" jsoniter "github.com/json-iterator/go" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/web/api/v1" - "github.com/gogo/protobuf/proto" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/util/stats" + v1 "github.com/prometheus/prometheus/web/api/v1" + + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querier/tripperware" ) + type ProtobufCodec struct{} func (p ProtobufCodec) ContentType() v1.MIMEType { @@ -74,7 +76,7 @@ func createPrometheusQueryResponse(resp *v1.Response) (*tripperware.PrometheusRe }, ErrorType: string(resp.ErrorType), Error: resp.Error, - Warnings: resp.Warnings, + Warnings: resp.Warnings, }, nil } @@ -152,7 +154,7 @@ func getStats(builtin *stats.BuiltinStats) *tripperware.PrometheusResponseSample statSamples := tripperware.PrometheusResponseSamplesStats{ TotalQueryableSamples: builtin.Samples.TotalQueryableSamples, TotalQueryableSamplesPerStep: queryableSamplesStatsPerStep, - PeakSamples: int64(builtin.Samples.PeakSamples), + PeakSamples: int64(builtin.Samples.PeakSamples), } return &statSamples diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 4fc9064dbc8..d83ae20ae91 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -13,6 +13,7 @@ import ( 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/weaveworks/common/httpgrpc" "google.golang.org/grpc/status" @@ -105,6 +106,22 @@ func (instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, _ return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } + // protobuf serialization treats empty slices as nil + switch resp.Data.ResultType { + case model.ValMatrix.String(): + if resp.Data.Result.GetMatrix().SampleStreams == nil { + resp.Data.Result.GetMatrix().SampleStreams = []tripperware.SampleStream{} + } + case model.ValVector.String(): + if resp.Data.Result.GetVector().Samples == nil { + resp.Data.Result.GetVector().Samples = []tripperware.Sample{} + } + } + + if resp.Headers == nil { + resp.Headers = []*tripperware.PrometheusResponseHeader{} + } + for h, hv := range r.Header { resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) } diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 094351bcfe5..46f1affb48a 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -214,6 +214,15 @@ func (c prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } + // protobuf serialization treats empty slices as nil + if resp.Data.ResultType == model.ValMatrix.String() && resp.Data.Result.GetMatrix().SampleStreams == nil { + resp.Data.Result.GetMatrix().SampleStreams = []tripperware.SampleStream{} + } + + if resp.Headers == nil { + resp.Headers = []*tripperware.PrometheusResponseHeader{} + } + for h, hv := range r.Header { resp.Headers = append(resp.Headers, &tripperware.PrometheusResponseHeader{Name: h, Values: hv}) } From 4901a281c5dae381c43a38337a9f2450ed8048ee Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 20 Sep 2024 09:18:42 -0700 Subject: [PATCH 32/38] add integration test for protobuf codec Signed-off-by: Ahmed Hassan --- integration/query_frontend_test.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/integration/query_frontend_test.go b/integration/query_frontend_test.go index a2c81ed95ff..e0334d113af 100644 --- a/integration/query_frontend_test.go +++ b/integration/query_frontend_test.go @@ -203,6 +203,25 @@ func TestQueryFrontendWithVerticalShardingQueryScheduler(t *testing.T) { }) } +func TestQueryFrontendProtobufCodec(t *testing.T) { + runQueryFrontendTest(t, queryFrontendTestConfig{ + testMissingMetricName: false, + querySchedulerEnabled: true, + queryStatsEnabled: true, + setup: func(t *testing.T, s *e2e.Scenario) (configFile string, flags map[string]string) { + require.NoError(t, writeFileToSharedDir(s, cortexConfigFile, []byte(BlocksStorageConfig))) + + minio := e2edb.NewMinio(9000, BlocksStorageFlags()["-blocks-storage.s3.bucket-name"]) + require.NoError(t, s.StartAndWaitReady(minio)) + + flags = mergeFlags(e2e.EmptyFlags(), map[string]string{ + "-api.querier-default-codec": "protobuf", + }) + return cortexConfigFile, flags + }, + }) +} + func TestQueryFrontendRemoteRead(t *testing.T) { runQueryFrontendTest(t, queryFrontendTestConfig{ remoteReadEnabled: true, From 0a7d80727f325bacc98fdc375045c25e72bd9b2e Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 23 Sep 2024 08:32:02 -0700 Subject: [PATCH 33/38] add gzip compression to integration test Signed-off-by: Ahmed Hassan --- integration/query_frontend_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/integration/query_frontend_test.go b/integration/query_frontend_test.go index e0334d113af..c5a11b612ae 100644 --- a/integration/query_frontend_test.go +++ b/integration/query_frontend_test.go @@ -215,7 +215,8 @@ func TestQueryFrontendProtobufCodec(t *testing.T) { require.NoError(t, s.StartAndWaitReady(minio)) flags = mergeFlags(e2e.EmptyFlags(), map[string]string{ - "-api.querier-default-codec": "protobuf", + "-api.querier-default-codec": "protobuf", + "-querier.response-compression": "gzip", }) return cortexConfigFile, flags }, From 0f6e8bdd7debe0502fcf9ac08fc9dfc00671240e Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 23 Sep 2024 17:56:05 -0700 Subject: [PATCH 34/38] fix recompile proto files Signed-off-by: Ahmed Hassan --- pkg/querier/tripperware/query.pb.go | 154 ++++++++++++++-------------- 1 file changed, 77 insertions(+), 77 deletions(-) diff --git a/pkg/querier/tripperware/query.pb.go b/pkg/querier/tripperware/query.pb.go index 76fb276e0ee..2e16fc9c6db 100644 --- a/pkg/querier/tripperware/query.pb.go +++ b/pkg/querier/tripperware/query.pb.go @@ -994,83 +994,83 @@ func init() { func init() { proto.RegisterFile("query.proto", fileDescriptor_5c6ac9b241082464) } var fileDescriptor_5c6ac9b241082464 = []byte{ - // 1207 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, 0x76, 0x8e, 0xd3, 0xa4, 0xff, 0x9b, 0x3e, 0x9c, 0xfe, 0xcb, 0x8c, 0x19, - 0x81, 0x14, 0x04, 0x75, 0x44, 0x10, 0x54, 0x80, 0x54, 0xd1, 0x81, 0x42, 0x5a, 0x28, 0x6d, 0x6f, - 0xaa, 0x22, 0xb1, 0xa9, 0xae, 0xed, 0x5b, 0x67, 0x88, 0xe7, 0xd1, 0x3b, 0x77, 0x9a, 0x98, 0x15, - 0x6b, 0x16, 0x88, 0x35, 0x12, 0x0b, 0x76, 0x2c, 0xf8, 0x20, 0x59, 0x76, 0x59, 0x21, 0x31, 0x22, - 0xce, 0x06, 0xcd, 0xaa, 0x1f, 0x01, 0xdd, 0xc7, 0xd8, 0xe3, 0xc4, 0x49, 0xd4, 0x15, 0x1b, 0x67, - 0xce, 0x39, 0xbf, 0xf3, 0xbc, 0xe7, 0x11, 0x68, 0x3e, 0x4d, 0x28, 0x1b, 0x75, 0x22, 0x16, 0xf2, - 0x10, 0x35, 0x39, 0xf3, 0xa2, 0x88, 0xb2, 0x5d, 0xc2, 0xe8, 0x95, 0x0b, 0x83, 0x70, 0x10, 0x4a, - 0xfe, 0xba, 0xf8, 0x52, 0x90, 0x2b, 0xd6, 0x20, 0x0c, 0x07, 0x43, 0xba, 0x2e, 0xa9, 0x6e, 0xf2, - 0x64, 0xbd, 0x9f, 0x30, 0xc2, 0xbd, 0x30, 0xd0, 0xf2, 0xd5, 0xa3, 0x72, 0x12, 0x68, 0xeb, 0x57, - 0x3e, 0x1c, 0x78, 0x7c, 0x3b, 0xe9, 0x76, 0x7a, 0xa1, 0xbf, 0xde, 0x0b, 0x19, 0xa7, 0x7b, 0x11, - 0x0b, 0xbf, 0xa3, 0x3d, 0xae, 0xa9, 0xf5, 0x68, 0x67, 0x90, 0x0b, 0xba, 0xfa, 0x43, 0xa9, 0x3a, - 0x3f, 0x56, 0x00, 0xdd, 0x67, 0xa1, 0x4f, 0xf9, 0x36, 0x4d, 0x62, 0x4c, 0xe3, 0x28, 0x0c, 0x62, - 0x8a, 0x1c, 0x30, 0xb7, 0x38, 0xe1, 0x49, 0xdc, 0x32, 0xda, 0xc6, 0xda, 0x82, 0x0b, 0x59, 0x6a, - 0x9b, 0xb1, 0xe4, 0x60, 0x2d, 0x41, 0x5f, 0x40, 0xf5, 0x33, 0xc2, 0x49, 0xab, 0xdc, 0x36, 0xd6, - 0x9a, 0x1b, 0xff, 0xef, 0x14, 0x52, 0xec, 0x4c, 0x4d, 0x0a, 0x88, 0x7b, 0x69, 0x3f, 0xb5, 0x4b, - 0x59, 0x6a, 0x2f, 0xf5, 0x09, 0x27, 0xef, 0x84, 0xbe, 0xc7, 0xa9, 0x1f, 0xf1, 0x11, 0x96, 0x06, - 0xd0, 0xfb, 0xb0, 0x70, 0x8b, 0xb1, 0x90, 0x3d, 0x1c, 0x45, 0xb4, 0x55, 0x91, 0xfe, 0x2e, 0x67, - 0xa9, 0xbd, 0x42, 0x73, 0x66, 0x41, 0x63, 0x8a, 0x44, 0x6f, 0x41, 0x4d, 0x12, 0xad, 0xaa, 0x54, - 0x59, 0xc9, 0x52, 0x7b, 0x59, 0xaa, 0x14, 0xe0, 0x0a, 0x81, 0x3e, 0x87, 0xfa, 0x26, 0x25, 0x7d, - 0xca, 0xe2, 0x56, 0xad, 0x5d, 0x59, 0x6b, 0x6e, 0xbc, 0x79, 0x42, 0xb4, 0x79, 0x01, 0x14, 0xda, - 0xad, 0x65, 0xa9, 0x6d, 0x5c, 0xc3, 0xb9, 0x32, 0xda, 0x80, 0xc6, 0x37, 0x84, 0x05, 0x5e, 0x30, - 0x88, 0x5b, 0x66, 0xbb, 0xb2, 0xb6, 0xe0, 0x5e, 0xca, 0x52, 0x1b, 0xed, 0x6a, 0x5e, 0xc1, 0xf1, - 0x04, 0x27, 0xc2, 0xbc, 0x1d, 0x3c, 0x09, 0xe3, 0x56, 0x5d, 0x2a, 0xc8, 0x30, 0x3d, 0xc1, 0x28, - 0x86, 0x29, 0x11, 0xce, 0x5f, 0x06, 0x2c, 0xcd, 0x56, 0x0e, 0x75, 0x00, 0x30, 0x8d, 0x93, 0x21, - 0x97, 0xc5, 0x51, 0x8f, 0xb1, 0x94, 0xa5, 0x36, 0xb0, 0x09, 0x17, 0x17, 0x10, 0xe8, 0x0e, 0x98, - 0x8a, 0xd2, 0xcf, 0xe2, 0x9c, 0x90, 0xe8, 0x03, 0xd1, 0x9c, 0x0a, 0xe9, 0x2e, 0xe9, 0xd7, 0x31, - 0x95, 0x4d, 0xac, 0x2d, 0xa0, 0x7b, 0x50, 0x13, 0x4f, 0x1e, 0xcb, 0x37, 0x69, 0x6e, 0xbc, 0x71, - 0x46, 0xcd, 0x44, 0x5b, 0xc4, 0x2a, 0x3f, 0xa9, 0x56, 0xcc, 0x4f, 0x32, 0x9c, 0x1d, 0x58, 0xfa, - 0x94, 0xf4, 0xb6, 0x69, 0x7f, 0xd2, 0x67, 0xab, 0x50, 0xd9, 0xa1, 0x23, 0x9d, 0x57, 0x3d, 0x4b, - 0x6d, 0x41, 0x62, 0xf1, 0x83, 0x6e, 0x40, 0x9d, 0xee, 0x71, 0x1a, 0xf0, 0xb8, 0x55, 0x96, 0x6f, - 0xb6, 0x32, 0xe3, 0xff, 0x96, 0x94, 0xb9, 0xcb, 0x3a, 0xf6, 0x1c, 0x8b, 0xf3, 0x0f, 0xe7, 0x0f, - 0x03, 0x4c, 0x05, 0x42, 0xb6, 0x4c, 0x84, 0x71, 0xe9, 0xa7, 0xe2, 0x2e, 0x64, 0xa9, 0xad, 0x18, - 0x58, 0xfd, 0x11, 0x61, 0xd0, 0xa0, 0x2f, 0x4b, 0x56, 0x51, 0x61, 0xd0, 0xa0, 0x8f, 0xc5, 0x0f, - 0x6a, 0x43, 0x83, 0x33, 0xd2, 0xa3, 0x8f, 0xbd, 0xbe, 0x6e, 0xb4, 0xbc, 0x29, 0x24, 0xfb, 0x76, - 0x1f, 0xdd, 0x80, 0x06, 0xd3, 0xf9, 0xb4, 0x6a, 0xb2, 0x52, 0x17, 0x3a, 0x6a, 0x56, 0x3b, 0xf9, - 0xac, 0x76, 0x6e, 0x06, 0x23, 0x77, 0x31, 0x4b, 0xed, 0x09, 0x12, 0x4f, 0xbe, 0xee, 0x54, 0x1b, - 0x95, 0xf3, 0x55, 0xe7, 0x97, 0x32, 0x2c, 0x6e, 0x11, 0x3f, 0x1a, 0xd2, 0x2d, 0xce, 0x28, 0xf1, - 0xd1, 0x1e, 0x98, 0x43, 0xd2, 0xa5, 0x43, 0x31, 0x82, 0x2a, 0xfd, 0x7c, 0x82, 0x3b, 0x5f, 0x09, - 0xfe, 0x7d, 0xe2, 0x31, 0xf7, 0x4b, 0x91, 0xfe, 0x9f, 0xa9, 0xfd, 0x4a, 0x1b, 0x40, 0xe9, 0xdf, - 0xec, 0x93, 0x88, 0x53, 0x26, 0xde, 0xdd, 0xa7, 0x9c, 0x79, 0x3d, 0xac, 0xfd, 0xa1, 0x8f, 0xa0, - 0x1e, 0xcb, 0x48, 0xf2, 0xca, 0x9f, 0x9f, 0xba, 0x56, 0x21, 0x4e, 0x5b, 0xe6, 0x19, 0x19, 0x26, - 0x34, 0xc6, 0xb9, 0x02, 0x7a, 0x08, 0xb0, 0xed, 0xc5, 0x3c, 0x1c, 0x30, 0xe2, 0x8b, 0xc6, 0x11, - 0xea, 0xed, 0x99, 0x87, 0x53, 0x16, 0x36, 0x73, 0x90, 0x4c, 0x03, 0x69, 0x73, 0x05, 0x5d, 0x5c, - 0xf8, 0x76, 0xbe, 0x87, 0x95, 0x39, 0x6a, 0xe8, 0x75, 0x58, 0xe4, 0x9e, 0x4f, 0x63, 0x4e, 0xfc, - 0xe8, 0xb1, 0xaf, 0x76, 0x55, 0x05, 0x37, 0x27, 0xbc, 0xbb, 0x31, 0xfa, 0x04, 0x16, 0x26, 0x76, - 0xf4, 0x48, 0x5c, 0x3d, 0x2d, 0x1c, 0xb7, 0x2a, 0x42, 0xc1, 0x53, 0x25, 0xe7, 0x29, 0x2c, 0x1f, - 0xc1, 0xa0, 0x0b, 0x50, 0xeb, 0x85, 0x49, 0xa0, 0xfa, 0xc9, 0xc0, 0x8a, 0x40, 0xe7, 0xa1, 0x12, - 0x27, 0xca, 0x89, 0x81, 0xc5, 0x27, 0xfa, 0x00, 0xea, 0xdd, 0xa4, 0xb7, 0x43, 0x79, 0x5e, 0x89, - 0x59, 0xd7, 0x53, 0xa7, 0x12, 0x84, 0x73, 0xb0, 0x13, 0xc3, 0xf2, 0x11, 0x19, 0xb2, 0x00, 0xba, - 0x61, 0x12, 0xf4, 0x09, 0xf3, 0xa8, 0x4a, 0xb4, 0x86, 0x0b, 0x1c, 0x11, 0xd2, 0x30, 0xdc, 0xa5, - 0x4c, 0xbb, 0x57, 0x84, 0xe0, 0x26, 0xc2, 0x9d, 0x9c, 0x60, 0x03, 0x2b, 0x62, 0x1a, 0x7e, 0xb5, - 0x10, 0xbe, 0xe3, 0xc3, 0xe5, 0x13, 0x66, 0x1a, 0xe1, 0x69, 0x43, 0x18, 0xb2, 0x84, 0x6f, 0x9f, - 0xb5, 0x0a, 0x14, 0x5a, 0x6d, 0x84, 0xa6, 0x18, 0x4f, 0xad, 0x3f, 0x69, 0x14, 0x67, 0xbf, 0x0c, - 0xd6, 0xe9, 0x8a, 0xe8, 0x1e, 0x5c, 0xe4, 0x21, 0x27, 0x43, 0xb9, 0xab, 0x48, 0x77, 0x98, 0x4b, - 0xf5, 0x18, 0xaf, 0x66, 0xa9, 0x3d, 0x1f, 0x80, 0xe7, 0xb3, 0xd1, 0x6f, 0x06, 0x5c, 0x9d, 0x2b, - 0xb9, 0x4f, 0xd9, 0x16, 0xa7, 0x91, 0x6e, 0xf7, 0x8f, 0xcf, 0xc8, 0xee, 0xa8, 0xb6, 0x8c, 0x56, - 0x9b, 0x70, 0xdb, 0x59, 0x6a, 0x9f, 0xea, 0x04, 0x9f, 0x2a, 0x45, 0xef, 0x42, 0x33, 0xa2, 0x64, - 0x27, 0x4f, 0xb5, 0x22, 0x53, 0x5d, 0xce, 0x52, 0xbb, 0xc8, 0xc6, 0x45, 0xc2, 0xf1, 0xe0, 0x15, - 0x83, 0x14, 0x1d, 0x20, 0x07, 0x57, 0x4f, 0x8c, 0x22, 0x8e, 0x8d, 0x53, 0xf9, 0xd8, 0x38, 0x39, - 0x0f, 0xa1, 0x75, 0xd2, 0xb1, 0x44, 0xab, 0x50, 0xfd, 0x9a, 0xf8, 0xf9, 0x91, 0xd2, 0x5b, 0x52, - 0xb2, 0xd0, 0x6b, 0x60, 0x3e, 0x92, 0x8b, 0x42, 0x56, 0x78, 0x22, 0xd4, 0x4c, 0xe7, 0x57, 0x03, - 0x2e, 0xce, 0x3d, 0x4d, 0xe8, 0x1a, 0x98, 0xcf, 0x68, 0x8f, 0x87, 0x4c, 0x37, 0xde, 0xec, 0x0d, - 0x78, 0x24, 0x45, 0x9b, 0x25, 0xac, 0x41, 0xe8, 0x2a, 0x34, 0x18, 0xd9, 0x75, 0x47, 0x9c, 0xaa, - 0xe8, 0x17, 0x37, 0x4b, 0x78, 0xc2, 0x11, 0xc6, 0x7c, 0xc2, 0x99, 0xb7, 0xa7, 0x0f, 0xda, 0xac, - 0xb1, 0xbb, 0x52, 0x24, 0x8c, 0x29, 0x90, 0xdb, 0x00, 0x7d, 0x10, 0x9d, 0xeb, 0x60, 0x2a, 0x57, - 0xe8, 0x5a, 0x71, 0x12, 0x8e, 0x1f, 0x25, 0x55, 0xeb, 0x69, 0x93, 0xff, 0x54, 0x06, 0x53, 0xf1, - 0xfe, 0xc3, 0x75, 0x7e, 0x1d, 0x4c, 0x15, 0x8f, 0xde, 0x7f, 0xc7, 0xb7, 0xf9, 0xb9, 0xfd, 0xd4, - 0x36, 0xc4, 0x51, 0x94, 0x7d, 0x80, 0x35, 0x1c, 0x3d, 0x28, 0xee, 0x4e, 0x55, 0xb2, 0xb3, 0x57, - 0xf9, 0xff, 0xb4, 0xad, 0xa9, 0x6a, 0x71, 0x99, 0xde, 0x03, 0x53, 0xd5, 0x19, 0xdd, 0x82, 0x73, - 0x71, 0xe1, 0xdc, 0xe5, 0x65, 0x59, 0x9d, 0xe3, 0x40, 0x21, 0xf4, 0x66, 0x9e, 0xd5, 0x72, 0x6f, - 0x3e, 0x3f, 0xb0, 0x4a, 0x2f, 0x0e, 0xac, 0xd2, 0xcb, 0x03, 0xcb, 0xf8, 0x61, 0x6c, 0x19, 0xbf, - 0x8f, 0x2d, 0x63, 0x7f, 0x6c, 0x19, 0xcf, 0xc7, 0x96, 0xf1, 0xf7, 0xd8, 0x32, 0xfe, 0x19, 0x5b, - 0xa5, 0x97, 0x63, 0xcb, 0xf8, 0xf9, 0xd0, 0x2a, 0x3d, 0x3f, 0xb4, 0x4a, 0x2f, 0x0e, 0xad, 0xd2, - 0xb7, 0xc5, 0x7f, 0xc7, 0xbb, 0xa6, 0xbc, 0xd2, 0xef, 0xfd, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x31, - 0xdc, 0x8d, 0x85, 0xb1, 0x0b, 0x00, 0x00, + // 1208 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x4b, 0x6f, 0x1b, 0x55, + 0x14, 0xf6, 0xf8, 0x31, 0x76, 0x8e, 0xd3, 0xa4, 0xdc, 0xf4, 0xe1, 0x94, 0x32, 0x63, 0x46, 0x20, + 0x05, 0x41, 0x1d, 0x91, 0x0a, 0x10, 0x20, 0x2a, 0x3a, 0x50, 0x48, 0x0b, 0xa5, 0xed, 0x4d, 0x55, + 0x24, 0x36, 0xd5, 0xb5, 0x7d, 0xeb, 0x0c, 0xf1, 0x3c, 0x7a, 0xe7, 0x4e, 0x13, 0xb3, 0x62, 0xcd, + 0x02, 0xb1, 0x46, 0x62, 0xc1, 0x8e, 0x05, 0x3f, 0x24, 0xcb, 0x2e, 0x2b, 0x24, 0x46, 0xd4, 0xd9, + 0xa0, 0x59, 0xf5, 0x27, 0xa0, 0xfb, 0x18, 0x7b, 0x9c, 0x38, 0x89, 0xba, 0x62, 0xe3, 0xcc, 0x39, + 0xe7, 0x3b, 0xcf, 0x7b, 0x1e, 0x81, 0xe6, 0xe3, 0x84, 0xb2, 0x51, 0x27, 0x62, 0x21, 0x0f, 0x51, + 0x93, 0x33, 0x2f, 0x8a, 0x28, 0xdb, 0x25, 0x8c, 0x5e, 0x3a, 0x37, 0x08, 0x07, 0xa1, 0xe4, 0xaf, + 0x8b, 0x2f, 0x05, 0xb9, 0x64, 0x0d, 0xc2, 0x70, 0x30, 0xa4, 0xeb, 0x92, 0xea, 0x26, 0x8f, 0xd6, + 0xfb, 0x09, 0x23, 0xdc, 0x0b, 0x03, 0x2d, 0x5f, 0x3d, 0x2c, 0x27, 0x81, 0xb6, 0x7e, 0xe9, 0xc3, + 0x81, 0xc7, 0xb7, 0x93, 0x6e, 0xa7, 0x17, 0xfa, 0xeb, 0xbd, 0x90, 0x71, 0xba, 0x17, 0xb1, 0xf0, + 0x7b, 0xda, 0xe3, 0x9a, 0x5a, 0x8f, 0x76, 0x06, 0xb9, 0xa0, 0xab, 0x3f, 0x94, 0xaa, 0xf3, 0x53, + 0x05, 0xd0, 0x5d, 0x16, 0xfa, 0x94, 0x6f, 0xd3, 0x24, 0xc6, 0x34, 0x8e, 0xc2, 0x20, 0xa6, 0xc8, + 0x01, 0x73, 0x8b, 0x13, 0x9e, 0xc4, 0x2d, 0xa3, 0x6d, 0xac, 0x2d, 0xb8, 0x90, 0xa5, 0xb6, 0x19, + 0x4b, 0x0e, 0xd6, 0x12, 0xf4, 0x25, 0x54, 0x3f, 0x27, 0x9c, 0xb4, 0xca, 0x6d, 0x63, 0xad, 0xb9, + 0xf1, 0x6a, 0xa7, 0x90, 0x62, 0x67, 0x6a, 0x52, 0x40, 0xdc, 0x0b, 0xfb, 0xa9, 0x5d, 0xca, 0x52, + 0x7b, 0xa9, 0x4f, 0x38, 0x79, 0x27, 0xf4, 0x3d, 0x4e, 0xfd, 0x88, 0x8f, 0xb0, 0x34, 0x80, 0xde, + 0x83, 0x85, 0x1b, 0x8c, 0x85, 0xec, 0xfe, 0x28, 0xa2, 0xad, 0x8a, 0xf4, 0x77, 0x31, 0x4b, 0xed, + 0x15, 0x9a, 0x33, 0x0b, 0x1a, 0x53, 0x24, 0x7a, 0x0b, 0x6a, 0x92, 0x68, 0x55, 0xa5, 0xca, 0x4a, + 0x96, 0xda, 0xcb, 0x52, 0xa5, 0x00, 0x57, 0x08, 0xf4, 0x05, 0xd4, 0x37, 0x29, 0xe9, 0x53, 0x16, + 0xb7, 0x6a, 0xed, 0xca, 0x5a, 0x73, 0xe3, 0xcd, 0x63, 0xa2, 0xcd, 0x0b, 0xa0, 0xd0, 0x6e, 0x2d, + 0x4b, 0x6d, 0xe3, 0x0a, 0xce, 0x95, 0xd1, 0x06, 0x34, 0xbe, 0x25, 0x2c, 0xf0, 0x82, 0x41, 0xdc, + 0x32, 0xdb, 0x95, 0xb5, 0x05, 0xf7, 0x42, 0x96, 0xda, 0x68, 0x57, 0xf3, 0x0a, 0x8e, 0x27, 0x38, + 0x11, 0xe6, 0xcd, 0xe0, 0x51, 0x18, 0xb7, 0xea, 0x52, 0x41, 0x86, 0xe9, 0x09, 0x46, 0x31, 0x4c, + 0x89, 0x70, 0xfe, 0x36, 0x60, 0x69, 0xb6, 0x72, 0xa8, 0x03, 0x80, 0x69, 0x9c, 0x0c, 0xb9, 0x2c, + 0x8e, 0x7a, 0x8c, 0xa5, 0x2c, 0xb5, 0x81, 0x4d, 0xb8, 0xb8, 0x80, 0x40, 0xb7, 0xc0, 0x54, 0x94, + 0x7e, 0x16, 0xe7, 0x98, 0x44, 0xef, 0x89, 0xe6, 0x54, 0x48, 0x77, 0x49, 0xbf, 0x8e, 0xa9, 0x6c, + 0x62, 0x6d, 0x01, 0xdd, 0x81, 0x9a, 0x78, 0xf2, 0x58, 0xbe, 0x49, 0x73, 0xe3, 0x8d, 0x53, 0x6a, + 0x26, 0xda, 0x22, 0x56, 0xf9, 0x49, 0xb5, 0x62, 0x7e, 0x92, 0xe1, 0xec, 0xc0, 0xd2, 0x67, 0xa4, + 0xb7, 0x4d, 0xfb, 0x93, 0x3e, 0x5b, 0x85, 0xca, 0x0e, 0x1d, 0xe9, 0xbc, 0xea, 0x59, 0x6a, 0x0b, + 0x12, 0x8b, 0x1f, 0x74, 0x0d, 0xea, 0x74, 0x8f, 0xd3, 0x80, 0xc7, 0xad, 0xb2, 0x7c, 0xb3, 0x95, + 0x19, 0xff, 0x37, 0xa4, 0xcc, 0x5d, 0xd6, 0xb1, 0xe7, 0x58, 0x9c, 0x7f, 0x38, 0x7f, 0x1a, 0x60, + 0x2a, 0x10, 0xb2, 0x65, 0x22, 0x8c, 0x4b, 0x3f, 0x15, 0x77, 0x21, 0x4b, 0x6d, 0xc5, 0xc0, 0xea, + 0x8f, 0x08, 0x83, 0x06, 0x7d, 0x59, 0xb2, 0x8a, 0x0a, 0x83, 0x06, 0x7d, 0x2c, 0x7e, 0x50, 0x1b, + 0x1a, 0x9c, 0x91, 0x1e, 0x7d, 0xe8, 0xf5, 0x75, 0xa3, 0xe5, 0x4d, 0x21, 0xd9, 0x37, 0xfb, 0xe8, + 0x1a, 0x34, 0x98, 0xce, 0xa7, 0x55, 0x93, 0x95, 0x3a, 0xd7, 0x51, 0xb3, 0xda, 0xc9, 0x67, 0xb5, + 0x73, 0x3d, 0x18, 0xb9, 0x8b, 0x59, 0x6a, 0x4f, 0x90, 0x78, 0xf2, 0x75, 0xab, 0xda, 0xa8, 0x9c, + 0xad, 0x3a, 0xbf, 0x96, 0x61, 0x71, 0x8b, 0xf8, 0xd1, 0x90, 0x6e, 0x71, 0x46, 0x89, 0x8f, 0xf6, + 0xc0, 0x1c, 0x92, 0x2e, 0x1d, 0x8a, 0x11, 0x54, 0xe9, 0xe7, 0x13, 0xdc, 0xf9, 0x5a, 0xf0, 0xef, + 0x12, 0x8f, 0xb9, 0x5f, 0x89, 0xf4, 0xff, 0x4a, 0xed, 0x97, 0xda, 0x00, 0x4a, 0xff, 0x7a, 0x9f, + 0x44, 0x9c, 0x32, 0xf1, 0xee, 0x3e, 0xe5, 0xcc, 0xeb, 0x61, 0xed, 0x0f, 0x7d, 0x04, 0xf5, 0x58, + 0x46, 0x92, 0x57, 0xfe, 0xec, 0xd4, 0xb5, 0x0a, 0x71, 0xda, 0x32, 0x4f, 0xc8, 0x30, 0xa1, 0x31, + 0xce, 0x15, 0xd0, 0x7d, 0x80, 0x6d, 0x2f, 0xe6, 0xe1, 0x80, 0x11, 0x5f, 0x34, 0x8e, 0x50, 0x6f, + 0xcf, 0x3c, 0x9c, 0xb2, 0xb0, 0x99, 0x83, 0x64, 0x1a, 0x48, 0x9b, 0x2b, 0xe8, 0xe2, 0xc2, 0xb7, + 0xf3, 0x03, 0xac, 0xcc, 0x51, 0x43, 0xaf, 0xc3, 0x22, 0xf7, 0x7c, 0x1a, 0x73, 0xe2, 0x47, 0x0f, + 0x7d, 0xb5, 0xab, 0x2a, 0xb8, 0x39, 0xe1, 0xdd, 0x8e, 0xd1, 0xa7, 0xb0, 0x30, 0xb1, 0xa3, 0x47, + 0xe2, 0xf2, 0x49, 0xe1, 0xb8, 0x55, 0x11, 0x0a, 0x9e, 0x2a, 0x39, 0x8f, 0x61, 0xf9, 0x10, 0x06, + 0x9d, 0x83, 0x5a, 0x2f, 0x4c, 0x02, 0xd5, 0x4f, 0x06, 0x56, 0x04, 0x3a, 0x0b, 0x95, 0x38, 0x51, + 0x4e, 0x0c, 0x2c, 0x3e, 0xd1, 0xfb, 0x50, 0xef, 0x26, 0xbd, 0x1d, 0xca, 0xf3, 0x4a, 0xcc, 0xba, + 0x9e, 0x3a, 0x95, 0x20, 0x9c, 0x83, 0x9d, 0x18, 0x96, 0x0f, 0xc9, 0x90, 0x05, 0xd0, 0x0d, 0x93, + 0xa0, 0x4f, 0x98, 0x47, 0x55, 0xa2, 0x35, 0x5c, 0xe0, 0x88, 0x90, 0x86, 0xe1, 0x2e, 0x65, 0xda, + 0xbd, 0x22, 0x04, 0x37, 0x11, 0xee, 0xe4, 0x04, 0x1b, 0x58, 0x11, 0xd3, 0xf0, 0xab, 0x85, 0xf0, + 0x1d, 0x1f, 0x2e, 0x1e, 0x33, 0xd3, 0x08, 0x4f, 0x1b, 0xc2, 0x90, 0x25, 0x7c, 0xfb, 0xb4, 0x55, + 0xa0, 0xd0, 0x6a, 0x23, 0x34, 0xc5, 0x78, 0x6a, 0xfd, 0x49, 0xa3, 0x38, 0xfb, 0x65, 0xb0, 0x4e, + 0x56, 0x44, 0x77, 0xe0, 0x3c, 0x0f, 0x39, 0x19, 0xca, 0x5d, 0x45, 0xba, 0xc3, 0x5c, 0xaa, 0xc7, + 0x78, 0x35, 0x4b, 0xed, 0xf9, 0x00, 0x3c, 0x9f, 0x8d, 0x7e, 0x37, 0xe0, 0xf2, 0x5c, 0xc9, 0x5d, + 0xca, 0xb6, 0x38, 0x8d, 0x74, 0xbb, 0x7f, 0x7c, 0x4a, 0x76, 0x87, 0xb5, 0x65, 0xb4, 0xda, 0x84, + 0xdb, 0xce, 0x52, 0xfb, 0x44, 0x27, 0xf8, 0x44, 0x29, 0x7a, 0x17, 0x9a, 0x11, 0x25, 0x3b, 0x79, + 0xaa, 0x15, 0x99, 0xea, 0x72, 0x96, 0xda, 0x45, 0x36, 0x2e, 0x12, 0x8e, 0x07, 0x2f, 0x19, 0xa4, + 0xe8, 0x00, 0x39, 0xb8, 0x7a, 0x62, 0x14, 0x71, 0x64, 0x9c, 0xca, 0x47, 0xc6, 0xc9, 0xb9, 0x0f, + 0xad, 0xe3, 0x8e, 0x25, 0x5a, 0x85, 0xea, 0x37, 0xc4, 0xcf, 0x8f, 0x94, 0xde, 0x92, 0x92, 0x85, + 0x5e, 0x03, 0xf3, 0x81, 0x5c, 0x14, 0xb2, 0xc2, 0x13, 0xa1, 0x66, 0x3a, 0xbf, 0x19, 0x70, 0x7e, + 0xee, 0x69, 0x42, 0x57, 0xc0, 0x7c, 0x42, 0x7b, 0x3c, 0x64, 0xba, 0xf1, 0x66, 0x6f, 0xc0, 0x03, + 0x29, 0xda, 0x2c, 0x61, 0x0d, 0x42, 0x97, 0xa1, 0xc1, 0xc8, 0xae, 0x3b, 0xe2, 0x54, 0x45, 0xbf, + 0xb8, 0x59, 0xc2, 0x13, 0x8e, 0x30, 0xe6, 0x13, 0xce, 0xbc, 0x3d, 0x7d, 0xd0, 0x66, 0x8d, 0xdd, + 0x96, 0x22, 0x61, 0x4c, 0x81, 0xdc, 0x06, 0xe8, 0x83, 0xe8, 0x7c, 0x02, 0xa6, 0x72, 0x85, 0xae, + 0x16, 0x27, 0xe1, 0xe8, 0x51, 0xd2, 0xdb, 0x51, 0xed, 0x90, 0x49, 0xab, 0xff, 0x5c, 0x06, 0x53, + 0x49, 0xfe, 0xc7, 0xa5, 0xfe, 0x01, 0x98, 0x2a, 0x1e, 0xbd, 0x05, 0x8f, 0xee, 0xf4, 0x33, 0xfb, + 0xa9, 0x6d, 0x88, 0xd3, 0x28, 0xbb, 0x01, 0x6b, 0x38, 0xba, 0x57, 0xdc, 0xa0, 0xaa, 0x70, 0xa7, + 0x2f, 0xf4, 0x57, 0xb4, 0xad, 0xa9, 0x6a, 0x71, 0xa5, 0xde, 0x01, 0x53, 0x55, 0x1b, 0xdd, 0x80, + 0x33, 0x71, 0xe1, 0xe8, 0xe5, 0x65, 0x59, 0x9d, 0xe3, 0x40, 0x21, 0x74, 0x6d, 0x67, 0xb5, 0xdc, + 0xeb, 0x4f, 0x9f, 0x5b, 0xa5, 0x67, 0xcf, 0xad, 0xd2, 0x8b, 0xe7, 0x96, 0xf1, 0xe3, 0xd8, 0x32, + 0xfe, 0x18, 0x5b, 0xc6, 0xfe, 0xd8, 0x32, 0x9e, 0x8e, 0x2d, 0xe3, 0x9f, 0xb1, 0x65, 0xfc, 0x3b, + 0xb6, 0x4a, 0x2f, 0xc6, 0x96, 0xf1, 0xcb, 0x81, 0x55, 0x7a, 0x7a, 0x60, 0x95, 0x9e, 0x1d, 0x58, + 0xa5, 0xef, 0x8a, 0xff, 0x94, 0x77, 0x4d, 0x79, 0xab, 0xaf, 0xfe, 0x17, 0x00, 0x00, 0xff, 0xff, + 0xc9, 0x8e, 0x9e, 0x8e, 0xb7, 0x0b, 0x00, 0x00, } func (this *PrometheusResponse) Equal(that interface{}) bool { From 7730ad2e89c7e40b54b96956c79c2c3e727ffa98 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 23 Sep 2024 18:19:53 -0700 Subject: [PATCH 35/38] update config docs Signed-off-by: Ahmed Hassan --- docs/blocks-storage/querier.md | 5 +++++ docs/configuration/config-file-reference.md | 10 ++++++++++ 2 files changed, 15 insertions(+) diff --git a/docs/blocks-storage/querier.md b/docs/blocks-storage/querier.md index 41d0f433bf2..6c50cd40a25 100644 --- a/docs/blocks-storage/querier.md +++ b/docs/blocks-storage/querier.md @@ -126,6 +126,11 @@ querier: # CLI flag: -querier.per-step-stats-enabled [per_step_stats_enabled: | default = false] + # Use compression for metrics query API or instant and range query APIs. + # Supports 'gzip' and '' (disable compression) + # CLI flag: -querier.response-compression + [response_compression: | default = ""] + # The time after which a metric should be queried from storage and not just # ingesters. 0 means all queries are sent to store. When running the blocks # storage, if this option is enabled, the time range of the query sent to the diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index a8ff9ef9280..4089b4763da 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -95,6 +95,11 @@ api: # CLI flag: -api.build-info-enabled [build_info_enabled: | default = false] + # Choose default codec for querier response serialization. Supports 'json' and + # 'protobuf'. + # CLI flag: -api.querier-default-codec + [querier_default_codec: | default = "json"] + # The server_config configures the HTTP and gRPC server of the launched # service(s). [server: ] @@ -3718,6 +3723,11 @@ The `querier_config` configures the Cortex querier. # CLI flag: -querier.per-step-stats-enabled [per_step_stats_enabled: | default = false] +# Use compression for metrics query API or instant and range query APIs. +# Supports 'gzip' and '' (disable compression) +# CLI flag: -querier.response-compression +[response_compression: | default = ""] + # The time after which a metric should be queried from storage and not just # ingesters. 0 means all queries are sent to store. When running the blocks # storage, if this option is enabled, the time range of the query sent to the From 5d8658e9efc724d3ac15eb9ef8eb62685ea50354 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 24 Sep 2024 15:15:47 -0700 Subject: [PATCH 36/38] add changelog for protobuf codec Signed-off-by: Ahmed Hassan --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 41dfac4a91c..9c692798ac7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ ## master / unreleased +* [FEATURE] Query Frontend/Querier: Add protobuf codec `-api.querier-default-codec` and the option to choose response compression type `-querier.response-compression`. #5527 * [CHANGE] Enable Compactor and Alertmanager in target all. #6204 * [FEATURE] Ruler: Experimental: Add `ruler.frontend-address` to allow query to query frontends instead of ingesters. #6151 * [FEATURE] Ruler: Minimize chances of missed rule group evaluations that can occur due to OOM kills, bad underlying nodes, or due to an unhealthy ruler that appears in the ring as healthy. This feature is enabled via `-ruler.enable-ha-evaluation` flag. #6129 From 91e13663c467cf48807338de79f74490d6932710 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 24 Sep 2024 18:05:45 -0700 Subject: [PATCH 37/38] make gzip default compression Signed-off-by: Ahmed Hassan --- docs/blocks-storage/querier.md | 2 +- docs/configuration/config-file-reference.md | 2 +- pkg/querier/querier.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/blocks-storage/querier.md b/docs/blocks-storage/querier.md index 6c50cd40a25..7fdfaf89a4d 100644 --- a/docs/blocks-storage/querier.md +++ b/docs/blocks-storage/querier.md @@ -129,7 +129,7 @@ querier: # Use compression for metrics query API or instant and range query APIs. # Supports 'gzip' and '' (disable compression) # CLI flag: -querier.response-compression - [response_compression: | default = ""] + [response_compression: | default = "gzip"] # The time after which a metric should be queried from storage and not just # ingesters. 0 means all queries are sent to store. When running the blocks diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 4089b4763da..58c3fdec637 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3726,7 +3726,7 @@ The `querier_config` configures the Cortex querier. # Use compression for metrics query API or instant and range query APIs. # Supports 'gzip' and '' (disable compression) # CLI flag: -querier.response-compression -[response_compression: | default = ""] +[response_compression: | default = "gzip"] # The time after which a metric should be queried from storage and not just # ingesters. 0 means all queries are sent to store. When running the blocks diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 4f6b28d6ffc..4c481cbee4d 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -115,7 +115,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxSamples, "querier.max-samples", 50e6, "Maximum number of samples a single query can load into memory.") f.DurationVar(&cfg.QueryIngestersWithin, "querier.query-ingesters-within", 0, "Maximum lookback beyond which queries are not sent to ingester. 0 means all queries are sent to ingester.") f.BoolVar(&cfg.EnablePerStepStats, "querier.per-step-stats-enabled", false, "Enable returning samples stats per steps in query response.") - f.StringVar(&cfg.ResponseCompression, "querier.response-compression", "", "Use compression for metrics query API or instant and range query APIs. Supports 'gzip' and '' (disable compression)") + f.StringVar(&cfg.ResponseCompression, "querier.response-compression", "gzip", "Use compression for metrics query API or instant and range query APIs. Supports 'gzip' and '' (disable compression)") f.DurationVar(&cfg.MaxQueryIntoFuture, "querier.max-query-into-future", 10*time.Minute, "Maximum duration into the future you can query. 0 to disable.") f.DurationVar(&cfg.DefaultEvaluationInterval, "querier.default-evaluation-interval", time.Minute, "The default evaluation interval or step size for subqueries.") f.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 0, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. When running the blocks storage, if this option is enabled, the time range of the query sent to the store will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") From 390e37007e2c4b0de26f50a1f396ebbbeb88a7ac Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 24 Sep 2024 19:03:14 -0700 Subject: [PATCH 38/38] add protobuf codec to experimental features Signed-off-by: Ahmed Hassan --- docs/configuration/v1-guarantees.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index aba6f3695d5..c1627547806 100644 --- a/docs/configuration/v1-guarantees.md +++ b/docs/configuration/v1-guarantees.md @@ -115,3 +115,4 @@ Currently experimental features are: - String interning for metrics labels - Enable string interning for metrics labels by setting `-ingester.labels-string-interning-enabled` on Ingester. - Query-frontend: query rejection (`-frontend.query-rejection.enabled`) +- Querier: protobuf codec (`-api.querier-default-codec`)