From 923ddf6a5f63ae8b09aa88da18ebb932f3e69dcc Mon Sep 17 00:00:00 2001 From: rubywtl Date: Wed, 16 Jul 2025 14:31:01 -0700 Subject: [PATCH 01/14] frontend: Implement logical query plan middleware under new distributed execution feature flag Signed-off-by: rubywtl --- pkg/cortex/modules.go | 6 +- pkg/frontend/config.go | 6 +- .../instantquery/instant_logicalplan_gen.go | 84 +++++ .../instant_logicalplan_gen_test.go | 353 ++++++++++++++++++ .../tripperware/instantquery/instant_query.go | 6 +- .../instantquery/instant_query_middlewares.go | 9 + .../instant_query_middlewares_test.go | 121 ++++++ .../instantquery/instant_query_test.go | 2 +- pkg/querier/tripperware/query.go | 10 + .../tripperware/queryrange/query_range.go | 6 +- .../queryrange/query_range_middlewares.go | 9 + .../query_range_middlewares_test.go | 13 +- .../queryrange/query_range_test.go | 4 +- .../queryrange/range_logicalplan_gen.go | 86 +++++ .../queryrange/range_logicalplan_gen_test.go | 336 +++++++++++++++++ .../queryrange/results_cache_test.go | 2 +- .../queryrange/split_by_interval_test.go | 6 +- pkg/querier/tripperware/roundtrip_test.go | 2 +- .../tripperware/test_shard_by_query_utils.go | 2 +- 19 files changed, 1045 insertions(+), 18 deletions(-) create mode 100644 pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go create mode 100644 pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go create mode 100644 pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go create mode 100644 pkg/querier/tripperware/queryrange/range_logicalplan_gen.go create mode 100644 pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 5f4679b9d4d..febb4f33a59 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -533,12 +533,16 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, + t.Cfg.Querier.EnablePerStepStats, + t.Cfg.Frontend.DistributedExecEnabled, + t.Cfg.Frontend.DisableDuplicateLabelChecks, ) if err != nil { return nil, err } - instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta) + instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Querier.EnablePerStepStats, + t.Cfg.Frontend.DistributedExecEnabled, t.Cfg.Frontend.DisableDuplicateLabelChecks) if err != nil { return nil, err } diff --git a/pkg/frontend/config.go b/pkg/frontend/config.go index 03dff13980e..75212e65d5c 100644 --- a/pkg/frontend/config.go +++ b/pkg/frontend/config.go @@ -20,7 +20,9 @@ type CombinedFrontendConfig struct { FrontendV1 v1.Config `yaml:",inline"` FrontendV2 v2.Config `yaml:",inline"` - DownstreamURL string `yaml:"downstream_url"` + DownstreamURL string `yaml:"downstream_url"` + DistributedExecEnabled bool `yaml:"distributed_exec_enabled"` + DisableDuplicateLabelChecks bool `yaml:"disable_duplicate_label_checks"` } func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { @@ -29,6 +31,8 @@ func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { cfg.FrontendV2.RegisterFlags(f) f.StringVar(&cfg.DownstreamURL, "frontend.downstream-url", "", "URL of downstream Prometheus.") + f.BoolVar(&cfg.DistributedExecEnabled, "frontend.distributed_exec_enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan fragments to downstream components.") + f.BoolVar(&cfg.DisableDuplicateLabelChecks, "frontend.disable_duplicate_label_checks", false, "Experimental: Disables duplicate label checks during logical plan generation") } // InitFrontend initializes frontend (either V1 -- without scheduler, or V2 -- with scheduler) or no frontend at diff --git a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go new file mode 100644 index 00000000000..942a62423ec --- /dev/null +++ b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go @@ -0,0 +1,84 @@ +package instantquery + +import ( + "context" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/prometheus/prometheus/promql/parser" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + "github.com/weaveworks/common/httpgrpc" + "net/http" + "time" +) + +const ( + stepBatch = 10 +) + +func InstantLogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) tripperware.Middleware { + return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { + return instantLogicalPlanGen{ + lookbackDelta: lookbackDelta, + enabledPerStepStats: enablePerStepStats, + disableDuplicateLabelChecks: disableDuplicateLabelChecks, + next: next, + } + }) +} + +type instantLogicalPlanGen struct { + next tripperware.Handler + lookbackDelta time.Duration + enabledPerStepStats bool + disableDuplicateLabelChecks bool +} + +func (l instantLogicalPlanGen) NewInstantLogicalPlan(qs string, ts time.Time) ([]byte, error) { + + qOpts := query.Options{ + Start: ts, + End: ts, + Step: 0, + StepsBatch: stepBatch, + LookbackDelta: l.lookbackDelta, + EnablePerStepStats: l.disableDuplicateLabelChecks, + } + + expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() + if err != nil { + return nil, err + } + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) + + // TODO: Add distributed optimizer for remote node insertion + + byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) + if err != nil { + return nil, err + } + + return byteLP, nil +} + +func (l instantLogicalPlanGen) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { + promReq, ok := r.(*tripperware.PrometheusRequest) + if !ok { + return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") + } + + instantTime := time.Unix(0, promReq.Start*int64(time.Millisecond)) + + byteLP, err := l.NewInstantLogicalPlan(promReq.Query, instantTime) + if err != nil { + return nil, err + } + promReq.LogicalPlan = byteLP + + return l.next.Do(ctx, r) +} diff --git a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go new file mode 100644 index 00000000000..70db0d4974f --- /dev/null +++ b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go @@ -0,0 +1,353 @@ +package instantquery + +import ( + "context" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" + "github.com/go-kit/log" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + "github.com/thanos-io/thanos/pkg/querysharding" + "github.com/weaveworks/common/middleware" + "github.com/weaveworks/common/user" + "io" + "net/http" + "net/http/httptest" + "net/url" + "strconv" + "testing" + "time" +) + +// TestInstantLogicalPlan ensures that the instant logical plan generation middleware +// correctly produces a logical plan and encodes it into the Prometheus request body. + +func TestInstantLogicalPlan(t *testing.T) { + for i, tc := range []struct { + name string + input *tripperware.PrometheusRequest + err error + }{ + { + name: "rate vector selector", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + err: nil, + }, + { + name: "memory usage expression", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + err: nil, + }, + { + name: "scalar only query", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "42", + }, + err: nil, + }, + { + name: "vector arithmetic", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", + }, + err: nil, + }, + { + name: "avg_over_time with nested rate", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", + }, + err: nil, + }, + } { + tc := tc + t.Run(strconv.Itoa(i), func(t *testing.T) { + t.Parallel() + + lpm := InstantLogicalPlanGenMiddleware( + time.Duration(1000), + false, + true, + ) + handler := lpm.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + + // Test Group 1: Execute middleware to populate the logical plan + _, _ = handler.Do(context.Background(), tc.input) + require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") + + // Test Group 2: Ensure the logical plan can be deserialized back + qOpts := query.Options{ + Start: time.Unix(tc.input.Start, 0), + End: time.Unix(tc.input.End, 0), + Step: time.Duration(1000), + } + planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: true} + _, err := logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) + require.NoError(t, err) + + // Test 3: Encode the request and validate method and body + httpReq, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.input) + require.NoError(t, err) + require.Equal(t, httpReq.Method, http.MethodPost, "Method should be POST") + + bodyBytes, err := io.ReadAll(httpReq.Body) + require.NoError(t, err) + require.NotEmpty(t, bodyBytes, "HTTP body should not be empty") + require.Equal(t, bodyBytes, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") + }) + } +} + +// Test 4: Integration test for query round-trip with distributed execution enabled (feature flag). +// Checks logical plan is generated, included in request body, and processed correctly. + +func TestRoundTripWithDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + + instantQueryMiddlewares, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{shardSize: 2}, + nil, + qa, + 5*time.Minute, + false, + true, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddlewares, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range instantQueryMiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, pReq tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.NotEmpty(t, body) + require.Equal(t, body, tc.pReq.LogicalPlan) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +func TestRoundTripWithoutDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + instantQueryMiddlewares, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{shardSize: 2}, + nil, + qa, + 5*time.Minute, + false, + false, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddlewares, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range instantQueryMiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.Empty(t, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +type mockLimitsShard struct { + maxQueryLookback time.Duration + maxQueryLength time.Duration + maxCacheFreshness time.Duration + maxQueryResponseSize int64 + shardSize int + queryPriority validation.QueryPriority + queryRejection validation.QueryRejection +} + +func (m mockLimitsShard) MaxQueryLookback(string) time.Duration { + return m.maxQueryLookback +} + +func (m mockLimitsShard) MaxQueryLength(string) time.Duration { + return m.maxQueryLength +} + +func (mockLimitsShard) MaxQueryParallelism(string) int { + return 14 // Flag default. +} + +func (m mockLimitsShard) MaxCacheFreshness(string) time.Duration { + return m.maxCacheFreshness +} + +func (m mockLimitsShard) MaxQueryResponseSize(string) int64 { + return m.maxQueryResponseSize +} + +func (m mockLimitsShard) QueryVerticalShardSize(userID string) int { + return m.shardSize +} + +func (m mockLimitsShard) QueryPriority(userID string) validation.QueryPriority { + return m.queryPriority +} + +func (m mockLimitsShard) QueryRejection(userID string) validation.QueryRejection { + return m.queryRejection +} diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index 54fe4aeba0d..cb4a7c01b4e 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -168,6 +168,8 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } } + h.Add("Content-Type", "application/json") + isSourceRuler := strings.Contains(h.Get("User-Agent"), tripperware.RulerUserAgent) if !isSourceRuler { // When the source is the Ruler, skip set header @@ -175,10 +177,10 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ } req := &http.Request{ - Method: "GET", + Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: http.NoBody, + Body: io.NopCloser(bytes.NewReader(promReq.LogicalPlan)), Header: h, } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 0fd48760480..b6ceeb1cdea 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -15,10 +15,19 @@ func Middlewares( merger tripperware.Merger, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, + enablePerStepStats bool, + distributedExecEnabled bool, + disableDuplicateLabelChecks bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ NewLimitsMiddleware(limits, lookbackDelta), tripperware.ShardByMiddleware(log, limits, merger, queryAnalyzer), } + + if distributedExecEnabled { + m = append(m, + InstantLogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + } + return m, nil } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go new file mode 100644 index 00000000000..63a65ef4fb9 --- /dev/null +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -0,0 +1,121 @@ +package instantquery + +import ( + "context" + "io" + "net/http" + "net/http/httptest" + "net/url" + "strconv" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/stretchr/testify/require" + "github.com/thanos-io/thanos/pkg/querysharding" + "github.com/weaveworks/common/middleware" + "github.com/weaveworks/common/user" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" +) + +const ( + distributedExecEnabled = false + queryAll = "/api/v1/instant_query?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536716898&stats=all" + responseBody = `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}]}}` +) + +func TestRoundTrip(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + instantQueryMiddleware, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{maxQueryLookback: 2}, + nil, + qa, + 5*time.Minute, + false, + distributedExecEnabled, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddleware, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + path, expectedBody string + }{ + {"/foo", "bar"}, + {queryAll, responseBody}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + req, err := http.NewRequest("POST", tc.path, http.NoBody) + require.NoError(t, err) + + ctx := user.InjectOrgID(context.Background(), "1") + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + bs, err := io.ReadAll(resp.Body) + require.NoError(t, err) + require.Equal(t, tc.expectedBody, string(bs)) + }) + } +} + +type singleHostRoundTripper struct { + host string + next http.RoundTripper +} + +func (s singleHostRoundTripper) RoundTrip(r *http.Request) (*http.Response, error) { + r.URL.Scheme = "http" + r.URL.Host = s.host + return s.next.RoundTrip(r) +} diff --git a/pkg/querier/tripperware/instantquery/instant_query_test.go b/pkg/querier/tripperware/instantquery/instant_query_test.go index b9dec5bbef9..6aa4e797842 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_test.go @@ -93,7 +93,7 @@ func TestRequest(t *testing.T) { tc := tc t.Run(tc.url, func(t *testing.T) { t.Parallel() - r, err := http.NewRequest("GET", tc.url, nil) + r, err := http.NewRequest("POST", tc.url, http.NoBody) require.NoError(t, err) r.Header.Add("Test-Header", "test") diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index e20ab6e3c4e..d549eb1a310 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -96,6 +96,8 @@ type Request interface { GetStep() int64 // GetQuery returns the query of the request. GetQuery() string + // GetLogicalPlan returns the serialized logical plan + GetLogicalPlan() []byte // WithStartEnd clone the current request with different start and end timestamp. WithStartEnd(startTime int64, endTime int64) Request // WithQuery clone the current request with a different query. @@ -152,6 +154,7 @@ type PrometheusRequest struct { Headers http.Header Stats string CachingOptions CachingOptions + LogicalPlan []byte } func (m *PrometheusRequest) GetPath() string { @@ -217,6 +220,13 @@ func (m *PrometheusRequest) GetStats() string { return "" } +func (m *PrometheusRequest) GetLogicalPlan() []byte { + if m != nil { + return m.LogicalPlan + } + return []byte{} +} + // WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. func (m *PrometheusRequest) WithStartEnd(start int64, end int64) Request { new := *m diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 9d82031fc0b..522ce2e202c 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -175,13 +175,15 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques } } + h.Add("Content-Type", "application/json") + tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) req := &http.Request{ - Method: "GET", + Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: http.NoBody, + Body: io.NopCloser(bytes.NewReader(promReq.LogicalPlan)), Header: h, } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 69185351bdc..0b80d2cdbc1 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -102,6 +102,9 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, + enablePerStepStats bool, + distributedExecEnabled bool, + disableDuplicateLabelChecks bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -137,5 +140,11 @@ func Middlewares( queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("shardBy", metrics), tripperware.ShardByMiddleware(log, limits, shardedPrometheusCodec, queryAnalyzer)) + if distributedExecEnabled { + queryRangeMiddleware = append(queryRangeMiddleware, + tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), + RangeLogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + } + return queryRangeMiddleware, c, nil } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index b5cdef60386..a56362cb2a2 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -25,13 +25,17 @@ var ( ShardedPrometheusCodec = NewPrometheusCodec(false, "", "protobuf") ) +const ( + distributedExecEnabled = false +) + func TestRoundTrip(t *testing.T) { s := httptest.NewServer( middleware.AuthenticateUser.Wrap( http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { var err error switch r.RequestURI { - case query: + case queryAll: _, err = w.Write([]byte(responseBody)) case queryWithWarnings: _, err = w.Write([]byte(responseBodyWithWarnings)) @@ -66,6 +70,9 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, + false, + distributedExecEnabled, + true, ) require.NoError(t, err) @@ -90,11 +97,11 @@ func TestRoundTrip(t *testing.T) { path, expectedBody string }{ {"/foo", "bar"}, - {query, responseBody}, + {queryAll, responseBody}, } { t.Run(strconv.Itoa(i), func(t *testing.T) { //parallel testing causes data race - req, err := http.NewRequest("GET", tc.path, http.NoBody) + req, err := http.NewRequest("POST", tc.path, http.NoBody) require.NoError(t, err) // query-frontend doesn't actually authenticate requests, we rely on diff --git a/pkg/querier/tripperware/queryrange/query_range_test.go b/pkg/querier/tripperware/queryrange/query_range_test.go index 1f3ebb137d8..27fba6b1bab 100644 --- a/pkg/querier/tripperware/queryrange/query_range_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_test.go @@ -43,7 +43,7 @@ func TestRequest(t *testing.T) { expectedErr error }{ { - url: query, + url: queryAll, expected: &parsedRequestWithHeaders, }, { @@ -74,7 +74,7 @@ func TestRequest(t *testing.T) { tc := tc t.Run(tc.url, func(t *testing.T) { t.Parallel() - r, err := http.NewRequest("GET", tc.url, nil) + r, err := http.NewRequest("POST", tc.url, http.NoBody) require.NoError(t, err) r.Header.Add("Test-Header", "test") diff --git a/pkg/querier/tripperware/queryrange/range_logicalplan_gen.go b/pkg/querier/tripperware/queryrange/range_logicalplan_gen.go new file mode 100644 index 00000000000..c154bff866b --- /dev/null +++ b/pkg/querier/tripperware/queryrange/range_logicalplan_gen.go @@ -0,0 +1,86 @@ +package queryrange + +import ( + "context" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/prometheus/prometheus/promql/parser" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + "github.com/weaveworks/common/httpgrpc" + "net/http" + "time" +) + +const ( + stepBatch = 10 +) + +func RangeLogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) tripperware.Middleware { + return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { + return rangeLogicalPlanGen{ + lookbackDelta: lookbackDelta, + enabledPerStepStats: enablePerStepStats, + next: next, + disableDuplicateLabelChecks: disableDuplicateLabelChecks, + } + }) +} + +type rangeLogicalPlanGen struct { + lookbackDelta time.Duration + enabledPerStepStats bool + next tripperware.Handler + disableDuplicateLabelChecks bool +} + +// NewRangeLogicalPlan generates an optimized and serialized logical query plan +func (l rangeLogicalPlanGen) NewRangeLogicalPlan(qs string, start, end time.Time, interval time.Duration) ([]byte, error) { + + qOpts := query.Options{ + Start: start, + End: end, + Step: interval, + StepsBatch: stepBatch, + LookbackDelta: l.lookbackDelta, + EnablePerStepStats: l.enabledPerStepStats, + } + + expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() + if err != nil { + return nil, err + } + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, + } + + lPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + optimizedPlan, _ := lPlan.Optimize(logicalplan.DefaultOptimizers) + byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) + if err != nil { + return nil, err + } + + // TODO: Add distributed optimizer for remote node insertion + + return byteLP, nil +} + +func (l rangeLogicalPlanGen) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { + promReq, ok := r.(*tripperware.PrometheusRequest) + if !ok { + return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") + } + + startTime := time.Unix(0, promReq.Start*int64(time.Millisecond)) + endTime := time.Unix(0, promReq.End*int64(time.Millisecond)) + duration := time.Duration(promReq.Step) * time.Millisecond + + byteLP, err := l.NewRangeLogicalPlan(promReq.Query, startTime, endTime, duration) + if err != nil { + return nil, err + } + promReq.LogicalPlan = byteLP + + return l.next.Do(ctx, r) +} diff --git a/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go b/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go new file mode 100644 index 00000000000..7cda4b13335 --- /dev/null +++ b/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go @@ -0,0 +1,336 @@ +package queryrange + +import ( + "context" + "github.com/cortexproject/cortex/pkg/util/validation" + "github.com/go-kit/log" + "github.com/thanos-io/thanos/pkg/querysharding" + "github.com/weaveworks/common/middleware" + "github.com/weaveworks/common/user" + "io" + "net/http" + "net/http/httptest" + "net/url" + "strconv" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" +) + +// TestRangeLogicalPlan validates the range logical plan generation middleware. +func TestRangeLogicalPlan(t *testing.T) { + testCases := []struct { + name string + input *tripperware.PrometheusRequest + }{ + { + name: "rate vector over time", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + }, + { + name: "memory usage ratio", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 30000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + }, + { + name: "avg_over_time function", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 60000, + Query: "avg_over_time(http_requests_total[5m])", + }, + }, + { + name: "vector arithmetic with range", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 10000, + Query: "rate(node_network_receive_bytes_total[1m]) / rate(node_network_transmit_bytes_total[1m])", + }, + }, + { + name: "simple scalar operation", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "2 + 2", + }, + }, + } + + for i, tc := range testCases { + tc := tc + t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { + t.Parallel() + + middleware := RangeLogicalPlanGenMiddleware( + 5*time.Minute, + true, + true, + ) + + handler := middleware.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + + // Test Group 1: Execute middleware to populate the logical plan + _, err := handler.Do(context.Background(), tc.input) + require.NoError(t, err) + require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") + + // Test Group 2: Ensure the logical plan can be deserialized back + start := time.Unix(0, tc.input.Start*int64(time.Millisecond)) + end := time.Unix(0, tc.input.End*int64(time.Millisecond)) + step := time.Duration(tc.input.Step) * time.Millisecond + + qOpts := query.Options{ + Start: start, + End: end, + Step: step, + StepsBatch: 10, + LookbackDelta: 5 * time.Minute, + EnablePerStepStats: true, + } + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: true, + } + _, err = logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) + require.NoError(t, err, "logical plan should be valid and de-serializable") + + // Test 3: Encode the request and validate method and body + httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) + require.NoError(t, err) + require.Equal(t, http.MethodPost, httpReq.Method) + + body, err := io.ReadAll(httpReq.Body) + require.NoError(t, err) + require.NotEmpty(t, body, "HTTP body should not be empty") + require.Equal(t, body, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") + }) + } +} + +// Test 4: Integration test for query round-trip with distributed execution enabled (feature flag). +// Checks logical plan is generated, included in request body, and processed correctly. +func TestRoundTripWithDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + case queryWithWarnings: + _, err = w.Write([]byte(responseBodyWithWarnings)) + case queryWithInfos: + _, err = w.Write([]byte(responseBodyWithInfos)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + queyrangemiddlewares, _, err := Middlewares(Config{}, + log.NewNopLogger(), + mockLimits{}, + nil, + nil, + qa, + PrometheusCodec, + ShardedPrometheusCodec, + 5*time.Minute, + false, + true, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + queyrangemiddlewares, + nil, + PrometheusCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range queyrangemiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.NotEmpty(t, body) + require.Equal(t, tc.pReq.LogicalPlan, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +func TestRoundTripWithoutDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + case queryWithWarnings: + _, err = w.Write([]byte(responseBodyWithWarnings)) + case queryWithInfos: + _, err = w.Write([]byte(responseBodyWithInfos)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + queyrangemiddlewares, _, err := Middlewares(Config{}, + log.NewNopLogger(), + mockLimits{}, + nil, + nil, + qa, + PrometheusCodec, + ShardedPrometheusCodec, + 5*time.Minute, + false, + false, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + queyrangemiddlewares, + nil, + PrometheusCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range queyrangemiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.Empty(t, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} diff --git a/pkg/querier/tripperware/queryrange/results_cache_test.go b/pkg/querier/tripperware/queryrange/results_cache_test.go index b029e85443c..8d4c32cfd25 100644 --- a/pkg/querier/tripperware/queryrange/results_cache_test.go +++ b/pkg/querier/tripperware/queryrange/results_cache_test.go @@ -26,7 +26,7 @@ import ( ) const ( - query = "/api/v1/query_range?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536673680&stats=all&step=120" + queryAll = "/api/v1/query_range?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536673680&stats=all&step=120" queryWithWarnings = "/api/v1/query_range?end=1536716898&query=sumsum%28warnings%29&start=1536673680&stats=all&step=120&warnings=true" queryWithInfos = "/api/v1/query_range?end=1536716898&query=rate%28go_gc_gogc_percent%5B5m%5D%29&start=1536673680&stats=all&step=120" responseBody = `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}]}}` diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index c854959be03..2f219182bdb 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -294,7 +294,7 @@ func TestSplitByDay(t *testing.T) { intervalFn IntervalFn }{ { - path: query, + path: queryAll, expectedBody: string(mergedHTTPResponseBody), expectedQueryCount: 2, intervalFn: func(ctx context.Context, _ tripperware.Request) (context.Context, time.Duration, error) { @@ -302,7 +302,7 @@ func TestSplitByDay(t *testing.T) { }, }, { - path: query, + path: queryAll, expectedBody: string(mergedHTTPResponseBody), expectedQueryCount: 2, intervalFn: dynamicIntervalFn(Config{SplitQueriesByInterval: day}, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta), @@ -344,7 +344,7 @@ func TestSplitByDay(t *testing.T) { next: http.DefaultTransport, }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(tc.intervalFn, mockLimits{}, PrometheusCodec, nil, lookbackDelta)) - req, err := http.NewRequest("GET", tc.path, http.NoBody) + req, err := http.NewRequest("POST", tc.path, http.NoBody) require.NoError(t, err) ctx := user.InjectOrgID(context.Background(), "1") diff --git a/pkg/querier/tripperware/roundtrip_test.go b/pkg/querier/tripperware/roundtrip_test.go index 0ef5e1519d5..5b50fac7337 100644 --- a/pkg/querier/tripperware/roundtrip_test.go +++ b/pkg/querier/tripperware/roundtrip_test.go @@ -321,7 +321,7 @@ cortex_query_frontend_queries_total{op="query", source="api", user="1"} 1 } { t.Run(tc.path, func(t *testing.T) { //parallel testing causes data race - req, err := http.NewRequest("GET", tc.path, http.NoBody) + req, err := http.NewRequest("POST", tc.path, http.NoBody) require.NoError(t, err) // query-frontend doesn't actually authenticate requests, we rely on diff --git a/pkg/querier/tripperware/test_shard_by_query_utils.go b/pkg/querier/tripperware/test_shard_by_query_utils.go index bb13f3e1438..f03555f7fd4 100644 --- a/pkg/querier/tripperware/test_shard_by_query_utils.go +++ b/pkg/querier/tripperware/test_shard_by_query_utils.go @@ -453,7 +453,7 @@ http_requests_total`, ctx := user.InjectOrgID(context.Background(), "1") - req, err := http.NewRequest("GET", tt.path, http.NoBody) + req, err := http.NewRequest("POST", tt.path, http.NoBody) req = req.WithContext(ctx) require.NoError(t, err) From dcfccb56e9e41923479b2e3ef9fee7f2dcc5be6f Mon Sep 17 00:00:00 2001 From: rubywtl Date: Wed, 16 Jul 2025 16:05:30 -0700 Subject: [PATCH 02/14] Combine instant and range logical query plan gen middleware + Update tests Signed-off-by: rubywtl --- .../instantquery/instant_logicalplan_gen.go | 84 ----- .../instant_logicalplan_gen_test.go | 353 ------------------ .../instantquery/instant_query_middlewares.go | 2 +- .../instant_query_middlewares_test.go | 237 ++++++++++++ .../logical_plan_gen_instant_test.go | 109 ++++++ ..._logicalplan_gen.go => logicalplan_gen.go} | 62 ++- .../queryrange/logical_plan_gen_range_test.go | 120 ++++++ .../queryrange/query_range_middlewares.go | 2 +- .../query_range_middlewares_test.go | 209 +++++++++++ .../queryrange/range_logicalplan_gen_test.go | 336 ----------------- pkg/querier/tripperware/roundtrip_test.go | 2 +- 11 files changed, 728 insertions(+), 788 deletions(-) delete mode 100644 pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go delete mode 100644 pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go create mode 100644 pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go rename pkg/querier/tripperware/{queryrange/range_logicalplan_gen.go => logicalplan_gen.go} (52%) create mode 100644 pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go delete mode 100644 pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go diff --git a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go deleted file mode 100644 index 942a62423ec..00000000000 --- a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen.go +++ /dev/null @@ -1,84 +0,0 @@ -package instantquery - -import ( - "context" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/prometheus/prometheus/promql/parser" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" - "github.com/weaveworks/common/httpgrpc" - "net/http" - "time" -) - -const ( - stepBatch = 10 -) - -func InstantLogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) tripperware.Middleware { - return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { - return instantLogicalPlanGen{ - lookbackDelta: lookbackDelta, - enabledPerStepStats: enablePerStepStats, - disableDuplicateLabelChecks: disableDuplicateLabelChecks, - next: next, - } - }) -} - -type instantLogicalPlanGen struct { - next tripperware.Handler - lookbackDelta time.Duration - enabledPerStepStats bool - disableDuplicateLabelChecks bool -} - -func (l instantLogicalPlanGen) NewInstantLogicalPlan(qs string, ts time.Time) ([]byte, error) { - - qOpts := query.Options{ - Start: ts, - End: ts, - Step: 0, - StepsBatch: stepBatch, - LookbackDelta: l.lookbackDelta, - EnablePerStepStats: l.disableDuplicateLabelChecks, - } - - expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() - if err != nil { - return nil, err - } - - planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, - } - - logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) - optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) - - // TODO: Add distributed optimizer for remote node insertion - - byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) - if err != nil { - return nil, err - } - - return byteLP, nil -} - -func (l instantLogicalPlanGen) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { - promReq, ok := r.(*tripperware.PrometheusRequest) - if !ok { - return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") - } - - instantTime := time.Unix(0, promReq.Start*int64(time.Millisecond)) - - byteLP, err := l.NewInstantLogicalPlan(promReq.Query, instantTime) - if err != nil { - return nil, err - } - promReq.LogicalPlan = byteLP - - return l.next.Do(ctx, r) -} diff --git a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go b/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go deleted file mode 100644 index 70db0d4974f..00000000000 --- a/pkg/querier/tripperware/instantquery/instant_logicalplan_gen_test.go +++ /dev/null @@ -1,353 +0,0 @@ -package instantquery - -import ( - "context" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/util/validation" - "github.com/go-kit/log" - "github.com/stretchr/testify/require" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" - "github.com/thanos-io/thanos/pkg/querysharding" - "github.com/weaveworks/common/middleware" - "github.com/weaveworks/common/user" - "io" - "net/http" - "net/http/httptest" - "net/url" - "strconv" - "testing" - "time" -) - -// TestInstantLogicalPlan ensures that the instant logical plan generation middleware -// correctly produces a logical plan and encodes it into the Prometheus request body. - -func TestInstantLogicalPlan(t *testing.T) { - for i, tc := range []struct { - name string - input *tripperware.PrometheusRequest - err error - }{ - { - name: "rate vector selector", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", - }, - err: nil, - }, - { - name: "memory usage expression", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", - }, - err: nil, - }, - { - name: "scalar only query", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "42", - }, - err: nil, - }, - { - name: "vector arithmetic", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", - }, - err: nil, - }, - { - name: "avg_over_time with nested rate", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", - }, - err: nil, - }, - } { - tc := tc - t.Run(strconv.Itoa(i), func(t *testing.T) { - t.Parallel() - - lpm := InstantLogicalPlanGenMiddleware( - time.Duration(1000), - false, - true, - ) - handler := lpm.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - - // Test Group 1: Execute middleware to populate the logical plan - _, _ = handler.Do(context.Background(), tc.input) - require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") - - // Test Group 2: Ensure the logical plan can be deserialized back - qOpts := query.Options{ - Start: time.Unix(tc.input.Start, 0), - End: time.Unix(tc.input.End, 0), - Step: time.Duration(1000), - } - planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: true} - _, err := logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) - require.NoError(t, err) - - // Test 3: Encode the request and validate method and body - httpReq, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.input) - require.NoError(t, err) - require.Equal(t, httpReq.Method, http.MethodPost, "Method should be POST") - - bodyBytes, err := io.ReadAll(httpReq.Body) - require.NoError(t, err) - require.NotEmpty(t, bodyBytes, "HTTP body should not be empty") - require.Equal(t, bodyBytes, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") - }) - } -} - -// Test 4: Integration test for query round-trip with distributed execution enabled (feature flag). -// Checks logical plan is generated, included in request body, and processed correctly. - -func TestRoundTripWithDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - - instantQueryMiddlewares, err := Middlewares( - log.NewNopLogger(), - mockLimitsShard{shardSize: 2}, - nil, - qa, - 5*time.Minute, - false, - true, - true, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - nil, - instantQueryMiddlewares, - testInstantQueryCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range instantQueryMiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, pReq tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) - require.NoError(t, err) - } - - req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.NotEmpty(t, body) - require.Equal(t, body, tc.pReq.LogicalPlan) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) - require.NoError(t, err) - }) - } -} - -func TestRoundTripWithoutDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - instantQueryMiddlewares, err := Middlewares( - log.NewNopLogger(), - mockLimitsShard{shardSize: 2}, - nil, - qa, - 5*time.Minute, - false, - false, - true, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - nil, - instantQueryMiddlewares, - testInstantQueryCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range instantQueryMiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) - require.NoError(t, err) - } - - req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.Empty(t, body) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) - require.NoError(t, err) - }) - } -} - -type mockLimitsShard struct { - maxQueryLookback time.Duration - maxQueryLength time.Duration - maxCacheFreshness time.Duration - maxQueryResponseSize int64 - shardSize int - queryPriority validation.QueryPriority - queryRejection validation.QueryRejection -} - -func (m mockLimitsShard) MaxQueryLookback(string) time.Duration { - return m.maxQueryLookback -} - -func (m mockLimitsShard) MaxQueryLength(string) time.Duration { - return m.maxQueryLength -} - -func (mockLimitsShard) MaxQueryParallelism(string) int { - return 14 // Flag default. -} - -func (m mockLimitsShard) MaxCacheFreshness(string) time.Duration { - return m.maxCacheFreshness -} - -func (m mockLimitsShard) MaxQueryResponseSize(string) int64 { - return m.maxQueryResponseSize -} - -func (m mockLimitsShard) QueryVerticalShardSize(userID string) int { - return m.shardSize -} - -func (m mockLimitsShard) QueryPriority(userID string) validation.QueryPriority { - return m.queryPriority -} - -func (m mockLimitsShard) QueryRejection(userID string) validation.QueryRejection { - return m.queryRejection -} diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index b6ceeb1cdea..ab26e52e3ad 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -26,7 +26,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - InstantLogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + tripperware.LogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 63a65ef4fb9..83adea9601f 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -109,6 +109,243 @@ func TestRoundTrip(t *testing.T) { } } +// Integration test for query round-trip with distributed execution enabled (feature flag). +// Checks logical plan is generated, included in request body, and processed correctly. + +func TestRoundTripWithDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + + instantQueryMiddlewares, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{shardSize: 2}, + nil, + qa, + 5*time.Minute, + false, + true, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddlewares, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range instantQueryMiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, pReq tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.NotEmpty(t, body) + require.Equal(t, body, tc.pReq.LogicalPlan) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +func TestRoundTripWithoutDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + instantQueryMiddlewares, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{shardSize: 2}, + nil, + qa, + 5*time.Minute, + false, + false, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddlewares, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range instantQueryMiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.Empty(t, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +type mockLimitsShard struct { + maxQueryLookback time.Duration + maxQueryLength time.Duration + maxCacheFreshness time.Duration + maxQueryResponseSize int64 + shardSize int + queryPriority validation.QueryPriority + queryRejection validation.QueryRejection +} + +func (m mockLimitsShard) MaxQueryLookback(string) time.Duration { + return m.maxQueryLookback +} + +func (m mockLimitsShard) MaxQueryLength(string) time.Duration { + return m.maxQueryLength +} + +func (mockLimitsShard) MaxQueryParallelism(string) int { + return 14 // Flag default. +} + +func (m mockLimitsShard) MaxCacheFreshness(string) time.Duration { + return m.maxCacheFreshness +} + +func (m mockLimitsShard) MaxQueryResponseSize(string) int64 { + return m.maxQueryResponseSize +} + +func (m mockLimitsShard) QueryVerticalShardSize(userID string) int { + return m.shardSize +} + +func (m mockLimitsShard) QueryPriority(userID string) validation.QueryPriority { + return m.queryPriority +} + +func (m mockLimitsShard) QueryRejection(userID string) validation.QueryRejection { + return m.queryRejection +} + type singleHostRoundTripper struct { host string next http.RoundTripper diff --git a/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go b/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go new file mode 100644 index 00000000000..c89bfec86a3 --- /dev/null +++ b/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go @@ -0,0 +1,109 @@ +package instantquery + +import ( + "context" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" + "io" + "net/http" + "strconv" + "testing" + "time" +) + +// TestInstantLogicalPlan ensures that the instant logical plan generation middleware +// correctly produces a logical plan and encodes it into the Prometheus request body. + +func TestInstantLogicalPlan(t *testing.T) { + for i, tc := range []struct { + name string + input *tripperware.PrometheusRequest + err error + }{ + { + name: "rate vector selector", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + err: nil, + }, + { + name: "memory usage expression", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + err: nil, + }, + { + name: "scalar only query", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "42", + }, + err: nil, + }, + { + name: "vector arithmetic", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", + }, + err: nil, + }, + { + name: "avg_over_time with nested rate", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", + }, + err: nil, + }, + } { + tc := tc + t.Run(strconv.Itoa(i), func(t *testing.T) { + t.Parallel() + + lpm := tripperware.LogicalPlanGenMiddleware( + time.Duration(1000), + false, + true, + ) + handler := lpm.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + + // Test Group 1: Execute middleware to populate the logical plan + _, _ = handler.Do(context.Background(), tc.input) + require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") + + // Test Group 2: Ensure the logical plan can be deserialized back + qOpts := query.Options{ + Start: time.Unix(tc.input.Start, 0), + End: time.Unix(tc.input.End, 0), + Step: time.Duration(1000), + } + planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: true} + _, err := logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) + require.NoError(t, err) + + // Test 3: Encode the request and validate method and body + httpReq, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.input) + require.NoError(t, err) + require.Equal(t, httpReq.Method, http.MethodPost, "Method should be POST") + + bodyBytes, err := io.ReadAll(httpReq.Body) + require.NoError(t, err) + require.NotEmpty(t, bodyBytes, "HTTP body should not be empty") + require.Equal(t, bodyBytes, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") + }) + } +} diff --git a/pkg/querier/tripperware/queryrange/range_logicalplan_gen.go b/pkg/querier/tripperware/logicalplan_gen.go similarity index 52% rename from pkg/querier/tripperware/queryrange/range_logicalplan_gen.go rename to pkg/querier/tripperware/logicalplan_gen.go index c154bff866b..c77c15d4955 100644 --- a/pkg/querier/tripperware/queryrange/range_logicalplan_gen.go +++ b/pkg/querier/tripperware/logicalplan_gen.go @@ -1,8 +1,7 @@ -package queryrange +package tripperware import ( "context" - "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/prometheus/prometheus/promql/parser" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/promql-engine/query" @@ -15,9 +14,9 @@ const ( stepBatch = 10 ) -func RangeLogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) tripperware.Middleware { - return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { - return rangeLogicalPlanGen{ +func LogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) Middleware { + return MiddlewareFunc(func(next Handler) Handler { + return logicalPlanGen{ lookbackDelta: lookbackDelta, enabledPerStepStats: enablePerStepStats, next: next, @@ -26,15 +25,47 @@ func RangeLogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepSta }) } -type rangeLogicalPlanGen struct { +type logicalPlanGen struct { lookbackDelta time.Duration enabledPerStepStats bool - next tripperware.Handler + next Handler disableDuplicateLabelChecks bool } -// NewRangeLogicalPlan generates an optimized and serialized logical query plan -func (l rangeLogicalPlanGen) NewRangeLogicalPlan(qs string, start, end time.Time, interval time.Duration) ([]byte, error) { +func (l logicalPlanGen) NewInstantLogicalPlan(qs string, ts time.Time) ([]byte, error) { + + qOpts := query.Options{ + Start: ts, + End: ts, + Step: 0, + StepsBatch: stepBatch, + LookbackDelta: l.lookbackDelta, + EnablePerStepStats: l.disableDuplicateLabelChecks, + } + + expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() + if err != nil { + return nil, err + } + + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, + } + + logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) + optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) + + // TODO: Add distributed optimizer for remote node insertion + + byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) + if err != nil { + return nil, err + } + + return byteLP, nil +} + +func (l logicalPlanGen) NewRangeLogicalPlan(qs string, start, end time.Time, interval time.Duration) ([]byte, error) { qOpts := query.Options{ Start: start, @@ -66,8 +97,8 @@ func (l rangeLogicalPlanGen) NewRangeLogicalPlan(qs string, start, end time.Time return byteLP, nil } -func (l rangeLogicalPlanGen) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { - promReq, ok := r.(*tripperware.PrometheusRequest) +func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { + promReq, ok := r.(*PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") } @@ -76,7 +107,14 @@ func (l rangeLogicalPlanGen) Do(ctx context.Context, r tripperware.Request) (tri endTime := time.Unix(0, promReq.End*int64(time.Millisecond)) duration := time.Duration(promReq.Step) * time.Millisecond - byteLP, err := l.NewRangeLogicalPlan(promReq.Query, startTime, endTime, duration) + var byteLP []byte + var err error + if promReq.Step != 0 { + byteLP, err = l.NewRangeLogicalPlan(promReq.Query, startTime, endTime, duration) + } else { + byteLP, err = l.NewInstantLogicalPlan(promReq.Query, startTime) + } + if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go b/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go new file mode 100644 index 00000000000..3eb5af6a17b --- /dev/null +++ b/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go @@ -0,0 +1,120 @@ +package queryrange + +import ( + "context" + "io" + "net/http" + "strconv" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" +) + +// TestRangeLogicalPlan validates the range logical plan generation middleware. +func TestRangeLogicalPlan(t *testing.T) { + testCases := []struct { + name string + input *tripperware.PrometheusRequest + }{ + { + name: "rate vector over time", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + }, + { + name: "memory usage ratio", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 30000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + }, + { + name: "avg_over_time function", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 60000, + Query: "avg_over_time(http_requests_total[5m])", + }, + }, + { + name: "vector arithmetic with range", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 10000, + Query: "rate(node_network_receive_bytes_total[1m]) / rate(node_network_transmit_bytes_total[1m])", + }, + }, + { + name: "simple scalar operation", + input: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "2 + 2", + }, + }, + } + + for i, tc := range testCases { + tc := tc + t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { + t.Parallel() + + middleware := tripperware.LogicalPlanGenMiddleware( + 5*time.Minute, + true, + true, + ) + + handler := middleware.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + + // Test Group 1: Execute middleware to populate the logical plan + _, err := handler.Do(context.Background(), tc.input) + require.NoError(t, err) + require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") + + // Test Group 2: Ensure the logical plan can be deserialized back + start := time.Unix(0, tc.input.Start*int64(time.Millisecond)) + end := time.Unix(0, tc.input.End*int64(time.Millisecond)) + step := time.Duration(tc.input.Step) * time.Millisecond + + qOpts := query.Options{ + Start: start, + End: end, + Step: step, + StepsBatch: 10, + LookbackDelta: 5 * time.Minute, + EnablePerStepStats: true, + } + planOpts := logicalplan.PlanOptions{ + DisableDuplicateLabelCheck: true, + } + _, err = logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) + require.NoError(t, err, "logical plan should be valid and de-serializable") + + // Test 3: Encode the request and validate method and body + httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) + require.NoError(t, err) + require.Equal(t, http.MethodPost, httpReq.Method) + + body, err := io.ReadAll(httpReq.Body) + require.NoError(t, err) + require.NotEmpty(t, body, "HTTP body should not be empty") + require.Equal(t, body, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") + }) + } +} diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 0b80d2cdbc1..541f2555fd6 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -143,7 +143,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - RangeLogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + tripperware.LogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index a56362cb2a2..47d205aeb6c 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -123,6 +123,215 @@ func TestRoundTrip(t *testing.T) { } } +// Integration test for query round-trip with distributed execution enabled (feature flag). +// Checks logical plan is generated, included in request body, and processed correctly. +func TestRoundTripWithDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + case queryWithWarnings: + _, err = w.Write([]byte(responseBodyWithWarnings)) + case queryWithInfos: + _, err = w.Write([]byte(responseBodyWithInfos)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + queyrangemiddlewares, _, err := Middlewares(Config{}, + log.NewNopLogger(), + mockLimits{}, + nil, + nil, + qa, + PrometheusCodec, + ShardedPrometheusCodec, + 5*time.Minute, + false, + true, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + queyrangemiddlewares, + nil, + PrometheusCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range queyrangemiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.NotEmpty(t, body) + require.Equal(t, tc.pReq.LogicalPlan, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + +func TestRoundTripWithoutDistributedExec(t *testing.T) { + s := httptest.NewServer( + middleware.AuthenticateUser.Wrap( + http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var err error + switch r.RequestURI { + case queryAll: + _, err = w.Write([]byte(responseBody)) + case queryWithWarnings: + _, err = w.Write([]byte(responseBodyWithWarnings)) + case queryWithInfos: + _, err = w.Write([]byte(responseBodyWithInfos)) + default: + _, err = w.Write([]byte("bar")) + } + if err != nil { + t.Fatal(err) + } + }), + ), + ) + defer s.Close() + + u, err := url.Parse(s.URL) + require.NoError(t, err) + + downstream := singleHostRoundTripper{ + host: u.Host, + next: http.DefaultTransport, + } + + qa := querysharding.NewQueryAnalyzer() + queyrangemiddlewares, _, err := Middlewares(Config{}, + log.NewNopLogger(), + mockLimits{}, + nil, + nil, + qa, + PrometheusCodec, + ShardedPrometheusCodec, + 5*time.Minute, + false, + false, + true, + ) + require.NoError(t, err) + + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + queyrangemiddlewares, + nil, + PrometheusCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) + + for i, tc := range []struct { + pReq *tripperware.PrometheusRequest + }{ + {pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + ctx := user.InjectOrgID(context.Background(), "1") + + for _, mw := range queyrangemiddlewares { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { + return nil, nil + })) + _, err := handler.Do(ctx, tc.pReq) + require.NoError(t, err) + } + + req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) + require.NoError(t, err) + req = req.WithContext(ctx) + err = user.InjectOrgIDIntoHTTPRequest(ctx, req) + require.NoError(t, err) + body, err := io.ReadAll(req.Body) + require.Empty(t, body) + + resp, err := tw(downstream).RoundTrip(req) + require.NoError(t, err) + require.Equal(t, 200, resp.StatusCode) + + _, err = io.ReadAll(resp.Body) + require.NoError(t, err) + }) + } +} + type singleHostRoundTripper struct { host string next http.RoundTripper diff --git a/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go b/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go deleted file mode 100644 index 7cda4b13335..00000000000 --- a/pkg/querier/tripperware/queryrange/range_logicalplan_gen_test.go +++ /dev/null @@ -1,336 +0,0 @@ -package queryrange - -import ( - "context" - "github.com/cortexproject/cortex/pkg/util/validation" - "github.com/go-kit/log" - "github.com/thanos-io/thanos/pkg/querysharding" - "github.com/weaveworks/common/middleware" - "github.com/weaveworks/common/user" - "io" - "net/http" - "net/http/httptest" - "net/url" - "strconv" - "testing" - "time" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/stretchr/testify/require" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" -) - -// TestRangeLogicalPlan validates the range logical plan generation middleware. -func TestRangeLogicalPlan(t *testing.T) { - testCases := []struct { - name string - input *tripperware.PrometheusRequest - }{ - { - name: "rate vector over time", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", - }, - }, - { - name: "memory usage ratio", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 30000, - Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", - }, - }, - { - name: "avg_over_time function", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 60000, - Query: "avg_over_time(http_requests_total[5m])", - }, - }, - { - name: "vector arithmetic with range", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 10000, - Query: "rate(node_network_receive_bytes_total[1m]) / rate(node_network_transmit_bytes_total[1m])", - }, - }, - { - name: "simple scalar operation", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "2 + 2", - }, - }, - } - - for i, tc := range testCases { - tc := tc - t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { - t.Parallel() - - middleware := RangeLogicalPlanGenMiddleware( - 5*time.Minute, - true, - true, - ) - - handler := middleware.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - - // Test Group 1: Execute middleware to populate the logical plan - _, err := handler.Do(context.Background(), tc.input) - require.NoError(t, err) - require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") - - // Test Group 2: Ensure the logical plan can be deserialized back - start := time.Unix(0, tc.input.Start*int64(time.Millisecond)) - end := time.Unix(0, tc.input.End*int64(time.Millisecond)) - step := time.Duration(tc.input.Step) * time.Millisecond - - qOpts := query.Options{ - Start: start, - End: end, - Step: step, - StepsBatch: 10, - LookbackDelta: 5 * time.Minute, - EnablePerStepStats: true, - } - planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: true, - } - _, err = logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) - require.NoError(t, err, "logical plan should be valid and de-serializable") - - // Test 3: Encode the request and validate method and body - httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) - require.NoError(t, err) - require.Equal(t, http.MethodPost, httpReq.Method) - - body, err := io.ReadAll(httpReq.Body) - require.NoError(t, err) - require.NotEmpty(t, body, "HTTP body should not be empty") - require.Equal(t, body, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") - }) - } -} - -// Test 4: Integration test for query round-trip with distributed execution enabled (feature flag). -// Checks logical plan is generated, included in request body, and processed correctly. -func TestRoundTripWithDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - case queryWithWarnings: - _, err = w.Write([]byte(responseBodyWithWarnings)) - case queryWithInfos: - _, err = w.Write([]byte(responseBodyWithInfos)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - queyrangemiddlewares, _, err := Middlewares(Config{}, - log.NewNopLogger(), - mockLimits{}, - nil, - nil, - qa, - PrometheusCodec, - ShardedPrometheusCodec, - 5*time.Minute, - false, - true, - true, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - queyrangemiddlewares, - nil, - PrometheusCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range queyrangemiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) - require.NoError(t, err) - } - - req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.NotEmpty(t, body) - require.Equal(t, tc.pReq.LogicalPlan, body) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) - require.NoError(t, err) - }) - } -} - -func TestRoundTripWithoutDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - case queryWithWarnings: - _, err = w.Write([]byte(responseBodyWithWarnings)) - case queryWithInfos: - _, err = w.Write([]byte(responseBodyWithInfos)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - queyrangemiddlewares, _, err := Middlewares(Config{}, - log.NewNopLogger(), - mockLimits{}, - nil, - nil, - qa, - PrometheusCodec, - ShardedPrometheusCodec, - 5*time.Minute, - false, - false, - true, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - queyrangemiddlewares, - nil, - PrometheusCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range queyrangemiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) - require.NoError(t, err) - } - - req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.Empty(t, body) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) - require.NoError(t, err) - }) - } -} diff --git a/pkg/querier/tripperware/roundtrip_test.go b/pkg/querier/tripperware/roundtrip_test.go index 5b50fac7337..c519d16c4c8 100644 --- a/pkg/querier/tripperware/roundtrip_test.go +++ b/pkg/querier/tripperware/roundtrip_test.go @@ -28,7 +28,7 @@ import ( const ( queryRange = "/api/v1/query_range?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536673680&stats=all&step=120" - query = "/api/v1/query?time=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29" + queryAll = "/api/v1/query?time=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29" queryNonShardable = "/api/v1/query?time=1536716898&query=container_memory_rss" queryExemplar = "/api/v1/query_exemplars?query=test_exemplar_metric_total&start=2020-09-14T15:22:25.479Z&end=2020-09-14T15:23:25.479Z'" querySubqueryStepSizeTooSmall = "/api/v1/query?query=up%5B30d%3A%5D" From 16e1f87d66f2fecf35e3c6d5c4754e7ecea1e0c6 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Thu, 17 Jul 2025 13:23:52 -0700 Subject: [PATCH 03/14] remove unneccessary configs in query frontend + restructure logical-plan-gen middleware helper functions Signed-off-by: rubywtl --- docs/configuration/config-file-reference.md | 5 + pkg/cortex/modules.go | 5 +- pkg/frontend/config.go | 6 +- .../tripperware/instantquery/instant_query.go | 22 ++++- .../instantquery/instant_query_middlewares.go | 4 +- .../instant_query_middlewares_test.go | 14 +-- .../logical_plan_gen_instant_test.go | 39 ++++---- pkg/querier/tripperware/logicalplan_gen.go | 97 ++++++------------- pkg/querier/tripperware/query.go | 9 +- .../queryrange/logical_plan_gen_range_test.go | 42 ++++---- .../tripperware/queryrange/query_range.go | 23 ++++- .../queryrange/query_range_middlewares.go | 4 +- .../query_range_middlewares_test.go | 14 +-- 13 files changed, 143 insertions(+), 141 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 2963a87348c..27d80558597 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4451,6 +4451,11 @@ grpc_client_config: # URL of downstream Prometheus. # CLI flag: -frontend.downstream-url [downstream_url: | default = ""] + +# Experimental: Enables distributed execution of queries by passing logical +# query plan fragments to downstream components. +# CLI flag: -frontend.distributed_exec_enabled +[distributed_exec_enabled: | default = false] ``` ### `query_range_config` diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index febb4f33a59..6e9402da66f 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -533,16 +533,13 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, - t.Cfg.Querier.EnablePerStepStats, t.Cfg.Frontend.DistributedExecEnabled, - t.Cfg.Frontend.DisableDuplicateLabelChecks, ) if err != nil { return nil, err } - instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Querier.EnablePerStepStats, - t.Cfg.Frontend.DistributedExecEnabled, t.Cfg.Frontend.DisableDuplicateLabelChecks) + instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled) if err != nil { return nil, err } diff --git a/pkg/frontend/config.go b/pkg/frontend/config.go index 75212e65d5c..633f8f53e74 100644 --- a/pkg/frontend/config.go +++ b/pkg/frontend/config.go @@ -20,9 +20,8 @@ type CombinedFrontendConfig struct { FrontendV1 v1.Config `yaml:",inline"` FrontendV2 v2.Config `yaml:",inline"` - DownstreamURL string `yaml:"downstream_url"` - DistributedExecEnabled bool `yaml:"distributed_exec_enabled"` - DisableDuplicateLabelChecks bool `yaml:"disable_duplicate_label_checks"` + DownstreamURL string `yaml:"downstream_url"` + DistributedExecEnabled bool `yaml:"distributed_exec_enabled"` } func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { @@ -32,7 +31,6 @@ func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { f.StringVar(&cfg.DownstreamURL, "frontend.downstream-url", "", "URL of downstream Prometheus.") f.BoolVar(&cfg.DistributedExecEnabled, "frontend.distributed_exec_enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan fragments to downstream components.") - f.BoolVar(&cfg.DisableDuplicateLabelChecks, "frontend.disable_duplicate_label_checks", false, "Experimental: Disables duplicate label checks during logical plan generation") } // InitFrontend initializes frontend (either V1 -- without scheduler, or V2 -- with scheduler) or no frontend at diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index cb4a7c01b4e..ad0b4ae8732 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -23,6 +23,8 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/spanlogger" + + "github.com/thanos-io/promql-engine/logicalplan" ) var ( @@ -141,6 +143,19 @@ func (c instantQueryCodec) DecodeResponse(ctx context.Context, r *http.Response, return &resp, nil } +func (c instantQueryCodec) getSerializedBody(promReq *tripperware.PrometheusRequest) ([]byte, error) { + var byteLP []byte + var err error + + if promReq.LogicalPlan != nil && *promReq.LogicalPlan != nil { + byteLP, err = logicalplan.Marshal((*promReq.LogicalPlan).Root()) + if err != nil { + return nil, err + } + } + return byteLP, nil +} + func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { promReq, ok := r.(*tripperware.PrometheusRequest) if !ok { @@ -176,11 +191,16 @@ func (c instantQueryCodec) EncodeRequest(ctx context.Context, r tripperware.Requ tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) } + byteBody, err := c.getSerializedBody(promReq) + if err != nil { + return nil, err + } + req := &http.Request{ Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: io.NopCloser(bytes.NewReader(promReq.LogicalPlan)), + Body: io.NopCloser(bytes.NewReader(byteBody)), Header: h, } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index ab26e52e3ad..2bdffe1be23 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -15,9 +15,7 @@ func Middlewares( merger tripperware.Merger, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, - enablePerStepStats bool, distributedExecEnabled bool, - disableDuplicateLabelChecks bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ NewLimitsMiddleware(limits, lookbackDelta), @@ -26,7 +24,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.LogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + tripperware.LogicalPlanGenMiddleware()) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 83adea9601f..ed331083545 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -2,6 +2,7 @@ package instantquery import ( "context" + "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/http/httptest" @@ -60,9 +61,7 @@ func TestRoundTrip(t *testing.T) { nil, qa, 5*time.Minute, - false, distributedExecEnabled, - true, ) require.NoError(t, err) @@ -147,8 +146,6 @@ func TestRoundTripWithDistributedExec(t *testing.T) { nil, qa, 5*time.Minute, - false, - true, true, ) require.NoError(t, err) @@ -195,9 +192,14 @@ func TestRoundTripWithDistributedExec(t *testing.T) { req = req.WithContext(ctx) err = user.InjectOrgIDIntoHTTPRequest(ctx, req) require.NoError(t, err) + body, err := io.ReadAll(req.Body) require.NotEmpty(t, body) - require.Equal(t, body, tc.pReq.LogicalPlan) + require.NoError(t, err) + + byteLP, err := logicalplan.Marshal((*tc.pReq.LogicalPlan).Root()) + require.NoError(t, err) + require.Equal(t, byteLP, body) resp, err := tw(downstream).RoundTrip(req) require.NoError(t, err) @@ -244,8 +246,6 @@ func TestRoundTripWithoutDistributedExec(t *testing.T) { qa, 5*time.Minute, false, - false, - true, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go b/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go index c89bfec86a3..01e1ecb9cf0 100644 --- a/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go +++ b/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go @@ -72,11 +72,7 @@ func TestInstantLogicalPlan(t *testing.T) { t.Run(strconv.Itoa(i), func(t *testing.T) { t.Parallel() - lpm := tripperware.LogicalPlanGenMiddleware( - time.Duration(1000), - false, - true, - ) + lpm := tripperware.LogicalPlanGenMiddleware() handler := lpm.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { return nil, nil })) @@ -85,17 +81,7 @@ func TestInstantLogicalPlan(t *testing.T) { _, _ = handler.Do(context.Background(), tc.input) require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") - // Test Group 2: Ensure the logical plan can be deserialized back - qOpts := query.Options{ - Start: time.Unix(tc.input.Start, 0), - End: time.Unix(tc.input.End, 0), - Step: time.Duration(1000), - } - planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: true} - _, err := logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) - require.NoError(t, err) - - // Test 3: Encode the request and validate method and body + // Test 2: Encode the request and validate method and body httpReq, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.input) require.NoError(t, err) require.Equal(t, httpReq.Method, http.MethodPost, "Method should be POST") @@ -103,7 +89,26 @@ func TestInstantLogicalPlan(t *testing.T) { bodyBytes, err := io.ReadAll(httpReq.Body) require.NoError(t, err) require.NotEmpty(t, bodyBytes, "HTTP body should not be empty") - require.Equal(t, bodyBytes, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") + + // Test Group 3: Ensure the logical plan can be deserialized back + qOpts := query.Options{ + Start: time.Unix(tc.input.Start, 0), + End: time.Unix(tc.input.Start, 0), + Step: 0, + } + + qOpts = query.Options{ + Start: time.Unix(tc.input.Start, 0), + End: time.Unix(tc.input.Start, 0), + Step: 0, + StepsBatch: 10, + LookbackDelta: 0, + EnablePerStepStats: false, + } + + planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: false} + _, err = logicalplan.NewFromBytes(bodyBytes, &qOpts, planOpts) + require.NoError(t, err) }) } } diff --git a/pkg/querier/tripperware/logicalplan_gen.go b/pkg/querier/tripperware/logicalplan_gen.go index c77c15d4955..e89776edeab 100644 --- a/pkg/querier/tripperware/logicalplan_gen.go +++ b/pkg/querier/tripperware/logicalplan_gen.go @@ -14,33 +14,39 @@ const ( stepBatch = 10 ) -func LogicalPlanGenMiddleware(lookbackDelta time.Duration, enablePerStepStats bool, disableDuplicateLabelChecks bool) Middleware { +func LogicalPlanGenMiddleware() Middleware { return MiddlewareFunc(func(next Handler) Handler { return logicalPlanGen{ - lookbackDelta: lookbackDelta, - enabledPerStepStats: enablePerStepStats, - next: next, - disableDuplicateLabelChecks: disableDuplicateLabelChecks, + next: next, } }) } type logicalPlanGen struct { - lookbackDelta time.Duration - enabledPerStepStats bool - next Handler - disableDuplicateLabelChecks bool + next Handler } -func (l logicalPlanGen) NewInstantLogicalPlan(qs string, ts time.Time) ([]byte, error) { - - qOpts := query.Options{ - Start: ts, - End: ts, - Step: 0, - StepsBatch: stepBatch, - LookbackDelta: l.lookbackDelta, - EnablePerStepStats: l.disableDuplicateLabelChecks, +func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { + + qOpts := query.Options{} + if step == 0 { + qOpts = query.Options{ + Start: start, + End: start, + Step: 0, + StepsBatch: stepBatch, + LookbackDelta: 0, + EnablePerStepStats: false, + } + } else { + qOpts = query.Options{ + Start: start, + End: end, + Step: step, + StepsBatch: stepBatch, + LookbackDelta: 0, + EnablePerStepStats: false, + } } expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() @@ -49,52 +55,15 @@ func (l logicalPlanGen) NewInstantLogicalPlan(qs string, ts time.Time) ([]byte, } planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, + DisableDuplicateLabelCheck: false, } logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) - // TODO: Add distributed optimizer for remote node insertion - - byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) - if err != nil { - return nil, err - } + //TODO: Add distributed query optimizer - return byteLP, nil -} - -func (l logicalPlanGen) NewRangeLogicalPlan(qs string, start, end time.Time, interval time.Duration) ([]byte, error) { - - qOpts := query.Options{ - Start: start, - End: end, - Step: interval, - StepsBatch: stepBatch, - LookbackDelta: l.lookbackDelta, - EnablePerStepStats: l.enabledPerStepStats, - } - - expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() - if err != nil { - return nil, err - } - - planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: l.disableDuplicateLabelChecks, - } - - lPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) - optimizedPlan, _ := lPlan.Optimize(logicalplan.DefaultOptimizers) - byteLP, err := logicalplan.Marshal(optimizedPlan.Root()) - if err != nil { - return nil, err - } - - // TODO: Add distributed optimizer for remote node insertion - - return byteLP, nil + return &optimizedPlan, nil } func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { @@ -105,20 +74,16 @@ func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { startTime := time.Unix(0, promReq.Start*int64(time.Millisecond)) endTime := time.Unix(0, promReq.End*int64(time.Millisecond)) - duration := time.Duration(promReq.Step) * time.Millisecond + step := time.Duration(promReq.Step) * time.Millisecond - var byteLP []byte var err error - if promReq.Step != 0 { - byteLP, err = l.NewRangeLogicalPlan(promReq.Query, startTime, endTime, duration) - } else { - byteLP, err = l.NewInstantLogicalPlan(promReq.Query, startTime) - } + newLogicalPlan, err := l.NewLogicalPlan(promReq.Query, startTime, endTime, step) if err != nil { return nil, err } - promReq.LogicalPlan = byteLP + + promReq.LogicalPlan = newLogicalPlan return l.next.Do(ctx, r) } diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index d549eb1a310..b5d857a42b2 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -6,6 +6,7 @@ import ( "context" "encoding/binary" "fmt" + "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "strconv" @@ -97,7 +98,7 @@ type Request interface { // GetQuery returns the query of the request. GetQuery() string // GetLogicalPlan returns the serialized logical plan - GetLogicalPlan() []byte + GetLogicalPlan() *logicalplan.Plan // WithStartEnd clone the current request with different start and end timestamp. WithStartEnd(startTime int64, endTime int64) Request // WithQuery clone the current request with a different query. @@ -154,7 +155,7 @@ type PrometheusRequest struct { Headers http.Header Stats string CachingOptions CachingOptions - LogicalPlan []byte + LogicalPlan *logicalplan.Plan } func (m *PrometheusRequest) GetPath() string { @@ -220,11 +221,11 @@ func (m *PrometheusRequest) GetStats() string { return "" } -func (m *PrometheusRequest) GetLogicalPlan() []byte { +func (m *PrometheusRequest) GetLogicalPlan() *logicalplan.Plan { if m != nil { return m.LogicalPlan } - return []byte{} + return nil } // WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. diff --git a/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go b/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go index 3eb5af6a17b..a5bd3f0f062 100644 --- a/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go +++ b/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go @@ -2,16 +2,15 @@ package queryrange import ( "context" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" + "github.com/thanos-io/promql-engine/query" "io" "net/http" "strconv" "testing" "time" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/stretchr/testify/require" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" ) // TestRangeLogicalPlan validates the range logical plan generation middleware. @@ -72,11 +71,7 @@ func TestRangeLogicalPlan(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := tripperware.LogicalPlanGenMiddleware( - 5*time.Minute, - true, - true, - ) + middleware := tripperware.LogicalPlanGenMiddleware() handler := middleware.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { return nil, nil @@ -87,7 +82,16 @@ func TestRangeLogicalPlan(t *testing.T) { require.NoError(t, err) require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") - // Test Group 2: Ensure the logical plan can be deserialized back + // Test 2: Encode the request and validate method and body + httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) + require.NoError(t, err) + require.Equal(t, http.MethodPost, httpReq.Method) + + body, err := io.ReadAll(httpReq.Body) + require.NoError(t, err) + require.NotEmpty(t, body, "HTTP body should not be empty") + + // Test Group 3: Ensure the logical plan can be deserialized back start := time.Unix(0, tc.input.Start*int64(time.Millisecond)) end := time.Unix(0, tc.input.End*int64(time.Millisecond)) step := time.Duration(tc.input.Step) * time.Millisecond @@ -98,23 +102,13 @@ func TestRangeLogicalPlan(t *testing.T) { Step: step, StepsBatch: 10, LookbackDelta: 5 * time.Minute, - EnablePerStepStats: true, + EnablePerStepStats: false, } planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: true, + DisableDuplicateLabelCheck: false, } - _, err = logicalplan.NewFromBytes(tc.input.LogicalPlan, &qOpts, planOpts) + _, err = logicalplan.NewFromBytes(body, &qOpts, planOpts) require.NoError(t, err, "logical plan should be valid and de-serializable") - - // Test 3: Encode the request and validate method and body - httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) - require.NoError(t, err) - require.Equal(t, http.MethodPost, httpReq.Method) - - body, err := io.ReadAll(httpReq.Body) - require.NoError(t, err) - require.NotEmpty(t, body, "HTTP body should not be empty") - require.Equal(t, body, tc.input.LogicalPlan, "logical plan in request body does not match expected bytes") }) } } diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 522ce2e202c..52423237f0e 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -3,6 +3,7 @@ package queryrange import ( "bytes" "context" + "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/url" @@ -151,6 +152,21 @@ func (c prometheusCodec) DecodeRequest(_ context.Context, r *http.Request, forwa return &result, nil } +// getSerializedBody serializes the logical plan from a Prometheus request. +// Returns an empty byte array if the logical plan is nil. +func (c prometheusCodec) getSerializedBody(promReq *tripperware.PrometheusRequest) ([]byte, error) { + var byteLP []byte + var err error + + if promReq.LogicalPlan != nil && *promReq.LogicalPlan != nil { + byteLP, err = logicalplan.Marshal((*promReq.LogicalPlan).Root()) + if err != nil { + return nil, err + } + } + return byteLP, nil +} + func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Request) (*http.Request, error) { promReq, ok := r.(*tripperware.PrometheusRequest) if !ok { @@ -179,11 +195,16 @@ func (c prometheusCodec) EncodeRequest(ctx context.Context, r tripperware.Reques tripperware.SetRequestHeaders(h, c.defaultCodecType, c.compression) + bodyBytes, err := c.getSerializedBody(promReq) + if err != nil { + return nil, err + } + req := &http.Request{ Method: "POST", RequestURI: u.String(), // This is what the httpgrpc code looks at. URL: u, - Body: io.NopCloser(bytes.NewReader(promReq.LogicalPlan)), + Body: io.NopCloser(bytes.NewReader(bodyBytes)), Header: h, } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 541f2555fd6..3aca642fcd4 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -102,9 +102,7 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, - enablePerStepStats bool, distributedExecEnabled bool, - disableDuplicateLabelChecks bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -143,7 +141,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.LogicalPlanGenMiddleware(lookbackDelta, enablePerStepStats, disableDuplicateLabelChecks)) + tripperware.LogicalPlanGenMiddleware()) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 47d205aeb6c..65165ce47dc 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/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/http/httptest" @@ -71,8 +72,6 @@ func TestRoundTrip(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, false, - distributedExecEnabled, - true, ) require.NoError(t, err) @@ -166,8 +165,6 @@ func TestRoundTripWithDistributedExec(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - false, - true, true, ) require.NoError(t, err) @@ -215,9 +212,14 @@ func TestRoundTripWithDistributedExec(t *testing.T) { req = req.WithContext(ctx) err = user.InjectOrgIDIntoHTTPRequest(ctx, req) require.NoError(t, err) + body, err := io.ReadAll(req.Body) require.NotEmpty(t, body) - require.Equal(t, tc.pReq.LogicalPlan, body) + require.NoError(t, err) + + byteLP, err := logicalplan.Marshal((*tc.pReq.LogicalPlan).Root()) + require.NoError(t, err) + require.Equal(t, byteLP, body) resp, err := tw(downstream).RoundTrip(req) require.NoError(t, err) @@ -271,8 +273,6 @@ func TestRoundTripWithoutDistributedExec(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, false, - false, - true, ) require.NoError(t, err) From bd386f3dfbc02d597b5c48ed8399c8f30cd8eb92 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Thu, 17 Jul 2025 15:17:55 -0700 Subject: [PATCH 04/14] adjust prom request logicalplan type and update tests Signed-off-by: rubywtl --- .../tripperware/instantquery/instant_query.go | 4 +- .../instant_query_middlewares_test.go | 2 +- .../logical_plan_gen_instant_test.go | 114 ------------- .../tripperware/logical_plan_gen_test.go | 153 ++++++++++++++++++ pkg/querier/tripperware/logicalplan_gen.go | 2 +- pkg/querier/tripperware/query.go | 8 +- .../queryrange/logical_plan_gen_range_test.go | 114 ------------- .../tripperware/queryrange/query_range.go | 4 +- .../query_range_middlewares_test.go | 2 +- pkg/querier/tripperware/roundtrip_test.go | 4 +- 10 files changed, 166 insertions(+), 241 deletions(-) delete mode 100644 pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go create mode 100644 pkg/querier/tripperware/logical_plan_gen_test.go delete mode 100644 pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go diff --git a/pkg/querier/tripperware/instantquery/instant_query.go b/pkg/querier/tripperware/instantquery/instant_query.go index ad0b4ae8732..a3977207199 100644 --- a/pkg/querier/tripperware/instantquery/instant_query.go +++ b/pkg/querier/tripperware/instantquery/instant_query.go @@ -147,8 +147,8 @@ func (c instantQueryCodec) getSerializedBody(promReq *tripperware.PrometheusRequ var byteLP []byte var err error - if promReq.LogicalPlan != nil && *promReq.LogicalPlan != nil { - byteLP, err = logicalplan.Marshal((*promReq.LogicalPlan).Root()) + if promReq.LogicalPlan != nil { + byteLP, err = logicalplan.Marshal(promReq.LogicalPlan.Root()) if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index ed331083545..591c7e57d3f 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -197,7 +197,7 @@ func TestRoundTripWithDistributedExec(t *testing.T) { require.NotEmpty(t, body) require.NoError(t, err) - byteLP, err := logicalplan.Marshal((*tc.pReq.LogicalPlan).Root()) + byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) require.NoError(t, err) require.Equal(t, byteLP, body) diff --git a/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go b/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go deleted file mode 100644 index 01e1ecb9cf0..00000000000 --- a/pkg/querier/tripperware/instantquery/logical_plan_gen_instant_test.go +++ /dev/null @@ -1,114 +0,0 @@ -package instantquery - -import ( - "context" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/stretchr/testify/require" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" - "io" - "net/http" - "strconv" - "testing" - "time" -) - -// TestInstantLogicalPlan ensures that the instant logical plan generation middleware -// correctly produces a logical plan and encodes it into the Prometheus request body. - -func TestInstantLogicalPlan(t *testing.T) { - for i, tc := range []struct { - name string - input *tripperware.PrometheusRequest - err error - }{ - { - name: "rate vector selector", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", - }, - err: nil, - }, - { - name: "memory usage expression", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", - }, - err: nil, - }, - { - name: "scalar only query", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "42", - }, - err: nil, - }, - { - name: "vector arithmetic", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", - }, - err: nil, - }, - { - name: "avg_over_time with nested rate", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", - }, - err: nil, - }, - } { - tc := tc - t.Run(strconv.Itoa(i), func(t *testing.T) { - t.Parallel() - - lpm := tripperware.LogicalPlanGenMiddleware() - handler := lpm.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - - // Test Group 1: Execute middleware to populate the logical plan - _, _ = handler.Do(context.Background(), tc.input) - require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") - - // Test 2: Encode the request and validate method and body - httpReq, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.input) - require.NoError(t, err) - require.Equal(t, httpReq.Method, http.MethodPost, "Method should be POST") - - bodyBytes, err := io.ReadAll(httpReq.Body) - require.NoError(t, err) - require.NotEmpty(t, bodyBytes, "HTTP body should not be empty") - - // Test Group 3: Ensure the logical plan can be deserialized back - qOpts := query.Options{ - Start: time.Unix(tc.input.Start, 0), - End: time.Unix(tc.input.Start, 0), - Step: 0, - } - - qOpts = query.Options{ - Start: time.Unix(tc.input.Start, 0), - End: time.Unix(tc.input.Start, 0), - Step: 0, - StepsBatch: 10, - LookbackDelta: 0, - EnablePerStepStats: false, - } - - planOpts := logicalplan.PlanOptions{DisableDuplicateLabelCheck: false} - _, err = logicalplan.NewFromBytes(bodyBytes, &qOpts, planOpts) - require.NoError(t, err) - }) - } -} diff --git a/pkg/querier/tripperware/logical_plan_gen_test.go b/pkg/querier/tripperware/logical_plan_gen_test.go new file mode 100644 index 00000000000..f4d25d36d0d --- /dev/null +++ b/pkg/querier/tripperware/logical_plan_gen_test.go @@ -0,0 +1,153 @@ +package tripperware + +import ( + "context" + "github.com/stretchr/testify/require" + "strconv" + "testing" +) + +// TestInstantLogicalPlan ensures that the instant logical plan generation middleware +// correctly produces a logical plan and insert it into the Prometheus request body. + +func TestInstantLogicalPlan(t *testing.T) { + for i, tc := range []struct { + name string + input *PrometheusRequest + err error + }{ + { + name: "rate vector selector", + input: &PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + err: nil, + }, + { + name: "memory usage expression", + input: &PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + err: nil, + }, + { + name: "scalar only query", + input: &PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "42", + }, + err: nil, + }, + { + name: "vector arithmetic", + input: &PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", + }, + err: nil, + }, + { + name: "avg_over_time with nested rate", + input: &PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", + }, + err: nil, + }, + } { + tc := tc + t.Run(strconv.Itoa(i), func(t *testing.T) { + t.Parallel() + + lpm := LogicalPlanGenMiddleware() + handler := lpm.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { + return nil, nil + })) + + // Test: Execute middleware to populate the logical plan + _, err := handler.Do(context.Background(), tc.input) + require.NoError(t, err) + require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") + }) + } +} + +// TestRangeLogicalPlan validates the range logical plan generation middleware. +func TestRangeLogicalPlan(t *testing.T) { + testCases := []struct { + name string + input *PrometheusRequest + }{ + { + name: "rate vector over time", + input: &PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", + }, + }, + { + name: "memory usage ratio", + input: &PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 30000, + Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", + }, + }, + { + name: "avg_over_time function", + input: &PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 60000, + Query: "avg_over_time(http_requests_total[5m])", + }, + }, + { + name: "vector arithmetic with range", + input: &PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 10000, + Query: "rate(node_network_receive_bytes_total[1m]) / rate(node_network_transmit_bytes_total[1m])", + }, + }, + { + name: "simple scalar operation", + input: &PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "2 + 2", + }, + }, + } + + for i, tc := range testCases { + tc := tc + t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { + t.Parallel() + + middleware := LogicalPlanGenMiddleware() + + handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { + return nil, nil + })) + + // Test: Execute middleware to populate the logical plan + _, err := handler.Do(context.Background(), tc.input) + require.NoError(t, err) + require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") + + }) + } +} diff --git a/pkg/querier/tripperware/logicalplan_gen.go b/pkg/querier/tripperware/logicalplan_gen.go index e89776edeab..7edb98ec6a6 100644 --- a/pkg/querier/tripperware/logicalplan_gen.go +++ b/pkg/querier/tripperware/logicalplan_gen.go @@ -83,7 +83,7 @@ func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { return nil, err } - promReq.LogicalPlan = newLogicalPlan + promReq.LogicalPlan = *newLogicalPlan return l.next.Do(ctx, r) } diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index b5d857a42b2..f95ead12ff9 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -97,8 +97,8 @@ type Request interface { GetStep() int64 // GetQuery returns the query of the request. GetQuery() string - // GetLogicalPlan returns the serialized logical plan - GetLogicalPlan() *logicalplan.Plan + // GetLogicalPlan returns the logical plan + GetLogicalPlan() logicalplan.Plan // WithStartEnd clone the current request with different start and end timestamp. WithStartEnd(startTime int64, endTime int64) Request // WithQuery clone the current request with a different query. @@ -155,7 +155,7 @@ type PrometheusRequest struct { Headers http.Header Stats string CachingOptions CachingOptions - LogicalPlan *logicalplan.Plan + LogicalPlan logicalplan.Plan } func (m *PrometheusRequest) GetPath() string { @@ -221,7 +221,7 @@ func (m *PrometheusRequest) GetStats() string { return "" } -func (m *PrometheusRequest) GetLogicalPlan() *logicalplan.Plan { +func (m *PrometheusRequest) GetLogicalPlan() logicalplan.Plan { if m != nil { return m.LogicalPlan } diff --git a/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go b/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go deleted file mode 100644 index a5bd3f0f062..00000000000 --- a/pkg/querier/tripperware/queryrange/logical_plan_gen_range_test.go +++ /dev/null @@ -1,114 +0,0 @@ -package queryrange - -import ( - "context" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/stretchr/testify/require" - "github.com/thanos-io/promql-engine/logicalplan" - "github.com/thanos-io/promql-engine/query" - "io" - "net/http" - "strconv" - "testing" - "time" -) - -// TestRangeLogicalPlan validates the range logical plan generation middleware. -func TestRangeLogicalPlan(t *testing.T) { - testCases := []struct { - name string - input *tripperware.PrometheusRequest - }{ - { - name: "rate vector over time", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", - }, - }, - { - name: "memory usage ratio", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 30000, - Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", - }, - }, - { - name: "avg_over_time function", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 60000, - Query: "avg_over_time(http_requests_total[5m])", - }, - }, - { - name: "vector arithmetic with range", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 10000, - Query: "rate(node_network_receive_bytes_total[1m]) / rate(node_network_transmit_bytes_total[1m])", - }, - }, - { - name: "simple scalar operation", - input: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "2 + 2", - }, - }, - } - - for i, tc := range testCases { - tc := tc - t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { - t.Parallel() - - middleware := tripperware.LogicalPlanGenMiddleware() - - handler := middleware.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - - // Test Group 1: Execute middleware to populate the logical plan - _, err := handler.Do(context.Background(), tc.input) - require.NoError(t, err) - require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") - - // Test 2: Encode the request and validate method and body - httpReq, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.input) - require.NoError(t, err) - require.Equal(t, http.MethodPost, httpReq.Method) - - body, err := io.ReadAll(httpReq.Body) - require.NoError(t, err) - require.NotEmpty(t, body, "HTTP body should not be empty") - - // Test Group 3: Ensure the logical plan can be deserialized back - start := time.Unix(0, tc.input.Start*int64(time.Millisecond)) - end := time.Unix(0, tc.input.End*int64(time.Millisecond)) - step := time.Duration(tc.input.Step) * time.Millisecond - - qOpts := query.Options{ - Start: start, - End: end, - Step: step, - StepsBatch: 10, - LookbackDelta: 5 * time.Minute, - EnablePerStepStats: false, - } - planOpts := logicalplan.PlanOptions{ - DisableDuplicateLabelCheck: false, - } - _, err = logicalplan.NewFromBytes(body, &qOpts, planOpts) - require.NoError(t, err, "logical plan should be valid and de-serializable") - }) - } -} diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 52423237f0e..0cb4e284aaa 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -158,8 +158,8 @@ func (c prometheusCodec) getSerializedBody(promReq *tripperware.PrometheusReques var byteLP []byte var err error - if promReq.LogicalPlan != nil && *promReq.LogicalPlan != nil { - byteLP, err = logicalplan.Marshal((*promReq.LogicalPlan).Root()) + if promReq.LogicalPlan != nil { + byteLP, err = logicalplan.Marshal(promReq.LogicalPlan.Root()) if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 65165ce47dc..47ae8ce41e3 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -217,7 +217,7 @@ func TestRoundTripWithDistributedExec(t *testing.T) { require.NotEmpty(t, body) require.NoError(t, err) - byteLP, err := logicalplan.Marshal((*tc.pReq.LogicalPlan).Root()) + byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) require.NoError(t, err) require.Equal(t, byteLP, body) diff --git a/pkg/querier/tripperware/roundtrip_test.go b/pkg/querier/tripperware/roundtrip_test.go index c519d16c4c8..a73623a0b70 100644 --- a/pkg/querier/tripperware/roundtrip_test.go +++ b/pkg/querier/tripperware/roundtrip_test.go @@ -232,7 +232,7 @@ cortex_query_frontend_queries_total{op="query_range", source="api", user="1"} 1 `, }, { - path: query, + path: queryAll, expectedBody: instantResponseBody, limits: defaultOverrides, maxSubQuerySteps: 11000, @@ -256,7 +256,7 @@ cortex_query_frontend_queries_total{op="query", source="api", user="1"} 1 `, }, { - path: query, + path: queryAll, expectedBody: instantResponseBody, limits: shardingOverrides, maxSubQuerySteps: 11000, From 55386c3d1ee6901b8ef4e6f1a4d9ec04690adc94 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Sun, 20 Jul 2025 10:50:25 -0700 Subject: [PATCH 05/14] merge related tests and improve readability Signed-off-by: rubywtl --- .../instant_query_middlewares_test.go | 214 ++++++----------- .../tripperware/logical_plan_gen_test.go | 88 ++++--- pkg/querier/tripperware/logicalplan_gen.go | 5 +- pkg/querier/tripperware/query.go | 3 +- .../query_range_middlewares_test.go | 223 ++++++------------ 5 files changed, 189 insertions(+), 344 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 591c7e57d3f..ec638e14af8 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -2,7 +2,6 @@ package instantquery import ( "context" - "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/http/httptest" @@ -19,12 +18,12 @@ import ( "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/util/validation" + "github.com/thanos-io/promql-engine/logicalplan" ) const ( - distributedExecEnabled = false - queryAll = "/api/v1/instant_query?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536716898&stats=all" - responseBody = `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}]}}` + queryAll = "/api/v1/instant_query?end=1536716898&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536716898&stats=all" + responseBody = `{"status":"success","data":{"resultType":"matrix","result":[{"metric":{"foo":"bar"},"values":[[1536673680,"137"],[1536673780,"137"]]}]}}` ) func TestRoundTrip(t *testing.T) { @@ -61,7 +60,7 @@ func TestRoundTrip(t *testing.T) { nil, qa, 5*time.Minute, - distributedExecEnabled, + false, // distributedExecEnabled ) require.NoError(t, err) @@ -108,10 +107,8 @@ func TestRoundTrip(t *testing.T) { } } -// Integration test for query round-trip with distributed execution enabled (feature flag). -// Checks logical plan is generated, included in request body, and processed correctly. - -func TestRoundTripWithDistributedExec(t *testing.T) { +func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { + // Common test server setup s := httptest.NewServer( middleware.AuthenticateUser.Wrap( http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { @@ -138,162 +135,95 @@ func TestRoundTripWithDistributedExec(t *testing.T) { next: http.DefaultTransport, } - qa := querysharding.NewQueryAnalyzer() + testCases := []struct { + name string + distributedEnabled bool + pReq *tripperware.PrometheusRequest + expectEmptyBody bool + }{ + { + name: "With distributed execution", + distributedEnabled: true, + pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }, + expectEmptyBody: false, + }, + { + name: "Without distributed execution", + distributedEnabled: false, + pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 100000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }, + expectEmptyBody: true, + }, + } - instantQueryMiddlewares, err := Middlewares( - log.NewNopLogger(), - mockLimitsShard{shardSize: 2}, - nil, - qa, - 5*time.Minute, - true, - ) - require.NoError(t, err) + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + qa := querysharding.NewQueryAnalyzer() + instantQueryMiddlewares, err := Middlewares( + log.NewNopLogger(), + mockLimitsShard{shardSize: 2}, + nil, + qa, + 5*time.Minute, + tc.distributedEnabled, + ) + require.NoError(t, err) - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + nil, + instantQueryMiddlewares, + testInstantQueryCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - nil, - instantQueryMiddlewares, - testInstantQueryCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { ctx := user.InjectOrgID(context.Background(), "1") + // test middlewares for _, mw := range instantQueryMiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, pReq tripperware.Request) (tripperware.Response, error) { + handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { return nil, nil })) _, err := handler.Do(ctx, tc.pReq) require.NoError(t, err) } + // encode and prepare request req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) require.NoError(t, err) req = req.WithContext(ctx) err = user.InjectOrgIDIntoHTTPRequest(ctx, req) require.NoError(t, err) + // check request body body, err := io.ReadAll(req.Body) - require.NotEmpty(t, body) require.NoError(t, err) - - byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) - require.NoError(t, err) - require.Equal(t, byteLP, body) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) - require.NoError(t, err) - }) - } -} - -func TestRoundTripWithoutDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - instantQueryMiddlewares, err := Middlewares( - log.NewNopLogger(), - mockLimitsShard{shardSize: 2}, - nil, - qa, - 5*time.Minute, - false, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - nil, - instantQueryMiddlewares, - testInstantQueryCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 100000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range instantQueryMiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) + if tc.expectEmptyBody { + require.Empty(t, body) + } else { + require.NotEmpty(t, body) + byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) require.NoError(t, err) + require.Equal(t, byteLP, body) } - req, err := tripperware.Codec.EncodeRequest(testInstantQueryCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.Empty(t, body) - + // test round trip resp, err := tw(downstream).RoundTrip(req) require.NoError(t, err) require.Equal(t, 200, resp.StatusCode) diff --git a/pkg/querier/tripperware/logical_plan_gen_test.go b/pkg/querier/tripperware/logical_plan_gen_test.go index f4d25d36d0d..6a43dcbcb09 100644 --- a/pkg/querier/tripperware/logical_plan_gen_test.go +++ b/pkg/querier/tripperware/logical_plan_gen_test.go @@ -2,91 +2,70 @@ package tripperware import ( "context" - "github.com/stretchr/testify/require" "strconv" "testing" -) -// TestInstantLogicalPlan ensures that the instant logical plan generation middleware -// correctly produces a logical plan and insert it into the Prometheus request body. + "github.com/stretchr/testify/require" +) -func TestInstantLogicalPlan(t *testing.T) { - for i, tc := range []struct { - name string - input *PrometheusRequest - err error +func TestLogicalPlanGeneration(t *testing.T) { + testCases := []struct { + name string + queryType string // "instant" or "range" + input *PrometheusRequest + err error }{ + // instant query test cases { - name: "rate vector selector", + name: "instant - rate vector selector", + queryType: "instant", input: &PrometheusRequest{ Start: 100000, End: 100000, Query: "rate(node_cpu_seconds_total{mode!=\"idle\"}[5m])", }, - err: nil, }, { - name: "memory usage expression", + name: "instant - memory usage expression", + queryType: "instant", input: &PrometheusRequest{ Start: 100000, End: 100000, Query: "100 * (1 - (node_memory_MemAvailable_bytes / node_memory_MemTotal_bytes))", }, - err: nil, }, { - name: "scalar only query", + name: "instant - scalar only query", + queryType: "instant", input: &PrometheusRequest{ Start: 100000, End: 100000, Query: "42", }, - err: nil, }, { - name: "vector arithmetic", + name: "instant - vector arithmetic", + queryType: "instant", input: &PrometheusRequest{ Start: 100000, End: 100000, Query: "node_load1 / ignoring(cpu) node_cpu_seconds_total", }, - err: nil, }, { - name: "avg_over_time with nested rate", + name: "instant - avg_over_time with nested rate", + queryType: "instant", input: &PrometheusRequest{ Start: 100000, End: 100000, Query: "avg_over_time(rate(http_requests_total[5m])[30m:5m])", }, - err: nil, }, - } { - tc := tc - t.Run(strconv.Itoa(i), func(t *testing.T) { - t.Parallel() - - lpm := LogicalPlanGenMiddleware() - handler := lpm.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { - return nil, nil - })) - // Test: Execute middleware to populate the logical plan - _, err := handler.Do(context.Background(), tc.input) - require.NoError(t, err) - require.NotEmpty(t, tc.input.LogicalPlan, "prom request should not be empty") - }) - } -} - -// TestRangeLogicalPlan validates the range logical plan generation middleware. -func TestRangeLogicalPlan(t *testing.T) { - testCases := []struct { - name string - input *PrometheusRequest - }{ + // query range test cases { - name: "rate vector over time", + name: "range - rate vector over time", + queryType: "range", input: &PrometheusRequest{ Start: 100000, End: 200000, @@ -95,7 +74,8 @@ func TestRangeLogicalPlan(t *testing.T) { }, }, { - name: "memory usage ratio", + name: "range - memory usage ratio", + queryType: "range", input: &PrometheusRequest{ Start: 100000, End: 200000, @@ -104,7 +84,8 @@ func TestRangeLogicalPlan(t *testing.T) { }, }, { - name: "avg_over_time function", + name: "range - avg_over_time function", + queryType: "range", input: &PrometheusRequest{ Start: 100000, End: 200000, @@ -113,7 +94,8 @@ func TestRangeLogicalPlan(t *testing.T) { }, }, { - name: "vector arithmetic with range", + name: "range - vector arithmetic with range", + queryType: "range", input: &PrometheusRequest{ Start: 100000, End: 200000, @@ -122,7 +104,8 @@ func TestRangeLogicalPlan(t *testing.T) { }, }, { - name: "simple scalar operation", + name: "range - simple scalar operation", + queryType: "range", input: &PrometheusRequest{ Start: 100000, End: 200000, @@ -143,7 +126,16 @@ func TestRangeLogicalPlan(t *testing.T) { return nil, nil })) - // Test: Execute middleware to populate the logical plan + // additional validation on the test cases based on query type + if tc.queryType == "range" { + require.NotZero(t, tc.input.Step, "range query should have non-zero step") + require.NotEqual(t, tc.input.Start, tc.input.End, "range query should have different start and end times") + } else { + require.Equal(t, tc.input.Start, tc.input.End, "instant query should have equal start and end times") + require.Zero(t, tc.input.Step, "instant query should have zero step") + } + + // test: execute middleware to populate the logical plan _, err := handler.Do(context.Background(), tc.input) require.NoError(t, err) require.NotEmpty(t, tc.input.LogicalPlan, "logical plan should be populated") diff --git a/pkg/querier/tripperware/logicalplan_gen.go b/pkg/querier/tripperware/logicalplan_gen.go index 7edb98ec6a6..687a6653df5 100644 --- a/pkg/querier/tripperware/logicalplan_gen.go +++ b/pkg/querier/tripperware/logicalplan_gen.go @@ -2,12 +2,13 @@ package tripperware import ( "context" + "net/http" + "time" + "github.com/prometheus/prometheus/promql/parser" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/promql-engine/query" "github.com/weaveworks/common/httpgrpc" - "net/http" - "time" ) const ( diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index f95ead12ff9..c2009171865 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -6,7 +6,6 @@ import ( "context" "encoding/binary" "fmt" - "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "strconv" @@ -30,6 +29,8 @@ import ( "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/runutil" + + "github.com/thanos-io/promql-engine/logicalplan" ) var ( diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 47ae8ce41e3..99a3d7cb7ef 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -26,10 +26,6 @@ var ( ShardedPrometheusCodec = NewPrometheusCodec(false, "", "protobuf") ) -const ( - distributedExecEnabled = false -) - func TestRoundTrip(t *testing.T) { s := httptest.NewServer( middleware.AuthenticateUser.Wrap( @@ -71,7 +67,7 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - false, + false, // distributedExecEnabled ) require.NoError(t, err) @@ -122,9 +118,7 @@ func TestRoundTrip(t *testing.T) { } } -// Integration test for query round-trip with distributed execution enabled (feature flag). -// Checks logical plan is generated, included in request body, and processed correctly. -func TestRoundTripWithDistributedExec(t *testing.T) { +func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { s := httptest.NewServer( middleware.AuthenticateUser.Wrap( http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { @@ -155,50 +149,72 @@ func TestRoundTripWithDistributedExec(t *testing.T) { next: http.DefaultTransport, } - qa := querysharding.NewQueryAnalyzer() - queyrangemiddlewares, _, err := Middlewares(Config{}, - log.NewNopLogger(), - mockLimits{}, - nil, - nil, - qa, - PrometheusCodec, - ShardedPrometheusCodec, - 5*time.Minute, - true, - ) - require.NoError(t, err) + testCases := []struct { + name string + distributedEnabled bool + pReq *tripperware.PrometheusRequest + expectEmptyBody bool + }{ + { + name: "With distributed execution", + distributedEnabled: true, + pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }, + expectEmptyBody: false, + }, + { + name: "Without distributed execution", + distributedEnabled: false, + pReq: &tripperware.PrometheusRequest{ + Start: 100000, + End: 200000, + Step: 15000, + Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", + }, + expectEmptyBody: true, + }, + } - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + qa := querysharding.NewQueryAnalyzer() + queyrangemiddlewares, _, err := Middlewares(Config{}, + log.NewNopLogger(), + mockLimits{}, + nil, + nil, + qa, + PrometheusCodec, + ShardedPrometheusCodec, + 5*time.Minute, + tc.distributedEnabled, + ) + require.NoError(t, err) - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - queyrangemiddlewares, - nil, - PrometheusCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) + defaultLimits := validation.NewOverrides(validation.Limits{}, nil) + + tw := tripperware.NewQueryTripperware(log.NewNopLogger(), + nil, + nil, + queyrangemiddlewares, + nil, + PrometheusCodec, + nil, + defaultLimits, + qa, + time.Minute, + 0, + 0, + false, + ) - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { ctx := user.InjectOrgID(context.Background(), "1") + // test middlewares for _, mw := range queyrangemiddlewares { handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { return nil, nil @@ -207,121 +223,26 @@ func TestRoundTripWithDistributedExec(t *testing.T) { require.NoError(t, err) } + // encode and prepare request req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) require.NoError(t, err) req = req.WithContext(ctx) err = user.InjectOrgIDIntoHTTPRequest(ctx, req) require.NoError(t, err) + // check request body body, err := io.ReadAll(req.Body) - require.NotEmpty(t, body) - require.NoError(t, err) - - byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) - require.NoError(t, err) - require.Equal(t, byteLP, body) - - resp, err := tw(downstream).RoundTrip(req) - require.NoError(t, err) - require.Equal(t, 200, resp.StatusCode) - - _, err = io.ReadAll(resp.Body) require.NoError(t, err) - }) - } -} - -func TestRoundTripWithoutDistributedExec(t *testing.T) { - s := httptest.NewServer( - middleware.AuthenticateUser.Wrap( - http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - var err error - switch r.RequestURI { - case queryAll: - _, err = w.Write([]byte(responseBody)) - case queryWithWarnings: - _, err = w.Write([]byte(responseBodyWithWarnings)) - case queryWithInfos: - _, err = w.Write([]byte(responseBodyWithInfos)) - default: - _, err = w.Write([]byte("bar")) - } - if err != nil { - t.Fatal(err) - } - }), - ), - ) - defer s.Close() - - u, err := url.Parse(s.URL) - require.NoError(t, err) - - downstream := singleHostRoundTripper{ - host: u.Host, - next: http.DefaultTransport, - } - - qa := querysharding.NewQueryAnalyzer() - queyrangemiddlewares, _, err := Middlewares(Config{}, - log.NewNopLogger(), - mockLimits{}, - nil, - nil, - qa, - PrometheusCodec, - ShardedPrometheusCodec, - 5*time.Minute, - false, - ) - require.NoError(t, err) - - defaultLimits := validation.NewOverrides(validation.Limits{}, nil) - - tw := tripperware.NewQueryTripperware(log.NewNopLogger(), - nil, - nil, - queyrangemiddlewares, - nil, - PrometheusCodec, - nil, - defaultLimits, - qa, - time.Minute, - 0, - 0, - false, - ) - - for i, tc := range []struct { - pReq *tripperware.PrometheusRequest - }{ - {pReq: &tripperware.PrometheusRequest{ - Start: 100000, - End: 200000, - Step: 15000, - Query: "node_cpu_seconds_total{mode!=\"idle\"}[5m]", - }}, - } { - t.Run(strconv.Itoa(i), func(t *testing.T) { - ctx := user.InjectOrgID(context.Background(), "1") - - for _, mw := range queyrangemiddlewares { - handler := mw.Wrap(tripperware.HandlerFunc(func(_ context.Context, req tripperware.Request) (tripperware.Response, error) { - return nil, nil - })) - _, err := handler.Do(ctx, tc.pReq) + if tc.expectEmptyBody { + require.Empty(t, body) + } else { + require.NotEmpty(t, body) + byteLP, err := logicalplan.Marshal(tc.pReq.LogicalPlan.Root()) require.NoError(t, err) + require.Equal(t, byteLP, body) } - req, err := tripperware.Codec.EncodeRequest(PrometheusCodec, context.Background(), tc.pReq) - require.NoError(t, err) - req = req.WithContext(ctx) - err = user.InjectOrgIDIntoHTTPRequest(ctx, req) - require.NoError(t, err) - body, err := io.ReadAll(req.Body) - require.Empty(t, body) - + // test round trip resp, err := tw(downstream).RoundTrip(req) require.NoError(t, err) require.Equal(t, 200, resp.StatusCode) From 1f2bf91515f6b2f023d16b25d2e39be203782668 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 12:18:03 -0700 Subject: [PATCH 06/14] edit feature flag name and visibility Signed-off-by: rubywtl --- docs/configuration/config-file-reference.md | 5 ----- pkg/frontend/config.go | 4 ++-- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 27d80558597..2963a87348c 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4451,11 +4451,6 @@ grpc_client_config: # URL of downstream Prometheus. # CLI flag: -frontend.downstream-url [downstream_url: | default = ""] - -# Experimental: Enables distributed execution of queries by passing logical -# query plan fragments to downstream components. -# CLI flag: -frontend.distributed_exec_enabled -[distributed_exec_enabled: | default = false] ``` ### `query_range_config` diff --git a/pkg/frontend/config.go b/pkg/frontend/config.go index 633f8f53e74..a1109f213ad 100644 --- a/pkg/frontend/config.go +++ b/pkg/frontend/config.go @@ -21,7 +21,7 @@ type CombinedFrontendConfig struct { FrontendV2 v2.Config `yaml:",inline"` DownstreamURL string `yaml:"downstream_url"` - DistributedExecEnabled bool `yaml:"distributed_exec_enabled"` + DistributedExecEnabled bool `yaml:"distributed_exec_enabled" doc:"hidden"` } func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { @@ -30,7 +30,7 @@ func (cfg *CombinedFrontendConfig) RegisterFlags(f *flag.FlagSet) { cfg.FrontendV2.RegisterFlags(f) f.StringVar(&cfg.DownstreamURL, "frontend.downstream-url", "", "URL of downstream Prometheus.") - f.BoolVar(&cfg.DistributedExecEnabled, "frontend.distributed_exec_enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan fragments to downstream components.") + f.BoolVar(&cfg.DistributedExecEnabled, "frontend.distributed-exec-enabled", false, "Experimental: Enables distributed execution of queries by passing logical query plan fragments to downstream components.") } // InitFrontend initializes frontend (either V1 -- without scheduler, or V2 -- with scheduler) or no frontend at From 5770302b781fab679c3d5550906a55a998a6801a Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 13:09:27 -0700 Subject: [PATCH 07/14] Add lookbackDelta and EnableStepStats back to query frontend config Signed-off-by: rubywtl --- pkg/cortex/modules.go | 3 ++- .../instantquery/instant_query_middlewares.go | 3 ++- .../instant_query_middlewares_test.go | 2 ++ .../tripperware/logical_plan_gen_test.go | 3 ++- pkg/querier/tripperware/logicalplan_gen.go | 18 +++++++++++------- .../queryrange/query_range_middlewares.go | 3 ++- .../queryrange/query_range_middlewares_test.go | 2 ++ 7 files changed, 23 insertions(+), 11 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 6e9402da66f..552cc5dfc57 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -534,12 +534,13 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled, + t.Cfg.Querier.EnablePerStepStats, ) if err != nil { return nil, err } - instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled) + instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled, t.Cfg.Querier.EnablePerStepStats) if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 2bdffe1be23..55256b453ad 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -16,6 +16,7 @@ func Middlewares( queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, distributedExecEnabled bool, + enablePerStepStats bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ NewLimitsMiddleware(limits, lookbackDelta), @@ -24,7 +25,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.LogicalPlanGenMiddleware()) + tripperware.LogicalPlanGenMiddleware(enablePerStepStats, lookbackDelta)) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index ec638e14af8..72009f1de3e 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -61,6 +61,7 @@ func TestRoundTrip(t *testing.T) { qa, 5*time.Minute, false, // distributedExecEnabled + false, // enablePerStepStats ) require.NoError(t, err) @@ -173,6 +174,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { qa, 5*time.Minute, tc.distributedEnabled, + false, // enablePerStepStats ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/logical_plan_gen_test.go b/pkg/querier/tripperware/logical_plan_gen_test.go index 6a43dcbcb09..ef00ba5d3fd 100644 --- a/pkg/querier/tripperware/logical_plan_gen_test.go +++ b/pkg/querier/tripperware/logical_plan_gen_test.go @@ -4,6 +4,7 @@ import ( "context" "strconv" "testing" + "time" "github.com/stretchr/testify/require" ) @@ -120,7 +121,7 @@ func TestLogicalPlanGeneration(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := LogicalPlanGenMiddleware() + middleware := LogicalPlanGenMiddleware(false, 5*time.Minute) handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { return nil, nil diff --git a/pkg/querier/tripperware/logicalplan_gen.go b/pkg/querier/tripperware/logicalplan_gen.go index 687a6653df5..c7e7e6136cf 100644 --- a/pkg/querier/tripperware/logicalplan_gen.go +++ b/pkg/querier/tripperware/logicalplan_gen.go @@ -15,16 +15,20 @@ const ( stepBatch = 10 ) -func LogicalPlanGenMiddleware() Middleware { +func LogicalPlanGenMiddleware(enablePerStepStats bool, lookBackDelta time.Duration) Middleware { return MiddlewareFunc(func(next Handler) Handler { return logicalPlanGen{ - next: next, + next: next, + enablePerStepStats: enablePerStepStats, + lookBackDelta: lookBackDelta, } }) } type logicalPlanGen struct { - next Handler + next Handler + enablePerStepStats bool + lookBackDelta time.Duration } func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { @@ -36,8 +40,8 @@ func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time End: start, Step: 0, StepsBatch: stepBatch, - LookbackDelta: 0, - EnablePerStepStats: false, + LookbackDelta: l.lookBackDelta, + EnablePerStepStats: l.enablePerStepStats, } } else { qOpts = query.Options{ @@ -45,8 +49,8 @@ func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time End: end, Step: step, StepsBatch: stepBatch, - LookbackDelta: 0, - EnablePerStepStats: false, + LookbackDelta: l.lookBackDelta, + EnablePerStepStats: l.enablePerStepStats, } } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 3aca642fcd4..ef6ca813eff 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -103,6 +103,7 @@ func Middlewares( shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, distributedExecEnabled bool, + enablePerStepStats bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -141,7 +142,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.LogicalPlanGenMiddleware()) + tripperware.LogicalPlanGenMiddleware(enablePerStepStats, lookbackDelta)) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 99a3d7cb7ef..6619a3c118b 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -68,6 +68,7 @@ func TestRoundTrip(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, false, // distributedExecEnabled + false, // enablePerStepStats ) require.NoError(t, err) @@ -192,6 +193,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, tc.distributedEnabled, + false, // enablePerStepStats ) require.NoError(t, err) From 9ab3f816bb3d28f20824283d0273e67e6a2cd8a2 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 14:03:02 -0700 Subject: [PATCH 08/14] style: code cleanup and readability improvements Signed-off-by: rubywtl --- ...ogicalplan_gen.go => distributed_query.go} | 49 +++++++++---------- ..._gen_test.go => distributed_query_test.go} | 2 +- .../instantquery/instant_query_middlewares.go | 2 +- pkg/querier/tripperware/query.go | 6 +-- .../queryrange/query_range_middlewares.go | 2 +- 5 files changed, 29 insertions(+), 32 deletions(-) rename pkg/querier/tripperware/{logicalplan_gen.go => distributed_query.go} (59%) rename pkg/querier/tripperware/{logical_plan_gen_test.go => distributed_query_test.go} (98%) diff --git a/pkg/querier/tripperware/logicalplan_gen.go b/pkg/querier/tripperware/distributed_query.go similarity index 59% rename from pkg/querier/tripperware/logicalplan_gen.go rename to pkg/querier/tripperware/distributed_query.go index c7e7e6136cf..14f61aa2f6d 100644 --- a/pkg/querier/tripperware/logicalplan_gen.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -15,9 +15,9 @@ const ( stepBatch = 10 ) -func LogicalPlanGenMiddleware(enablePerStepStats bool, lookBackDelta time.Duration) Middleware { +func DistributedQueryMiddleware(enablePerStepStats bool, lookBackDelta time.Duration) Middleware { return MiddlewareFunc(func(next Handler) Handler { - return logicalPlanGen{ + return distributedQueryMiddleware{ next: next, enablePerStepStats: enablePerStepStats, lookBackDelta: lookBackDelta, @@ -25,33 +25,30 @@ func LogicalPlanGenMiddleware(enablePerStepStats bool, lookBackDelta time.Durati }) } -type logicalPlanGen struct { +func getStartAndEnd(start time.Time, end time.Time, step time.Duration) (time.Time, time.Time) { + if step == 0 { + return start, start + } + return start, end +} + +type distributedQueryMiddleware struct { next Handler enablePerStepStats bool lookBackDelta time.Duration } -func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { +func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { - qOpts := query.Options{} - if step == 0 { - qOpts = query.Options{ - Start: start, - End: start, - Step: 0, - StepsBatch: stepBatch, - LookbackDelta: l.lookBackDelta, - EnablePerStepStats: l.enablePerStepStats, - } - } else { - qOpts = query.Options{ - Start: start, - End: end, - Step: step, - StepsBatch: stepBatch, - LookbackDelta: l.lookBackDelta, - EnablePerStepStats: l.enablePerStepStats, - } + start, end = getStartAndEnd(start, end, step) + + qOpts := query.Options{ + Start: start, + End: end, + Step: step, + StepsBatch: stepBatch, + LookbackDelta: d.lookBackDelta, + EnablePerStepStats: d.enablePerStepStats, } expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() @@ -71,7 +68,7 @@ func (l logicalPlanGen) NewLogicalPlan(qs string, start time.Time, end time.Time return &optimizedPlan, nil } -func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { +func (d distributedQueryMiddleware) Do(ctx context.Context, r Request) (Response, error) { promReq, ok := r.(*PrometheusRequest) if !ok { return nil, httpgrpc.Errorf(http.StatusBadRequest, "invalid request format") @@ -83,12 +80,12 @@ func (l logicalPlanGen) Do(ctx context.Context, r Request) (Response, error) { var err error - newLogicalPlan, err := l.NewLogicalPlan(promReq.Query, startTime, endTime, step) + newLogicalPlan, err := d.newLogicalPlan(promReq.Query, startTime, endTime, step) if err != nil { return nil, err } promReq.LogicalPlan = *newLogicalPlan - return l.next.Do(ctx, r) + return d.next.Do(ctx, r) } diff --git a/pkg/querier/tripperware/logical_plan_gen_test.go b/pkg/querier/tripperware/distributed_query_test.go similarity index 98% rename from pkg/querier/tripperware/logical_plan_gen_test.go rename to pkg/querier/tripperware/distributed_query_test.go index ef00ba5d3fd..68f80fd823b 100644 --- a/pkg/querier/tripperware/logical_plan_gen_test.go +++ b/pkg/querier/tripperware/distributed_query_test.go @@ -121,7 +121,7 @@ func TestLogicalPlanGeneration(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := LogicalPlanGenMiddleware(false, 5*time.Minute) + middleware := DistributedQueryMiddleware(false, 5*time.Minute) handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { return nil, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 55256b453ad..fb9b3bd40ae 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -25,7 +25,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.LogicalPlanGenMiddleware(enablePerStepStats, lookbackDelta)) + tripperware.DistributedQueryMiddleware(enablePerStepStats, lookbackDelta)) } return m, nil diff --git a/pkg/querier/tripperware/query.go b/pkg/querier/tripperware/query.go index c2009171865..42e2d9eebf0 100644 --- a/pkg/querier/tripperware/query.go +++ b/pkg/querier/tripperware/query.go @@ -223,10 +223,10 @@ func (m *PrometheusRequest) GetStats() string { } func (m *PrometheusRequest) GetLogicalPlan() logicalplan.Plan { - if m != nil { - return m.LogicalPlan + if m == nil { + return nil } - return nil + return m.LogicalPlan } // WithStartEnd clones the current `PrometheusRequest` with a new `start` and `end` timestamp. diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index ef6ca813eff..e337802ba60 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -142,7 +142,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.LogicalPlanGenMiddleware(enablePerStepStats, lookbackDelta)) + tripperware.DistributedQueryMiddleware(enablePerStepStats, lookbackDelta)) } return queryRangeMiddleware, c, nil From 03c17ef6c50b8738eb4325d9373d4ccfab7f5ebb Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 15:08:46 -0700 Subject: [PATCH 09/14] fix import files lint error Signed-off-by: rubywtl --- pkg/querier/tripperware/distributed_query.go | 2 -- .../tripperware/instantquery/instant_query_middlewares.go | 3 +-- .../instantquery/instant_query_middlewares_test.go | 7 +++---- pkg/querier/tripperware/queryrange/query_range.go | 2 +- .../tripperware/queryrange/query_range_middlewares_test.go | 7 +++---- 5 files changed, 8 insertions(+), 13 deletions(-) diff --git a/pkg/querier/tripperware/distributed_query.go b/pkg/querier/tripperware/distributed_query.go index 14f61aa2f6d..f481ecc20a7 100644 --- a/pkg/querier/tripperware/distributed_query.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -63,8 +63,6 @@ func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, e logicalPlan := logicalplan.NewFromAST(expr, &qOpts, planOpts) optimizedPlan, _ := logicalPlan.Optimize(logicalplan.DefaultOptimizers) - //TODO: Add distributed query optimizer - return &optimizedPlan, nil } diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index fb9b3bd40ae..72c599ebe8f 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -3,10 +3,9 @@ package instantquery import ( "time" + "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/go-kit/log" "github.com/thanos-io/thanos/pkg/querysharding" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" ) func Middlewares( diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 72009f1de3e..f9b9d364a17 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -10,15 +10,14 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" "github.com/go-kit/log" "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/user" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/util/validation" - "github.com/thanos-io/promql-engine/logicalplan" ) const ( diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index 0cb4e284aaa..a660dbcc430 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -3,7 +3,6 @@ package queryrange import ( "bytes" "context" - "github.com/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/url" @@ -24,6 +23,7 @@ import ( "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/spanlogger" + "github.com/thanos-io/promql-engine/logicalplan" ) // StatusSuccess Prometheus success result. diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 6619a3c118b..d3ce0ff4fd3 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/thanos-io/promql-engine/logicalplan" "io" "net/http" "net/http/httptest" @@ -11,14 +10,14 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" "github.com/go-kit/log" "github.com/stretchr/testify/require" + "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/user" - - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/util/validation" ) var ( From d8a1239ab3d9226ea2d1cb9f7c69204db6b1c50f Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 15:26:38 -0700 Subject: [PATCH 10/14] lint: fix import orders Signed-off-by: rubywtl --- .../tripperware/instantquery/instant_query_middlewares.go | 3 ++- .../instantquery/instant_query_middlewares_test.go | 5 +++-- pkg/querier/tripperware/queryrange/query_range.go | 3 +-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 72c599ebe8f..fb9b3bd40ae 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -3,9 +3,10 @@ package instantquery import ( "time" - "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/go-kit/log" "github.com/thanos-io/thanos/pkg/querysharding" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" ) func Middlewares( diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index f9b9d364a17..8d5ee94c861 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -10,14 +10,15 @@ import ( "testing" "time" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/util/validation" "github.com/go-kit/log" "github.com/stretchr/testify/require" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/user" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" ) const ( diff --git a/pkg/querier/tripperware/queryrange/query_range.go b/pkg/querier/tripperware/queryrange/query_range.go index a660dbcc430..df721146f66 100644 --- a/pkg/querier/tripperware/queryrange/query_range.go +++ b/pkg/querier/tripperware/queryrange/query_range.go @@ -14,16 +14,15 @@ import ( "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" "github.com/prometheus/common/model" + "github.com/thanos-io/promql-engine/logicalplan" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/api/queryapi" "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/util" - "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/spanlogger" - "github.com/thanos-io/promql-engine/logicalplan" ) // StatusSuccess Prometheus success result. From bab8813ea1cebb71cfec06fdb87156c357b9c252 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Tue, 22 Jul 2025 16:31:28 -0700 Subject: [PATCH 11/14] lint: fix import order Signed-off-by: rubywtl --- .../tripperware/queryrange/query_range_middlewares_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index d3ce0ff4fd3..c46911712ca 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -10,14 +10,15 @@ import ( "testing" "time" - "github.com/cortexproject/cortex/pkg/querier/tripperware" - "github.com/cortexproject/cortex/pkg/util/validation" "github.com/go-kit/log" "github.com/stretchr/testify/require" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/middleware" "github.com/weaveworks/common/user" + + "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/util/validation" ) var ( From 970ab81d251a4d4809dded707b6a9d787715dbb0 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Wed, 23 Jul 2025 10:29:07 -0700 Subject: [PATCH 12/14] remove enableStepStats from query frontend Signed-off-by: rubywtl --- pkg/cortex/modules.go | 3 +-- pkg/querier/tripperware/distributed_query.go | 14 ++++++-------- pkg/querier/tripperware/distributed_query_test.go | 2 +- .../instantquery/instant_query_middlewares.go | 3 +-- .../instantquery/instant_query_middlewares_test.go | 2 -- .../queryrange/query_range_middlewares.go | 3 +-- .../queryrange/query_range_middlewares_test.go | 2 -- 7 files changed, 10 insertions(+), 19 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 552cc5dfc57..6e9402da66f 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -534,13 +534,12 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled, - t.Cfg.Querier.EnablePerStepStats, ) if err != nil { return nil, err } - instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled, t.Cfg.Querier.EnablePerStepStats) + instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled) if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/distributed_query.go b/pkg/querier/tripperware/distributed_query.go index f481ecc20a7..45a0f69bf5b 100644 --- a/pkg/querier/tripperware/distributed_query.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -15,12 +15,11 @@ const ( stepBatch = 10 ) -func DistributedQueryMiddleware(enablePerStepStats bool, lookBackDelta time.Duration) Middleware { +func DistributedQueryMiddleware(lookBackDelta time.Duration) Middleware { return MiddlewareFunc(func(next Handler) Handler { return distributedQueryMiddleware{ - next: next, - enablePerStepStats: enablePerStepStats, - lookBackDelta: lookBackDelta, + next: next, + lookBackDelta: lookBackDelta, } }) } @@ -33,9 +32,8 @@ func getStartAndEnd(start time.Time, end time.Time, step time.Duration) (time.Ti } type distributedQueryMiddleware struct { - next Handler - enablePerStepStats bool - lookBackDelta time.Duration + next Handler + lookBackDelta time.Duration } func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { @@ -48,7 +46,7 @@ func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, e Step: step, StepsBatch: stepBatch, LookbackDelta: d.lookBackDelta, - EnablePerStepStats: d.enablePerStepStats, + EnablePerStepStats: false, // Hardcoded value that will be re-populated again in the querier stage } expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() diff --git a/pkg/querier/tripperware/distributed_query_test.go b/pkg/querier/tripperware/distributed_query_test.go index 68f80fd823b..a73edc1a341 100644 --- a/pkg/querier/tripperware/distributed_query_test.go +++ b/pkg/querier/tripperware/distributed_query_test.go @@ -121,7 +121,7 @@ func TestLogicalPlanGeneration(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := DistributedQueryMiddleware(false, 5*time.Minute) + middleware := DistributedQueryMiddleware(5 * time.Minute) handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { return nil, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index fb9b3bd40ae..56ecba81a05 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -16,7 +16,6 @@ func Middlewares( queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, distributedExecEnabled bool, - enablePerStepStats bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ NewLimitsMiddleware(limits, lookbackDelta), @@ -25,7 +24,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.DistributedQueryMiddleware(enablePerStepStats, lookbackDelta)) + tripperware.DistributedQueryMiddleware(lookbackDelta)) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 8d5ee94c861..37f697de784 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -61,7 +61,6 @@ func TestRoundTrip(t *testing.T) { qa, 5*time.Minute, false, // distributedExecEnabled - false, // enablePerStepStats ) require.NoError(t, err) @@ -174,7 +173,6 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { qa, 5*time.Minute, tc.distributedEnabled, - false, // enablePerStepStats ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index e337802ba60..d8041a0b1ee 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -103,7 +103,6 @@ func Middlewares( shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, distributedExecEnabled bool, - enablePerStepStats bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -142,7 +141,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.DistributedQueryMiddleware(enablePerStepStats, lookbackDelta)) + tripperware.DistributedQueryMiddleware(lookbackDelta)) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index c46911712ca..684108f746c 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -68,7 +68,6 @@ func TestRoundTrip(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, false, // distributedExecEnabled - false, // enablePerStepStats ) require.NoError(t, err) @@ -193,7 +192,6 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, tc.distributedEnabled, - false, // enablePerStepStats ) require.NoError(t, err) From 6662c5efe87a166f1766c81dd5bcf6c6c0a99809 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Wed, 23 Jul 2025 11:32:39 -0700 Subject: [PATCH 13/14] add logical plan optimization-related config param to middleware Signed-off-by: rubywtl --- pkg/cortex/modules.go | 15 +++++++++++- pkg/querier/tripperware/distributed_query.go | 24 ++++++++++--------- .../tripperware/distributed_query_test.go | 4 +++- .../instantquery/instant_query_middlewares.go | 3 ++- .../instant_query_middlewares_test.go | 8 ++++++- .../queryrange/query_range_middlewares.go | 3 ++- .../query_range_middlewares_test.go | 8 ++++++- 7 files changed, 48 insertions(+), 17 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 6e9402da66f..ce2aad7ffbc 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -8,6 +8,7 @@ import ( "net/http" "runtime" "runtime/debug" + "time" "github.com/go-kit/log/level" "github.com/opentracing-contrib/go-stdlib/nethttp" @@ -533,13 +534,25 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, + func(time.Duration) time.Duration { + return t.Cfg.Querier.DefaultEvaluationInterval + }, t.Cfg.Frontend.DistributedExecEnabled, ) if err != nil { return nil, err } - instantQueryMiddlewares, err := instantquery.Middlewares(util_log.Logger, t.Overrides, instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, t.Cfg.Frontend.DistributedExecEnabled) + instantQueryMiddlewares, err := instantquery.Middlewares( + util_log.Logger, + t.Overrides, + instantQueryCodec, + queryAnalyzer, + t.Cfg.Querier.LookbackDelta, + func(time.Duration) time.Duration { + return t.Cfg.Querier.DefaultEvaluationInterval + }, + t.Cfg.Frontend.DistributedExecEnabled) if err != nil { return nil, err } diff --git a/pkg/querier/tripperware/distributed_query.go b/pkg/querier/tripperware/distributed_query.go index 45a0f69bf5b..a2789beb3a8 100644 --- a/pkg/querier/tripperware/distributed_query.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -15,11 +15,11 @@ const ( stepBatch = 10 ) -func DistributedQueryMiddleware(lookBackDelta time.Duration) Middleware { +func DistributedQueryMiddleware(noStepSubqueryIntervalFn func(time.Duration) time.Duration) Middleware { return MiddlewareFunc(func(next Handler) Handler { return distributedQueryMiddleware{ - next: next, - lookBackDelta: lookBackDelta, + next: next, + noStepSubqueryIntervalFn: noStepSubqueryIntervalFn, } }) } @@ -32,8 +32,8 @@ func getStartAndEnd(start time.Time, end time.Time, step time.Duration) (time.Ti } type distributedQueryMiddleware struct { - next Handler - lookBackDelta time.Duration + next Handler + noStepSubqueryIntervalFn func(time.Duration) time.Duration } func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { @@ -41,12 +41,14 @@ func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, e start, end = getStartAndEnd(start, end, step) qOpts := query.Options{ - Start: start, - End: end, - Step: step, - StepsBatch: stepBatch, - LookbackDelta: d.lookBackDelta, - EnablePerStepStats: false, // Hardcoded value that will be re-populated again in the querier stage + Start: start, + End: end, + Step: step, + StepsBatch: stepBatch, + NoStepSubqueryIntervalFn: d.noStepSubqueryIntervalFn, + // Hardcoded value for execution-time-params that will be re-populated again in the querier stage + LookbackDelta: 0, + EnablePerStepStats: false, } expr, err := parser.NewParser(qs, parser.WithFunctions(parser.Functions)).ParseExpr() diff --git a/pkg/querier/tripperware/distributed_query_test.go b/pkg/querier/tripperware/distributed_query_test.go index a73edc1a341..68b61a1fe9d 100644 --- a/pkg/querier/tripperware/distributed_query_test.go +++ b/pkg/querier/tripperware/distributed_query_test.go @@ -121,7 +121,9 @@ func TestLogicalPlanGeneration(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := DistributedQueryMiddleware(5 * time.Minute) + middleware := DistributedQueryMiddleware(func(time.Duration) time.Duration { + return time.Minute + }) handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { return nil, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 56ecba81a05..8a54d3b0380 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -15,6 +15,7 @@ func Middlewares( merger tripperware.Merger, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, + noStepSubqueryIntervalFn func(time.Duration) time.Duration, distributedExecEnabled bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ @@ -24,7 +25,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.DistributedQueryMiddleware(lookbackDelta)) + tripperware.DistributedQueryMiddleware(noStepSubqueryIntervalFn)) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 37f697de784..318470eed92 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -60,7 +60,10 @@ func TestRoundTrip(t *testing.T) { nil, qa, 5*time.Minute, - false, // distributedExecEnabled + func(time.Duration) time.Duration { + return time.Minute + }, + false, ) require.NoError(t, err) @@ -172,6 +175,9 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { nil, qa, 5*time.Minute, + func(time.Duration) time.Duration { + return 5 * time.Minute + }, tc.distributedEnabled, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index d8041a0b1ee..6fc8c40e3af 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -102,6 +102,7 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, + noStepSubqueryIntervalFn func(time.Duration) time.Duration, distributedExecEnabled bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. @@ -141,7 +142,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.DistributedQueryMiddleware(lookbackDelta)) + tripperware.DistributedQueryMiddleware(noStepSubqueryIntervalFn)) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 684108f746c..cb05c6526e7 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -67,7 +67,10 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - false, // distributedExecEnabled + func(time.Duration) time.Duration { + return time.Minute + }, + false, ) require.NoError(t, err) @@ -191,6 +194,9 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, + func(time.Duration) time.Duration { + return time.Minute + }, tc.distributedEnabled, ) require.NoError(t, err) From 8fdebab22ebe8eb3e273608c4f20333921866b48 Mon Sep 17 00:00:00 2001 From: rubywtl Date: Wed, 23 Jul 2025 12:42:20 -0700 Subject: [PATCH 14/14] change params passed-in in logical plan middleware Signed-off-by: rubywtl --- pkg/cortex/modules.go | 9 ++----- pkg/querier/tripperware/distributed_query.go | 26 +++++++++++-------- .../tripperware/distributed_query_test.go | 4 +-- .../instantquery/instant_query_middlewares.go | 4 +-- .../instant_query_middlewares_test.go | 8 ++---- .../queryrange/query_range_middlewares.go | 4 +-- .../query_range_middlewares_test.go | 8 ++---- 7 files changed, 26 insertions(+), 37 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index ce2aad7ffbc..a13f35e6a9d 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -8,7 +8,6 @@ import ( "net/http" "runtime" "runtime/debug" - "time" "github.com/go-kit/log/level" "github.com/opentracing-contrib/go-stdlib/nethttp" @@ -534,9 +533,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, - func(time.Duration) time.Duration { - return t.Cfg.Querier.DefaultEvaluationInterval - }, + t.Cfg.Querier.DefaultEvaluationInterval, t.Cfg.Frontend.DistributedExecEnabled, ) if err != nil { @@ -549,9 +546,7 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro instantQueryCodec, queryAnalyzer, t.Cfg.Querier.LookbackDelta, - func(time.Duration) time.Duration { - return t.Cfg.Querier.DefaultEvaluationInterval - }, + t.Cfg.Querier.DefaultEvaluationInterval, t.Cfg.Frontend.DistributedExecEnabled) if err != nil { return nil, err diff --git a/pkg/querier/tripperware/distributed_query.go b/pkg/querier/tripperware/distributed_query.go index a2789beb3a8..02a0692153d 100644 --- a/pkg/querier/tripperware/distributed_query.go +++ b/pkg/querier/tripperware/distributed_query.go @@ -15,11 +15,12 @@ const ( stepBatch = 10 ) -func DistributedQueryMiddleware(noStepSubqueryIntervalFn func(time.Duration) time.Duration) Middleware { +func DistributedQueryMiddleware(defaultEvaluationInterval time.Duration, lookbackDelta time.Duration) Middleware { return MiddlewareFunc(func(next Handler) Handler { return distributedQueryMiddleware{ - next: next, - noStepSubqueryIntervalFn: noStepSubqueryIntervalFn, + next: next, + lookbackDelta: lookbackDelta, + defaultEvaluationInterval: defaultEvaluationInterval, } }) } @@ -32,8 +33,9 @@ func getStartAndEnd(start time.Time, end time.Time, step time.Duration) (time.Ti } type distributedQueryMiddleware struct { - next Handler - noStepSubqueryIntervalFn func(time.Duration) time.Duration + next Handler + defaultEvaluationInterval time.Duration + lookbackDelta time.Duration } func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, end time.Time, step time.Duration) (*logicalplan.Plan, error) { @@ -41,13 +43,15 @@ func (d distributedQueryMiddleware) newLogicalPlan(qs string, start time.Time, e start, end = getStartAndEnd(start, end, step) qOpts := query.Options{ - Start: start, - End: end, - Step: step, - StepsBatch: stepBatch, - NoStepSubqueryIntervalFn: d.noStepSubqueryIntervalFn, + Start: start, + End: end, + Step: step, + StepsBatch: stepBatch, + NoStepSubqueryIntervalFn: func(duration time.Duration) time.Duration { + return d.defaultEvaluationInterval + }, // Hardcoded value for execution-time-params that will be re-populated again in the querier stage - LookbackDelta: 0, + LookbackDelta: d.lookbackDelta, EnablePerStepStats: false, } diff --git a/pkg/querier/tripperware/distributed_query_test.go b/pkg/querier/tripperware/distributed_query_test.go index 68b61a1fe9d..d11a3dfbbaa 100644 --- a/pkg/querier/tripperware/distributed_query_test.go +++ b/pkg/querier/tripperware/distributed_query_test.go @@ -121,9 +121,7 @@ func TestLogicalPlanGeneration(t *testing.T) { t.Run(strconv.Itoa(i)+"_"+tc.name, func(t *testing.T) { t.Parallel() - middleware := DistributedQueryMiddleware(func(time.Duration) time.Duration { - return time.Minute - }) + middleware := DistributedQueryMiddleware(time.Minute, 5*time.Minute) handler := middleware.Wrap(HandlerFunc(func(_ context.Context, req Request) (Response, error) { return nil, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go index 8a54d3b0380..a74a9ad1479 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares.go @@ -15,7 +15,7 @@ func Middlewares( merger tripperware.Merger, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration, - noStepSubqueryIntervalFn func(time.Duration) time.Duration, + defaultEvaluationInterval time.Duration, distributedExecEnabled bool, ) ([]tripperware.Middleware, error) { m := []tripperware.Middleware{ @@ -25,7 +25,7 @@ func Middlewares( if distributedExecEnabled { m = append(m, - tripperware.DistributedQueryMiddleware(noStepSubqueryIntervalFn)) + tripperware.DistributedQueryMiddleware(defaultEvaluationInterval, lookbackDelta)) } return m, nil diff --git a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go index 318470eed92..b6d445fe20b 100644 --- a/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go +++ b/pkg/querier/tripperware/instantquery/instant_query_middlewares_test.go @@ -60,9 +60,7 @@ func TestRoundTrip(t *testing.T) { nil, qa, 5*time.Minute, - func(time.Duration) time.Duration { - return time.Minute - }, + time.Minute, false, ) require.NoError(t, err) @@ -175,9 +173,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { nil, qa, 5*time.Minute, - func(time.Duration) time.Duration { - return 5 * time.Minute - }, + time.Minute, tc.distributedEnabled, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 6fc8c40e3af..38493f54130 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -102,7 +102,7 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, - noStepSubqueryIntervalFn func(time.Duration) time.Duration, + defaultEvaluationInterval time.Duration, distributedExecEnabled bool, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. @@ -142,7 +142,7 @@ func Middlewares( if distributedExecEnabled { queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("range_logical_plan_gen", metrics), - tripperware.DistributedQueryMiddleware(noStepSubqueryIntervalFn)) + tripperware.DistributedQueryMiddleware(defaultEvaluationInterval, lookbackDelta)) } return queryRangeMiddleware, c, nil diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index cb05c6526e7..4e19fe84a02 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -67,9 +67,7 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - func(time.Duration) time.Duration { - return time.Minute - }, + time.Minute, false, ) require.NoError(t, err) @@ -194,9 +192,7 @@ func TestRoundTripWithAndWithoutDistributedExec(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - func(time.Duration) time.Duration { - return time.Minute - }, + time.Minute, tc.distributedEnabled, ) require.NoError(t, err)