From b874e4e103ceee250d9ec395a5451c536b46d302 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 24 Dec 2024 10:34:37 -0800 Subject: [PATCH 01/20] add limit for range query max splits by interval Signed-off-by: Ahmed Hassan --- .../queryrange/query_range_middlewares.go | 28 ++++++++++++++----- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 8af1ba1f84..eaf1724c1e 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -17,6 +17,7 @@ package queryrange import ( "flag" + "math" "time" "github.com/go-kit/log" @@ -34,11 +35,12 @@ const day = 24 * time.Hour // Config for query_range middleware chain. type Config struct { - SplitQueriesByInterval time.Duration `yaml:"split_queries_by_interval"` - AlignQueriesWithStep bool `yaml:"align_queries_with_step"` - ResultsCacheConfig `yaml:"results_cache"` - CacheResults bool `yaml:"cache_results"` - MaxRetries int `yaml:"max_retries"` + SplitQueriesByInterval time.Duration `yaml:"split_queries_by_interval"` + SplitQueriesByIntervalMaxSplits int `yaml:"split_queries_by_interval_max_splits"` + AlignQueriesWithStep bool `yaml:"align_queries_with_step"` + ResultsCacheConfig `yaml:"results_cache"` + CacheResults bool `yaml:"cache_results"` + MaxRetries int `yaml:"max_retries"` // List of headers which query_range middleware chain would forward to downstream querier. ForwardHeaders flagext.StringSlice `yaml:"forward_headers_list"` @@ -50,6 +52,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxRetries, "querier.max-retries-per-request", 5, "Maximum number of retries for a single request; beyond this, the downstream error is returned.") f.DurationVar(&cfg.SplitQueriesByInterval, "querier.split-queries-by-interval", 0, "Split queries by an interval and execute in parallel, 0 disables it. You should use a multiple of 24 hours (same as the storage bucketing scheme), to avoid queriers downloading and processing the same chunks. This also determines how cache keys are chosen when result caching is enabled") + f.IntVar(&cfg.SplitQueriesByIntervalMaxSplits, "querier.split-queries-by-interval-max-splits", 0, "Maximum number of splits by interval for a query, 0 disables it. Uses a multiple of `split-queries-by-interval` to ensure the number of splits remain below the limit.") f.BoolVar(&cfg.AlignQueriesWithStep, "querier.align-querier-with-step", false, "Mutate incoming queries to align their start and end with their step.") f.BoolVar(&cfg.CacheResults, "querier.cache-results", false, "Cache query results.") f.Var(&cfg.ForwardHeaders, "frontend.forward-headers-list", "List of headers forwarded by the query Frontend to downstream querier.") @@ -66,6 +69,9 @@ func (cfg *Config) Validate(qCfg querier.Config) error { return errors.Wrap(err, "invalid ResultsCache config") } } + if cfg.SplitQueriesByIntervalMaxSplits > 0 && cfg.SplitQueriesByInterval <= 0 { + return errors.New("split-queries-by-interval-max-splits requires that a value for split-queries-by-interval is set.") + } return nil } @@ -89,8 +95,16 @@ func Middlewares( queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("step_align", metrics), StepAlignMiddleware) } if cfg.SplitQueriesByInterval != 0 { - staticIntervalFn := func(_ tripperware.Request) time.Duration { return cfg.SplitQueriesByInterval } - queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(staticIntervalFn, limits, prometheusCodec, registerer)) + intervalFn := func(_ tripperware.Request) time.Duration { return cfg.SplitQueriesByInterval } + if cfg.SplitQueriesByIntervalMaxSplits != 0 { + intervalFn = func(r tripperware.Request) time.Duration { + queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) + baseInterval := cfg.SplitQueriesByInterval + n := int(math.Ceil(float64(queryRange) / float64(baseInterval*time.Duration(cfg.SplitQueriesByIntervalMaxSplits)))) + return time.Duration(n) * cfg.SplitQueriesByInterval + } + } + queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer)) } var c cache.Cache From 61069782fb07d44efcf7862b23513dc3fd65af97 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 30 Dec 2024 17:20:49 -0800 Subject: [PATCH 02/20] Change dynamic interval sharding to take into account vertical sharding Signed-off-by: Ahmed Hassan --- docs/configuration/config-file-reference.md | 7 ++++++ .../queryrange/query_range_middlewares.go | 12 +++------- .../queryrange/split_by_interval.go | 23 +++++++++++++++++++ 3 files changed, 33 insertions(+), 9 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 5479bb4d00..599e787b73 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4193,6 +4193,13 @@ The `query_range_config` configures the query splitting and caching in the Corte # CLI flag: -querier.split-queries-by-interval [split_queries_by_interval: | default = 0s] +# Maximum number of splits for a range query, 0 disables it. Uses a multiple of +# `split-queries-by-interval` to maintain the number of splits below the limit. +# If vertical sharding is enabled for a query, the combined total number of +# vertical and interval shards is kept below this limit +# CLI flag: -querier.split-queries-by-interval-max-splits +[split_queries_by_interval_max_splits: | default = 0] + # Mutate incoming queries to align their start and end with their step. # CLI flag: -querier.align-querier-with-step [align_queries_with_step: | default = false] diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index eaf1724c1e..887c2bc9aa 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -17,7 +17,6 @@ package queryrange import ( "flag" - "math" "time" "github.com/go-kit/log" @@ -52,7 +51,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxRetries, "querier.max-retries-per-request", 5, "Maximum number of retries for a single request; beyond this, the downstream error is returned.") f.DurationVar(&cfg.SplitQueriesByInterval, "querier.split-queries-by-interval", 0, "Split queries by an interval and execute in parallel, 0 disables it. You should use a multiple of 24 hours (same as the storage bucketing scheme), to avoid queriers downloading and processing the same chunks. This also determines how cache keys are chosen when result caching is enabled") - f.IntVar(&cfg.SplitQueriesByIntervalMaxSplits, "querier.split-queries-by-interval-max-splits", 0, "Maximum number of splits by interval for a query, 0 disables it. Uses a multiple of `split-queries-by-interval` to ensure the number of splits remain below the limit.") + f.IntVar(&cfg.SplitQueriesByIntervalMaxSplits, "querier.split-queries-by-interval-max-splits", 0, "Maximum number of splits for a range query, 0 disables it. Uses a multiple of `split-queries-by-interval` to maintain the number of splits below the limit. If vertical sharding is enabled for a query, the combined total number of vertical and interval shards is kept below this limit") f.BoolVar(&cfg.AlignQueriesWithStep, "querier.align-querier-with-step", false, "Mutate incoming queries to align their start and end with their step.") f.BoolVar(&cfg.CacheResults, "querier.cache-results", false, "Cache query results.") f.Var(&cfg.ForwardHeaders, "frontend.forward-headers-list", "List of headers forwarded by the query Frontend to downstream querier.") @@ -95,14 +94,9 @@ func Middlewares( queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("step_align", metrics), StepAlignMiddleware) } if cfg.SplitQueriesByInterval != 0 { - intervalFn := func(_ tripperware.Request) time.Duration { return cfg.SplitQueriesByInterval } + intervalFn := staticIntervalFn(cfg) if cfg.SplitQueriesByIntervalMaxSplits != 0 { - intervalFn = func(r tripperware.Request) time.Duration { - queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - baseInterval := cfg.SplitQueriesByInterval - n := int(math.Ceil(float64(queryRange) / float64(baseInterval*time.Duration(cfg.SplitQueriesByIntervalMaxSplits)))) - return time.Duration(n) * cfg.SplitQueriesByInterval - } + intervalFn = dynamicIntervalFn(cfg, queryAnalyzer) } queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer)) } diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 064b53d760..3220345647 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -8,6 +8,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/promql/parser" + "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/querier/tripperware" @@ -135,3 +136,25 @@ func nextIntervalBoundary(t, step int64, interval time.Duration) int64 { } return target } + +func staticIntervalFn(cfg Config) func(r tripperware.Request) time.Duration { + return func(_ tripperware.Request) time.Duration { + return cfg.SplitQueriesByInterval + } +} + +func dynamicIntervalFn(cfg Config, queryAnalyzer querysharding.Analyzer) func(r tripperware.Request) time.Duration { + return func(r tripperware.Request) time.Duration { + queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) + baseInterval := cfg.SplitQueriesByInterval + + analysis, _ := queryAnalyzer.Analyze(r.GetQuery()) + maxSplits := time.Duration(cfg.SplitQueriesByIntervalMaxSplits) + if cfg.VerticalShardSize > 0 && analysis.IsShardable() { + maxSplits = time.Duration(cfg.SplitQueriesByIntervalMaxSplits / cfg.VerticalShardSize) + } + + n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + return n * cfg.SplitQueriesByInterval + } +} From 01c121c1868ea39fc66a988f2efc86df159825bf Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Thu, 16 Jan 2025 11:00:27 -0800 Subject: [PATCH 03/20] add dynamic sharding based on total days of data fetched for query Signed-off-by: Ahmed Hassan --- pkg/cortex/modules.go | 2 + pkg/frontend/transport/handler.go | 5 + pkg/querier/querier.go | 4 + pkg/querier/stats/stats.go | 9 + .../queryrange/query_range_middlewares.go | 8 +- .../query_range_middlewares_test.go | 2 + .../queryrange/split_by_interval.go | 165 ++++++++++++++++-- pkg/util/time.go | 67 +++++++ 8 files changed, 243 insertions(+), 19 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index a771c22116..d8bb4a3150 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -487,6 +487,8 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, + t.Cfg.Querier.QueryStoreAfter, + t.Cfg.Querier.MaxDaysOfDataFetched, ) if err != nil { return nil, err diff --git a/pkg/frontend/transport/handler.go b/pkg/frontend/transport/handler.go index bba995988d..20f224e716 100644 --- a/pkg/frontend/transport/handler.go +++ b/pkg/frontend/transport/handler.go @@ -351,6 +351,7 @@ func (f *Handler) reportQueryStats(r *http.Request, userID string, queryString u splitQueries := stats.LoadSplitQueries() dataSelectMaxTime := stats.LoadDataSelectMaxTime() dataSelectMinTime := stats.LoadDataSelectMinTime() + splitInterval := stats.LoadSplitInterval() // Track stats. f.querySeconds.WithLabelValues(userID).Add(wallTime.Seconds()) @@ -425,6 +426,10 @@ func (f *Handler) reportQueryStats(r *http.Request, userID string, queryString u logMessage = append(logMessage, "query_storage_wall_time_seconds", sws) } + if splitInterval > 0 { + logMessage = append(logMessage, "split_interval", splitInterval.String()) + } + if error != nil { s, ok := status.FromError(error) if !ok { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index e9a80374cd..22b260308d 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -62,6 +62,9 @@ type Config struct { // Limit of number of steps allowed for every subquery expression in a query. MaxSubQuerySteps int64 `yaml:"max_subquery_steps"` + // Max number of days of data fetched for a query, used to calculate appropriate interval and vertical shard size. + MaxDaysOfDataFetched int `yaml:"max_days_of_data_fetched"` + // Directory for ActiveQueryTracker. If empty, ActiveQueryTracker will be disabled and MaxConcurrent will not be applied (!). // ActiveQueryTracker logs queries that were active during the last crash, but logs them on the next startup. // However, we need to use active query tracker, otherwise we cannot limit Max Concurrent queries in the PromQL @@ -131,6 +134,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Int64Var(&cfg.MaxSubQuerySteps, "querier.max-subquery-steps", 0, "Max number of steps allowed for every subquery expression in query. Number of steps is calculated using subquery range / step. A value > 0 enables it.") f.BoolVar(&cfg.IgnoreMaxQueryLength, "querier.ignore-max-query-length", false, "If enabled, ignore max query length check at Querier select method. Users can choose to ignore it since the validation can be done before Querier evaluation like at Query Frontend or Ruler.") f.BoolVar(&cfg.EnablePromQLExperimentalFunctions, "querier.enable-promql-experimental-functions", false, "[Experimental] If true, experimental promQL functions are enabled.") + f.IntVar(&cfg.MaxDaysOfDataFetched, "querier.max-days-of-data-fetched", 0, "Max number of days of data fetched for a query. This can be used to calculate appropriate interval and vertical shard size dynamically.") } // Validate the config diff --git a/pkg/querier/stats/stats.go b/pkg/querier/stats/stats.go index d4eda0f756..cdf432718d 100644 --- a/pkg/querier/stats/stats.go +++ b/pkg/querier/stats/stats.go @@ -21,6 +21,7 @@ type QueryStats struct { Priority int64 DataSelectMaxTime int64 DataSelectMinTime int64 + SplitInterval time.Duration m sync.Mutex } @@ -287,6 +288,14 @@ func (s *QueryStats) LoadDataSelectMinTime() int64 { return atomic.LoadInt64(&s.DataSelectMinTime) } +func (s *QueryStats) LoadSplitInterval() time.Duration { + if s == nil { + return 0 + } + + return s.SplitInterval +} + func (s *QueryStats) AddStoreGatewayTouchedPostings(count uint64) { if s == nil { return diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 887c2bc9aa..af8e003661 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -85,6 +85,8 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, + queryStoreAfter time.Duration, + maxDaysOfDataFetched int, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -95,10 +97,10 @@ func Middlewares( } if cfg.SplitQueriesByInterval != 0 { intervalFn := staticIntervalFn(cfg) - if cfg.SplitQueriesByIntervalMaxSplits != 0 { - intervalFn = dynamicIntervalFn(cfg, queryAnalyzer) + if cfg.SplitQueriesByIntervalMaxSplits > 0 || maxDaysOfDataFetched > 0 { + intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, queryStoreAfter, lookbackDelta, maxDaysOfDataFetched) } - queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer)) + queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, queryStoreAfter, lookbackDelta)) } var c cache.Cache diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index c8238a9a62..55be979a3d 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -64,6 +64,8 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, + 24*time.Hour, + 0, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 3220345647..e1998afc50 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -3,6 +3,7 @@ package queryrange import ( "context" "net/http" + "sort" "time" "github.com/prometheus/client_golang/prometheus" @@ -11,13 +12,25 @@ import ( "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/httpgrpc" + querier_stats "github.com/cortexproject/cortex/pkg/querier/stats" "github.com/cortexproject/cortex/pkg/querier/tripperware" + "github.com/cortexproject/cortex/pkg/tenant" + "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/util/validation" ) -type IntervalFn func(r tripperware.Request) time.Duration +// dayMillis is the L4 block range in milliseconds. +var dayMillis = util.DurationMilliseconds(24 * time.Hour) + +type IntervalFn func(ctx context.Context, r tripperware.Request) (time.Duration, error) + +type dayRange struct { + startDay int64 + endDay int64 +} // SplitByIntervalMiddleware creates a new Middleware that splits requests by a given interval. -func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, merger tripperware.Merger, registerer prometheus.Registerer) tripperware.Middleware { +func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, merger tripperware.Merger, registerer prometheus.Registerer, queryStoreAfter time.Duration, lookbackDelta time.Duration) tripperware.Middleware { return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { return splitByInterval{ next: next, @@ -29,6 +42,8 @@ func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, m Name: "frontend_split_queries_total", Help: "Total number of underlying query requests after the split by interval is applied", }), + queryStoreAfter: queryStoreAfter, + lookbackDelta: lookbackDelta, } }) } @@ -41,17 +56,28 @@ type splitByInterval struct { // Metrics. splitByCounter prometheus.Counter + + queryStoreAfter time.Duration + lookbackDelta time.Duration } func (s splitByInterval) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { // First we're going to build new requests, one for each day, taking care // to line up the boundaries with step. - reqs, err := splitQuery(r, s.interval(r)) + interval, err := s.interval(ctx, r) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + reqs, err := splitQuery(r, interval) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } s.splitByCounter.Add(float64(len(reqs))) + stats := querier_stats.FromContext(ctx) + if stats != nil { + stats.SplitInterval = interval + } reqResps, err := tripperware.DoRequests(ctx, s.next, reqs, s.limits) if err != nil { return nil, err @@ -137,24 +163,131 @@ func nextIntervalBoundary(t, step int64, interval time.Duration) int64 { return target } -func staticIntervalFn(cfg Config) func(r tripperware.Request) time.Duration { - return func(_ tripperware.Request) time.Duration { - return cfg.SplitQueriesByInterval +func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { + return func(_ context.Context, _ tripperware.Request) (time.Duration, error) { + return cfg.SplitQueriesByInterval, nil } } -func dynamicIntervalFn(cfg Config, queryAnalyzer querysharding.Analyzer) func(r tripperware.Request) time.Duration { - return func(r tripperware.Request) time.Duration { - queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - baseInterval := cfg.SplitQueriesByInterval +func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, queryStoreAfter time.Duration, lookbackDelta time.Duration, maxDaysOfDataFetched int) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { + return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { + tenantIDs, err := tenant.TenantIDs(ctx) + if err != nil { + return cfg.SplitQueriesByInterval, err + } + + queryDayRange := int((r.GetEnd() / dayMillis) - (r.GetStart() / dayMillis) + 1) + baseInterval := int(cfg.SplitQueriesByInterval / day) + analysis, err := queryAnalyzer.Analyze(r.GetQuery()) + if err != nil { + return cfg.SplitQueriesByInterval, err + } + + queryVerticalShardSize := validation.SmallestPositiveIntPerTenant(tenantIDs, limits.QueryVerticalShardSize) + if queryVerticalShardSize <= 0 || !analysis.IsShardable() { + queryVerticalShardSize = 1 + } + + queryExpr, err := parser.ParseExpr(r.GetQuery()) + if err != nil { + return cfg.SplitQueriesByInterval, err + } + daysFetchedWithoutSharding := getDaysFetchedByQuery(queryExpr, []tripperware.Request{r}, queryStoreAfter, lookbackDelta, time.Now()) + extraDaysFetchedPerShard := daysFetchedWithoutSharding - queryDayRange + + // if lookbackDelta is configured and the query start time is not 00:00 UTC, we need to account for 1 fetched day of data per split except for the first split + lookbackDeltaCompensation := 0 + if lookbackDelta > 0 && (r.GetStart()-util.DurationMilliseconds(lookbackDelta))/dayMillis == r.GetStart()/dayMillis { + lookbackDeltaCompensation = 1 + } + + var maxSplitsByFetchedDaysOfData int + if maxDaysOfDataFetched > 0 { + maxSplitsByFetchedDaysOfData = ((maxDaysOfDataFetched / queryVerticalShardSize) - queryDayRange - lookbackDeltaCompensation) / (extraDaysFetchedPerShard + lookbackDeltaCompensation) + } - analysis, _ := queryAnalyzer.Analyze(r.GetQuery()) - maxSplits := time.Duration(cfg.SplitQueriesByIntervalMaxSplits) - if cfg.VerticalShardSize > 0 && analysis.IsShardable() { - maxSplits = time.Duration(cfg.SplitQueriesByIntervalMaxSplits / cfg.VerticalShardSize) + var maxSplitsByConfig int + if cfg.SplitQueriesByIntervalMaxSplits > 0 { + maxSplitsByConfig = cfg.SplitQueriesByIntervalMaxSplits / queryVerticalShardSize } - n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) - return n * cfg.SplitQueriesByInterval + var maxSplits int + switch { + case maxSplitsByFetchedDaysOfData <= 0 && maxSplitsByConfig <= 0: + maxSplits = 1 + case maxSplitsByFetchedDaysOfData <= 0: + maxSplits = maxSplitsByConfig + case maxSplitsByConfig <= 0: + maxSplits = maxSplitsByFetchedDaysOfData + default: + // Use the more restricting shard limit + maxSplits = min(maxSplitsByConfig, maxSplitsByFetchedDaysOfData) + } + + n := (queryDayRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + return time.Duration(n) * cfg.SplitQueriesByInterval, nil + } +} + +// calculates the total number of days the query will have to fetch during execution, considering the query itself, +// queryStoreAfter and lookbackDelta. +func getDaysFetchedByQuery(expr parser.Expr, reqs []tripperware.Request, queryStoreAfter, lookbackDelta time.Duration, now time.Time) int { + count := 0 + queryStoreMaxT := util.TimeToMillis(now.Add(-queryStoreAfter)) + var evalRange time.Duration + + for _, req := range reqs { + var ranges []dayRange + parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + switch n := node.(type) { + case *parser.VectorSelector: + start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), lookbackDelta, n, path, evalRange) + // Query shouldn't touch Store Gateway. + if start > queryStoreMaxT { + return nil + } else { + // If the query split needs to query store, cap the max time to now - queryStoreAfter. + end = min(end, queryStoreMaxT) + } + + startDay := start / dayMillis + endDay := end / dayMillis + ranges = append(ranges, dayRange{startDay: startDay, endDay: endDay}) + evalRange = 0 + case *parser.MatrixSelector: + evalRange = n.Range + } + return nil + }) + nonOverlappingRanges := mergeDayRanges(ranges) + for _, dayRange := range nonOverlappingRanges { + count += int(dayRange.endDay-dayRange.startDay) + 1 + } + } + return count +} + +func mergeDayRanges(ranges []dayRange) []dayRange { + if len(ranges) == 0 { + return ranges + } + + // Sort ranges by their startDay + sort.Slice(ranges, func(i, j int) bool { + return ranges[i].startDay < ranges[j].startDay + }) + + // Merge overlapping ranges + merged := []dayRange{ranges[0]} + for _, current := range ranges { + last := &merged[len(merged)-1] + if current.startDay <= last.endDay { + if current.endDay > last.endDay { + last.endDay = current.endDay + } + } else { + merged = append(merged, current) + } } + return merged } diff --git a/pkg/util/time.go b/pkg/util/time.go index 3f19a71da9..2656065650 100644 --- a/pkg/util/time.go +++ b/pkg/util/time.go @@ -240,3 +240,70 @@ func ParseDurationMs(s string) (int64, error) { } return 0, httpgrpc.Errorf(http.StatusBadRequest, "cannot parse %q to a valid duration", s) } + +func DurationMilliseconds(d time.Duration) int64 { + return int64(d / (time.Millisecond / time.Nanosecond)) +} + +func GetTimeRangesForSelector(start, end int64, lookbackDelta time.Duration, n *parser.VectorSelector, path []parser.Node, evalRange time.Duration) (int64, int64) { + subqOffset, subqRange, subqTs := subqueryTimes(path) + + if subqTs != nil { + // The timestamp on the subquery overrides the eval statement time ranges. + start = *subqTs + end = *subqTs + } + + if n.Timestamp != nil { + // The timestamp on the selector overrides everything. + start = *n.Timestamp + end = *n.Timestamp + } else { + offsetMilliseconds := DurationMilliseconds(subqOffset) + start = start - offsetMilliseconds - DurationMilliseconds(subqRange) + end -= offsetMilliseconds + } + + if evalRange == 0 { + start -= DurationMilliseconds(lookbackDelta) + } else { + // For all matrix queries we want to ensure that we have (end-start) + range selected + // this way we have `range` data before the start time + start -= DurationMilliseconds(evalRange) + } + + offsetMilliseconds := DurationMilliseconds(n.OriginalOffset) + start -= offsetMilliseconds + end -= offsetMilliseconds + + return start, end +} + +// subqueryTimes returns the sum of offsets and ranges of all subqueries in the path. +// If the @ modifier is used, then the offset and range is w.r.t. that timestamp +// (i.e. the sum is reset when we have @ modifier). +// The returned *int64 is the closest timestamp that was seen. nil for no @ modifier. +func subqueryTimes(path []parser.Node) (time.Duration, time.Duration, *int64) { + var ( + subqOffset, subqRange time.Duration + ts int64 = math.MaxInt64 + ) + for _, node := range path { + if n, ok := node.(*parser.SubqueryExpr); ok { + subqOffset += n.OriginalOffset + subqRange += n.Range + if n.Timestamp != nil { + // The @ modifier on subquery invalidates all the offset and + // range till now. Hence resetting it here. + subqOffset = n.OriginalOffset + subqRange = n.Range + ts = *n.Timestamp + } + } + } + var tsp *int64 + if ts != math.MaxInt64 { + tsp = &ts + } + return subqOffset, subqRange, tsp +} From b15dde6d24a8d1b753933ace0f2948a9e388a563 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 17 Jan 2025 10:00:23 -0800 Subject: [PATCH 04/20] add unit tests for dynamicIntervalFn Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval.go | 17 +- .../queryrange/split_by_interval_test.go | 147 +++++++++++++++++- 2 files changed, 153 insertions(+), 11 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index e1998afc50..8b64484afe 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -177,7 +177,6 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer } queryDayRange := int((r.GetEnd() / dayMillis) - (r.GetStart() / dayMillis) + 1) - baseInterval := int(cfg.SplitQueriesByInterval / day) analysis, err := queryAnalyzer.Analyze(r.GetQuery()) if err != nil { return cfg.SplitQueriesByInterval, err @@ -211,21 +210,23 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer maxSplitsByConfig = cfg.SplitQueriesByIntervalMaxSplits / queryVerticalShardSize } - var maxSplits int + var maxSplits time.Duration switch { case maxSplitsByFetchedDaysOfData <= 0 && maxSplitsByConfig <= 0: - maxSplits = 1 + return cfg.SplitQueriesByInterval, nil case maxSplitsByFetchedDaysOfData <= 0: - maxSplits = maxSplitsByConfig + maxSplits = time.Duration(maxSplitsByConfig) case maxSplitsByConfig <= 0: - maxSplits = maxSplitsByFetchedDaysOfData + maxSplits = time.Duration(maxSplitsByFetchedDaysOfData) default: // Use the more restricting shard limit - maxSplits = min(maxSplitsByConfig, maxSplitsByFetchedDaysOfData) + maxSplits = time.Duration(min(maxSplitsByConfig, maxSplitsByFetchedDaysOfData)) } - n := (queryDayRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) - return time.Duration(n) * cfg.SplitQueriesByInterval, nil + queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) + baseInterval := cfg.SplitQueriesByInterval + n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + return n * cfg.SplitQueriesByInterval, nil } } diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 23989ac64a..186ce9824b 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/thanos-io/thanos/pkg/querysharding" "github.com/weaveworks/common/httpgrpc" "github.com/prometheus/prometheus/promql/parser" @@ -21,7 +22,11 @@ import ( "github.com/cortexproject/cortex/pkg/querier/tripperware" ) -const seconds = 1e3 // 1e3 milliseconds per second. +const ( + seconds = 1e3 // 1e3 milliseconds per second. + queryStoreAfter = 24 * time.Hour + lookbackDelta = 5 * time.Minute +) func TestNextIntervalBoundary(t *testing.T) { t.Parallel() @@ -305,11 +310,11 @@ func TestSplitByDay(t *testing.T) { u, err := url.Parse(s.URL) require.NoError(t, err) - interval := func(_ tripperware.Request) time.Duration { return 24 * time.Hour } + interval := func(_ context.Context, _ tripperware.Request) (time.Duration, error) { return 24 * time.Hour, nil } roundtripper := tripperware.NewRoundTripper(singleHostRoundTripper{ host: u.Host, next: http.DefaultTransport, - }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(interval, mockLimits{}, PrometheusCodec, nil)) + }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(interval, mockLimits{}, PrometheusCodec, nil, queryStoreAfter, lookbackDelta)) req, err := http.NewRequest("GET", tc.path, http.NoBody) require.NoError(t, err) @@ -408,3 +413,139 @@ func Test_evaluateAtModifier(t *testing.T) { }) } } + +func TestDynamicIntervalFn(t *testing.T) { + for _, tc := range []struct { + name string + baseSplitInterval time.Duration + req tripperware.Request + expectedInterval time.Duration + expectedError bool + maxQueryIntervalSplits int + maxDaysOfDataFetched int + }{ + { + baseSplitInterval: day, + name: "failed to parse request, return default interval", + req: &tripperware.PrometheusRequest{ + Query: "up[aaa", + Start: 0, + End: 10 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + }, + maxQueryIntervalSplits: 30, + maxDaysOfDataFetched: 200, + expectedInterval: day, + expectedError: true, + }, + { + baseSplitInterval: day, + name: "30 day range no limits, expect split by 1 day", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 30 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up", + }, + expectedInterval: day, + }, + { + baseSplitInterval: day, + name: "30 day range with 20 max splits, expect split by 2 day", + req: &tripperware.PrometheusRequest{ + Start: 30 * 24 * 3600 * seconds, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up", + }, + maxQueryIntervalSplits: 20, + expectedInterval: 2 * day, + }, + { + baseSplitInterval: day, + name: "60 day range, expect split by 4 day", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up", + }, + maxQueryIntervalSplits: 15, + expectedInterval: 4 * day, + }, + { + baseSplitInterval: day, + name: "61 day range, expect split by 5 day", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 61 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up", + }, + maxQueryIntervalSplits: 15, + expectedInterval: 5 * day, + }, + { + baseSplitInterval: day, + name: "30 day range short matrix selector with 200 days fetched limit, expect split by 1 day", + req: &tripperware.PrometheusRequest{ + Start: 30 * 24 * 3600 * seconds, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "avg_over_time(up[1h])", + }, + maxDaysOfDataFetched: 200, + expectedInterval: day, + }, + { + baseSplitInterval: day, + name: "30 day range long matrix selector with 200 days fetched limit, expect split by 1 day", + req: &tripperware.PrometheusRequest{ + Start: 30 * 24 * 3600 * seconds, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "avg_over_time(up[20d])", + }, + maxDaysOfDataFetched: 200, + expectedInterval: 4 * day, + }, + { + baseSplitInterval: day, + name: "60 day range, expect split by 7 day", + req: &tripperware.PrometheusRequest{ + Start: (2 * 24 * 3600 * seconds) + (3600*seconds - 120), + End: (61 * 24 * 3600 * seconds) + (2*3600*seconds + 500), + Step: 30 * 60 * seconds, + Query: "rate(up[1d]) + rate(up[2d]) + rate(up[5d]) + rate(up[15d])", + }, + maxDaysOfDataFetched: 200, + expectedInterval: 7 * day, + }, + { + baseSplitInterval: day, + name: "30 day range, expect split by 7 day", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 100 * 24 * 3600 * seconds, + Step: 60 * 60 * seconds, + Query: "up[5d:10m]", + }, + maxQueryIntervalSplits: 100, + maxDaysOfDataFetched: 300, + expectedInterval: 4 * day, + }, + } { + t.Run(tc.name, func(t *testing.T) { + cfg := Config{ + SplitQueriesByInterval: tc.baseSplitInterval, + SplitQueriesByIntervalMaxSplits: tc.maxQueryIntervalSplits, + } + ctx := user.InjectOrgID(context.Background(), "1") + interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), queryStoreAfter, lookbackDelta, tc.maxDaysOfDataFetched)(ctx, tc.req) + require.Equal(t, tc.expectedInterval, interval) + if !tc.expectedError { + require.Nil(t, err) + } + }) + } +} From a1e7a3ae85136685cc157f127b6fff538637ba42 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 22 Jan 2025 11:26:38 -0800 Subject: [PATCH 05/20] allow using any base interval size for dynamicIntervalFn Signed-off-by: Ahmed Hassan --- docs/configuration/config-file-reference.md | 21 ++- pkg/cortex/modules.go | 1 - pkg/querier/querier.go | 4 - .../queryrange/dynamic_query_splits.go | 17 ++ .../queryrange/query_range_middlewares.go | 27 ++-- .../query_range_middlewares_test.go | 1 - .../queryrange/split_by_interval.go | 148 +++++++++--------- .../queryrange/split_by_interval_test.go | 118 ++++++++++---- pkg/util/time.go | 2 + 9 files changed, 205 insertions(+), 134 deletions(-) create mode 100644 pkg/querier/tripperware/queryrange/dynamic_query_splits.go diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 599e787b73..747571ac31 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4193,12 +4193,21 @@ The `query_range_config` configures the query splitting and caching in the Corte # CLI flag: -querier.split-queries-by-interval [split_queries_by_interval: | default = 0s] -# Maximum number of splits for a range query, 0 disables it. Uses a multiple of -# `split-queries-by-interval` to maintain the number of splits below the limit. -# If vertical sharding is enabled for a query, the combined total number of -# vertical and interval shards is kept below this limit -# CLI flag: -querier.split-queries-by-interval-max-splits -[split_queries_by_interval_max_splits: | default = 0] +dynamic_query_splits: + # [EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. + # Dynamically uses a multiple of `split-queries-by-interval` to maintain the + # number of splits below the limit. If vertical sharding is enabled for a + # query, the combined total number of vertical and interval shards is kept + # below this limit. + # CLI flag: -querier.max-shards-per-query + [max_shards_per_query: | default = 0] + + # [EXPERIMENTAL] Max total duration of data fetched by all query shards from + # storage, 0 disables it. Dynamically uses a multiple of + # `split-queries-by-interval` to ensure the total fetched duration of data is + # lower than the value set. + # CLI flag: -querier.max-duration-of-data-fetched-from-storage-per-query + [max_duration_of_data_fetched_from_storage_per_query: | default = 0s] # Mutate incoming queries to align their start and end with their step. # CLI flag: -querier.align-querier-with-step diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index d8bb4a3150..1a9ad3d415 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -488,7 +488,6 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, t.Cfg.Querier.QueryStoreAfter, - t.Cfg.Querier.MaxDaysOfDataFetched, ) if err != nil { return nil, err diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 22b260308d..e9a80374cd 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -62,9 +62,6 @@ type Config struct { // Limit of number of steps allowed for every subquery expression in a query. MaxSubQuerySteps int64 `yaml:"max_subquery_steps"` - // Max number of days of data fetched for a query, used to calculate appropriate interval and vertical shard size. - MaxDaysOfDataFetched int `yaml:"max_days_of_data_fetched"` - // Directory for ActiveQueryTracker. If empty, ActiveQueryTracker will be disabled and MaxConcurrent will not be applied (!). // ActiveQueryTracker logs queries that were active during the last crash, but logs them on the next startup. // However, we need to use active query tracker, otherwise we cannot limit Max Concurrent queries in the PromQL @@ -134,7 +131,6 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Int64Var(&cfg.MaxSubQuerySteps, "querier.max-subquery-steps", 0, "Max number of steps allowed for every subquery expression in query. Number of steps is calculated using subquery range / step. A value > 0 enables it.") f.BoolVar(&cfg.IgnoreMaxQueryLength, "querier.ignore-max-query-length", false, "If enabled, ignore max query length check at Querier select method. Users can choose to ignore it since the validation can be done before Querier evaluation like at Query Frontend or Ruler.") f.BoolVar(&cfg.EnablePromQLExperimentalFunctions, "querier.enable-promql-experimental-functions", false, "[Experimental] If true, experimental promQL functions are enabled.") - f.IntVar(&cfg.MaxDaysOfDataFetched, "querier.max-days-of-data-fetched", 0, "Max number of days of data fetched for a query. This can be used to calculate appropriate interval and vertical shard size dynamically.") } // Validate the config diff --git a/pkg/querier/tripperware/queryrange/dynamic_query_splits.go b/pkg/querier/tripperware/queryrange/dynamic_query_splits.go new file mode 100644 index 0000000000..1120967365 --- /dev/null +++ b/pkg/querier/tripperware/queryrange/dynamic_query_splits.go @@ -0,0 +1,17 @@ +package queryrange + +import ( + "flag" + "time" +) + +type DynamicQuerySplitsConfig struct { + MaxShardsPerQuery int `yaml:"max_shards_per_query"` + MaxDurationOfDataFetchedFromStoragePerQuery time.Duration `yaml:"max_duration_of_data_fetched_from_storage_per_query"` +} + +// RegisterFlags registers flags foy dynamic query splits +func (cfg *DynamicQuerySplitsConfig) RegisterFlags(f *flag.FlagSet) { + f.IntVar(&cfg.MaxShardsPerQuery, "querier.max-shards-per-query", 0, "[EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to maintain the number of splits below the limit. If vertical sharding is enabled for a query, the combined total number of vertical and interval shards is kept below this limit.") + f.DurationVar(&cfg.MaxDurationOfDataFetchedFromStoragePerQuery, "querier.max-duration-of-data-fetched-from-storage-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched by all query shards from storage, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to ensure the total fetched duration of data is lower than the value set.") +} diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index af8e003661..65c8b7712e 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -34,12 +34,14 @@ const day = 24 * time.Hour // Config for query_range middleware chain. type Config struct { - SplitQueriesByInterval time.Duration `yaml:"split_queries_by_interval"` - SplitQueriesByIntervalMaxSplits int `yaml:"split_queries_by_interval_max_splits"` - AlignQueriesWithStep bool `yaml:"align_queries_with_step"` - ResultsCacheConfig `yaml:"results_cache"` - CacheResults bool `yaml:"cache_results"` - MaxRetries int `yaml:"max_retries"` + // Query splits config + SplitQueriesByInterval time.Duration `yaml:"split_queries_by_interval"` + DynamicQuerySplitsConfig DynamicQuerySplitsConfig `yaml:"dynamic_query_splits"` + + AlignQueriesWithStep bool `yaml:"align_queries_with_step"` + ResultsCacheConfig `yaml:"results_cache"` + CacheResults bool `yaml:"cache_results"` + MaxRetries int `yaml:"max_retries"` // List of headers which query_range middleware chain would forward to downstream querier. ForwardHeaders flagext.StringSlice `yaml:"forward_headers_list"` @@ -51,11 +53,11 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxRetries, "querier.max-retries-per-request", 5, "Maximum number of retries for a single request; beyond this, the downstream error is returned.") f.DurationVar(&cfg.SplitQueriesByInterval, "querier.split-queries-by-interval", 0, "Split queries by an interval and execute in parallel, 0 disables it. You should use a multiple of 24 hours (same as the storage bucketing scheme), to avoid queriers downloading and processing the same chunks. This also determines how cache keys are chosen when result caching is enabled") - f.IntVar(&cfg.SplitQueriesByIntervalMaxSplits, "querier.split-queries-by-interval-max-splits", 0, "Maximum number of splits for a range query, 0 disables it. Uses a multiple of `split-queries-by-interval` to maintain the number of splits below the limit. If vertical sharding is enabled for a query, the combined total number of vertical and interval shards is kept below this limit") f.BoolVar(&cfg.AlignQueriesWithStep, "querier.align-querier-with-step", false, "Mutate incoming queries to align their start and end with their step.") f.BoolVar(&cfg.CacheResults, "querier.cache-results", false, "Cache query results.") f.Var(&cfg.ForwardHeaders, "frontend.forward-headers-list", "List of headers forwarded by the query Frontend to downstream querier.") cfg.ResultsCacheConfig.RegisterFlags(f) + cfg.DynamicQuerySplitsConfig.RegisterFlags(f) } // Validate validates the config. @@ -68,8 +70,10 @@ func (cfg *Config) Validate(qCfg querier.Config) error { return errors.Wrap(err, "invalid ResultsCache config") } } - if cfg.SplitQueriesByIntervalMaxSplits > 0 && cfg.SplitQueriesByInterval <= 0 { - return errors.New("split-queries-by-interval-max-splits requires that a value for split-queries-by-interval is set.") + if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { + if cfg.SplitQueriesByInterval <= 0 { + return errors.New("configs under dynamic-query-splits requires that a value for split-queries-by-interval is set.") + } } return nil } @@ -86,7 +90,6 @@ func Middlewares( shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, queryStoreAfter time.Duration, - maxDaysOfDataFetched int, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -97,8 +100,8 @@ func Middlewares( } if cfg.SplitQueriesByInterval != 0 { intervalFn := staticIntervalFn(cfg) - if cfg.SplitQueriesByIntervalMaxSplits > 0 || maxDaysOfDataFetched > 0 { - intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, queryStoreAfter, lookbackDelta, maxDaysOfDataFetched) + if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { + intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, queryStoreAfter, lookbackDelta) } queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, queryStoreAfter, lookbackDelta)) } diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 55be979a3d..14ccfd0ea7 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -65,7 +65,6 @@ func TestRoundTrip(t *testing.T) { ShardedPrometheusCodec, 5*time.Minute, 24*time.Hour, - 0, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 8b64484afe..038b5b04f8 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -3,7 +3,6 @@ package queryrange import ( "context" "net/http" - "sort" "time" "github.com/prometheus/client_golang/prometheus" @@ -19,16 +18,8 @@ import ( "github.com/cortexproject/cortex/pkg/util/validation" ) -// dayMillis is the L4 block range in milliseconds. -var dayMillis = util.DurationMilliseconds(24 * time.Hour) - type IntervalFn func(ctx context.Context, r tripperware.Request) (time.Duration, error) -type dayRange struct { - startDay int64 - endDay int64 -} - // SplitByIntervalMiddleware creates a new Middleware that splits requests by a given interval. func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, merger tripperware.Merger, registerer prometheus.Registerer, queryStoreAfter time.Duration, lookbackDelta time.Duration) tripperware.Middleware { return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { @@ -169,14 +160,13 @@ func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Reques } } -func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, queryStoreAfter time.Duration, lookbackDelta time.Duration, maxDaysOfDataFetched int) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { +func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, queryStoreAfter time.Duration, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { return cfg.SplitQueriesByInterval, err } - queryDayRange := int((r.GetEnd() / dayMillis) - (r.GetStart() / dayMillis) + 1) analysis, err := queryAnalyzer.Analyze(r.GetQuery()) if err != nil { return cfg.SplitQueriesByInterval, err @@ -191,23 +181,35 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer if err != nil { return cfg.SplitQueriesByInterval, err } - daysFetchedWithoutSharding := getDaysFetchedByQuery(queryExpr, []tripperware.Request{r}, queryStoreAfter, lookbackDelta, time.Now()) - extraDaysFetchedPerShard := daysFetchedWithoutSharding - queryDayRange - // if lookbackDelta is configured and the query start time is not 00:00 UTC, we need to account for 1 fetched day of data per split except for the first split - lookbackDeltaCompensation := 0 - if lookbackDelta > 0 && (r.GetStart()-util.DurationMilliseconds(lookbackDelta))/dayMillis == r.GetStart()/dayMillis { - lookbackDeltaCompensation = 1 - } + // Calculates: duration of data fetched if the query was not sharded, the original range covered by the query start and end times, + // and the duration of data fetched by lookbackDelta for the first split + durationFetchedWithoutSharding, originalRangeCount, firstSplitLookbackDeltaCompensation := durationFetchedByQuery(queryExpr, r, queryStoreAfter, lookbackDelta, cfg.SplitQueriesByInterval, time.Now()) + extraDaysFetchedPerShard := durationFetchedWithoutSharding - originalRangeCount + + // Calculate the extra duration of data fetched by lookbackDelta per each split except the first split + nextIntervalStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), cfg.SplitQueriesByInterval) + r.GetStep() + nextIntervalReq := r.WithStartEnd(nextIntervalStart, r.GetEnd()) + _, _, lookbackDeltaCompensation := durationFetchedByQuery(queryExpr, nextIntervalReq, queryStoreAfter, lookbackDelta, cfg.SplitQueriesByInterval, time.Now()) var maxSplitsByFetchedDaysOfData int - if maxDaysOfDataFetched > 0 { - maxSplitsByFetchedDaysOfData = ((maxDaysOfDataFetched / queryVerticalShardSize) - queryDayRange - lookbackDeltaCompensation) / (extraDaysFetchedPerShard + lookbackDeltaCompensation) + if cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { + if extraDaysFetchedPerShard == 0 { + extraDaysFetchedPerShard = 1 // prevent division by 0 + } + maxIntervalsFetchedByQuery := int(cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery / cfg.SplitQueriesByInterval) + maxSplitsByFetchedDaysOfData = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - originalRangeCount + firstSplitLookbackDeltaCompensation) / (extraDaysFetchedPerShard + lookbackDeltaCompensation) + if maxSplitsByFetchedDaysOfData <= 0 { + maxSplitsByFetchedDaysOfData = 1 + } } var maxSplitsByConfig int - if cfg.SplitQueriesByIntervalMaxSplits > 0 { - maxSplitsByConfig = cfg.SplitQueriesByIntervalMaxSplits / queryVerticalShardSize + if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 { + maxSplitsByConfig = cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery / queryVerticalShardSize + if maxSplitsByConfig <= 0 { + maxSplitsByConfig = 1 + } } var maxSplits time.Duration @@ -225,70 +227,64 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) baseInterval := cfg.SplitQueriesByInterval - n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + + // Calculate the multiple of interval needed to shard query to <= maxSplits + n1 := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + if n1 <= 0 { + n1 = 1 + } + + // The first split can be truncated and not cover the full length of n*interval. + // So we remove it and calculate the multiple of interval needed to shard <= maxSplits-1 + nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n1*baseInterval) + r.GetStep() + queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) + var n2 time.Duration + if maxSplits > 1 { + n2 = (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) + } else { + // If maxSplits is <= 1 then we should not shard at all + n1 += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval + } + n := max(n1, n2) return n * cfg.SplitQueriesByInterval, nil } } -// calculates the total number of days the query will have to fetch during execution, considering the query itself, -// queryStoreAfter and lookbackDelta. -func getDaysFetchedByQuery(expr parser.Expr, reqs []tripperware.Request, queryStoreAfter, lookbackDelta time.Duration, now time.Time) int { - count := 0 +// calculates the total duration of data the query will have to fetch from storage as a multiple of baseInterval. +// also returns the total time range fetched by the original query start and end times +func durationFetchedByQuery(expr parser.Expr, req tripperware.Request, queryStoreAfter, lookbackDelta time.Duration, baseInterval time.Duration, now time.Time) (durationFetchedCount int, originalRangeCount int, lookbackDeltaCount int) { + durationFetchedCount = 0 + originalRangeCount = 0 + lookbackDeltaCount = 0 + baseIntervalMillis := util.DurationMilliseconds(baseInterval) queryStoreMaxT := util.TimeToMillis(now.Add(-queryStoreAfter)) var evalRange time.Duration - for _, req := range reqs { - var ranges []dayRange - parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { - switch n := node.(type) { - case *parser.VectorSelector: - start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), lookbackDelta, n, path, evalRange) - // Query shouldn't touch Store Gateway. - if start > queryStoreMaxT { - return nil - } else { - // If the query split needs to query store, cap the max time to now - queryStoreAfter. - end = min(end, queryStoreMaxT) - } - - startDay := start / dayMillis - endDay := end / dayMillis - ranges = append(ranges, dayRange{startDay: startDay, endDay: endDay}) - evalRange = 0 - case *parser.MatrixSelector: - evalRange = n.Range + parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + switch n := node.(type) { + case *parser.VectorSelector: + originalRangeCount += int((req.GetEnd()/baseIntervalMillis)-(req.GetStart()/baseIntervalMillis)) + 1 + start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), 0, n, path, evalRange) + // Query shouldn't touch Store Gateway. + if start > queryStoreMaxT { + return nil + } else { + // If the query split needs to query store, cap the max time to now - queryStoreAfter. + end = min(end, queryStoreMaxT) } - return nil - }) - nonOverlappingRanges := mergeDayRanges(ranges) - for _, dayRange := range nonOverlappingRanges { - count += int(dayRange.endDay-dayRange.startDay) + 1 - } - } - return count -} -func mergeDayRanges(ranges []dayRange) []dayRange { - if len(ranges) == 0 { - return ranges - } + startIntervalIndex := start / baseIntervalMillis + endIntervalIndex := end / baseIntervalMillis + durationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 - // Sort ranges by their startDay - sort.Slice(ranges, func(i, j int) bool { - return ranges[i].startDay < ranges[j].startDay - }) - - // Merge overlapping ranges - merged := []dayRange{ranges[0]} - for _, current := range ranges { - last := &merged[len(merged)-1] - if current.startDay <= last.endDay { - if current.endDay > last.endDay { - last.endDay = current.endDay + if evalRange == 0 && (start-util.DurationMilliseconds(lookbackDelta))/baseIntervalMillis == start/baseIntervalMillis { + lookbackDeltaCount += 1 } - } else { - merged = append(merged, current) + evalRange = 0 + case *parser.MatrixSelector: + evalRange = n.Range } - } - return merged + return nil + }) + return durationFetchedCount, originalRangeCount, lookbackDeltaCount } diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 186ce9824b..5e87dc1a3b 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -416,13 +416,13 @@ func Test_evaluateAtModifier(t *testing.T) { func TestDynamicIntervalFn(t *testing.T) { for _, tc := range []struct { - name string - baseSplitInterval time.Duration - req tripperware.Request - expectedInterval time.Duration - expectedError bool - maxQueryIntervalSplits int - maxDaysOfDataFetched int + name string + baseSplitInterval time.Duration + req tripperware.Request + expectedInterval time.Duration + expectedError bool + maxQueryIntervalSplits int + maxDurationOfDataFetched time.Duration }{ { baseSplitInterval: day, @@ -433,14 +433,49 @@ func TestDynamicIntervalFn(t *testing.T) { End: 10 * 24 * 3600 * seconds, Step: 5 * 60 * seconds, }, - maxQueryIntervalSplits: 30, - maxDaysOfDataFetched: 200, - expectedInterval: day, - expectedError: true, + maxQueryIntervalSplits: 30, + maxDurationOfDataFetched: 200 * day, + expectedInterval: day, + expectedError: true, + }, + { + baseSplitInterval: time.Hour, + name: "48 hour, expect split by 1 hour", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 2 * 24 * 3600 * seconds, + Step: 60 * seconds, + Query: "up", + }, + expectedInterval: time.Hour, + }, + { + baseSplitInterval: time.Hour, + name: "5 days with 15 max splits, expect split by 8 hours", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 5 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up", + }, + maxQueryIntervalSplits: 15, + expectedInterval: 8 * time.Hour, + }, + { + baseSplitInterval: 2 * time.Hour, + name: "4 days with 200 hour max duration fetched, expect split by 4 hours", + req: &tripperware.PrometheusRequest{ + Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), + End: 7*24*3600*seconds + (2*3600*seconds + 60*seconds), + Step: 5 * 60 * seconds, + Query: "up[5m]", + }, + maxDurationOfDataFetched: 200 * time.Hour, + expectedInterval: 4 * time.Hour, }, { baseSplitInterval: day, - name: "30 day range no limits, expect split by 1 day", + name: "30 day range, expect split by 1 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 30 * 24 * 3600 * seconds, @@ -463,7 +498,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "60 day range, expect split by 4 day", + name: "60 day range with 15 max splits, expect split by 4 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 60 * 24 * 3600 * seconds, @@ -475,7 +510,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "61 day range, expect split by 5 day", + name: "61 day range with 15 max splits, expect split by 5 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 61 * 24 * 3600 * seconds, @@ -487,61 +522,76 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "30 day range short matrix selector with 200 days fetched limit, expect split by 1 day", + name: "30 day range short matrix selector with 200 days max duration fetched, expect split by 1 day", req: &tripperware.PrometheusRequest{ Start: 30 * 24 * 3600 * seconds, End: 60 * 24 * 3600 * seconds, Step: 5 * 60 * seconds, Query: "avg_over_time(up[1h])", }, - maxDaysOfDataFetched: 200, - expectedInterval: day, + maxDurationOfDataFetched: 200 * day, + expectedInterval: day, }, { baseSplitInterval: day, - name: "30 day range long matrix selector with 200 days fetched limit, expect split by 1 day", + name: "30 day range long matrix selector with 200 days max duration fetched, expect split by 4 days", req: &tripperware.PrometheusRequest{ Start: 30 * 24 * 3600 * seconds, End: 60 * 24 * 3600 * seconds, Step: 5 * 60 * seconds, Query: "avg_over_time(up[20d])", }, - maxDaysOfDataFetched: 200, - expectedInterval: 4 * day, + maxDurationOfDataFetched: 200 * day, + expectedInterval: 4 * day, }, { baseSplitInterval: day, - name: "60 day range, expect split by 7 day", + name: "query with multiple matrix selectors, expect split by 10 day", req: &tripperware.PrometheusRequest{ - Start: (2 * 24 * 3600 * seconds) + (3600*seconds - 120), - End: (61 * 24 * 3600 * seconds) + (2*3600*seconds + 500), - Step: 30 * 60 * seconds, - Query: "rate(up[1d]) + rate(up[2d]) + rate(up[5d]) + rate(up[15d])", + Start: (14 * 24 * 3600 * seconds) + (3600*seconds - 120*seconds), + End: (52 * 24 * 3600 * seconds) + (2*3600*seconds + 500*seconds), + Step: 60 * seconds, + Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", }, - maxDaysOfDataFetched: 200, - expectedInterval: 7 * day, + maxDurationOfDataFetched: 200 * day, + expectedInterval: 10 * day, }, { baseSplitInterval: day, - name: "30 day range, expect split by 7 day", + name: "30 day range with subquery, expect split by 13 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 100 * 24 * 3600 * seconds, Step: 60 * 60 * seconds, Query: "up[5d:10m]", }, - maxQueryIntervalSplits: 100, - maxDaysOfDataFetched: 300, - expectedInterval: 4 * day, + maxQueryIntervalSplits: 100, + maxDurationOfDataFetched: 150 * day, + expectedInterval: 13 * day, + }, + { + baseSplitInterval: 2 * time.Hour, + name: "duration of data fetched is much larger than config, expect large interval and no sharding", + req: &tripperware.PrometheusRequest{ + Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), + End: 7*24*3600*seconds + (1*3600*seconds + 60*seconds), + Step: 5 * 60 * seconds, + Query: "up[5m]", + }, + maxDurationOfDataFetched: 50 * time.Hour, + expectedInterval: 170 * time.Hour, }, } { t.Run(tc.name, func(t *testing.T) { cfg := Config{ - SplitQueriesByInterval: tc.baseSplitInterval, - SplitQueriesByIntervalMaxSplits: tc.maxQueryIntervalSplits, + SplitQueriesByInterval: tc.baseSplitInterval, + DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ + MaxShardsPerQuery: tc.maxQueryIntervalSplits, + MaxDurationOfDataFetchedFromStoragePerQuery: tc.maxDurationOfDataFetched, + }, } ctx := user.InjectOrgID(context.Background(), "1") - interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), queryStoreAfter, lookbackDelta, tc.maxDaysOfDataFetched)(ctx, tc.req) + interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), queryStoreAfter, lookbackDelta)(ctx, tc.req) require.Equal(t, tc.expectedInterval, interval) if !tc.expectedError { require.Nil(t, err) diff --git a/pkg/util/time.go b/pkg/util/time.go index 2656065650..5e52493292 100644 --- a/pkg/util/time.go +++ b/pkg/util/time.go @@ -245,6 +245,7 @@ func DurationMilliseconds(d time.Duration) int64 { return int64(d / (time.Millisecond / time.Nanosecond)) } +// Copied from https://github.com/prometheus/prometheus/blob/dfae954dc1137568f33564e8cffda321f2867925/promql/engine.go#L811 func GetTimeRangesForSelector(start, end int64, lookbackDelta time.Duration, n *parser.VectorSelector, path []parser.Node, evalRange time.Duration) (int64, int64) { subqOffset, subqRange, subqTs := subqueryTimes(path) @@ -279,6 +280,7 @@ func GetTimeRangesForSelector(start, end int64, lookbackDelta time.Duration, n * return start, end } +// Copied from https://github.com/prometheus/prometheus/blob/dfae954dc1137568f33564e8cffda321f2867925/promql/engine.go#L754 // subqueryTimes returns the sum of offsets and ranges of all subqueries in the path. // If the @ modifier is used, then the offset and range is w.r.t. that timestamp // (i.e. the sum is reset when we have @ modifier). From 08ec706415dccb45a0415940c342854c6202ab06 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 22 Jan 2025 11:44:36 -0800 Subject: [PATCH 06/20] add dynamic query splits to experimental features Signed-off-by: Ahmed Hassan --- docs/configuration/config-file-reference.md | 3 ++- docs/configuration/v1-guarantees.md | 3 +++ pkg/querier/tripperware/queryrange/dynamic_query_splits.go | 2 +- pkg/querier/tripperware/queryrange/split_by_interval.go | 2 +- 4 files changed, 7 insertions(+), 3 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 747571ac31..97e0568e05 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4205,7 +4205,8 @@ dynamic_query_splits: # [EXPERIMENTAL] Max total duration of data fetched by all query shards from # storage, 0 disables it. Dynamically uses a multiple of # `split-queries-by-interval` to ensure the total fetched duration of data is - # lower than the value set. + # lower than the value set. It takes into account additional data fetched by + # matrix selectors and subqueries. # CLI flag: -querier.max-duration-of-data-fetched-from-storage-per-query [max_duration_of_data_fetched_from_storage_per_query: | default = 0s] diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index 94da14fb10..07915432d5 100644 --- a/docs/configuration/v1-guarantees.md +++ b/docs/configuration/v1-guarantees.md @@ -118,3 +118,6 @@ Currently experimental features are: - Enable string interning for metrics labels by setting `-ingester.labels-string-interning-enabled` on Ingester. - Query-frontend: query rejection (`-frontend.query-rejection.enabled`) - Querier: protobuf codec (`-api.querier-default-codec`) +- Query-frontend: dynamic query splits + - `querier.max-shards-per-query` (int) CLI flag + - `querier.max-duration-of-data-fetched-from-storage-per-query` (duration) CLI flag diff --git a/pkg/querier/tripperware/queryrange/dynamic_query_splits.go b/pkg/querier/tripperware/queryrange/dynamic_query_splits.go index 1120967365..35aeeaf1d5 100644 --- a/pkg/querier/tripperware/queryrange/dynamic_query_splits.go +++ b/pkg/querier/tripperware/queryrange/dynamic_query_splits.go @@ -13,5 +13,5 @@ type DynamicQuerySplitsConfig struct { // RegisterFlags registers flags foy dynamic query splits func (cfg *DynamicQuerySplitsConfig) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxShardsPerQuery, "querier.max-shards-per-query", 0, "[EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to maintain the number of splits below the limit. If vertical sharding is enabled for a query, the combined total number of vertical and interval shards is kept below this limit.") - f.DurationVar(&cfg.MaxDurationOfDataFetchedFromStoragePerQuery, "querier.max-duration-of-data-fetched-from-storage-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched by all query shards from storage, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to ensure the total fetched duration of data is lower than the value set.") + f.DurationVar(&cfg.MaxDurationOfDataFetchedFromStoragePerQuery, "querier.max-duration-of-data-fetched-from-storage-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched by all query shards from storage, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to ensure the total fetched duration of data is lower than the value set. It takes into account additional data fetched by matrix selectors and subqueries.") } diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 038b5b04f8..f4f079e32f 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -57,7 +57,7 @@ func (s splitByInterval) Do(ctx context.Context, r tripperware.Request) (tripper // to line up the boundaries with step. interval, err := s.interval(ctx, r) if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + return nil, httpgrpc.Errorf(http.StatusInternalServerError, err.Error()) } reqs, err := splitQuery(r, interval) if err != nil { From 7349cfa17755177c0e1991e1d0113d807d905dbc Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Thu, 23 Jan 2025 10:16:00 -0800 Subject: [PATCH 07/20] rename dynamicIntervalFn unit tests Signed-off-by: Ahmed Hassan --- .../tripperware/queryrange/split_by_interval_test.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 5e87dc1a3b..52ce249b98 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -451,7 +451,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: time.Hour, - name: "5 days with 15 max splits, expect split by 8 hours", + name: "120 hour with 15 max splits, expect split by 8 hours", req: &tripperware.PrometheusRequest{ Start: 0, End: 5 * 24 * 3600 * seconds, @@ -463,10 +463,10 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: 2 * time.Hour, - name: "4 days with 200 hour max duration fetched, expect split by 4 hours", + name: "102 hours with 200 hour max duration fetched, expect split by 4 hours", req: &tripperware.PrometheusRequest{ Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), - End: 7*24*3600*seconds + (2*3600*seconds + 60*seconds), + End: (7 * 24 * 3600 * seconds) + (2*3600*seconds + 60*seconds), Step: 5 * 60 * seconds, Query: "up[5m]", }, @@ -558,7 +558,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "30 day range with subquery, expect split by 13 day", + name: "100 day range with subquery, expect split by 13 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 100 * 24 * 3600 * seconds, @@ -574,9 +574,9 @@ func TestDynamicIntervalFn(t *testing.T) { name: "duration of data fetched is much larger than config, expect large interval and no sharding", req: &tripperware.PrometheusRequest{ Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), - End: 7*24*3600*seconds + (1*3600*seconds + 60*seconds), + End: (7 * 24 * 3600 * seconds) + (1*3600*seconds + 60*seconds), Step: 5 * 60 * seconds, - Query: "up[5m]", + Query: "up[5h]", }, maxDurationOfDataFetched: 50 * time.Hour, expectedInterval: 170 * time.Hour, From 3e95b45aa9e2e9a368fb42e5508bc9cd8baf7469 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 24 Jan 2025 13:32:09 -0800 Subject: [PATCH 08/20] refactor dynamicIntervalFn to be more readable Signed-off-by: Ahmed Hassan --- pkg/cortex/modules.go | 1 - .../queryrange/query_range_middlewares.go | 5 +- .../query_range_middlewares_test.go | 1 - .../queryrange/split_by_interval.go | 228 ++++++++++-------- .../queryrange/split_by_interval_test.go | 4 +- 5 files changed, 138 insertions(+), 101 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 1a9ad3d415..a771c22116 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -487,7 +487,6 @@ func (t *Cortex) initQueryFrontendTripperware() (serv services.Service, err erro prometheusCodec, shardedPrometheusCodec, t.Cfg.Querier.LookbackDelta, - t.Cfg.Querier.QueryStoreAfter, ) if err != nil { return nil, err diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 65c8b7712e..b4ed9653e5 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -89,7 +89,6 @@ func Middlewares( prometheusCodec tripperware.Codec, shardedPrometheusCodec tripperware.Codec, lookbackDelta time.Duration, - queryStoreAfter time.Duration, ) ([]tripperware.Middleware, cache.Cache, error) { // Metric used to keep track of each middleware execution duration. metrics := tripperware.NewInstrumentMiddlewareMetrics(registerer) @@ -101,9 +100,9 @@ func Middlewares( if cfg.SplitQueriesByInterval != 0 { intervalFn := staticIntervalFn(cfg) if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { - intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, queryStoreAfter, lookbackDelta) + intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, lookbackDelta) } - queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, queryStoreAfter, lookbackDelta)) + queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, lookbackDelta)) } var c cache.Cache diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go index 14ccfd0ea7..c8238a9a62 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares_test.go @@ -64,7 +64,6 @@ func TestRoundTrip(t *testing.T) { PrometheusCodec, ShardedPrometheusCodec, 5*time.Minute, - 24*time.Hour, ) require.NoError(t, err) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index f4f079e32f..edc7686ab5 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -21,7 +21,7 @@ import ( type IntervalFn func(ctx context.Context, r tripperware.Request) (time.Duration, error) // SplitByIntervalMiddleware creates a new Middleware that splits requests by a given interval. -func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, merger tripperware.Merger, registerer prometheus.Registerer, queryStoreAfter time.Duration, lookbackDelta time.Duration) tripperware.Middleware { +func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, merger tripperware.Merger, registerer prometheus.Registerer, lookbackDelta time.Duration) tripperware.Middleware { return tripperware.MiddlewareFunc(func(next tripperware.Handler) tripperware.Handler { return splitByInterval{ next: next, @@ -33,23 +33,20 @@ func SplitByIntervalMiddleware(interval IntervalFn, limits tripperware.Limits, m Name: "frontend_split_queries_total", Help: "Total number of underlying query requests after the split by interval is applied", }), - queryStoreAfter: queryStoreAfter, - lookbackDelta: lookbackDelta, + lookbackDelta: lookbackDelta, } }) } type splitByInterval struct { - next tripperware.Handler - limits tripperware.Limits - merger tripperware.Merger - interval IntervalFn + next tripperware.Handler + limits tripperware.Limits + merger tripperware.Merger + interval IntervalFn + lookbackDelta time.Duration // Metrics. splitByCounter prometheus.Counter - - queryStoreAfter time.Duration - lookbackDelta time.Duration } func (s splitByInterval) Do(ctx context.Context, r tripperware.Request) (tripperware.Response, error) { @@ -160,125 +157,159 @@ func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Reques } } -func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, queryStoreAfter time.Duration, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { +func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { - tenantIDs, err := tenant.TenantIDs(ctx) - if err != nil { - return cfg.SplitQueriesByInterval, err - } + baseInterval := cfg.SplitQueriesByInterval + maxDurationFetchedConfig := cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery + maxSplitsConfig := cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery - analysis, err := queryAnalyzer.Analyze(r.GetQuery()) + queryVerticalShardSize, err := getMaxVerticalShardSizeForQuery(ctx, r, limits, queryAnalyzer) if err != nil { - return cfg.SplitQueriesByInterval, err - } - - queryVerticalShardSize := validation.SmallestPositiveIntPerTenant(tenantIDs, limits.QueryVerticalShardSize) - if queryVerticalShardSize <= 0 || !analysis.IsShardable() { - queryVerticalShardSize = 1 + return baseInterval, err } queryExpr, err := parser.ParseExpr(r.GetQuery()) if err != nil { - return cfg.SplitQueriesByInterval, err + return baseInterval, err } - // Calculates: duration of data fetched if the query was not sharded, the original range covered by the query start and end times, - // and the duration of data fetched by lookbackDelta for the first split - durationFetchedWithoutSharding, originalRangeCount, firstSplitLookbackDeltaCompensation := durationFetchedByQuery(queryExpr, r, queryStoreAfter, lookbackDelta, cfg.SplitQueriesByInterval, time.Now()) - extraDaysFetchedPerShard := durationFetchedWithoutSharding - originalRangeCount + // First we analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split + queryRangeIntervalsCount, extraIntervalsPerSplitCount, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r, baseInterval, lookbackDelta) + if extraIntervalsPerSplitCount == 0 { + extraIntervalsPerSplitCount = 1 // prevent division by 0 + } - // Calculate the extra duration of data fetched by lookbackDelta per each split except the first split - nextIntervalStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), cfg.SplitQueriesByInterval) + r.GetStep() + // Next we analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for all subsequent splits + nextIntervalStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), baseInterval) + r.GetStep() nextIntervalReq := r.WithStartEnd(nextIntervalStart, r.GetEnd()) - _, _, lookbackDeltaCompensation := durationFetchedByQuery(queryExpr, nextIntervalReq, queryStoreAfter, lookbackDelta, cfg.SplitQueriesByInterval, time.Now()) + _, _, otherSplitsLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, nextIntervalReq, baseInterval, lookbackDelta) - var maxSplitsByFetchedDaysOfData int - if cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { - if extraDaysFetchedPerShard == 0 { - extraDaysFetchedPerShard = 1 // prevent division by 0 - } - maxIntervalsFetchedByQuery := int(cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery / cfg.SplitQueriesByInterval) - maxSplitsByFetchedDaysOfData = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - originalRangeCount + firstSplitLookbackDeltaCompensation) / (extraDaysFetchedPerShard + lookbackDeltaCompensation) - if maxSplitsByFetchedDaysOfData <= 0 { - maxSplitsByFetchedDaysOfData = 1 + // By default we subtract the 'first split' duration fetched by loookbackDelta, and divide by the 'other splits' duration fetched by loookbackDelta. + if firstSplitLookbackDeltaIntervals > 0 && otherSplitsLookbackDeltaIntervals > 0 { + firstSplitLookbackDeltaIntervals = 0 // Dividing is enough if additional duration is fetched by loookbackDelta for all splits + } else if otherSplitsLookbackDeltaIntervals > 0 { + firstSplitLookbackDeltaIntervals = otherSplitsLookbackDeltaIntervals * -1 // Adding instead of subtracting for first split, if additional duration is fetched by loookbackDelta for all splits except first one + } + + var maxSplitsByDurationFetched int + if maxDurationFetchedConfig > 0 { + maxIntervalsFetchedByQuery := int(maxDurationFetchedConfig / baseInterval) + maxSplitsByDurationFetched = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - queryRangeIntervalsCount - firstSplitLookbackDeltaIntervals) / (extraIntervalsPerSplitCount + otherSplitsLookbackDeltaIntervals) + if maxSplitsByDurationFetched <= 0 { + maxSplitsByDurationFetched = 1 } } var maxSplitsByConfig int - if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 { - maxSplitsByConfig = cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery / queryVerticalShardSize + if maxSplitsConfig > 0 { + maxSplitsByConfig = maxSplitsConfig / queryVerticalShardSize if maxSplitsByConfig <= 0 { maxSplitsByConfig = 1 } } - var maxSplits time.Duration + var maxSplits int switch { - case maxSplitsByFetchedDaysOfData <= 0 && maxSplitsByConfig <= 0: - return cfg.SplitQueriesByInterval, nil - case maxSplitsByFetchedDaysOfData <= 0: - maxSplits = time.Duration(maxSplitsByConfig) - case maxSplitsByConfig <= 0: - maxSplits = time.Duration(maxSplitsByFetchedDaysOfData) - default: + case maxDurationFetchedConfig > 0 && maxSplitsConfig > 0: // Use the more restricting shard limit - maxSplits = time.Duration(min(maxSplitsByConfig, maxSplitsByFetchedDaysOfData)) + maxSplits = min(maxSplitsByConfig, maxSplitsByDurationFetched) + case maxSplitsConfig > 0: + maxSplits = maxSplitsByConfig + case maxDurationFetchedConfig > 0: + maxSplits = maxSplitsByDurationFetched + default: + return baseInterval, nil } - queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - baseInterval := cfg.SplitQueriesByInterval + interval := getIntervalFromMaxSplits(r, baseInterval, maxSplits) + return interval, nil + } +} - // Calculate the multiple of interval needed to shard query to <= maxSplits - n1 := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) - if n1 <= 0 { - n1 = 1 - } +func getMaxVerticalShardSizeForQuery(ctx context.Context, r tripperware.Request, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer) (int, error) { + tenantIDs, err := tenant.TenantIDs(ctx) + if err != nil { + return 1, err + } - // The first split can be truncated and not cover the full length of n*interval. - // So we remove it and calculate the multiple of interval needed to shard <= maxSplits-1 - nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n1*baseInterval) + r.GetStep() - queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) - var n2 time.Duration - if maxSplits > 1 { - n2 = (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) - } else { - // If maxSplits is <= 1 then we should not shard at all - n1 += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval - } - n := max(n1, n2) - return n * cfg.SplitQueriesByInterval, nil + analysis, err := queryAnalyzer.Analyze(r.GetQuery()) + if err != nil { + return 1, err + } + + queryVerticalShardSize := validation.SmallestPositiveIntPerTenant(tenantIDs, limits.QueryVerticalShardSize) + if queryVerticalShardSize <= 0 || !analysis.IsShardable() { + queryVerticalShardSize = 1 + } + return queryVerticalShardSize, nil +} + +func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplits int) time.Duration { + maxSplitsDuration := time.Duration(maxSplits) + queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) + + // Calculate the multiple of interval needed to shard query to <= maxSplits + n1 := (queryRange + baseInterval*maxSplitsDuration - 1) / (baseInterval * maxSplitsDuration) + if n1 <= 0 { + n1 = 1 } + + // The first split can be truncated and not cover the full length of n*interval. + // So we remove it and calculate the multiple of interval needed to shard <= maxSplits-1 + nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n1*baseInterval) + r.GetStep() + queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) + var n2 time.Duration + if maxSplitsDuration > 1 { + n2 = (queryRangeWithoutFirstSplit + baseInterval*(maxSplitsDuration-1) - 1) / (baseInterval * (maxSplitsDuration - 1)) + } else { + // If maxSplits is <= 1 then we should not shard at all + n1 += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval + } + n := max(n1, n2) + return n * baseInterval } -// calculates the total duration of data the query will have to fetch from storage as a multiple of baseInterval. -// also returns the total time range fetched by the original query start and end times -func durationFetchedByQuery(expr parser.Expr, req tripperware.Request, queryStoreAfter, lookbackDelta time.Duration, baseInterval time.Duration, now time.Time) (durationFetchedCount int, originalRangeCount int, lookbackDeltaCount int) { - durationFetchedCount = 0 - originalRangeCount = 0 - lookbackDeltaCount = 0 +// Analyzes the query to identify variables useful for calculating the duration of data +// that will be fetched from storage when the query is executed after being split. +// All variables are expressed as a count of multiples of the base split interval. +// +// Returns: +// - queryRangeIntervalsCount: The total count of intervals fetched by the original start-end +// range of the query. This value is constant and does not depend on the number of splits. +// - extraIntervalsPerSplitCount: The count of additional intervals fetched by matrix selectors +// or subqueries. This value will be fetched once for every split. +// - lookbackDeltaIntervalsCount: The count of additional intervals fetched by the lookbackDelta +// for the specified start time. +// +// Example: +// Query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta +// - queryRangeIntervalsCount = 30 +// - extraIntervalsPerSplitCount = 15 +// - lookbackDeltaIntervalsCount = 1 +func analyzeDurationFetchedByQuery(expr parser.Expr, req tripperware.Request, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervalsCount int, extraIntervalsPerSplitCount int, lookbackDeltaIntervalsCount int) { + queryRangeIntervalsCount = 0 + lookbackDeltaIntervalsCount = 0 baseIntervalMillis := util.DurationMilliseconds(baseInterval) - queryStoreMaxT := util.TimeToMillis(now.Add(-queryStoreAfter)) - var evalRange time.Duration + totalDurationFetchedCount := 0 + var evalRange time.Duration parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { switch n := node.(type) { case *parser.VectorSelector: - originalRangeCount += int((req.GetEnd()/baseIntervalMillis)-(req.GetStart()/baseIntervalMillis)) + 1 - start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), 0, n, path, evalRange) - // Query shouldn't touch Store Gateway. - if start > queryStoreMaxT { - return nil - } else { - // If the query split needs to query store, cap the max time to now - queryStoreAfter. - end = min(end, queryStoreMaxT) - } + // Increment count of intervals fetched by the original start-end time range + queryRangeIntervalsCount += int((req.GetEnd()/baseIntervalMillis)-(req.GetStart()/baseIntervalMillis)) + 1 - startIntervalIndex := start / baseIntervalMillis - endIntervalIndex := end / baseIntervalMillis - durationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 - - if evalRange == 0 && (start-util.DurationMilliseconds(lookbackDelta))/baseIntervalMillis == start/baseIntervalMillis { - lookbackDeltaCount += 1 + // Adjust start and end time based on matrix selectors or subquery, this excludes lookbackDelta + start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), 0, n, path, evalRange) + startIntervalIndex := floorDiv(start, baseIntervalMillis) + endIntervalIndex := floorDiv(end, baseIntervalMillis) + totalDurationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 + + // Adjust start time based on lookbackDelta and increment the additional count of intervals fetched by it + startLookbackDelta := start - util.DurationMilliseconds(lookbackDelta) + startLookbackDeltaIntervalIndex := floorDiv(startLookbackDelta, baseIntervalMillis) + if evalRange == 0 && startLookbackDeltaIntervalIndex < startIntervalIndex { + lookbackDeltaIntervalsCount += int(startIntervalIndex - startLookbackDeltaIntervalIndex) } evalRange = 0 case *parser.MatrixSelector: @@ -286,5 +317,14 @@ func durationFetchedByQuery(expr parser.Expr, req tripperware.Request, queryStor } return nil }) - return durationFetchedCount, originalRangeCount, lookbackDeltaCount + extraIntervalsPerSplitCount = totalDurationFetchedCount - queryRangeIntervalsCount + + return queryRangeIntervalsCount, extraIntervalsPerSplitCount, lookbackDeltaIntervalsCount +} + +func floorDiv(a, b int64) int64 { + if a < 0 && a%b != 0 { + return a/b - 1 + } + return a / b } diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 52ce249b98..9f87bd3013 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -314,7 +314,7 @@ func TestSplitByDay(t *testing.T) { roundtripper := tripperware.NewRoundTripper(singleHostRoundTripper{ host: u.Host, next: http.DefaultTransport, - }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(interval, mockLimits{}, PrometheusCodec, nil, queryStoreAfter, lookbackDelta)) + }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(interval, mockLimits{}, PrometheusCodec, nil, lookbackDelta)) req, err := http.NewRequest("GET", tc.path, http.NoBody) require.NoError(t, err) @@ -591,7 +591,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, } ctx := user.InjectOrgID(context.Background(), "1") - interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), queryStoreAfter, lookbackDelta)(ctx, tc.req) + interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta)(ctx, tc.req) require.Equal(t, tc.expectedInterval, interval) if !tc.expectedError { require.Nil(t, err) From cfc50780284d5a54efc234beb4475818b00b46e3 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 27 Jan 2025 00:37:09 -0800 Subject: [PATCH 09/20] add unit tests for getIntervalFromMaxSplits Signed-off-by: Ahmed Hassan --- .../queryrange/dynamic_query_splits.go | 17 -- .../tripperware/queryrange/limits_test.go | 9 +- .../queryrange/query_range_middlewares.go | 15 +- .../queryrange/split_by_interval.go | 77 +++++--- .../queryrange/split_by_interval_test.go | 183 +++++++++++++++++- 5 files changed, 236 insertions(+), 65 deletions(-) delete mode 100644 pkg/querier/tripperware/queryrange/dynamic_query_splits.go diff --git a/pkg/querier/tripperware/queryrange/dynamic_query_splits.go b/pkg/querier/tripperware/queryrange/dynamic_query_splits.go deleted file mode 100644 index 35aeeaf1d5..0000000000 --- a/pkg/querier/tripperware/queryrange/dynamic_query_splits.go +++ /dev/null @@ -1,17 +0,0 @@ -package queryrange - -import ( - "flag" - "time" -) - -type DynamicQuerySplitsConfig struct { - MaxShardsPerQuery int `yaml:"max_shards_per_query"` - MaxDurationOfDataFetchedFromStoragePerQuery time.Duration `yaml:"max_duration_of_data_fetched_from_storage_per_query"` -} - -// RegisterFlags registers flags foy dynamic query splits -func (cfg *DynamicQuerySplitsConfig) RegisterFlags(f *flag.FlagSet) { - f.IntVar(&cfg.MaxShardsPerQuery, "querier.max-shards-per-query", 0, "[EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to maintain the number of splits below the limit. If vertical sharding is enabled for a query, the combined total number of vertical and interval shards is kept below this limit.") - f.DurationVar(&cfg.MaxDurationOfDataFetchedFromStoragePerQuery, "querier.max-duration-of-data-fetched-from-storage-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched by all query shards from storage, 0 disables it. Dynamically uses a multiple of `split-queries-by-interval` to ensure the total fetched duration of data is lower than the value set. It takes into account additional data fetched by matrix selectors and subqueries.") -} diff --git a/pkg/querier/tripperware/queryrange/limits_test.go b/pkg/querier/tripperware/queryrange/limits_test.go index 6c3e257986..09cfe77b98 100644 --- a/pkg/querier/tripperware/queryrange/limits_test.go +++ b/pkg/querier/tripperware/queryrange/limits_test.go @@ -233,9 +233,10 @@ func TestLimitsMiddleware_MaxQueryLength(t *testing.T) { } type mockLimits struct { - maxQueryLookback time.Duration - maxQueryLength time.Duration - maxCacheFreshness time.Duration + maxQueryLookback time.Duration + maxQueryLength time.Duration + maxCacheFreshness time.Duration + queryVerticalShardSize int } func (m mockLimits) MaxQueryLookback(string) time.Duration { @@ -255,7 +256,7 @@ func (m mockLimits) MaxCacheFreshness(string) time.Duration { } func (m mockLimits) QueryVerticalShardSize(userID string) int { - return 0 + return m.queryVerticalShardSize } func (m mockLimits) QueryPriority(userID string) validation.QueryPriority { diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index b4ed9653e5..0f0bb1b64f 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -70,7 +70,7 @@ func (cfg *Config) Validate(qCfg querier.Config) error { return errors.Wrap(err, "invalid ResultsCache config") } } - if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { + if cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery > 0 { if cfg.SplitQueriesByInterval <= 0 { return errors.New("configs under dynamic-query-splits requires that a value for split-queries-by-interval is set.") } @@ -78,6 +78,17 @@ func (cfg *Config) Validate(qCfg querier.Config) error { return nil } +type DynamicQuerySplitsConfig struct { + MaxSplitsPerQuery int `yaml:"max_splits_per_query"` + MaxFetchedStorageDataDurationPerQuery time.Duration `yaml:"max_fetched_storage_data_duration_per_query"` +} + +// RegisterFlags registers flags foy dynamic query splits +func (cfg *DynamicQuerySplitsConfig) RegisterFlags(f *flag.FlagSet) { + f.IntVar(&cfg.MaxSplitsPerQuery, "querier.max-splits-per-query", 0, "[EXPERIMENTAL] Maximum number of splits for a query, 0 disables it. Dynamically uses a multiple of split interval to maintain a total number of splits below the set value. If vertical sharding is enabled for a query, the combined total number of vertical and interval splits is kept below this value.") + f.DurationVar(&cfg.MaxFetchedStorageDataDurationPerQuery, "querier.max-fetched-storage-data-duration-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched from storage by all query splits, 0 disables it. Dynamically uses a multiple of split interval to maintain a total fetched duration of data lower than the value set. It takes into account additional duration fetched by matrix selectors and subqueries.") +} + // Middlewares returns list of middlewares that should be applied for range query. func Middlewares( cfg Config, @@ -99,7 +110,7 @@ func Middlewares( } if cfg.SplitQueriesByInterval != 0 { intervalFn := staticIntervalFn(cfg) - if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery > 0 { + if cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery > 0 { intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, lookbackDelta) } queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, lookbackDelta)) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index edc7686ab5..7b7fd9fb89 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -160,8 +160,8 @@ func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Reques func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { baseInterval := cfg.SplitQueriesByInterval - maxDurationFetchedConfig := cfg.DynamicQuerySplitsConfig.MaxDurationOfDataFetchedFromStoragePerQuery - maxSplitsConfig := cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery + maxDurationFetchedConfig := cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery + maxSplitsConfig := cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery queryVerticalShardSize, err := getMaxVerticalShardSizeForQuery(ctx, r, limits, queryAnalyzer) if err != nil { @@ -173,33 +173,37 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer return baseInterval, err } - // First we analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split - queryRangeIntervalsCount, extraIntervalsPerSplitCount, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r, baseInterval, lookbackDelta) + // First analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split + queryRangeIntervalsCount, extraIntervalsPerSplitCount, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r.GetStart(), r.GetEnd(), baseInterval, lookbackDelta) if extraIntervalsPerSplitCount == 0 { - extraIntervalsPerSplitCount = 1 // prevent division by 0 + extraIntervalsPerSplitCount = 1 // avoid division by 0 } - // Next we analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for all subsequent splits + // Next analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for other subsequent splits nextIntervalStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), baseInterval) + r.GetStep() - nextIntervalReq := r.WithStartEnd(nextIntervalStart, r.GetEnd()) - _, _, otherSplitsLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, nextIntervalReq, baseInterval, lookbackDelta) + _, _, otherSplitsLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, nextIntervalStart, r.GetEnd(), baseInterval, lookbackDelta) - // By default we subtract the 'first split' duration fetched by loookbackDelta, and divide by the 'other splits' duration fetched by loookbackDelta. + // By default subtract the 'first split' duration fetched by loookbackDelta, and divide by the 'other splits' duration fetched by loookbackDelta. if firstSplitLookbackDeltaIntervals > 0 && otherSplitsLookbackDeltaIntervals > 0 { firstSplitLookbackDeltaIntervals = 0 // Dividing is enough if additional duration is fetched by loookbackDelta for all splits } else if otherSplitsLookbackDeltaIntervals > 0 { firstSplitLookbackDeltaIntervals = otherSplitsLookbackDeltaIntervals * -1 // Adding instead of subtracting for first split, if additional duration is fetched by loookbackDelta for all splits except first one } + // Find the max number of splits that will fetch less than MaxFetchedStorageDataDurationPerQuery var maxSplitsByDurationFetched int if maxDurationFetchedConfig > 0 { maxIntervalsFetchedByQuery := int(maxDurationFetchedConfig / baseInterval) + // Equation for max duration fetched by example query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta + // MaxFetchedStorageDataDurationPerQuery > (30 + ((15 + 1) x horizontal splits)) x vertical shards + // Rearranging the equation to find the max horizontal splits maxSplitsByDurationFetched = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - queryRangeIntervalsCount - firstSplitLookbackDeltaIntervals) / (extraIntervalsPerSplitCount + otherSplitsLookbackDeltaIntervals) if maxSplitsByDurationFetched <= 0 { maxSplitsByDurationFetched = 1 } } + // Find max number of splits from MaxSplitsPerQuery after accounting for vertical sharding var maxSplitsByConfig int if maxSplitsConfig > 0 { maxSplitsByConfig = maxSplitsConfig / queryVerticalShardSize @@ -244,34 +248,43 @@ func getMaxVerticalShardSizeForQuery(ctx context.Context, r tripperware.Request, return queryVerticalShardSize, nil } -func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplits int) time.Duration { - maxSplitsDuration := time.Duration(maxSplits) +func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplitsInt int) time.Duration { + maxSplits := time.Duration(maxSplitsInt) queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - // Calculate the multiple of interval needed to shard query to <= maxSplits - n1 := (queryRange + baseInterval*maxSplitsDuration - 1) / (baseInterval * maxSplitsDuration) - if n1 <= 0 { - n1 = 1 + // Calculate the multiple n of interval needed to shard query to <= maxSplits + n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + if n <= 0 { + n = 1 } - // The first split can be truncated and not cover the full length of n*interval. - // So we remove it and calculate the multiple of interval needed to shard <= maxSplits-1 - nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n1*baseInterval) + r.GetStep() - queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) - var n2 time.Duration - if maxSplitsDuration > 1 { - n2 = (queryRangeWithoutFirstSplit + baseInterval*(maxSplitsDuration-1) - 1) / (baseInterval * (maxSplitsDuration - 1)) + if maxSplits == 1 || queryRange < baseInterval { + // No splitting, interval should be long enough to result in 1 split only + nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() + if nextSplitStart < r.GetEnd() { + queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) + n += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval + } } else { - // If maxSplits is <= 1 then we should not shard at all - n1 += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval + for n <= 2*(queryRange/baseInterval) { + // The first split can be truncated and shorter than other splits. + // So it is removed to check if a larger interval is needed to shard <= maxSplits-1 + nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() + queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) + n_temp := (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) + if n >= n_temp { + break + } + n++ + } } - n := max(n1, n2) return n * baseInterval } -// Analyzes the query to identify variables useful for calculating the duration of data -// that will be fetched from storage when the query is executed after being split. -// All variables are expressed as a count of multiples of the base split interval. +// analyzeDurationFetchedByQuery analyzes the query to identify variables useful for +// calculating the duration of data that will be fetched from storage when the query +// is executed after being split. All variables are expressed as a count of multiples +// of the base split interval. // // Returns: // - queryRangeIntervalsCount: The total count of intervals fetched by the original start-end @@ -282,11 +295,11 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, // for the specified start time. // // Example: -// Query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta +// Query up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta // - queryRangeIntervalsCount = 30 // - extraIntervalsPerSplitCount = 15 // - lookbackDeltaIntervalsCount = 1 -func analyzeDurationFetchedByQuery(expr parser.Expr, req tripperware.Request, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervalsCount int, extraIntervalsPerSplitCount int, lookbackDeltaIntervalsCount int) { +func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervalsCount int, extraIntervalsPerSplitCount int, lookbackDeltaIntervalsCount int) { queryRangeIntervalsCount = 0 lookbackDeltaIntervalsCount = 0 baseIntervalMillis := util.DurationMilliseconds(baseInterval) @@ -297,10 +310,10 @@ func analyzeDurationFetchedByQuery(expr parser.Expr, req tripperware.Request, ba switch n := node.(type) { case *parser.VectorSelector: // Increment count of intervals fetched by the original start-end time range - queryRangeIntervalsCount += int((req.GetEnd()/baseIntervalMillis)-(req.GetStart()/baseIntervalMillis)) + 1 + queryRangeIntervalsCount += int((queryEnd/baseIntervalMillis)-(queryStart/baseIntervalMillis)) + 1 // Adjust start and end time based on matrix selectors or subquery, this excludes lookbackDelta - start, end := util.GetTimeRangesForSelector(req.GetStart(), req.GetEnd(), 0, n, path, evalRange) + start, end := util.GetTimeRangesForSelector(queryStart, queryEnd, 0, n, path, evalRange) startIntervalIndex := floorDiv(start, baseIntervalMillis) endIntervalIndex := floorDiv(end, baseIntervalMillis) totalDurationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 9f87bd3013..54e36b8af8 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -26,6 +26,7 @@ const ( seconds = 1e3 // 1e3 milliseconds per second. queryStoreAfter = 24 * time.Hour lookbackDelta = 5 * time.Minute + longQuery = "/api/v1/query_range?end=1539266098&query=sum%28container_memory_rss%29+by+%28namespace%29&start=1536673680&stats=all&step=1200" ) func TestNextIntervalBoundary(t *testing.T) { @@ -290,8 +291,32 @@ func TestSplitByDay(t *testing.T) { for i, tc := range []struct { path, expectedBody string expectedQueryCount int32 + intervalFn IntervalFn }{ - {query, string(mergedHTTPResponseBody), 2}, + { + path: query, + expectedBody: string(mergedHTTPResponseBody), + expectedQueryCount: 2, + intervalFn: func(_ context.Context, _ tripperware.Request) (time.Duration, error) { return 24 * time.Hour, nil }, + }, + { + path: query, + expectedBody: string(mergedHTTPResponseBody), + expectedQueryCount: 2, + intervalFn: dynamicIntervalFn(Config{SplitQueriesByInterval: day}, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta), + }, + { + path: longQuery, + expectedBody: string(mergedHTTPResponseBody), + expectedQueryCount: 31, + intervalFn: func(_ context.Context, _ tripperware.Request) (time.Duration, error) { return day, nil }, + }, + { + path: longQuery, + expectedBody: string(mergedHTTPResponseBody), + expectedQueryCount: 8, + intervalFn: dynamicIntervalFn(Config{SplitQueriesByInterval: day, DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{MaxSplitsPerQuery: 10}}, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta), + }, } { tc := tc t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -310,11 +335,10 @@ func TestSplitByDay(t *testing.T) { u, err := url.Parse(s.URL) require.NoError(t, err) - interval := func(_ context.Context, _ tripperware.Request) (time.Duration, error) { return 24 * time.Hour, nil } roundtripper := tripperware.NewRoundTripper(singleHostRoundTripper{ host: u.Host, next: http.DefaultTransport, - }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(interval, mockLimits{}, PrometheusCodec, nil, lookbackDelta)) + }, PrometheusCodec, nil, NewLimitsMiddleware(mockLimits{}, 5*time.Minute), SplitByIntervalMiddleware(tc.intervalFn, mockLimits{}, PrometheusCodec, nil, lookbackDelta)) req, err := http.NewRequest("GET", tc.path, http.NoBody) require.NoError(t, err) @@ -414,13 +438,14 @@ func Test_evaluateAtModifier(t *testing.T) { } } -func TestDynamicIntervalFn(t *testing.T) { +func Test_dynamicIntervalFn(t *testing.T) { for _, tc := range []struct { name string baseSplitInterval time.Duration req tripperware.Request expectedInterval time.Duration expectedError bool + verticalShardSize int maxQueryIntervalSplits int maxDurationOfDataFetched time.Duration }{ @@ -546,7 +571,7 @@ func TestDynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "query with multiple matrix selectors, expect split by 10 day", + name: "query with multiple matrix selectors, expect split by 9 days", req: &tripperware.PrometheusRequest{ Start: (14 * 24 * 3600 * seconds) + (3600*seconds - 120*seconds), End: (52 * 24 * 3600 * seconds) + (2*3600*seconds + 500*seconds), @@ -554,11 +579,11 @@ func TestDynamicIntervalFn(t *testing.T) { Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", }, maxDurationOfDataFetched: 200 * day, - expectedInterval: 10 * day, + expectedInterval: 9 * day, }, { baseSplitInterval: day, - name: "100 day range with subquery, expect split by 13 day", + name: "100 day range with subquery, expect split by 13 days", req: &tripperware.PrometheusRequest{ Start: 0, End: 100 * 24 * 3600 * seconds, @@ -569,6 +594,32 @@ func TestDynamicIntervalFn(t *testing.T) { maxDurationOfDataFetched: 150 * day, expectedInterval: 13 * day, }, + { + baseSplitInterval: day, + name: "60 day range with 3 vertical shard size and 15 max splits, expect split by 12 days", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "sum by (pod) (up)", + }, + verticalShardSize: 3, + maxQueryIntervalSplits: 15, + expectedInterval: 12 * day, + }, + { + baseSplitInterval: day, + name: "query with multiple matrix selectors and 3 vertical shard size, expect split by 33 day", + req: &tripperware.PrometheusRequest{ + Start: (14 * 24 * 3600 * seconds) + (3600*seconds - 120*seconds), + End: (32 * 24 * 3600 * seconds) + (2*3600*seconds + 500*seconds), + Step: 60 * seconds, + Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", + }, + verticalShardSize: 3, + maxDurationOfDataFetched: 200 * day, + expectedInterval: 33 * day, + }, { baseSplitInterval: 2 * time.Hour, name: "duration of data fetched is much larger than config, expect large interval and no sharding", @@ -586,12 +637,12 @@ func TestDynamicIntervalFn(t *testing.T) { cfg := Config{ SplitQueriesByInterval: tc.baseSplitInterval, DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ - MaxShardsPerQuery: tc.maxQueryIntervalSplits, - MaxDurationOfDataFetchedFromStoragePerQuery: tc.maxDurationOfDataFetched, + MaxSplitsPerQuery: tc.maxQueryIntervalSplits, + MaxFetchedStorageDataDurationPerQuery: tc.maxDurationOfDataFetched, }, } ctx := user.InjectOrgID(context.Background(), "1") - interval, err := dynamicIntervalFn(cfg, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta)(ctx, tc.req) + interval, err := dynamicIntervalFn(cfg, mockLimits{queryVerticalShardSize: tc.verticalShardSize}, querysharding.NewQueryAnalyzer(), lookbackDelta)(ctx, tc.req) require.Equal(t, tc.expectedInterval, interval) if !tc.expectedError { require.Nil(t, err) @@ -599,3 +650,115 @@ func TestDynamicIntervalFn(t *testing.T) { }) } } + +func Test_getIntervalFromMaxSplits(t *testing.T) { + for _, tc := range []struct { + name string + baseSplitInterval time.Duration + req tripperware.Request + maxSplits int + expectedInterval time.Duration + }{ + { + name: "24 hours with 30 max splits, expected to split by 1 hour", + baseSplitInterval: time.Hour, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24 * 3600 * seconds, + Step: 60 * seconds, + Query: "foo", + }, + maxSplits: 30, + expectedInterval: time.Hour, + }, + { + name: "24 hours with 10 max splits, expected to split by 3 hours", + baseSplitInterval: time.Hour, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24 * 3600 * seconds, + Step: 60 * seconds, + Query: "foo", + }, + maxSplits: 10, + expectedInterval: 3 * time.Hour, + }, + { + name: "120 hours with 20 max splits, expected to split by 6 hours", + baseSplitInterval: time.Hour, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 5 * 24 * 3600 * seconds, + Step: 60 * seconds, + Query: "foo", + }, + maxSplits: 20, + expectedInterval: 6 * time.Hour, + }, + { + name: "30 days with 10 max splits, expected to split by 3 days", + baseSplitInterval: day, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 30 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 10, + expectedInterval: 3 * day, + }, + { + name: "60 days with 15 max splits, expected to split by 4 days", + baseSplitInterval: day, + req: &tripperware.PrometheusRequest{ + Start: 0 * 24 * 3600 * seconds, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 15, + expectedInterval: 4 * day, + }, + { + name: "61 days with 15 max splits, expected to split by 5 days", + baseSplitInterval: day, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 61 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 15, + expectedInterval: 5 * day, + }, + { + name: "51 days with 5 max splits, expected to split by 9 days", + baseSplitInterval: day, + req: &tripperware.PrometheusRequest{ + Start: (13 * 24 * 3600 * seconds) + (7*3600*seconds - 1300*seconds), + End: (51 * 24 * 3600 * seconds) + (1*3600*seconds + 4900*seconds), + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 5, + expectedInterval: 9 * day, + }, + { + name: "101 hours with 7 max splits, expected to split by 16 hours", + baseSplitInterval: time.Hour, + req: &tripperware.PrometheusRequest{ + Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), + End: (7 * 24 * 3600 * seconds) + (1*3600*seconds + 60*seconds), + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 7, + expectedInterval: 16 * time.Hour, + }, + } { + t.Run(tc.name, func(t *testing.T) { + interval := getIntervalFromMaxSplits(tc.req, tc.baseSplitInterval, tc.maxSplits) + require.Equal(t, tc.expectedInterval, interval) + }) + } +} From 83efc5cb07485b1371cda5ffc4be49e31c2e1bce Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 27 Jan 2025 16:35:40 -0800 Subject: [PATCH 10/20] add unit tests for analyzeDurationFetchedByQuery Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval.go | 44 ++--- .../queryrange/split_by_interval_test.go | 166 +++++++++++++++++- 2 files changed, 186 insertions(+), 24 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 7b7fd9fb89..74ab984078 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -174,9 +174,9 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer } // First analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split - queryRangeIntervalsCount, extraIntervalsPerSplitCount, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r.GetStart(), r.GetEnd(), baseInterval, lookbackDelta) - if extraIntervalsPerSplitCount == 0 { - extraIntervalsPerSplitCount = 1 // avoid division by 0 + queryRangeIntervals, extraIntervalsPerSplit, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r.GetStart(), r.GetEnd(), baseInterval, lookbackDelta) + if extraIntervalsPerSplit == 0 { + extraIntervalsPerSplit = 1 // avoid division by 0 } // Next analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for other subsequent splits @@ -194,10 +194,10 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer var maxSplitsByDurationFetched int if maxDurationFetchedConfig > 0 { maxIntervalsFetchedByQuery := int(maxDurationFetchedConfig / baseInterval) - // Equation for max duration fetched by example query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta + // Example equation for duration fetched by query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta // MaxFetchedStorageDataDurationPerQuery > (30 + ((15 + 1) x horizontal splits)) x vertical shards // Rearranging the equation to find the max horizontal splits - maxSplitsByDurationFetched = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - queryRangeIntervalsCount - firstSplitLookbackDeltaIntervals) / (extraIntervalsPerSplitCount + otherSplitsLookbackDeltaIntervals) + maxSplitsByDurationFetched = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - queryRangeIntervals - firstSplitLookbackDeltaIntervals) / (extraIntervalsPerSplit + otherSplitsLookbackDeltaIntervals) if maxSplitsByDurationFetched <= 0 { maxSplitsByDurationFetched = 1 } @@ -287,42 +287,42 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, // of the base split interval. // // Returns: -// - queryRangeIntervalsCount: The total count of intervals fetched by the original start-end +// - queryRangeIntervals: The total number of intervals fetched by the original start-end // range of the query. This value is constant and does not depend on the number of splits. -// - extraIntervalsPerSplitCount: The count of additional intervals fetched by matrix selectors +// - extraIntervalsPerSplit: The number of additional intervals fetched by matrix selectors // or subqueries. This value will be fetched once for every split. -// - lookbackDeltaIntervalsCount: The count of additional intervals fetched by the lookbackDelta +// - lookbackDeltaIntervals: The number of additional intervals fetched by the lookbackDelta // for the specified start time. // // Example: // Query up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta -// - queryRangeIntervalsCount = 30 -// - extraIntervalsPerSplitCount = 15 -// - lookbackDeltaIntervalsCount = 1 -func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervalsCount int, extraIntervalsPerSplitCount int, lookbackDeltaIntervalsCount int) { - queryRangeIntervalsCount = 0 - lookbackDeltaIntervalsCount = 0 +// - queryRangeIntervals = 30 +// - extraIntervalsPerSplit = 15 +// - lookbackDeltaIntervals = 1 +func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervals int, extraIntervalsPerSplit int, lookbackDeltaIntervals int) { + queryRangeIntervals = 0 + lookbackDeltaIntervals = 0 baseIntervalMillis := util.DurationMilliseconds(baseInterval) - totalDurationFetchedCount := 0 + totalDurationFetched := 0 var evalRange time.Duration parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { switch n := node.(type) { case *parser.VectorSelector: - // Increment count of intervals fetched by the original start-end time range - queryRangeIntervalsCount += int((queryEnd/baseIntervalMillis)-(queryStart/baseIntervalMillis)) + 1 + // Increment of intervals fetched by the original start-end time range + queryRangeIntervals += int((queryEnd/baseIntervalMillis)-(queryStart/baseIntervalMillis)) + 1 // Adjust start and end time based on matrix selectors or subquery, this excludes lookbackDelta start, end := util.GetTimeRangesForSelector(queryStart, queryEnd, 0, n, path, evalRange) startIntervalIndex := floorDiv(start, baseIntervalMillis) endIntervalIndex := floorDiv(end, baseIntervalMillis) - totalDurationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 + totalDurationFetched += int(endIntervalIndex-startIntervalIndex) + 1 - // Adjust start time based on lookbackDelta and increment the additional count of intervals fetched by it + // Adjust start time based on lookbackDelta and increment the additional of intervals fetched by it startLookbackDelta := start - util.DurationMilliseconds(lookbackDelta) startLookbackDeltaIntervalIndex := floorDiv(startLookbackDelta, baseIntervalMillis) if evalRange == 0 && startLookbackDeltaIntervalIndex < startIntervalIndex { - lookbackDeltaIntervalsCount += int(startIntervalIndex - startLookbackDeltaIntervalIndex) + lookbackDeltaIntervals += int(startIntervalIndex - startLookbackDeltaIntervalIndex) } evalRange = 0 case *parser.MatrixSelector: @@ -330,9 +330,9 @@ func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd } return nil }) - extraIntervalsPerSplitCount = totalDurationFetchedCount - queryRangeIntervalsCount + extraIntervalsPerSplit = totalDurationFetched - queryRangeIntervals - return queryRangeIntervalsCount, extraIntervalsPerSplitCount, lookbackDeltaIntervalsCount + return queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals } func floorDiv(a, b int64) int64 { diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 54e36b8af8..bec2c4351a 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -603,7 +603,7 @@ func Test_dynamicIntervalFn(t *testing.T) { Step: 5 * 60 * seconds, Query: "sum by (pod) (up)", }, - verticalShardSize: 3, + verticalShardSize: 3, maxQueryIntervalSplits: 15, expectedInterval: 12 * day, }, @@ -616,7 +616,7 @@ func Test_dynamicIntervalFn(t *testing.T) { Step: 60 * seconds, Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", }, - verticalShardSize: 3, + verticalShardSize: 3, maxDurationOfDataFetched: 200 * day, expectedInterval: 33 * day, }, @@ -762,3 +762,165 @@ func Test_getIntervalFromMaxSplits(t *testing.T) { }) } } + +func Test_analyzeDurationFetchedByQuery(t *testing.T) { + for _, tc := range []struct { + name string + baseSplitInterval time.Duration + lookbackDelta time.Duration + req tripperware.Request + expectedQueryRangeIntervals int + expectedExtraIntervalsPerSplit int + expectedLookbackDeltaIntervals int + }{ + { + name: "query range 00:00 to 23:59", + baseSplitInterval: time.Hour, + lookbackDelta: 0, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24*3600*seconds - 1, + Step: 60 * seconds, + Query: "up", + }, + expectedQueryRangeIntervals: 24, + expectedExtraIntervalsPerSplit: 0, + expectedLookbackDeltaIntervals: 0, + }, + { + name: "query range 00:00 to 00:00 next day", + baseSplitInterval: time.Hour, + lookbackDelta: 0, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24 * 3600 * seconds, + Step: 60 * seconds, + Query: "up", + }, + expectedQueryRangeIntervals: 25, + expectedExtraIntervalsPerSplit: 0, + expectedLookbackDeltaIntervals: 0, + }, + { + name: "query range 00:00 to 23:59, with 5 min lookback", + baseSplitInterval: time.Hour, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24*3600*seconds - 1, + Step: 60 * seconds, + Query: "up", + }, + expectedQueryRangeIntervals: 24, + expectedExtraIntervalsPerSplit: 0, + expectedLookbackDeltaIntervals: 1, + }, + { + name: "query range 00:00 to 23:59, with 5 hour subquery", + baseSplitInterval: time.Hour, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24*3600*seconds - 1, + Step: 60 * seconds, + Query: "up[5h:10m]", + }, + expectedQueryRangeIntervals: 24, + expectedExtraIntervalsPerSplit: 5, + expectedLookbackDeltaIntervals: 1, + }, + { + name: "query range 00:00 to 23:59, with 2 hour matrix selector", + baseSplitInterval: time.Hour, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24*3600*seconds - 1, + Step: 60 * seconds, + Query: "rate(up[2h])", + }, + expectedQueryRangeIntervals: 24, + expectedExtraIntervalsPerSplit: 2, + expectedLookbackDeltaIntervals: 0, + }, + { + name: "query range 00:00 to 23:59, with multiple matrix selectors", + baseSplitInterval: time.Hour, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 24*3600*seconds - 1, + Step: 60 * seconds, + Query: "rate(up[2h]) + rate(up[5h]) + rate(up[7h])", + }, + expectedQueryRangeIntervals: 72, + expectedExtraIntervalsPerSplit: 14, + expectedLookbackDeltaIntervals: 0, + }, + { + name: "query range 60 day with 20 day subquery", + baseSplitInterval: day, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0 * 24 * 3600 * seconds, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up[20d:1h]", + }, + expectedQueryRangeIntervals: 61, + expectedExtraIntervalsPerSplit: 20, + expectedLookbackDeltaIntervals: 1, + }, + { + name: "query range 35 day, with 15 day subquery and 1 week offset", + baseSplitInterval: day, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 0 * 24 * 3600 * seconds, + End: 35 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "up[15d:1h] offset 1w", + }, + expectedQueryRangeIntervals: 36, + expectedExtraIntervalsPerSplit: 15, + expectedLookbackDeltaIntervals: 1, + }, + { + name: "query range 10 days, with multiple subqueries and offsets", + baseSplitInterval: day, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: 10 * 24 * 3600 * seconds, + End: 20 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "rate(up[2d:1h] offset 1w) + rate(up[5d:1h] offset 2w) + rate(up[7d:1h] offset 3w)", + }, + expectedQueryRangeIntervals: 33, + expectedExtraIntervalsPerSplit: 14, + expectedLookbackDeltaIntervals: 3, + }, + { + name: "query range spans 40 days with 4 day matrix selector", + baseSplitInterval: day, + lookbackDelta: lookbackDelta, + req: &tripperware.PrometheusRequest{ + Start: (13 * 24 * 3600 * seconds) - (7*3600*seconds - 1300*seconds), + End: (51 * 24 * 3600 * seconds) + (1*3600*seconds + 4900*seconds), + Step: 5 * 60 * seconds, + Query: "up[4d]", + }, + expectedQueryRangeIntervals: 40, + expectedExtraIntervalsPerSplit: 4, + expectedLookbackDeltaIntervals: 0, + }, + } { + t.Run(tc.name, func(t *testing.T) { + expr, err := parser.ParseExpr(tc.req.GetQuery()) + require.Nil(t, err) + queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals := analyzeDurationFetchedByQuery(expr, tc.req.GetStart(), tc.req.GetEnd(), tc.baseSplitInterval, tc.lookbackDelta) + require.Equal(t, tc.expectedQueryRangeIntervals, queryRangeIntervals) + require.Equal(t, tc.expectedExtraIntervalsPerSplit, extraIntervalsPerSplit) + require.Equal(t, tc.expectedLookbackDeltaIntervals, lookbackDeltaIntervals) + }) + } +} From 427f5b2950e2eee7cb8dcfcba12219f0296170e6 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 27 Jan 2025 19:21:41 -0800 Subject: [PATCH 11/20] fix formatting Signed-off-by: Ahmed Hassan --- pkg/querier/tripperware/queryrange/query_range_middlewares.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 0f0bb1b64f..20991cb3c3 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -79,7 +79,7 @@ func (cfg *Config) Validate(qCfg querier.Config) error { } type DynamicQuerySplitsConfig struct { - MaxSplitsPerQuery int `yaml:"max_splits_per_query"` + MaxSplitsPerQuery int `yaml:"max_splits_per_query"` MaxFetchedStorageDataDurationPerQuery time.Duration `yaml:"max_fetched_storage_data_duration_per_query"` } From c931aa6b9414a1c682fb3685c230b682e668ef3f Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 27 Jan 2025 19:37:48 -0800 Subject: [PATCH 12/20] update docs Signed-off-by: Ahmed Hassan --- docs/configuration/config-file-reference.md | 29 ++++++++++----------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 97e0568e05..aec96d1bf8 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4194,21 +4194,20 @@ The `query_range_config` configures the query splitting and caching in the Corte [split_queries_by_interval: | default = 0s] dynamic_query_splits: - # [EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. - # Dynamically uses a multiple of `split-queries-by-interval` to maintain the - # number of splits below the limit. If vertical sharding is enabled for a - # query, the combined total number of vertical and interval shards is kept - # below this limit. - # CLI flag: -querier.max-shards-per-query - [max_shards_per_query: | default = 0] - - # [EXPERIMENTAL] Max total duration of data fetched by all query shards from - # storage, 0 disables it. Dynamically uses a multiple of - # `split-queries-by-interval` to ensure the total fetched duration of data is - # lower than the value set. It takes into account additional data fetched by - # matrix selectors and subqueries. - # CLI flag: -querier.max-duration-of-data-fetched-from-storage-per-query - [max_duration_of_data_fetched_from_storage_per_query: | default = 0s] + # [EXPERIMENTAL] Maximum number of splits for a query, 0 disables it. + # Dynamically uses a multiple of split interval to maintain a total number of + # splits below the set value. If vertical sharding is enabled for a query, the + # combined total number of vertical and interval splits is kept below this + # value. + # CLI flag: -querier.max-splits-per-query + [max_splits_per_query: | default = 0] + + # [EXPERIMENTAL] Max total duration of data fetched from storage by all query + # splits, 0 disables it. Dynamically uses a multiple of split interval to + # maintain a total fetched duration of data lower than the value set. It takes + # into account additional duration fetched by matrix selectors and subqueries. + # CLI flag: -querier.max-fetched-storage-data-duration-per-query + [max_fetched_storage_data_duration_per_query: | default = 0s] # Mutate incoming queries to align their start and end with their step. # CLI flag: -querier.align-querier-with-step From c1ae047c6c363e5ae1fa012dab6fb569fa59a61d Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Tue, 28 Jan 2025 08:17:54 -0800 Subject: [PATCH 13/20] update experimental features Signed-off-by: Ahmed Hassan --- docs/configuration/v1-guarantees.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index 07915432d5..2c8e633764 100644 --- a/docs/configuration/v1-guarantees.md +++ b/docs/configuration/v1-guarantees.md @@ -119,5 +119,5 @@ Currently experimental features are: - Query-frontend: query rejection (`-frontend.query-rejection.enabled`) - Querier: protobuf codec (`-api.querier-default-codec`) - Query-frontend: dynamic query splits - - `querier.max-shards-per-query` (int) CLI flag - - `querier.max-duration-of-data-fetched-from-storage-per-query` (duration) CLI flag + - `querier.max-splits-per-query` (int) CLI flag + - `querier.max-fetched-storage-data-duration-per-query` (duration) CLI flag From ba90bf0b37e3ed2957861aee4e0e5731d08e06e5 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 31 Jan 2025 09:59:47 -0800 Subject: [PATCH 14/20] fix dynamic splitting when query range is shorter than base interval Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval.go | 4 ++-- .../queryrange/split_by_interval_test.go | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 74ab984078..7568e9f0c5 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -258,7 +258,7 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, n = 1 } - if maxSplits == 1 || queryRange < baseInterval { + if maxSplits == 1 { // No splitting, interval should be long enough to result in 1 split only nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() if nextSplitStart < r.GetEnd() { @@ -266,7 +266,7 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, n += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval } } else { - for n <= 2*(queryRange/baseInterval) { + for n <= 2*((queryRange+baseInterval-1)/baseInterval) { // The first split can be truncated and shorter than other splits. // So it is removed to check if a larger interval is needed to shard <= maxSplits-1 nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index bec2c4351a..919eca3a43 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -498,6 +498,17 @@ func Test_dynamicIntervalFn(t *testing.T) { maxDurationOfDataFetched: 200 * time.Hour, expectedInterval: 4 * time.Hour, }, + { + baseSplitInterval: day, + name: "10 hour range, expect split by 1 day", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 19 * 3600 * seconds, + Step: 60 * seconds, + Query: "up", + }, + expectedInterval: day, + }, { baseSplitInterval: day, name: "30 day range, expect split by 1 day", @@ -695,6 +706,18 @@ func Test_getIntervalFromMaxSplits(t *testing.T) { maxSplits: 20, expectedInterval: 6 * time.Hour, }, + { + name: "23h with 10 max splits, expected to split by 1 day", + baseSplitInterval: day, + req: &tripperware.PrometheusRequest{ + Start: 12 * 3600 * seconds, + End: 35 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "foo", + }, + maxSplits: 10, + expectedInterval: 1 * day, + }, { name: "30 days with 10 max splits, expected to split by 3 days", baseSplitInterval: day, From 83858149f5997dbb81223bb57f4d1bdd15c53ff9 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Wed, 5 Feb 2025 17:35:53 -0800 Subject: [PATCH 15/20] refactor dynamic query splitting into smaller helper functions Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval.go | 189 ++++++++++-------- .../queryrange/split_by_interval_test.go | 82 ++++---- 2 files changed, 148 insertions(+), 123 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 7568e9f0c5..c03db327be 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -160,8 +160,8 @@ func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Reques func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { baseInterval := cfg.SplitQueriesByInterval - maxDurationFetchedConfig := cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery - maxSplitsConfig := cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery + maxSplitsConfigValue := cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery + maxDurationFetchedConfigValue := cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery queryVerticalShardSize, err := getMaxVerticalShardSizeForQuery(ctx, r, limits, queryAnalyzer) if err != nil { @@ -173,54 +173,19 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer return baseInterval, err } - // First analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split - queryRangeIntervals, extraIntervalsPerSplit, firstSplitLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, r.GetStart(), r.GetEnd(), baseInterval, lookbackDelta) - if extraIntervalsPerSplit == 0 { - extraIntervalsPerSplit = 1 // avoid division by 0 - } - - // Next analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for other subsequent splits - nextIntervalStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), baseInterval) + r.GetStep() - _, _, otherSplitsLookbackDeltaIntervals := analyzeDurationFetchedByQuery(queryExpr, nextIntervalStart, r.GetEnd(), baseInterval, lookbackDelta) - - // By default subtract the 'first split' duration fetched by loookbackDelta, and divide by the 'other splits' duration fetched by loookbackDelta. - if firstSplitLookbackDeltaIntervals > 0 && otherSplitsLookbackDeltaIntervals > 0 { - firstSplitLookbackDeltaIntervals = 0 // Dividing is enough if additional duration is fetched by loookbackDelta for all splits - } else if otherSplitsLookbackDeltaIntervals > 0 { - firstSplitLookbackDeltaIntervals = otherSplitsLookbackDeltaIntervals * -1 // Adding instead of subtracting for first split, if additional duration is fetched by loookbackDelta for all splits except first one - } - - // Find the max number of splits that will fetch less than MaxFetchedStorageDataDurationPerQuery - var maxSplitsByDurationFetched int - if maxDurationFetchedConfig > 0 { - maxIntervalsFetchedByQuery := int(maxDurationFetchedConfig / baseInterval) - // Example equation for duration fetched by query: up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta - // MaxFetchedStorageDataDurationPerQuery > (30 + ((15 + 1) x horizontal splits)) x vertical shards - // Rearranging the equation to find the max horizontal splits - maxSplitsByDurationFetched = ((maxIntervalsFetchedByQuery / queryVerticalShardSize) - queryRangeIntervals - firstSplitLookbackDeltaIntervals) / (extraIntervalsPerSplit + otherSplitsLookbackDeltaIntervals) - if maxSplitsByDurationFetched <= 0 { - maxSplitsByDurationFetched = 1 - } - } - - // Find max number of splits from MaxSplitsPerQuery after accounting for vertical sharding - var maxSplitsByConfig int - if maxSplitsConfig > 0 { - maxSplitsByConfig = maxSplitsConfig / queryVerticalShardSize - if maxSplitsByConfig <= 0 { - maxSplitsByConfig = 1 - } - } + // Get the max number of splits allowed by each of the two configs + maxSplitsFromConfig := getMaxSplitsFromConfig(maxSplitsConfigValue, queryVerticalShardSize) + maxSplitsFromDurationFetched := getMaxSplitsByDurationFetched(maxDurationFetchedConfigValue, queryVerticalShardSize, queryExpr, r.GetStart(), r.GetEnd(), r.GetStep(), baseInterval, lookbackDelta) + // Use the more restrictive max splits limit var maxSplits int switch { - case maxDurationFetchedConfig > 0 && maxSplitsConfig > 0: - // Use the more restricting shard limit - maxSplits = min(maxSplitsByConfig, maxSplitsByDurationFetched) - case maxSplitsConfig > 0: - maxSplits = maxSplitsByConfig - case maxDurationFetchedConfig > 0: - maxSplits = maxSplitsByDurationFetched + case maxSplitsConfigValue > 0 && maxDurationFetchedConfigValue > 0: + maxSplits = min(maxSplitsFromConfig, maxSplitsFromDurationFetched) + case maxSplitsConfigValue > 0: + maxSplits = maxSplitsFromConfig + case maxDurationFetchedConfigValue > 0: + maxSplits = maxSplitsFromDurationFetched default: return baseInterval, nil } @@ -258,71 +223,131 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, n = 1 } - if maxSplits == 1 { - // No splitting, interval should be long enough to result in 1 split only + for n <= 2*((queryRange+baseInterval-1)/baseInterval) { + // The first split could be truncated and shorter than other splits. + // So it is removed to check if a larger interval is needed nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() - if nextSplitStart < r.GetEnd() { - queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) - n += (queryRangeWithoutFirstSplit + baseInterval - 1) / baseInterval - } - } else { - for n <= 2*((queryRange+baseInterval-1)/baseInterval) { - // The first split can be truncated and shorter than other splits. - // So it is removed to check if a larger interval is needed to shard <= maxSplits-1 - nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() + if maxSplits == 1 { + // No splitting. Exit loop if first split is long enough to cover the full query range + if nextSplitStart >= r.GetEnd() { + break + } + } else { + // Recalculate n for remaining query range after removing first split. + // Exit loop if a larger n is not needed to split into <= maxSplits-1 queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) n_temp := (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) if n >= n_temp { break } - n++ } + n++ } return n * baseInterval } -// analyzeDurationFetchedByQuery analyzes the query to identify variables useful for +func getMaxSplitsFromConfig(maxSplitsConfigValue int, queryVerticalShardSize int) int { + // Find max number of allowed splits by MaxSplitsPerQuery config after accounting for vertical sharding + var maxSplitsFromConfig int + if maxSplitsConfigValue > 0 { + maxSplitsFromConfig = maxSplitsConfigValue / queryVerticalShardSize + } + if maxSplitsFromConfig <= 0 { + maxSplitsFromConfig = 1 + } + return maxSplitsFromConfig +} + +func getMaxSplitsByDurationFetched(maxDurationFetchedConfigValue time.Duration, queryVerticalShardSize int, expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) int { + // Get variables needed to calculate the max number of allowed splits by MaxFetchedStorageDataDurationPerQuery config + intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit, intervalsFetchedByLookbackDeltaForOtherSplits := getVariablesToCalculateMaxSplitsByDurationFetched(expr, queryStart, queryEnd, queryStep, baseInterval, lookbackDelta) + + // Handle different cases for lookbackDelta fetching additional duration of data + if intervalsFetchedByLookbackDeltaForFirstSplit > 0 && intervalsFetchedByLookbackDeltaForOtherSplits > 0 { + // lookbackDelta is fetching additional duration for all splits + extraIntervalsFetchedPerSplit += intervalsFetchedByLookbackDeltaForOtherSplits + } else if intervalsFetchedByLookbackDeltaForOtherSplits > 0 { + // lookbackDelta is fetching additional duration for all splits except first one + extraIntervalsFetchedPerSplit += intervalsFetchedByLookbackDeltaForOtherSplits + intervalsFetchedByQueryRange -= intervalsFetchedByLookbackDeltaForOtherSplits + } else if intervalsFetchedByLookbackDeltaForFirstSplit > 0 { + // lookbackDelta is fetching additional duration for first split only + intervalsFetchedByQueryRange += intervalsFetchedByLookbackDeltaForFirstSplit + } + + var maxSplitsByDurationFetched int + if maxDurationFetchedConfigValue > 0 { + // Example equation for duration fetched by query up[15d:1h] with a range of 30 days, 1 day base split interval, and 5 min lookbackDelta + // [duration_fetched_by_range + (extra_duration_fetched_per_split x horizontal_splits)] x vertical_shards <= maxDurationFetchedConfigValue + // [30 + (16 x horizontal_splits)] x vertical_shards <= maxDurationFetchedConfigValue + // Rearranging the equation to find the max horizontal splits + maxDurationFetchedAsMultipleOfInterval := int(maxDurationFetchedConfigValue / baseInterval) + maxSplitsByDurationFetched = ((maxDurationFetchedAsMultipleOfInterval / queryVerticalShardSize) - intervalsFetchedByQueryRange) / extraIntervalsFetchedPerSplit + } + if maxSplitsByDurationFetched <= 0 { + maxSplitsByDurationFetched = 1 + } + return maxSplitsByDurationFetched +} + +func getVariablesToCalculateMaxSplitsByDurationFetched(expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) (intervalsFetchedByQueryRange int, extraIntervalsFetchedPerSplit int, intervalsFetchedByLookbackDeltaForFirstSplit int, intervalsFetchedByLookbackDeltaForOtherSplits int) { + // First analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split + intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit = analyzeDurationFetchedByQueryExpr(expr, queryStart, queryEnd, baseInterval, lookbackDelta) + if extraIntervalsFetchedPerSplit == 0 { + extraIntervalsFetchedPerSplit = 1 // avoid division by 0 + } + + // Next analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for splits other than first one + nextIntervalStart := nextIntervalBoundary(queryStart, queryStep, baseInterval) + queryStep + _, _, intervalsFetchedByLookbackDeltaForOtherSplits = analyzeDurationFetchedByQueryExpr(expr, nextIntervalStart, queryEnd, baseInterval, lookbackDelta) + + return intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit, intervalsFetchedByLookbackDeltaForOtherSplits +} + +// analyzeDurationFetchedByQueryExpr analyzes the query to identify variables useful for // calculating the duration of data that will be fetched from storage when the query -// is executed after being split. All variables are expressed as a count of multiples +// is executed after being split. All variables are expressed as an integer count of multiples // of the base split interval. // // Returns: -// - queryRangeIntervals: The total number of intervals fetched by the original start-end -// range of the query. This value is constant and does not depend on the number of splits. -// - extraIntervalsPerSplit: The number of additional intervals fetched by matrix selectors -// or subqueries. This value will be fetched once for every split. -// - lookbackDeltaIntervals: The number of additional intervals fetched by the lookbackDelta +// - intervalsFetchedByQueryRange: The total number of intervals fetched by the original start-end +// range of the query. This value does not depend on the number of splits. +// - extraIntervalsFetchedPerSplit: The number of additional intervals fetched by matrix selectors +// and/or subqueries. This duration will be fetched by every query split. +// - intervalsFetchedByLookbackDelta: The number of additional intervals fetched by lookbackDelta // for the specified start time. // // Example: -// Query up[15d:1h] with a range of 30 days, a base split interval of 24 hours, and 5 min lookbackDelta -// - queryRangeIntervals = 30 -// - extraIntervalsPerSplit = 15 -// - lookbackDeltaIntervals = 1 -func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (queryRangeIntervals int, extraIntervalsPerSplit int, lookbackDeltaIntervals int) { - queryRangeIntervals = 0 - lookbackDeltaIntervals = 0 +// Query up[15d:1h] with a range of 30 days, 1 day base split interval, and 5 min lookbackDelta +// - intervalsFetchedByQueryRange = 30 +// - extraIntervalsFetchedPerSplit = 15 +// - intervalsFetchedByLookbackDelta = 1 +func analyzeDurationFetchedByQueryExpr(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (intervalsFetchedByQueryRange int, extraIntervalsFetchedPerSplit int, intervalsFetchedByLookbackDelta int) { + intervalsFetchedByQueryRange = 0 + intervalsFetchedByLookbackDelta = 0 baseIntervalMillis := util.DurationMilliseconds(baseInterval) - totalDurationFetched := 0 + totalIntervalsFetched := 0 var evalRange time.Duration parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { switch n := node.(type) { case *parser.VectorSelector: - // Increment of intervals fetched by the original start-end time range - queryRangeIntervals += int((queryEnd/baseIntervalMillis)-(queryStart/baseIntervalMillis)) + 1 + // Increment intervals fetched by the original start-end time range + queryStartIntervalIndex := floorDiv(queryStart, baseIntervalMillis) + queryEndIntervalIndex := floorDiv(queryEnd, baseIntervalMillis) + intervalsFetchedByQueryRange += int(queryEndIntervalIndex-queryStartIntervalIndex) + 1 - // Adjust start and end time based on matrix selectors or subquery, this excludes lookbackDelta + // Adjust start and end time based on matrix selectors or subquery and increment total intervals fetched, this excludes lookbackDelta start, end := util.GetTimeRangesForSelector(queryStart, queryEnd, 0, n, path, evalRange) startIntervalIndex := floorDiv(start, baseIntervalMillis) endIntervalIndex := floorDiv(end, baseIntervalMillis) - totalDurationFetched += int(endIntervalIndex-startIntervalIndex) + 1 + totalIntervalsFetched += int(endIntervalIndex-startIntervalIndex) + 1 - // Adjust start time based on lookbackDelta and increment the additional of intervals fetched by it + // Increment intervals fetched by lookbackDelta startLookbackDelta := start - util.DurationMilliseconds(lookbackDelta) startLookbackDeltaIntervalIndex := floorDiv(startLookbackDelta, baseIntervalMillis) if evalRange == 0 && startLookbackDeltaIntervalIndex < startIntervalIndex { - lookbackDeltaIntervals += int(startIntervalIndex - startLookbackDeltaIntervalIndex) + intervalsFetchedByLookbackDelta += int(startIntervalIndex - startLookbackDeltaIntervalIndex) } evalRange = 0 case *parser.MatrixSelector: @@ -330,9 +355,9 @@ func analyzeDurationFetchedByQuery(expr parser.Expr, queryStart int64, queryEnd } return nil }) - extraIntervalsPerSplit = totalDurationFetched - queryRangeIntervals - return queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals + extraIntervalsFetchedPerSplit = totalIntervalsFetched - intervalsFetchedByQueryRange + return intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDelta } func floorDiv(a, b int64) int64 { diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 919eca3a43..74ac3b71e5 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -788,13 +788,13 @@ func Test_getIntervalFromMaxSplits(t *testing.T) { func Test_analyzeDurationFetchedByQuery(t *testing.T) { for _, tc := range []struct { - name string - baseSplitInterval time.Duration - lookbackDelta time.Duration - req tripperware.Request - expectedQueryRangeIntervals int - expectedExtraIntervalsPerSplit int - expectedLookbackDeltaIntervals int + name string + baseSplitInterval time.Duration + lookbackDelta time.Duration + req tripperware.Request + expectedIntervalsFetchedByQueryRange int + expectedExtraIntervalsFetchedPerSplit int + expectedIntervalsFetchedByLookbackDelta int }{ { name: "query range 00:00 to 23:59", @@ -806,9 +806,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedQueryRangeIntervals: 24, - expectedExtraIntervalsPerSplit: 0, - expectedLookbackDeltaIntervals: 0, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 0, }, { name: "query range 00:00 to 00:00 next day", @@ -820,9 +820,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedQueryRangeIntervals: 25, - expectedExtraIntervalsPerSplit: 0, - expectedLookbackDeltaIntervals: 0, + expectedIntervalsFetchedByQueryRange: 25, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 0, }, { name: "query range 00:00 to 23:59, with 5 min lookback", @@ -834,9 +834,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedQueryRangeIntervals: 24, - expectedExtraIntervalsPerSplit: 0, - expectedLookbackDeltaIntervals: 1, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 1, }, { name: "query range 00:00 to 23:59, with 5 hour subquery", @@ -848,9 +848,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up[5h:10m]", }, - expectedQueryRangeIntervals: 24, - expectedExtraIntervalsPerSplit: 5, - expectedLookbackDeltaIntervals: 1, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 5, + expectedIntervalsFetchedByLookbackDelta: 1, }, { name: "query range 00:00 to 23:59, with 2 hour matrix selector", @@ -862,9 +862,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "rate(up[2h])", }, - expectedQueryRangeIntervals: 24, - expectedExtraIntervalsPerSplit: 2, - expectedLookbackDeltaIntervals: 0, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 2, + expectedIntervalsFetchedByLookbackDelta: 0, }, { name: "query range 00:00 to 23:59, with multiple matrix selectors", @@ -876,9 +876,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "rate(up[2h]) + rate(up[5h]) + rate(up[7h])", }, - expectedQueryRangeIntervals: 72, - expectedExtraIntervalsPerSplit: 14, - expectedLookbackDeltaIntervals: 0, + expectedIntervalsFetchedByQueryRange: 72, + expectedExtraIntervalsFetchedPerSplit: 14, + expectedIntervalsFetchedByLookbackDelta: 0, }, { name: "query range 60 day with 20 day subquery", @@ -890,9 +890,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[20d:1h]", }, - expectedQueryRangeIntervals: 61, - expectedExtraIntervalsPerSplit: 20, - expectedLookbackDeltaIntervals: 1, + expectedIntervalsFetchedByQueryRange: 61, + expectedExtraIntervalsFetchedPerSplit: 20, + expectedIntervalsFetchedByLookbackDelta: 1, }, { name: "query range 35 day, with 15 day subquery and 1 week offset", @@ -904,9 +904,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[15d:1h] offset 1w", }, - expectedQueryRangeIntervals: 36, - expectedExtraIntervalsPerSplit: 15, - expectedLookbackDeltaIntervals: 1, + expectedIntervalsFetchedByQueryRange: 36, + expectedExtraIntervalsFetchedPerSplit: 15, + expectedIntervalsFetchedByLookbackDelta: 1, }, { name: "query range 10 days, with multiple subqueries and offsets", @@ -918,9 +918,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "rate(up[2d:1h] offset 1w) + rate(up[5d:1h] offset 2w) + rate(up[7d:1h] offset 3w)", }, - expectedQueryRangeIntervals: 33, - expectedExtraIntervalsPerSplit: 14, - expectedLookbackDeltaIntervals: 3, + expectedIntervalsFetchedByQueryRange: 33, + expectedExtraIntervalsFetchedPerSplit: 14, + expectedIntervalsFetchedByLookbackDelta: 3, }, { name: "query range spans 40 days with 4 day matrix selector", @@ -932,18 +932,18 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[4d]", }, - expectedQueryRangeIntervals: 40, - expectedExtraIntervalsPerSplit: 4, - expectedLookbackDeltaIntervals: 0, + expectedIntervalsFetchedByQueryRange: 40, + expectedExtraIntervalsFetchedPerSplit: 4, + expectedIntervalsFetchedByLookbackDelta: 0, }, } { t.Run(tc.name, func(t *testing.T) { expr, err := parser.ParseExpr(tc.req.GetQuery()) require.Nil(t, err) - queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals := analyzeDurationFetchedByQuery(expr, tc.req.GetStart(), tc.req.GetEnd(), tc.baseSplitInterval, tc.lookbackDelta) - require.Equal(t, tc.expectedQueryRangeIntervals, queryRangeIntervals) - require.Equal(t, tc.expectedExtraIntervalsPerSplit, extraIntervalsPerSplit) - require.Equal(t, tc.expectedLookbackDeltaIntervals, lookbackDeltaIntervals) + queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals := analyzeDurationFetchedByQueryExpr(expr, tc.req.GetStart(), tc.req.GetEnd(), tc.baseSplitInterval, tc.lookbackDelta) + require.Equal(t, tc.expectedIntervalsFetchedByQueryRange, queryRangeIntervals) + require.Equal(t, tc.expectedExtraIntervalsFetchedPerSplit, extraIntervalsPerSplit) + require.Equal(t, tc.expectedIntervalsFetchedByLookbackDelta, lookbackDeltaIntervals) }) } } From 9225d2360540c26ed1916be65554e0441f8596ed Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Fri, 7 Feb 2025 16:23:25 -0800 Subject: [PATCH 16/20] use duration instead of int for dynamic query splitting calculation Signed-off-by: Ahmed Hassan --- docs/configuration/config-file-reference.md | 18 +-- docs/configuration/v1-guarantees.md | 4 +- .../queryrange/query_range_middlewares.go | 12 +- .../queryrange/split_by_interval.go | 147 +++++++++--------- .../queryrange/split_by_interval_test.go | 86 +++++----- 5 files changed, 134 insertions(+), 133 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index aec96d1bf8..46e6953fdf 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4194,20 +4194,20 @@ The `query_range_config` configures the query splitting and caching in the Corte [split_queries_by_interval: | default = 0s] dynamic_query_splits: - # [EXPERIMENTAL] Maximum number of splits for a query, 0 disables it. + # [EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. # Dynamically uses a multiple of split interval to maintain a total number of - # splits below the set value. If vertical sharding is enabled for a query, the - # combined total number of vertical and interval splits is kept below this - # value. - # CLI flag: -querier.max-splits-per-query - [max_splits_per_query: | default = 0] + # shards below the set value. If vertical sharding is enabled for a query, the + # combined total number of interval splits and vertical shards is kept below + # this value. + # CLI flag: -querier.max-shards-per-query + [max_shards_per_query: | default = 0] # [EXPERIMENTAL] Max total duration of data fetched from storage by all query - # splits, 0 disables it. Dynamically uses a multiple of split interval to + # shards, 0 disables it. Dynamically uses a multiple of split interval to # maintain a total fetched duration of data lower than the value set. It takes # into account additional duration fetched by matrix selectors and subqueries. - # CLI flag: -querier.max-fetched-storage-data-duration-per-query - [max_fetched_storage_data_duration_per_query: | default = 0s] + # CLI flag: -querier.max-fetched-data-duration-per-query + [max_fetched_data_duration_per_query: | default = 0s] # Mutate incoming queries to align their start and end with their step. # CLI flag: -querier.align-querier-with-step diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index 2c8e633764..722d312e83 100644 --- a/docs/configuration/v1-guarantees.md +++ b/docs/configuration/v1-guarantees.md @@ -119,5 +119,5 @@ Currently experimental features are: - Query-frontend: query rejection (`-frontend.query-rejection.enabled`) - Querier: protobuf codec (`-api.querier-default-codec`) - Query-frontend: dynamic query splits - - `querier.max-splits-per-query` (int) CLI flag - - `querier.max-fetched-storage-data-duration-per-query` (duration) CLI flag + - `querier.max-shards-per-query` (int) CLI flag + - `querier.max-fetched-data-duration-per-query` (duration) CLI flag diff --git a/pkg/querier/tripperware/queryrange/query_range_middlewares.go b/pkg/querier/tripperware/queryrange/query_range_middlewares.go index 20991cb3c3..1a79b6f656 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -70,7 +70,7 @@ func (cfg *Config) Validate(qCfg querier.Config) error { return errors.Wrap(err, "invalid ResultsCache config") } } - if cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery > 0 { + if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedDataDurationPerQuery > 0 { if cfg.SplitQueriesByInterval <= 0 { return errors.New("configs under dynamic-query-splits requires that a value for split-queries-by-interval is set.") } @@ -79,14 +79,14 @@ func (cfg *Config) Validate(qCfg querier.Config) error { } type DynamicQuerySplitsConfig struct { - MaxSplitsPerQuery int `yaml:"max_splits_per_query"` - MaxFetchedStorageDataDurationPerQuery time.Duration `yaml:"max_fetched_storage_data_duration_per_query"` + MaxShardsPerQuery int `yaml:"max_shards_per_query"` + MaxFetchedDataDurationPerQuery time.Duration `yaml:"max_fetched_data_duration_per_query"` } // RegisterFlags registers flags foy dynamic query splits func (cfg *DynamicQuerySplitsConfig) RegisterFlags(f *flag.FlagSet) { - f.IntVar(&cfg.MaxSplitsPerQuery, "querier.max-splits-per-query", 0, "[EXPERIMENTAL] Maximum number of splits for a query, 0 disables it. Dynamically uses a multiple of split interval to maintain a total number of splits below the set value. If vertical sharding is enabled for a query, the combined total number of vertical and interval splits is kept below this value.") - f.DurationVar(&cfg.MaxFetchedStorageDataDurationPerQuery, "querier.max-fetched-storage-data-duration-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched from storage by all query splits, 0 disables it. Dynamically uses a multiple of split interval to maintain a total fetched duration of data lower than the value set. It takes into account additional duration fetched by matrix selectors and subqueries.") + f.IntVar(&cfg.MaxShardsPerQuery, "querier.max-shards-per-query", 0, "[EXPERIMENTAL] Maximum number of shards for a query, 0 disables it. Dynamically uses a multiple of split interval to maintain a total number of shards below the set value. If vertical sharding is enabled for a query, the combined total number of interval splits and vertical shards is kept below this value.") + f.DurationVar(&cfg.MaxFetchedDataDurationPerQuery, "querier.max-fetched-data-duration-per-query", 0, "[EXPERIMENTAL] Max total duration of data fetched from storage by all query shards, 0 disables it. Dynamically uses a multiple of split interval to maintain a total fetched duration of data lower than the value set. It takes into account additional duration fetched by matrix selectors and subqueries.") } // Middlewares returns list of middlewares that should be applied for range query. @@ -110,7 +110,7 @@ func Middlewares( } if cfg.SplitQueriesByInterval != 0 { intervalFn := staticIntervalFn(cfg) - if cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery > 0 { + if cfg.DynamicQuerySplitsConfig.MaxShardsPerQuery > 0 || cfg.DynamicQuerySplitsConfig.MaxFetchedDataDurationPerQuery > 0 { intervalFn = dynamicIntervalFn(cfg, limits, queryAnalyzer, lookbackDelta) } queryRangeMiddleware = append(queryRangeMiddleware, tripperware.InstrumentMiddleware("split_by_interval", metrics), SplitByIntervalMiddleware(intervalFn, limits, prometheusCodec, registerer, lookbackDelta)) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index c03db327be..77a5793110 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -151,19 +151,23 @@ func nextIntervalBoundary(t, step int64, interval time.Duration) int64 { return target } +// Returns a fixed split interval func staticIntervalFn(cfg Config) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(_ context.Context, _ tripperware.Request) (time.Duration, error) { return cfg.SplitQueriesByInterval, nil } } +// Returns a dynamic multiple of base interval adjusted depending on configured 'max_shards_per_query' and 'max_fetched_data_duration_per_query' func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer, lookbackDelta time.Duration) func(ctx context.Context, r tripperware.Request) (time.Duration, error) { return func(ctx context.Context, r tripperware.Request) (time.Duration, error) { baseInterval := cfg.SplitQueriesByInterval - maxSplitsConfigValue := cfg.DynamicQuerySplitsConfig.MaxSplitsPerQuery - maxDurationFetchedConfigValue := cfg.DynamicQuerySplitsConfig.MaxFetchedStorageDataDurationPerQuery + dynamicSplitCfg := cfg.DynamicQuerySplitsConfig + if dynamicSplitCfg.MaxShardsPerQuery == 0 && dynamicSplitCfg.MaxFetchedDataDurationPerQuery == 0 { + return baseInterval, nil + } - queryVerticalShardSize, err := getMaxVerticalShardSizeForQuery(ctx, r, limits, queryAnalyzer) + queryVerticalShardSize, err := getMaxVerticalShardSize(ctx, r, limits, queryAnalyzer) if err != nil { return baseInterval, err } @@ -173,21 +177,18 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer return baseInterval, err } - // Get the max number of splits allowed by each of the two configs - maxSplitsFromConfig := getMaxSplitsFromConfig(maxSplitsConfigValue, queryVerticalShardSize) - maxSplitsFromDurationFetched := getMaxSplitsByDurationFetched(maxDurationFetchedConfigValue, queryVerticalShardSize, queryExpr, r.GetStart(), r.GetEnd(), r.GetStep(), baseInterval, lookbackDelta) + maxSplitsPerQuery := getMaxSplitsFromConfig(dynamicSplitCfg.MaxShardsPerQuery, queryVerticalShardSize) + maxSplitsFromDurationFetched := getMaxSplitsByDurationFetched(dynamicSplitCfg.MaxFetchedDataDurationPerQuery, queryVerticalShardSize, queryExpr, r.GetStart(), r.GetEnd(), r.GetStep(), baseInterval, lookbackDelta) // Use the more restrictive max splits limit var maxSplits int switch { - case maxSplitsConfigValue > 0 && maxDurationFetchedConfigValue > 0: - maxSplits = min(maxSplitsFromConfig, maxSplitsFromDurationFetched) - case maxSplitsConfigValue > 0: - maxSplits = maxSplitsFromConfig - case maxDurationFetchedConfigValue > 0: + case dynamicSplitCfg.MaxShardsPerQuery > 0 && dynamicSplitCfg.MaxFetchedDataDurationPerQuery > 0: + maxSplits = min(maxSplitsPerQuery, maxSplitsFromDurationFetched) + case dynamicSplitCfg.MaxShardsPerQuery > 0: + maxSplits = maxSplitsPerQuery + case dynamicSplitCfg.MaxFetchedDataDurationPerQuery > 0: maxSplits = maxSplitsFromDurationFetched - default: - return baseInterval, nil } interval := getIntervalFromMaxSplits(r, baseInterval, maxSplits) @@ -195,7 +196,7 @@ func dynamicIntervalFn(cfg Config, limits tripperware.Limits, queryAnalyzer quer } } -func getMaxVerticalShardSizeForQuery(ctx context.Context, r tripperware.Request, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer) (int, error) { +func getMaxVerticalShardSize(ctx context.Context, r tripperware.Request, limits tripperware.Limits, queryAnalyzer querysharding.Analyzer) (int, error) { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { return 1, err @@ -213,11 +214,11 @@ func getMaxVerticalShardSizeForQuery(ctx context.Context, r tripperware.Request, return queryVerticalShardSize, nil } +// Returns the minimum multiple of base interval needed to split query into less than maxSplits func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplitsInt int) time.Duration { maxSplits := time.Duration(maxSplitsInt) queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - - // Calculate the multiple n of interval needed to shard query to <= maxSplits + // Calculate the multiple n of interval needed to shard query into <= maxSplits n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) if n <= 0 { n = 1 @@ -246,8 +247,8 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, return n * baseInterval } +// Return max allowed number of splits by MaxShardsPerQuery config after accounting for vertical sharding func getMaxSplitsFromConfig(maxSplitsConfigValue int, queryVerticalShardSize int) int { - // Find max number of allowed splits by MaxSplitsPerQuery config after accounting for vertical sharding var maxSplitsFromConfig int if maxSplitsConfigValue > 0 { maxSplitsFromConfig = maxSplitsConfigValue / queryVerticalShardSize @@ -258,31 +259,18 @@ func getMaxSplitsFromConfig(maxSplitsConfigValue int, queryVerticalShardSize int return maxSplitsFromConfig } -func getMaxSplitsByDurationFetched(maxDurationFetchedConfigValue time.Duration, queryVerticalShardSize int, expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) int { - // Get variables needed to calculate the max number of allowed splits by MaxFetchedStorageDataDurationPerQuery config - intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit, intervalsFetchedByLookbackDeltaForOtherSplits := getVariablesToCalculateMaxSplitsByDurationFetched(expr, queryStart, queryEnd, queryStep, baseInterval, lookbackDelta) - - // Handle different cases for lookbackDelta fetching additional duration of data - if intervalsFetchedByLookbackDeltaForFirstSplit > 0 && intervalsFetchedByLookbackDeltaForOtherSplits > 0 { - // lookbackDelta is fetching additional duration for all splits - extraIntervalsFetchedPerSplit += intervalsFetchedByLookbackDeltaForOtherSplits - } else if intervalsFetchedByLookbackDeltaForOtherSplits > 0 { - // lookbackDelta is fetching additional duration for all splits except first one - extraIntervalsFetchedPerSplit += intervalsFetchedByLookbackDeltaForOtherSplits - intervalsFetchedByQueryRange -= intervalsFetchedByLookbackDeltaForOtherSplits - } else if intervalsFetchedByLookbackDeltaForFirstSplit > 0 { - // lookbackDelta is fetching additional duration for first split only - intervalsFetchedByQueryRange += intervalsFetchedByLookbackDeltaForFirstSplit +// Return max allowed number of splits by MaxFetchedDataDurationPerQuery config after accounting for vertical sharding +func getMaxSplitsByDurationFetched(maxFetchedDataDurationPerQuery time.Duration, queryVerticalShardSize int, expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) int { + fixedDurationFetched, perSplitDurationFetched := getDurationFetchedByQuerySplitting(expr, queryStart, queryEnd, queryStep, baseInterval, lookbackDelta) + if perSplitDurationFetched == 0 { + perSplitDurationFetched = baseInterval // should always divide by a multiple of base interval to get max splits } var maxSplitsByDurationFetched int - if maxDurationFetchedConfigValue > 0 { - // Example equation for duration fetched by query up[15d:1h] with a range of 30 days, 1 day base split interval, and 5 min lookbackDelta - // [duration_fetched_by_range + (extra_duration_fetched_per_split x horizontal_splits)] x vertical_shards <= maxDurationFetchedConfigValue - // [30 + (16 x horizontal_splits)] x vertical_shards <= maxDurationFetchedConfigValue + if maxFetchedDataDurationPerQuery > 0 { + // Duration fetched by query after splitting = fixedDurationFetched + perSplitDurationFetched x numOfShards // Rearranging the equation to find the max horizontal splits - maxDurationFetchedAsMultipleOfInterval := int(maxDurationFetchedConfigValue / baseInterval) - maxSplitsByDurationFetched = ((maxDurationFetchedAsMultipleOfInterval / queryVerticalShardSize) - intervalsFetchedByQueryRange) / extraIntervalsFetchedPerSplit + maxSplitsByDurationFetched = int(((maxFetchedDataDurationPerQuery / time.Duration(queryVerticalShardSize)) - fixedDurationFetched) / perSplitDurationFetched) } if maxSplitsByDurationFetched <= 0 { maxSplitsByDurationFetched = 1 @@ -290,64 +278,77 @@ func getMaxSplitsByDurationFetched(maxDurationFetchedConfigValue time.Duration, return maxSplitsByDurationFetched } -func getVariablesToCalculateMaxSplitsByDurationFetched(expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) (intervalsFetchedByQueryRange int, extraIntervalsFetchedPerSplit int, intervalsFetchedByLookbackDeltaForFirstSplit int, intervalsFetchedByLookbackDeltaForOtherSplits int) { - // First analyze the query using original start-end time. Additional duration fetched by lookbackDelta here only reflects the start time of first split - intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit = analyzeDurationFetchedByQueryExpr(expr, queryStart, queryEnd, baseInterval, lookbackDelta) - if extraIntervalsFetchedPerSplit == 0 { - extraIntervalsFetchedPerSplit = 1 // avoid division by 0 - } +// Return the fixed base duration fetched by the query regardless of the number of splits, and the duration that is fetched once for every split +func getDurationFetchedByQuerySplitting(expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) (fixedDurationFetched time.Duration, perSplitDurationFetched time.Duration) { + // First analyze the query using original start-end time. Duration fetched by lookbackDelta here only reflects the start time of first split + durationFetchedByRange, durationFetchedBySelectors, durationFetchedByLookbackDeltaFirstSplit := analyzeDurationFetchedByQueryExpr(expr, queryStart, queryEnd, baseInterval, lookbackDelta) + + fixedDurationFetched += durationFetchedByRange // Duration fetched by the query range is constant regardless of how many splits the query has + perSplitDurationFetched += durationFetchedBySelectors // Duration fetched by selectors is fetched once for every query split - // Next analyze the query using the next split start time to find the additional duration fetched by lookbackDelta for splits other than first one + // Next analyze the query using the next split start time to find the duration fetched by lookbackDelta for splits other than first one nextIntervalStart := nextIntervalBoundary(queryStart, queryStep, baseInterval) + queryStep - _, _, intervalsFetchedByLookbackDeltaForOtherSplits = analyzeDurationFetchedByQueryExpr(expr, nextIntervalStart, queryEnd, baseInterval, lookbackDelta) + _, _, durationFetchedByLookbackDeltaOtherSplits := analyzeDurationFetchedByQueryExpr(expr, nextIntervalStart, queryEnd, baseInterval, lookbackDelta) + + // Handle different cases for lookbackDelta + if durationFetchedByLookbackDeltaFirstSplit > 0 && durationFetchedByLookbackDeltaOtherSplits > 0 { + // lookbackDelta is fetching additional duration for all splits + perSplitDurationFetched += durationFetchedByLookbackDeltaOtherSplits + } else if durationFetchedByLookbackDeltaOtherSplits > 0 { + // lookbackDelta is fetching additional duration for all splits except first one + perSplitDurationFetched += durationFetchedByLookbackDeltaOtherSplits + fixedDurationFetched -= durationFetchedByLookbackDeltaOtherSplits + } else if durationFetchedByLookbackDeltaFirstSplit > 0 { + // lookbackDelta is fetching additional duration for first split only + fixedDurationFetched += durationFetchedByLookbackDeltaFirstSplit + } - return intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDeltaForFirstSplit, intervalsFetchedByLookbackDeltaForOtherSplits + return fixedDurationFetched, perSplitDurationFetched } -// analyzeDurationFetchedByQueryExpr analyzes the query to identify variables useful for -// calculating the duration of data that will be fetched from storage when the query -// is executed after being split. All variables are expressed as an integer count of multiples -// of the base split interval. +// analyzeDurationFetchedByQueryExpr analyzes the query to calculate +// the duration of data that will be fetched from storage by different +// parts of the query // // Returns: -// - intervalsFetchedByQueryRange: The total number of intervals fetched by the original start-end -// range of the query. This value does not depend on the number of splits. -// - extraIntervalsFetchedPerSplit: The number of additional intervals fetched by matrix selectors -// and/or subqueries. This duration will be fetched by every query split. -// - intervalsFetchedByLookbackDelta: The number of additional intervals fetched by lookbackDelta -// for the specified start time. +// - durationFetchedByRange: The total duration fetched by the original start-end +// range of the query. +// - durationFetchedBySelectors: The duration fetched by matrix selectors +// and/or subqueries. This duration will be fetched once by every query split. +// - durationFetchedByLookbackDelta: The duration fetched by lookbackDelta +// for the specified query start time. // // Example: -// Query up[15d:1h] with a range of 30 days, 1 day base split interval, and 5 min lookbackDelta -// - intervalsFetchedByQueryRange = 30 -// - extraIntervalsFetchedPerSplit = 15 -// - intervalsFetchedByLookbackDelta = 1 -func analyzeDurationFetchedByQueryExpr(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (intervalsFetchedByQueryRange int, extraIntervalsFetchedPerSplit int, intervalsFetchedByLookbackDelta int) { - intervalsFetchedByQueryRange = 0 - intervalsFetchedByLookbackDelta = 0 +// Query up[15d:1h] with a range of 30 days, 1 day base split interval, and 5 min lookbackDelta with 00:00 UTC start time +// - durationFetchedByRange = 30 day +// - durationFetchedBySelectors = 15 day +// - durationFetchedByLookbackDelta = 1 day +func analyzeDurationFetchedByQueryExpr(expr parser.Expr, queryStart int64, queryEnd int64, baseInterval time.Duration, lookbackDelta time.Duration) (durationFetchedByRange time.Duration, durationFetchedBySelectors time.Duration, durationFetchedByLookbackDelta time.Duration) { + durationFetchedByRangeCount := 0 + durationFetchedByLookbackDeltaCount := 0 baseIntervalMillis := util.DurationMilliseconds(baseInterval) - totalIntervalsFetched := 0 + totalDurationFetchedCount := 0 var evalRange time.Duration parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { switch n := node.(type) { case *parser.VectorSelector: - // Increment intervals fetched by the original start-end time range + // Increment duration fetched by the original start-end time range queryStartIntervalIndex := floorDiv(queryStart, baseIntervalMillis) queryEndIntervalIndex := floorDiv(queryEnd, baseIntervalMillis) - intervalsFetchedByQueryRange += int(queryEndIntervalIndex-queryStartIntervalIndex) + 1 + durationFetchedByRangeCount += int(queryEndIntervalIndex-queryStartIntervalIndex) + 1 - // Adjust start and end time based on matrix selectors or subquery and increment total intervals fetched, this excludes lookbackDelta + // Adjust start and end time based on matrix selectors and/or subquery selector and increment total duration fetched, this excludes lookbackDelta start, end := util.GetTimeRangesForSelector(queryStart, queryEnd, 0, n, path, evalRange) startIntervalIndex := floorDiv(start, baseIntervalMillis) endIntervalIndex := floorDiv(end, baseIntervalMillis) - totalIntervalsFetched += int(endIntervalIndex-startIntervalIndex) + 1 + totalDurationFetchedCount += int(endIntervalIndex-startIntervalIndex) + 1 - // Increment intervals fetched by lookbackDelta + // Increment duration fetched by lookbackDelta startLookbackDelta := start - util.DurationMilliseconds(lookbackDelta) startLookbackDeltaIntervalIndex := floorDiv(startLookbackDelta, baseIntervalMillis) if evalRange == 0 && startLookbackDeltaIntervalIndex < startIntervalIndex { - intervalsFetchedByLookbackDelta += int(startIntervalIndex - startLookbackDeltaIntervalIndex) + durationFetchedByLookbackDeltaCount += int(startIntervalIndex - startLookbackDeltaIntervalIndex) } evalRange = 0 case *parser.MatrixSelector: @@ -356,8 +357,8 @@ func analyzeDurationFetchedByQueryExpr(expr parser.Expr, queryStart int64, query return nil }) - extraIntervalsFetchedPerSplit = totalIntervalsFetched - intervalsFetchedByQueryRange - return intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDelta + durationFetchedBySelectorsCount := totalDurationFetchedCount - durationFetchedByRangeCount + return time.Duration(durationFetchedByRangeCount) * baseInterval, time.Duration(durationFetchedBySelectorsCount) * baseInterval, time.Duration(durationFetchedByLookbackDeltaCount) * baseInterval } func floorDiv(a, b int64) int64 { diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 74ac3b71e5..5e3a911355 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -315,7 +315,7 @@ func TestSplitByDay(t *testing.T) { path: longQuery, expectedBody: string(mergedHTTPResponseBody), expectedQueryCount: 8, - intervalFn: dynamicIntervalFn(Config{SplitQueriesByInterval: day, DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{MaxSplitsPerQuery: 10}}, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta), + intervalFn: dynamicIntervalFn(Config{SplitQueriesByInterval: day, DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{MaxShardsPerQuery: 10}}, mockLimits{}, querysharding.NewQueryAnalyzer(), lookbackDelta), }, } { tc := tc @@ -648,8 +648,8 @@ func Test_dynamicIntervalFn(t *testing.T) { cfg := Config{ SplitQueriesByInterval: tc.baseSplitInterval, DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ - MaxSplitsPerQuery: tc.maxQueryIntervalSplits, - MaxFetchedStorageDataDurationPerQuery: tc.maxDurationOfDataFetched, + MaxShardsPerQuery: tc.maxQueryIntervalSplits, + MaxFetchedDataDurationPerQuery: tc.maxDurationOfDataFetched, }, } ctx := user.InjectOrgID(context.Background(), "1") @@ -788,13 +788,13 @@ func Test_getIntervalFromMaxSplits(t *testing.T) { func Test_analyzeDurationFetchedByQuery(t *testing.T) { for _, tc := range []struct { - name string - baseSplitInterval time.Duration - lookbackDelta time.Duration - req tripperware.Request - expectedIntervalsFetchedByQueryRange int - expectedExtraIntervalsFetchedPerSplit int - expectedIntervalsFetchedByLookbackDelta int + name string + baseSplitInterval time.Duration + lookbackDelta time.Duration + req tripperware.Request + expectedDurationFetchedByRange time.Duration + expectedDurationFetchedBySelectors time.Duration + expectedDurationFetchedByLookbackDelta time.Duration }{ { name: "query range 00:00 to 23:59", @@ -806,9 +806,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedIntervalsFetchedByQueryRange: 24, - expectedExtraIntervalsFetchedPerSplit: 0, - expectedIntervalsFetchedByLookbackDelta: 0, + expectedDurationFetchedByRange: 24 * time.Hour, + expectedDurationFetchedBySelectors: 0 * time.Hour, + expectedDurationFetchedByLookbackDelta: 0 * time.Hour, }, { name: "query range 00:00 to 00:00 next day", @@ -820,9 +820,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedIntervalsFetchedByQueryRange: 25, - expectedExtraIntervalsFetchedPerSplit: 0, - expectedIntervalsFetchedByLookbackDelta: 0, + expectedDurationFetchedByRange: 25 * time.Hour, + expectedDurationFetchedBySelectors: 0 * time.Hour, + expectedDurationFetchedByLookbackDelta: 0 * time.Hour, }, { name: "query range 00:00 to 23:59, with 5 min lookback", @@ -834,9 +834,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up", }, - expectedIntervalsFetchedByQueryRange: 24, - expectedExtraIntervalsFetchedPerSplit: 0, - expectedIntervalsFetchedByLookbackDelta: 1, + expectedDurationFetchedByRange: 24 * time.Hour, + expectedDurationFetchedBySelectors: 0 * time.Hour, + expectedDurationFetchedByLookbackDelta: 1 * time.Hour, }, { name: "query range 00:00 to 23:59, with 5 hour subquery", @@ -848,9 +848,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "up[5h:10m]", }, - expectedIntervalsFetchedByQueryRange: 24, - expectedExtraIntervalsFetchedPerSplit: 5, - expectedIntervalsFetchedByLookbackDelta: 1, + expectedDurationFetchedByRange: 24 * time.Hour, + expectedDurationFetchedBySelectors: 5 * time.Hour, + expectedDurationFetchedByLookbackDelta: 1 * time.Hour, }, { name: "query range 00:00 to 23:59, with 2 hour matrix selector", @@ -862,9 +862,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "rate(up[2h])", }, - expectedIntervalsFetchedByQueryRange: 24, - expectedExtraIntervalsFetchedPerSplit: 2, - expectedIntervalsFetchedByLookbackDelta: 0, + expectedDurationFetchedByRange: 24 * time.Hour, + expectedDurationFetchedBySelectors: 2 * time.Hour, + expectedDurationFetchedByLookbackDelta: 0 * time.Hour, }, { name: "query range 00:00 to 23:59, with multiple matrix selectors", @@ -876,9 +876,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 60 * seconds, Query: "rate(up[2h]) + rate(up[5h]) + rate(up[7h])", }, - expectedIntervalsFetchedByQueryRange: 72, - expectedExtraIntervalsFetchedPerSplit: 14, - expectedIntervalsFetchedByLookbackDelta: 0, + expectedDurationFetchedByRange: 72 * time.Hour, + expectedDurationFetchedBySelectors: 14 * time.Hour, + expectedDurationFetchedByLookbackDelta: 0 * time.Hour, }, { name: "query range 60 day with 20 day subquery", @@ -890,9 +890,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[20d:1h]", }, - expectedIntervalsFetchedByQueryRange: 61, - expectedExtraIntervalsFetchedPerSplit: 20, - expectedIntervalsFetchedByLookbackDelta: 1, + expectedDurationFetchedByRange: 61 * day, + expectedDurationFetchedBySelectors: 20 * day, + expectedDurationFetchedByLookbackDelta: 1 * day, }, { name: "query range 35 day, with 15 day subquery and 1 week offset", @@ -904,9 +904,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[15d:1h] offset 1w", }, - expectedIntervalsFetchedByQueryRange: 36, - expectedExtraIntervalsFetchedPerSplit: 15, - expectedIntervalsFetchedByLookbackDelta: 1, + expectedDurationFetchedByRange: 36 * day, + expectedDurationFetchedBySelectors: 15 * day, + expectedDurationFetchedByLookbackDelta: 1 * day, }, { name: "query range 10 days, with multiple subqueries and offsets", @@ -918,9 +918,9 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "rate(up[2d:1h] offset 1w) + rate(up[5d:1h] offset 2w) + rate(up[7d:1h] offset 3w)", }, - expectedIntervalsFetchedByQueryRange: 33, - expectedExtraIntervalsFetchedPerSplit: 14, - expectedIntervalsFetchedByLookbackDelta: 3, + expectedDurationFetchedByRange: 33 * day, + expectedDurationFetchedBySelectors: 14 * day, + expectedDurationFetchedByLookbackDelta: 3 * day, }, { name: "query range spans 40 days with 4 day matrix selector", @@ -932,18 +932,18 @@ func Test_analyzeDurationFetchedByQuery(t *testing.T) { Step: 5 * 60 * seconds, Query: "up[4d]", }, - expectedIntervalsFetchedByQueryRange: 40, - expectedExtraIntervalsFetchedPerSplit: 4, - expectedIntervalsFetchedByLookbackDelta: 0, + expectedDurationFetchedByRange: 40 * day, + expectedDurationFetchedBySelectors: 4 * day, + expectedDurationFetchedByLookbackDelta: 0 * day, }, } { t.Run(tc.name, func(t *testing.T) { expr, err := parser.ParseExpr(tc.req.GetQuery()) require.Nil(t, err) queryRangeIntervals, extraIntervalsPerSplit, lookbackDeltaIntervals := analyzeDurationFetchedByQueryExpr(expr, tc.req.GetStart(), tc.req.GetEnd(), tc.baseSplitInterval, tc.lookbackDelta) - require.Equal(t, tc.expectedIntervalsFetchedByQueryRange, queryRangeIntervals) - require.Equal(t, tc.expectedExtraIntervalsFetchedPerSplit, extraIntervalsPerSplit) - require.Equal(t, tc.expectedIntervalsFetchedByLookbackDelta, lookbackDeltaIntervals) + require.Equal(t, tc.expectedDurationFetchedByRange, queryRangeIntervals) + require.Equal(t, tc.expectedDurationFetchedBySelectors, extraIntervalsPerSplit) + require.Equal(t, tc.expectedDurationFetchedByLookbackDelta, lookbackDeltaIntervals) }) } } From bea70adc1f8fa132bc66d33f2b25332f824c496c Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Sat, 8 Feb 2025 23:32:08 -0800 Subject: [PATCH 17/20] add comments for getIntervalFromMaxSplits Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 77a5793110..95006a8e96 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -218,30 +218,34 @@ func getMaxVerticalShardSize(ctx context.Context, r tripperware.Request, limits func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplitsInt int) time.Duration { maxSplits := time.Duration(maxSplitsInt) queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) - // Calculate the multiple n of interval needed to shard query into <= maxSplits - n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + // Calculate the multiple (n) of base interval needed to shard query into <= maxSplits + // Divide query range by base interval and max splits. Round up by adding (baseInterval*maxSplits - 1) + n := (queryRange + (baseInterval*maxSplits - 1)) / (baseInterval * maxSplits) if n <= 0 { n = 1 } + // Loop to handle cases where first split is truncated and shorter than remaining splits. + // Exits loop if interval (n) is sufficient after removing first split + // If no suitable interval was found terminates at a maximum of interval = 2 * query range for n <= 2*((queryRange+baseInterval-1)/baseInterval) { - // The first split could be truncated and shorter than other splits. - // So it is removed to check if a larger interval is needed + // Find new start time for query after removing first split nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() if maxSplits == 1 { - // No splitting. Exit loop if first split is long enough to cover the full query range + // If maxSplits == 1, the removed first split should cover the full query range. if nextSplitStart >= r.GetEnd() { break } } else { - // Recalculate n for remaining query range after removing first split. - // Exit loop if a larger n is not needed to split into <= maxSplits-1 queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) + // Recalculate n for the remaining query range with maxSplits-1. n_temp := (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) + // If a larger interval is needed after removing the first split, the initial n was insufficient. if n >= n_temp { break } } + // Increment n to check if larger interval fits the maxSplits constraint. n++ } return n * baseInterval From df492d854a1555faeabe333a44752c2a1c8147d7 Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Sun, 9 Feb 2025 14:25:46 -0800 Subject: [PATCH 18/20] add ceilDiv helper function Signed-off-by: Ahmed Hassan --- .../tripperware/queryrange/split_by_interval.go | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 95006a8e96..78b59b864f 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -219,8 +219,7 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, maxSplits := time.Duration(maxSplitsInt) queryRange := time.Duration((r.GetEnd() - r.GetStart()) * int64(time.Millisecond)) // Calculate the multiple (n) of base interval needed to shard query into <= maxSplits - // Divide query range by base interval and max splits. Round up by adding (baseInterval*maxSplits - 1) - n := (queryRange + (baseInterval*maxSplits - 1)) / (baseInterval * maxSplits) + n := ceilDiv(queryRange, baseInterval*maxSplits) if n <= 0 { n = 1 } @@ -228,7 +227,7 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, // Loop to handle cases where first split is truncated and shorter than remaining splits. // Exits loop if interval (n) is sufficient after removing first split // If no suitable interval was found terminates at a maximum of interval = 2 * query range - for n <= 2*((queryRange+baseInterval-1)/baseInterval) { + for n <= 2*ceilDiv(queryRange, baseInterval) { // Find new start time for query after removing first split nextSplitStart := nextIntervalBoundary(r.GetStart(), r.GetStep(), n*baseInterval) + r.GetStep() if maxSplits == 1 { @@ -239,7 +238,7 @@ func getIntervalFromMaxSplits(r tripperware.Request, baseInterval time.Duration, } else { queryRangeWithoutFirstSplit := time.Duration((r.GetEnd() - nextSplitStart) * int64(time.Millisecond)) // Recalculate n for the remaining query range with maxSplits-1. - n_temp := (queryRangeWithoutFirstSplit + baseInterval*(maxSplits-1) - 1) / (baseInterval * (maxSplits - 1)) + n_temp := ceilDiv(queryRangeWithoutFirstSplit, baseInterval*(maxSplits-1)) // If a larger interval is needed after removing the first split, the initial n was insufficient. if n >= n_temp { break @@ -371,3 +370,10 @@ func floorDiv(a, b int64) int64 { } return a / b } + +func ceilDiv(a, b time.Duration) time.Duration { + if a > 0 && a%b != 0 { + return a/b + 1 + } + return a / b +} From ce661ed50618378db4dd7c6f787e9b7d9bb87f4e Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 10 Feb 2025 18:37:36 -0800 Subject: [PATCH 19/20] add default max splits by duration fetched Signed-off-by: Ahmed Hassan --- pkg/querier/tripperware/queryrange/split_by_interval.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 78b59b864f..d7f7b18a88 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -266,13 +266,13 @@ func getMaxSplitsFromConfig(maxSplitsConfigValue int, queryVerticalShardSize int func getMaxSplitsByDurationFetched(maxFetchedDataDurationPerQuery time.Duration, queryVerticalShardSize int, expr parser.Expr, queryStart int64, queryEnd int64, queryStep int64, baseInterval time.Duration, lookbackDelta time.Duration) int { fixedDurationFetched, perSplitDurationFetched := getDurationFetchedByQuerySplitting(expr, queryStart, queryEnd, queryStep, baseInterval, lookbackDelta) if perSplitDurationFetched == 0 { - perSplitDurationFetched = baseInterval // should always divide by a multiple of base interval to get max splits + return int(maxFetchedDataDurationPerQuery / baseInterval) // Total duration fetched does not increase with number of splits, return default max splits } var maxSplitsByDurationFetched int if maxFetchedDataDurationPerQuery > 0 { // Duration fetched by query after splitting = fixedDurationFetched + perSplitDurationFetched x numOfShards - // Rearranging the equation to find the max horizontal splits + // Rearranging the equation to find the max horizontal splits after accounting for vertical shards maxSplitsByDurationFetched = int(((maxFetchedDataDurationPerQuery / time.Duration(queryVerticalShardSize)) - fixedDurationFetched) / perSplitDurationFetched) } if maxSplitsByDurationFetched <= 0 { From 701d39360653566f8b3d4202a762faa5e3214cfb Mon Sep 17 00:00:00 2001 From: Ahmed Hassan Date: Mon, 10 Feb 2025 18:47:33 -0800 Subject: [PATCH 20/20] add new unit tests for helper functions Signed-off-by: Ahmed Hassan --- .../queryrange/split_by_interval_test.go | 352 ++++++++++++++---- 1 file changed, 276 insertions(+), 76 deletions(-) diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 5e3a911355..94d2905598 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -446,79 +446,70 @@ func Test_dynamicIntervalFn(t *testing.T) { expectedInterval time.Duration expectedError bool verticalShardSize int - maxQueryIntervalSplits int + maxIntervalSplits int maxDurationOfDataFetched time.Duration }{ { baseSplitInterval: day, - name: "failed to parse request, return default interval", + name: "failed to parse query, return default interval", req: &tripperware.PrometheusRequest{ Query: "up[aaa", Start: 0, End: 10 * 24 * 3600 * seconds, Step: 5 * 60 * seconds, }, - maxQueryIntervalSplits: 30, + maxIntervalSplits: 30, maxDurationOfDataFetched: 200 * day, expectedInterval: day, expectedError: true, }, { - baseSplitInterval: time.Hour, - name: "48 hour, expect split by 1 hour", + baseSplitInterval: day, + name: "23 hour range with 10 max splits, expect split by 1 day", req: &tripperware.PrometheusRequest{ Start: 0, - End: 2 * 24 * 3600 * seconds, + End: 23 * 3600 * seconds, Step: 60 * seconds, Query: "up", }, - expectedInterval: time.Hour, + maxIntervalSplits: 10, + expectedInterval: day, }, { baseSplitInterval: time.Hour, - name: "120 hour with 15 max splits, expect split by 8 hours", + name: "24 hour range with 100 max splits, expect split by 1 hour", req: &tripperware.PrometheusRequest{ Start: 0, - End: 5 * 24 * 3600 * seconds, - Step: 5 * 60 * seconds, + End: 24 * 3600 * seconds, + Step: 60 * seconds, Query: "up", }, - maxQueryIntervalSplits: 15, - expectedInterval: 8 * time.Hour, - }, - { - baseSplitInterval: 2 * time.Hour, - name: "102 hours with 200 hour max duration fetched, expect split by 4 hours", - req: &tripperware.PrometheusRequest{ - Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), - End: (7 * 24 * 3600 * seconds) + (2*3600*seconds + 60*seconds), - Step: 5 * 60 * seconds, - Query: "up[5m]", - }, - maxDurationOfDataFetched: 200 * time.Hour, - expectedInterval: 4 * time.Hour, + maxIntervalSplits: 100, + expectedInterval: time.Hour, }, { - baseSplitInterval: day, - name: "10 hour range, expect split by 1 day", + baseSplitInterval: time.Hour, + name: "120 hour range with 15 max splits, expect split by 8 hours", req: &tripperware.PrometheusRequest{ Start: 0, - End: 19 * 3600 * seconds, + End: 5 * 24 * 3600 * seconds, Step: 60 * seconds, Query: "up", }, - expectedInterval: day, + maxIntervalSplits: 15, + expectedInterval: 8 * time.Hour, }, { baseSplitInterval: day, - name: "30 day range, expect split by 1 day", + name: "30 day range with 30 max splits, expect split by 1 day", req: &tripperware.PrometheusRequest{ Start: 0, End: 30 * 24 * 3600 * seconds, Step: 5 * 60 * seconds, Query: "up", }, - expectedInterval: day, + maxIntervalSplits: 30, + expectedInterval: day, }, { baseSplitInterval: day, @@ -529,8 +520,8 @@ func Test_dynamicIntervalFn(t *testing.T) { Step: 5 * 60 * seconds, Query: "up", }, - maxQueryIntervalSplits: 20, - expectedInterval: 2 * day, + maxIntervalSplits: 20, + expectedInterval: 2 * day, }, { baseSplitInterval: day, @@ -541,8 +532,8 @@ func Test_dynamicIntervalFn(t *testing.T) { Step: 5 * 60 * seconds, Query: "up", }, - maxQueryIntervalSplits: 15, - expectedInterval: 4 * day, + maxIntervalSplits: 15, + expectedInterval: 4 * day, }, { baseSplitInterval: day, @@ -553,12 +544,37 @@ func Test_dynamicIntervalFn(t *testing.T) { Step: 5 * 60 * seconds, Query: "up", }, - maxQueryIntervalSplits: 15, - expectedInterval: 5 * day, + maxIntervalSplits: 15, + expectedInterval: 5 * day, + }, + { + baseSplitInterval: day, + name: "60 day range with 15 max splits and 3 vertical shard size, expect split by 12 days", + req: &tripperware.PrometheusRequest{ + Start: 0, + End: 60 * 24 * 3600 * seconds, + Step: 5 * 60 * seconds, + Query: "sum by (pod) (up)", + }, + verticalShardSize: 3, + maxIntervalSplits: 15, + expectedInterval: 12 * day, + }, + { + baseSplitInterval: time.Hour, + name: "101 hours with 200 hour max duration fetched and 5m matrix selector, expect split by 2 hour", + req: &tripperware.PrometheusRequest{ + Start: (3 * 24 * 3600 * seconds) - (2 * 3600 * seconds), + End: (7 * 24 * 3600 * seconds) + (3 * 3600 * seconds) - 1, + Step: 60 * seconds, + Query: "up[5m]", + }, + maxDurationOfDataFetched: 200 * time.Hour, + expectedInterval: 2 * time.Hour, }, { baseSplitInterval: day, - name: "30 day range short matrix selector with 200 days max duration fetched, expect split by 1 day", + name: "30 day range with 200 days max duration fetched and 1h matrix selector, expect split by 1 day", req: &tripperware.PrometheusRequest{ Start: 30 * 24 * 3600 * seconds, End: 60 * 24 * 3600 * seconds, @@ -570,7 +586,7 @@ func Test_dynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "30 day range long matrix selector with 200 days max duration fetched, expect split by 4 days", + name: "30 day range with 200 days max duration fetched and 20d matrix selector, expect split by 4 days", req: &tripperware.PrometheusRequest{ Start: 30 * 24 * 3600 * seconds, End: 60 * 24 * 3600 * seconds, @@ -582,82 +598,266 @@ func Test_dynamicIntervalFn(t *testing.T) { }, { baseSplitInterval: day, - name: "query with multiple matrix selectors, expect split by 9 days", + name: "30 day range with multiple matrix selectors, expect split by 3 days", req: &tripperware.PrometheusRequest{ - Start: (14 * 24 * 3600 * seconds) + (3600*seconds - 120*seconds), - End: (52 * 24 * 3600 * seconds) + (2*3600*seconds + 500*seconds), - Step: 60 * seconds, + Start: (14 * 24 * 3600 * seconds) - (3600 * seconds), + End: (32 * 24 * 3600 * seconds) + (2 * 3600 * seconds), + Step: 5 * 60 * seconds, Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", }, maxDurationOfDataFetched: 200 * day, - expectedInterval: 9 * day, + expectedInterval: 3 * day, }, { baseSplitInterval: day, - name: "100 day range with subquery, expect split by 13 days", + name: "30 day range with multiple matrix selectors and 3 vertical shards, expect split by 6 days", + req: &tripperware.PrometheusRequest{ + Start: (14 * 24 * 3600 * seconds) - (3600 * seconds), + End: (32 * 24 * 3600 * seconds) + (2 * 3600 * seconds), + Step: 5 * 60 * seconds, + Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", + }, + verticalShardSize: 3, + maxDurationOfDataFetched: 350 * day, + expectedInterval: 6 * day, + }, + { + baseSplitInterval: day, + name: "100 day range with 5 day subquery, expect split by 13 days", req: &tripperware.PrometheusRequest{ Start: 0, End: 100 * 24 * 3600 * seconds, Step: 60 * 60 * seconds, Query: "up[5d:10m]", }, - maxQueryIntervalSplits: 100, + maxIntervalSplits: 100, maxDurationOfDataFetched: 150 * day, expectedInterval: 13 * day, }, { baseSplitInterval: day, - name: "60 day range with 3 vertical shard size and 15 max splits, expect split by 12 days", + name: "51 day range, longer than max duration fetched, expect split by 51 day", req: &tripperware.PrometheusRequest{ Start: 0, - End: 60 * 24 * 3600 * seconds, + End: (51 * 24 * 3600 * seconds), Step: 5 * 60 * seconds, - Query: "sum by (pod) (up)", + Query: "up[5d]", }, - verticalShardSize: 3, - maxQueryIntervalSplits: 15, - expectedInterval: 12 * day, + maxDurationOfDataFetched: 50 * day, + expectedInterval: 51 * day, }, + } { + t.Run(tc.name, func(t *testing.T) { + cfg := Config{ + SplitQueriesByInterval: tc.baseSplitInterval, + DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ + MaxShardsPerQuery: tc.maxIntervalSplits, + MaxFetchedDataDurationPerQuery: tc.maxDurationOfDataFetched, + }, + } + ctx := user.InjectOrgID(context.Background(), "1") + interval, err := dynamicIntervalFn(cfg, mockLimits{queryVerticalShardSize: tc.verticalShardSize}, querysharding.NewQueryAnalyzer(), lookbackDelta)(ctx, tc.req) + require.Equal(t, tc.expectedInterval, interval) + if !tc.expectedError { + require.Nil(t, err) + } + }) + } +} + +func Test_getMaxSplitsByDurationFetched(t *testing.T) { + for _, tc := range []struct { + name string + baseSplitInterval time.Duration + req tripperware.Request + verticalShardSize int + lookbackDelta time.Duration + maxDurationOfDataFetched time.Duration + expectedMaxSplits int + }{ { baseSplitInterval: day, - name: "query with multiple matrix selectors and 3 vertical shard size, expect split by 33 day", + name: "20 day range with no lookback, expect default max splits", req: &tripperware.PrometheusRequest{ - Start: (14 * 24 * 3600 * seconds) + (3600*seconds - 120*seconds), - End: (32 * 24 * 3600 * seconds) + (2*3600*seconds + 500*seconds), - Step: 60 * seconds, + Query: "up", + Start: 0, + End: (20 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + }, + verticalShardSize: 1, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 200, + }, + { + baseSplitInterval: day, + name: "20 days range with 5 min lookback, expect 180 max splits", + req: &tripperware.PrometheusRequest{ + Query: "up", + Start: 0, + End: (20 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + }, + verticalShardSize: 1, + lookbackDelta: 5 * time.Minute, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 180, + }, + { + baseSplitInterval: day, + name: "20 days range with 5 min lookback but first split starts at 30 min, expect 181 max splits", + req: &tripperware.PrometheusRequest{ + Query: "up", + Start: (30 * 60 * seconds), + End: (20 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + }, + verticalShardSize: 1, + lookbackDelta: 5 * time.Minute, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 181, + }, + { + baseSplitInterval: day, + name: "20 days range with 1 day subquery and 5 min lookback, expect 90 max splits", + req: &tripperware.PrometheusRequest{ + Query: "up[1d:1h]", + Start: 0, + End: (20 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + }, + verticalShardSize: 1, + lookbackDelta: 5 * time.Minute, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 90, + }, + { + baseSplitInterval: day, + // Certain step sizes can cause the start time of every split except the first one to shift forward. In these cases, + // lookbackDelta might not fetch additional days for all splits (except the first one if it starts at 00:00 UTC) + name: "20 days range with 1 day subquery and 5 min lookback but splits are step aligned forward, expect 179 max splits", + req: &tripperware.PrometheusRequest{ + Query: "up[1d:1h]", + Start: 0, + End: (20 * 24 * 3600 * seconds) - 1, + Step: 22 * 60 * seconds, + }, + verticalShardSize: 1, + lookbackDelta: 5 * time.Minute, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 179, + }, + { + baseSplitInterval: day, + name: "20 days range with 3 day matrix selector, expect 60 max splits", + req: &tripperware.PrometheusRequest{ + Query: "up[3d]", + Start: 0, + End: (20 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + }, + verticalShardSize: 1, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 60, + }, + { + baseSplitInterval: day, + name: "30 days range with 20 day subquery, expect 8 max splits", + req: &tripperware.PrometheusRequest{ + Start: 30 * 24 * 3600 * seconds, + End: (60 * 24 * 3600 * seconds) - 1, + Step: 5 * 60 * seconds, + Query: "avg_over_time(up[20d:1h])", + }, + verticalShardSize: 1, + lookbackDelta: 5 * time.Minute, + maxDurationOfDataFetched: 200 * day, + expectedMaxSplits: 8, + }, + { + baseSplitInterval: day, + // Each term in the query fetches 20 days of range + name: "20 days range with multiple matrix selectors, expect 20 max splits", + req: &tripperware.PrometheusRequest{ + Start: (14 * 24 * 3600 * seconds) - (3600 * seconds), + End: (32 * 24 * 3600 * seconds) + (2 * 3600 * seconds), + Step: 5 * 60 * seconds, + Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", + }, + verticalShardSize: 1, + maxDurationOfDataFetched: 350 * day, + expectedMaxSplits: 20, + }, + { + baseSplitInterval: day, + name: "20 days range with multiple matrix selectors and 3 vertical shard size, expect 4 max splits", + req: &tripperware.PrometheusRequest{ + Start: (14 * 24 * 3600 * seconds) - (3600 * seconds), + End: (32 * 24 * 3600 * seconds) + (2 * 3600 * seconds), + Step: 5 * 60 * seconds, Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", }, verticalShardSize: 3, - maxDurationOfDataFetched: 200 * day, - expectedInterval: 33 * day, + maxDurationOfDataFetched: 350 * day, + expectedMaxSplits: 4, }, { - baseSplitInterval: 2 * time.Hour, - name: "duration of data fetched is much larger than config, expect large interval and no sharding", + baseSplitInterval: day, + name: "around 37 days range with 5 day matrix selector and 3 vertical shards, expect 25 max splits", req: &tripperware.PrometheusRequest{ Start: (3 * 24 * 3600 * seconds) - (4*3600*seconds + 240*seconds), - End: (7 * 24 * 3600 * seconds) + (1*3600*seconds + 60*seconds), + End: (38 * 24 * 3600 * seconds) + (1*3600*seconds + 60*seconds), Step: 5 * 60 * seconds, - Query: "up[5h]", + Query: "up[5d]", }, - maxDurationOfDataFetched: 50 * time.Hour, - expectedInterval: 170 * time.Hour, + verticalShardSize: 3, + maxDurationOfDataFetched: 500 * day, + expectedMaxSplits: 25, }, } { t.Run(tc.name, func(t *testing.T) { - cfg := Config{ - SplitQueriesByInterval: tc.baseSplitInterval, - DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ - MaxShardsPerQuery: tc.maxQueryIntervalSplits, - MaxFetchedDataDurationPerQuery: tc.maxDurationOfDataFetched, - }, - } - ctx := user.InjectOrgID(context.Background(), "1") - interval, err := dynamicIntervalFn(cfg, mockLimits{queryVerticalShardSize: tc.verticalShardSize}, querysharding.NewQueryAnalyzer(), lookbackDelta)(ctx, tc.req) - require.Equal(t, tc.expectedInterval, interval) - if !tc.expectedError { - require.Nil(t, err) - } + queryExpr, err := parser.ParseExpr(tc.req.GetQuery()) + require.Nil(t, err) + maxSplits := getMaxSplitsByDurationFetched(tc.maxDurationOfDataFetched, tc.verticalShardSize, queryExpr, tc.req.GetStart(), tc.req.GetEnd(), tc.req.GetStep(), tc.baseSplitInterval, tc.lookbackDelta) + require.Equal(t, tc.expectedMaxSplits, maxSplits) + }) + } +} + +func Test_getMaxSplitsFromConfig(t *testing.T) { + for i, tc := range []struct { + maxIntervalSplits int + verticalShardSize int + expectedMaxSplits int + }{ + { + verticalShardSize: 1, + maxIntervalSplits: 100, + expectedMaxSplits: 100, + }, + { + verticalShardSize: 1, + maxIntervalSplits: 15, + expectedMaxSplits: 15, + }, + { + verticalShardSize: 1, + maxIntervalSplits: 0, + expectedMaxSplits: 1, + }, + { + verticalShardSize: 3, + maxIntervalSplits: 15, + expectedMaxSplits: 5, + }, + { + verticalShardSize: 5, + maxIntervalSplits: 103, + expectedMaxSplits: 20, + }, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + maxSplits := getMaxSplitsFromConfig(tc.maxIntervalSplits, tc.verticalShardSize) + require.Equal(t, tc.expectedMaxSplits, maxSplits) }) } }