diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 5479bb4d00..aec96d1bf8 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -4193,6 +4193,22 @@ 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] +dynamic_query_splits: + # [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 [align_queries_with_step: | default = false] diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index 94da14fb10..2c8e633764 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-splits-per-query` (int) CLI flag + - `querier.max-fetched-storage-data-duration-per-query` (duration) CLI flag 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/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/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 8af1ba1f84..20991cb3c3 100644 --- a/pkg/querier/tripperware/queryrange/query_range_middlewares.go +++ b/pkg/querier/tripperware/queryrange/query_range_middlewares.go @@ -34,11 +34,14 @@ 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"` + // 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"` @@ -54,6 +57,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { 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. @@ -66,9 +70,25 @@ 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.SplitQueriesByInterval <= 0 { + return errors.New("configs under dynamic-query-splits requires that a value for split-queries-by-interval is set.") + } + } 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, @@ -89,8 +109,11 @@ 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 := staticIntervalFn(cfg) + 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)) } var c cache.Cache diff --git a/pkg/querier/tripperware/queryrange/split_by_interval.go b/pkg/querier/tripperware/queryrange/split_by_interval.go index 064b53d760..c03db327be 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval.go @@ -8,15 +8,20 @@ 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" + 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 +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) 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, @@ -28,15 +33,17 @@ 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", }), + 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 @@ -45,12 +52,20 @@ type splitByInterval struct { 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.StatusInternalServerError, 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 @@ -135,3 +150,219 @@ func nextIntervalBoundary(t, step int64, interval time.Duration) int64 { } return target } + +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, 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 + + queryVerticalShardSize, err := getMaxVerticalShardSizeForQuery(ctx, r, limits, queryAnalyzer) + if err != nil { + return baseInterval, err + } + + queryExpr, err := parser.ParseExpr(r.GetQuery()) + if err != nil { + 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) + + // 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: + maxSplits = maxSplitsFromDurationFetched + default: + return baseInterval, nil + } + + interval := getIntervalFromMaxSplits(r, baseInterval, maxSplits) + return interval, nil + } +} + +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 + } + + 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, 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 + n := (queryRange + baseInterval*maxSplits - 1) / (baseInterval * maxSplits) + if n <= 0 { + n = 1 + } + + 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 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++ + } + return n * baseInterval +} + +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 an integer count of multiples +// of the base split interval. +// +// 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. +// +// 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 + baseIntervalMillis := util.DurationMilliseconds(baseInterval) + + 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 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 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) + totalIntervalsFetched += int(endIntervalIndex-startIntervalIndex) + 1 + + // Increment intervals fetched by lookbackDelta + startLookbackDelta := start - util.DurationMilliseconds(lookbackDelta) + startLookbackDeltaIntervalIndex := floorDiv(startLookbackDelta, baseIntervalMillis) + if evalRange == 0 && startLookbackDeltaIntervalIndex < startIntervalIndex { + intervalsFetchedByLookbackDelta += int(startIntervalIndex - startLookbackDeltaIntervalIndex) + } + evalRange = 0 + case *parser.MatrixSelector: + evalRange = n.Range + } + return nil + }) + + extraIntervalsFetchedPerSplit = totalIntervalsFetched - intervalsFetchedByQueryRange + return intervalsFetchedByQueryRange, extraIntervalsFetchedPerSplit, intervalsFetchedByLookbackDelta +} + +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 23989ac64a..74ac3b71e5 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,12 @@ 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 + 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) { t.Parallel() @@ -285,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) { @@ -305,11 +335,10 @@ 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 } 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(tc.intervalFn, mockLimits{}, PrometheusCodec, nil, lookbackDelta)) req, err := http.NewRequest("GET", tc.path, http.NoBody) require.NoError(t, err) @@ -408,3 +437,513 @@ func Test_evaluateAtModifier(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 + }{ + { + 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, + 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: "120 hour 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: "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, + }, + { + 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", + 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 with 15 max splits, 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 with 15 max splits, 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 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])", + }, + maxDurationOfDataFetched: 200 * day, + expectedInterval: day, + }, + { + baseSplitInterval: 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])", + }, + maxDurationOfDataFetched: 200 * day, + expectedInterval: 4 * day, + }, + { + baseSplitInterval: 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), + Step: 60 * seconds, + Query: "rate(up[2d]) + rate(up[5d]) + rate(up[7d])", + }, + maxDurationOfDataFetched: 200 * day, + expectedInterval: 9 * day, + }, + { + baseSplitInterval: day, + name: "100 day range with 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, + 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", + 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[5h]", + }, + maxDurationOfDataFetched: 50 * time.Hour, + expectedInterval: 170 * time.Hour, + }, + } { + t.Run(tc.name, func(t *testing.T) { + cfg := Config{ + SplitQueriesByInterval: tc.baseSplitInterval, + DynamicQuerySplitsConfig: DynamicQuerySplitsConfig{ + MaxSplitsPerQuery: tc.maxQueryIntervalSplits, + MaxFetchedStorageDataDurationPerQuery: 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_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: "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, + 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) + }) + } +} + +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: "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", + }, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 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", + }, + expectedIntervalsFetchedByQueryRange: 25, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 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", + }, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 0, + expectedIntervalsFetchedByLookbackDelta: 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]", + }, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 5, + expectedIntervalsFetchedByLookbackDelta: 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])", + }, + expectedIntervalsFetchedByQueryRange: 24, + expectedExtraIntervalsFetchedPerSplit: 2, + expectedIntervalsFetchedByLookbackDelta: 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])", + }, + expectedIntervalsFetchedByQueryRange: 72, + expectedExtraIntervalsFetchedPerSplit: 14, + expectedIntervalsFetchedByLookbackDelta: 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]", + }, + expectedIntervalsFetchedByQueryRange: 61, + expectedExtraIntervalsFetchedPerSplit: 20, + expectedIntervalsFetchedByLookbackDelta: 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", + }, + expectedIntervalsFetchedByQueryRange: 36, + expectedExtraIntervalsFetchedPerSplit: 15, + expectedIntervalsFetchedByLookbackDelta: 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)", + }, + expectedIntervalsFetchedByQueryRange: 33, + expectedExtraIntervalsFetchedPerSplit: 14, + expectedIntervalsFetchedByLookbackDelta: 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]", + }, + 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 := 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) + }) + } +} diff --git a/pkg/util/time.go b/pkg/util/time.go index 3f19a71da9..5e52493292 100644 --- a/pkg/util/time.go +++ b/pkg/util/time.go @@ -240,3 +240,72 @@ 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)) +} + +// 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) + + 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 +} + +// 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). +// 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 +}