Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/querier/queryrange/querysharding_test.go

948 lines
26 KiB

Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
package queryrange
import (
"context"
"errors"
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
"fmt"
"math"
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
"sort"
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
"strings"
"sync"
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
"testing"
"time"
"github.com/go-kit/log"
"github.com/grafana/dskit/user"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/v3/pkg/loghttp"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logql"
"github.com/grafana/loki/v3/pkg/logql/syntax"
"github.com/grafana/loki/v3/pkg/logqlmodel/stats"
"github.com/grafana/loki/v3/pkg/querier/plan"
"github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase/definitions"
"github.com/grafana/loki/v3/pkg/storage/config"
"github.com/grafana/loki/v3/pkg/storage/types"
"github.com/grafana/loki/v3/pkg/util"
"github.com/grafana/loki/v3/pkg/util/constants"
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
)
var (
nilShardingMetrics = logql.NewShardMapperMetrics(nil)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
defaultReq = func() *LokiRequest {
return &LokiRequest{
Step: 1000,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
Limit: 100,
StartTs: start,
EndTs: end,
Direction: logproto.BACKWARD,
Path: "/loki/api/v1/query_range",
}
}
lokiResps = []queryrangebase.Response{
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: defaultReq().Limit,
Version: 1,
Headers: []definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="debug"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 6), Line: "6"},
{Timestamp: time.Unix(0, 5), Line: "5"},
},
},
},
},
},
&LokiResponse{
Status: loghttp.QueryStatusSuccess,
Direction: logproto.BACKWARD,
Limit: 100,
Version: 1,
Headers: []definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{
{
Labels: `{foo="bar", level="error"}`,
Entries: []logproto.Entry{
{Timestamp: time.Unix(0, 2), Line: "2"},
{Timestamp: time.Unix(0, 1), Line: "1"},
},
},
},
},
},
}
)
func Test_shardSplitter(t *testing.T) {
req := defaultReq().WithStartEnd(start, end)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
for _, tc := range []struct {
desc string
lookback time.Duration
shouldShard bool
}{
{
desc: "older than lookback",
lookback: -time.Minute, // a negative lookback will ensure the entire query doesn't cross the sharding boundary & can safely be sharded.
shouldShard: true,
},
{
desc: "overlaps lookback",
lookback: end.Sub(start) / 2, // intersect the request causing it to avoid sharding
shouldShard: false,
},
{
desc: "newer than lookback",
lookback: end.Sub(start) + 1, // the entire query is in the ingester range and should avoid sharding.
shouldShard: false,
},
{
desc: "default",
lookback: 0,
shouldShard: true,
},
} {
t.Run(tc.desc, func(t *testing.T) {
var didShard bool
splitter := &shardSplitter{
shardingware: queryrangebase.HandlerFunc(func(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
didShard = true
return mockHandler(lokiResps[0], nil).Do(ctx, req)
}),
next: mockHandler(lokiResps[1], nil),
now: func() time.Time { return end },
limits: fakeLimits{
minShardingLookback: tc.lookback,
queryTimeout: time.Minute,
maxQueryParallelism: 1,
},
}
resp, err := splitter.Do(user.InjectOrgID(context.Background(), "1"), req)
require.Nil(t, err)
require.Equal(t, tc.shouldShard, didShard)
require.Nil(t, err)
if tc.shouldShard {
require.Equal(t, lokiResps[0], resp)
} else {
require.Equal(t, lokiResps[1], resp)
}
})
}
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
}
func Test_astMapper(t *testing.T) {
var lock sync.Mutex
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
called := 0
handler := queryrangebase.HandlerFunc(func(_ context.Context, _ queryrangebase.Request) (queryrangebase.Response, error) {
lock.Lock()
defer lock.Unlock()
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
resp := lokiResps[called]
called++
return resp, nil
})
mware := newASTMapperware(
ShardingConfigs{
config.PeriodConfig{
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
RowShards: 2,
},
},
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
testEngineOpts,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
handler,
handler,
nil,
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1, queryTimeout: time.Second},
0,
[]string{},
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
)
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
req := defaultReq()
req.Query = `{foo="bar"}`
req.Plan = &plan.QueryPlan{
AST: syntax.MustParseExpr(req.Query),
}
resp, err := mware.Do(user.InjectOrgID(context.Background(), "1"), req)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
require.Nil(t, err)
require.Equal(t, []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
}, resp.GetHeaders())
expected, err := DefaultCodec.MergeResponse(lokiResps...)
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
sort.Sort(logproto.Streams(expected.(*LokiResponse).Data.Result))
require.Nil(t, err)
require.Equal(t, called, 2)
require.Equal(t, expected.(*LokiResponse).Data, resp.(*LokiResponse).Data)
}
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
func Test_astMapper_QuerySizeLimits(t *testing.T) {
noErr := ""
for _, tc := range []struct {
desc string
query string
maxQuerierBytesSize int
err string
expectedStatsHandlerHits int
}{
{
desc: "Non shardable query",
query: `sum_over_time({app="foo"} |= "foo" | unwrap foo [1h])`,
maxQuerierBytesSize: 100,
err: noErr,
expectedStatsHandlerHits: 1,
},
{
desc: "Non shardable query too big",
query: `avg_over_time({job="foo"} | json busy="utilization" | unwrap busy [5m])`,
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
maxQuerierBytesSize: 10,
err: fmt.Sprintf(limErrQuerierTooManyBytesUnshardableTmpl, "100 B", "10 B"),
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
expectedStatsHandlerHits: 1,
},
{
desc: "Shardable query",
query: `count_over_time({app="foo"} |= "foo" [1h])`,
maxQuerierBytesSize: 100,
err: noErr,
expectedStatsHandlerHits: 1,
},
{
desc: "Shardable query too big",
query: `count_over_time({app="foo"} |= "foo" [1h])`,
maxQuerierBytesSize: 10,
err: fmt.Sprintf(limErrQuerierTooManyBytesShardableTmpl, "100 B", "10 B"),
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
expectedStatsHandlerHits: 1,
},
{
desc: "Partially Shardable query fitting",
query: `count_over_time({app="foo"} |= "foo" [1h]) - sum_over_time({app="foo"} |= "foo" | unwrap foo [1h])`,
maxQuerierBytesSize: 100,
err: noErr,
expectedStatsHandlerHits: 2,
},
{
desc: "Partially Shardable LHS too big",
query: `count_over_time({app="bar"} |= "bar" [1h]) - sum_over_time({app="foo"} |= "foo" | unwrap foo [1h])`,
maxQuerierBytesSize: 100,
err: fmt.Sprintf(limErrQuerierTooManyBytesShardableTmpl, "500 B", "100 B"),
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
expectedStatsHandlerHits: 2,
},
{
desc: "Partially Shardable RHS too big",
query: `count_over_time({app="foo"} |= "foo" [1h]) - sum_over_time({app="bar"} |= "bar" | unwrap foo [1h])`,
maxQuerierBytesSize: 100,
err: fmt.Sprintf(limErrQuerierTooManyBytesShardableTmpl, "500 B", "100 B"),
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
expectedStatsHandlerHits: 2,
},
} {
t.Run(tc.desc, func(t *testing.T) {
statsCalled := 0
handler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
if casted, ok := req.(*logproto.IndexStatsRequest); ok {
statsCalled++
var bytes uint64
if strings.Contains(casted.Matchers, `app="foo"`) {
bytes = 100
}
if strings.Contains(casted.Matchers, `app="bar"`) {
bytes = 500
}
return &IndexStatsResponse{
Response: &logproto.IndexStatsResponse{
Bytes: bytes,
},
}, nil
}
if _, ok := req.(*LokiRequest); ok {
return &LokiPromResponse{Response: &queryrangebase.PrometheusResponse{
Data: queryrangebase.PrometheusData{
ResultType: loghttp.ResultTypeVector,
Result: []queryrangebase.SampleStream{
{
Labels: []logproto.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []logproto.LegacySample{{Value: 10, TimestampMs: 10}},
},
},
},
}}, nil
}
return nil, nil
})
mware := newASTMapperware(
ShardingConfigs{
config.PeriodConfig{
RowShards: 2,
IndexType: types.TSDBType,
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
},
},
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
testEngineOpts,
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
handler,
handler,
nil,
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{
maxSeries: math.MaxInt32,
maxQueryParallelism: 1,
tsdbMaxQueryParallelism: 1,
queryTimeout: time.Minute,
maxQuerierBytesRead: tc.maxQuerierBytesSize,
},
0,
[]string{},
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
)
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
req := defaultReq()
req.Query = tc.query
req.Plan = &plan.QueryPlan{
AST: syntax.MustParseExpr(tc.query),
}
_, err := mware.Do(user.InjectOrgID(context.Background(), "1"), req)
Max bytes read limit (#8670) **What this PR does / why we need it**: This PR implements two new per-tenant limits that are enforced on log and metric queries (both range and instant) when TSDB is used: - `max_query_bytes_read`: Refuse queries that would read more than the configured bytes here. Overall limit regardless of splitting/sharding. The goal is to refuse queries that would take too long. The default value of 0 disables this limit. - `max_querier_bytes_read`: Refuse queries in which any of their subqueries after splitting and sharding would read more than the configured bytes here. The goal is to avoid a querier from running a query that would load too much data in memory and can potentially get OOMed. The default value of 0 disables this limit. These new limits can be configured per tenant and per query (see https://github.com/grafana/loki/pull/8727). The bytes a query would read are estimated through TSDB's index stats. Even though they are not exact, they are good enough to have a rough estimation of whether a query is too big to run or not. For more details on this refer to this discussion in the PR: https://github.com/grafana/loki/pull/8670#discussion_r1124858508. Both limits are implemented in the frontend. Even though we considered implementing `max_querier_bytes_read` in the querier, this way, the limits for pre and post splitting/sharding queries are enforced close to each other on the same component. Moreover, this way we can reduce the number of index stats requests issued to the index gateways by reusing the stats gathered while sharding the query. With regard to how index stats requests are issued: - We parallelize index stats requests by splitting them into queries that span up to 24h since our indices are sharded by 24h periods. On top of that, this prevents a single index gateway from processing a single huge request like `{app=~".+"} for 30d`. - If sharding is enabled and the query is shardable, for `max_querier_bytes_read`, we re-use the stats requests issued by the sharding ware. Specifically, we look at the [bytesPerShard][1] to enforce this limit. Note that once we merge this PR and enable these limits, the load of index stats requests will increase substantially and we may discover bottlenecks in our index gateways and TSDB. After speaking with @owen-d, we think it should be fine as, if needed, we can scale up our index gateways and support caching index stats requests. Here's a demo of this working: <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png"> <img width="1647" alt="image" src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png"> **Which issue(s) this PR fixes**: This PR addresses https://github.com/grafana/loki-private/issues/674. **Special notes for your reviewer**: - @jeschkies has reviewed the changes related to query-time limits. - I've done some refactoring in this PR: - Extracted logic to get stats for a set of matches into a new function [getStatsForMatchers][2]. - Extracted the _Handler_ interface implementation for [queryrangebase.roundTripper][3] into a new type [queryrangebase.roundTripperHandler][4]. This is used to create the handler that skips the rest of configured middlewares when sending an index stat quests ([example][5]). **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
if err != nil {
require.ErrorContains(t, err, tc.err)
}
require.Equal(t, tc.expectedStatsHandlerHits, statsCalled)
})
}
}
func Test_ShardingByPass(t *testing.T) {
called := 0
handler := queryrangebase.HandlerFunc(func(_ context.Context, _ queryrangebase.Request) (queryrangebase.Response, error) {
called++
return nil, nil
})
mware := newASTMapperware(
ShardingConfigs{
config.PeriodConfig{
RowShards: 2,
},
},
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
testEngineOpts,
handler,
handler,
nil,
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1},
0,
[]string{},
)
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
req := defaultReq()
req.Query = `1+1`
req.Plan = &plan.QueryPlan{
AST: syntax.MustParseExpr(req.Query),
}
_, err := mware.Do(user.InjectOrgID(context.Background(), "1"), req)
require.Nil(t, err)
require.Equal(t, called, 1)
}
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
func Test_hasShards(t *testing.T) {
for i, tc := range []struct {
input ShardingConfigs
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
expected bool
}{
{
input: ShardingConfigs{
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
{},
},
expected: false,
},
{
input: ShardingConfigs{
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
{RowShards: 16},
},
expected: true,
},
{
input: ShardingConfigs{
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
{},
{RowShards: 16},
{},
},
expected: true,
},
{
input: nil,
expected: false,
},
} {
t.Run(fmt.Sprintf("%d", i), func(t *testing.T) {
require.Equal(t, tc.expected, hasShards(tc.input))
})
}
}
// astmapper successful stream & prom conversion
func mockHandler(resp queryrangebase.Response, err error) queryrangebase.Handler {
return queryrangebase.HandlerFunc(func(ctx context.Context, _ queryrangebase.Request) (queryrangebase.Response, error) {
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
if expired := ctx.Err(); expired != nil {
return nil, expired
}
return resp, err
})
}
func Test_InstantSharding(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
var lock sync.Mutex
called := 0
shards := []string{}
cpyPeriodConf := testSchemas[0]
cpyPeriodConf.RowShards = 3
sharding := NewQueryShardMiddleware(log.NewNopLogger(), ShardingConfigs{
cpyPeriodConf,
}, testEngineOpts, queryrangebase.NewInstrumentMiddlewareMetrics(nil, constants.Loki),
nilShardingMetrics,
fakeLimits{
maxSeries: math.MaxInt32,
maxQueryParallelism: 10,
queryTimeout: time.Second,
},
0,
nil,
nil,
[]string{},
)
response, err := sharding.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
lock.Lock()
defer lock.Unlock()
called++
shards = append(shards, r.(*LokiInstantRequest).Shards...)
return &LokiPromResponse{Response: &queryrangebase.PrometheusResponse{
Data: queryrangebase.PrometheusData{
ResultType: loghttp.ResultTypeVector,
Result: []queryrangebase.SampleStream{
{
Labels: []logproto.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []logproto.LegacySample{{Value: 10, TimestampMs: 10}},
},
},
},
}}, nil
})).Do(ctx, &LokiInstantRequest{
Query: `rate({app="foo"}[1m])`,
TimeTs: util.TimeFromMillis(10),
Path: "/v1/query",
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
Plan: &plan.QueryPlan{
AST: syntax.MustParseExpr(`rate({app="foo"}[1m])`),
},
})
require.NoError(t, err)
require.Equal(t, 3, called, "expected 3 calls but got {}", called)
require.Len(t, response.(*LokiPromResponse).Response.Data.Result, 3)
require.ElementsMatch(t, []string{"0_of_3", "1_of_3", "2_of_3"}, shards)
require.Equal(t, queryrangebase.PrometheusData{
ResultType: loghttp.ResultTypeVector,
Result: []queryrangebase.SampleStream{
{
Labels: []logproto.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []logproto.LegacySample{{Value: 10, TimestampMs: 10}},
},
{
Labels: []logproto.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []logproto.LegacySample{{Value: 10, TimestampMs: 10}},
},
{
Labels: []logproto.LabelAdapter{{Name: "foo", Value: "bar"}},
Samples: []logproto.LegacySample{{Value: 10, TimestampMs: 10}},
},
},
}, response.(*LokiPromResponse).Response.Data)
require.Equal(t, loghttp.QueryStatusSuccess, response.(*LokiPromResponse).Response.Status)
}
func Test_SeriesShardingHandler(t *testing.T) {
sharding := NewSeriesQueryShardMiddleware(log.NewNopLogger(), ShardingConfigs{
config.PeriodConfig{
RowShards: 3,
},
},
queryrangebase.NewInstrumentMiddlewareMetrics(nil, constants.Loki),
nilShardingMetrics,
fakeLimits{
maxQueryParallelism: 10,
},
DefaultCodec,
)
ctx := user.InjectOrgID(context.Background(), "1")
response, err := sharding.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
req, ok := r.(*LokiSeriesRequest)
if !ok {
return nil, errors.New("not a series call")
}
return &LokiSeriesResponse{
Status: "success",
Version: 1,
Data: []logproto.SeriesIdentifier{
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "foo", Value: "bar"},
},
},
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "shard", Value: req.Shards[0]},
},
},
},
}, nil
})).Do(ctx, &LokiSeriesRequest{
Match: []string{"foo", "bar"},
StartTs: time.Unix(0, 1),
EndTs: time.Unix(0, 10),
Path: "foo",
})
expected := &LokiSeriesResponse{
Statistics: stats.Result{Summary: stats.Summary{Splits: 3}},
Status: "success",
Version: 1,
Data: []logproto.SeriesIdentifier{
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "foo", Value: "bar"},
},
},
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "shard", Value: "0_of_3"},
},
},
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "shard", Value: "1_of_3"},
},
},
{
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
Labels: []logproto.SeriesIdentifier_LabelsEntry{
{Key: "shard", Value: "2_of_3"},
},
},
},
}
actual := response.(*LokiSeriesResponse)
require.NoError(t, err)
Optimize series response format by using repeated fileds. (#11498) **What this PR does / why we need it**: The Protobuf map type is encodied as a repeated field of map entries. Decoding them to a slice is much faster than decoding them into a map. Since Loki is not using the fast key check for a map we can use the slice decoding. This change also allows us to decode the JSON directly into the right protobuf struct. This doulbes the JSON decoding speed and reduces the memory pressure by ~40%. ``` › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > before.log › go test -bench=. -run=^$ -count=10 ./pkg/querier/queryrange > after.log › benchstat before.log after.log goos: darwin goarch: arm64 pkg: github.com/grafana/loki/pkg/querier/queryrange │ before.log │ after.log │ │ sec/op │ sec/op vs base │ ResponseMerge/mergeStreams_unlimited-10 32.36m ± 0% 32.63m ± 2% ~ (p=0.393 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_unlimited-10 1.050m ± 1% 1.080m ± 3% +2.84% (p=0.005 n=10) ResponseMerge/mergeStreams_limited-10 33.02m ± 0% 32.60m ± 1% -1.29% (p=0.004 n=10) ResponseMerge/mergeOrderedNonOverlappingStreams_limited-10 15.11m ± 0% 15.07m ± 0% ~ (p=0.075 n=10) _CodecDecodeLogs-10 4.395m ± 1% 4.364m ± 0% -0.72% (p=0.005 n=10) _CodecDecodeSamples-10 16.97m ± 0% 16.84m ± 2% -0.77% (p=0.023 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 745.8µ ± 8% 736.8µ ± 12% ~ (p=0.739 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 15.37m ± 1% 10.60m ± 0% -31.03% (p=0.000 n=10) _MergeResponses-10 1186.9m ± 2% 149.8m ± 1% -87.38% (p=0.000 n=10) _UnwrapSeries-10 9.399m ± 1% 4.049m ± 0% -56.92% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 666.0m ± 3% 194.6m ± 0% -70.79% (p=0.000 n=10) geomean 18.87m 12.51m -33.70% │ before.log │ after.log │ │ B/op │ B/op vs base │ _CodecDecodeLogs-10 3.649Mi ± 0% 3.649Mi ± 0% ~ (p=0.364 n=10) _CodecDecodeSamples-10 18.12Mi ± 0% 18.12Mi ± 0% ~ (p=0.926 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 7.647Mi ± 0% 7.647Mi ± 0% ~ (p=0.587 n=10) _CodecDecodeSeries/application/json;_charset=utf-8-10 27.94Mi ± 0% 16.99Mi ± 0% -39.18% (p=0.000 n=10) _MergeResponses-10 2.362Mi ± 0% 2.408Mi ± 0% +1.98% (p=0.000 n=10) _UnwrapSeries-10 19.495Mi ± 0% 8.595Mi ± 0% -55.91% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 772.3Mi ± 0% 772.3Mi ± 0% ~ (p=0.912 n=10) geomean 17.50Mi 14.54Mi -16.91% │ before.log │ after.log │ │ allocs/op │ allocs/op vs base │ _CodecDecodeLogs-10 41.10k ± 0% 41.10k ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSamples-10 411.9k ± 0% 411.9k ± 0% ~ (p=1.000 n=10) _CodecDecodeSeries/application/vnd.google.protobuf-10 32.00 ± 0% 32.00 ± 0% ~ (p=1.000 n=10) ¹ _CodecDecodeSeries/application/json;_charset=utf-8-10 304.2k ± 0% 298.1k ± 0% -2.01% (p=0.000 n=10) _MergeResponses-10 100.1k ± 0% 100.1k ± 0% -0.00% (p=0.002 n=10) _UnwrapSeries-10 201.1k ± 0% 198.0k ± 0% -1.54% (p=0.000 n=10) _DecodeMergeEncodeCycle-10 203.1k ± 0% 203.1k ± 0% ~ (p=0.621 n=10) geomean 48.95k 48.70k -0.51% ¹ all samples are equal ``` **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
require.Equal(t, expected.Status, actual.Status)
require.ElementsMatch(t, expected.Data, actual.Data)
}
func TestShardingAcrossConfigs_ASTMapper(t *testing.T) {
now := model.Now()
confs := ShardingConfigs{
{
From: config.DayTime{Time: now.Add(-30 * 24 * time.Hour)},
RowShards: 2,
},
{
From: config.DayTime{Time: now.Add(-24 * time.Hour)},
RowShards: 3,
},
}
for _, tc := range []struct {
name string
req queryrangebase.Request
resp queryrangebase.Response
numExpectedShards int
}{
{
name: "logs query touching just the active schema config",
req: defaultReq().WithStartEnd(now.Add(-time.Hour).Time(), now.Time()).WithQuery(`{foo="bar"}`),
resp: &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Headers: []definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
numExpectedShards: 3,
},
{
name: "logs query touching just the prev schema config",
req: defaultReq().WithStartEnd(confs[0].From.Time.Time(), confs[0].From.Time.Add(time.Hour).Time()).WithQuery(`{foo="bar"}`),
resp: &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Headers: []definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
numExpectedShards: 2,
},
{
name: "metric query touching just the active schema config",
req: defaultReq().WithStartEnd(confs[1].From.Time.Add(5*time.Minute).Time(), confs[1].From.Time.Add(time.Hour).Time()).WithQuery(`rate({foo="bar"}[1m])`),
resp: &LokiPromResponse{
Response: &queryrangebase.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrangebase.PrometheusData{
ResultType: "",
Result: []queryrangebase.SampleStream{},
},
Headers: []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
},
numExpectedShards: 3,
},
{
name: "metric query touching just the prev schema config",
req: defaultReq().WithStartEnd(confs[0].From.Time.Add(time.Hour).Time(), confs[0].From.Time.Add(2*time.Hour).Time()).WithQuery(`rate({foo="bar"}[1m])`),
resp: &LokiPromResponse{
Response: &queryrangebase.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrangebase.PrometheusData{
ResultType: "",
Result: []queryrangebase.SampleStream{},
},
Headers: []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
},
numExpectedShards: 2,
},
{
name: "logs query covering both schemas",
req: defaultReq().WithStartEnd(confs[0].From.Time.Time(), now.Time()).WithQuery(`{foo="bar"}`),
resp: &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Headers: []definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
numExpectedShards: 1,
},
{
name: "metric query covering both schemas",
req: defaultReq().WithStartEnd(confs[0].From.Time.Time(), now.Time()).WithQuery(`rate({foo="bar"}[1m])`),
resp: &LokiPromResponse{
Response: &queryrangebase.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrangebase.PrometheusData{
ResultType: "",
Result: []queryrangebase.SampleStream{},
},
Headers: []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
},
numExpectedShards: 1,
},
{
name: "metric query with start/end within first schema but with large enough range to cover previous schema too",
req: defaultReq().WithStartEnd(confs[1].From.Time.Add(5*time.Minute).Time(), confs[1].From.Time.Add(time.Hour).Time()).WithQuery(`rate({foo="bar"}[24h])`),
resp: &LokiPromResponse{
Response: &queryrangebase.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrangebase.PrometheusData{
ResultType: "",
Result: []queryrangebase.SampleStream{},
},
Headers: []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
},
numExpectedShards: 1,
},
{
name: "metric query with start/end within first schema but with large enough offset to shift it to previous schema",
req: defaultReq().WithStartEnd(confs[1].From.Time.Add(5*time.Minute).Time(), now.Time()).WithQuery(`rate({foo="bar"}[1m] offset 12h)`),
resp: &LokiPromResponse{
Response: &queryrangebase.PrometheusResponse{
Status: loghttp.QueryStatusSuccess,
Data: queryrangebase.PrometheusData{
ResultType: "",
Result: []queryrangebase.SampleStream{},
},
Headers: []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
},
},
},
numExpectedShards: 1,
},
} {
t.Run(tc.name, func(t *testing.T) {
var lock sync.Mutex
called := 0
handler := queryrangebase.HandlerFunc(func(_ context.Context, _ queryrangebase.Request) (queryrangebase.Response, error) {
lock.Lock()
defer lock.Unlock()
called++
return tc.resp, nil
})
mware := newASTMapperware(
confs,
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
testEngineOpts,
handler,
handler,
nil,
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, maxQueryParallelism: 1, queryTimeout: time.Second},
0,
[]string{},
)
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
// currently all the tests call `defaultReq()` which creates an instance of the type LokiRequest
// if in the future that isn't true, we need another way to access the Plan field of an arbitrary query type
// or we should set the Plan in calls to `GetExpression` if the Plan is nil by calling `ParseExpr` or similar
tc.req.(*LokiRequest).Plan = &plan.QueryPlan{
AST: syntax.MustParseExpr(tc.req.GetQuery()),
}
resp, err := mware.Do(user.InjectOrgID(context.Background(), "1"), tc.req)
require.Nil(t, err)
require.Equal(t, []*definitions.PrometheusResponseHeader{
{Name: "Header", Values: []string{"value"}},
}, resp.GetHeaders())
require.Equal(t, tc.numExpectedShards, called)
})
}
}
func TestShardingAcrossConfigs_SeriesSharding(t *testing.T) {
now := model.Now()
confs := ShardingConfigs{
{
From: config.DayTime{Time: now.Add(-30 * 24 * time.Hour)},
RowShards: 2,
},
{
From: config.DayTime{Time: now.Add(-24 * time.Hour)},
RowShards: 3,
},
}
for _, tc := range []struct {
name string
req *LokiSeriesRequest
numExpectedShards int
}{
{
name: "series query touching just the active schema config",
req: &LokiSeriesRequest{
Match: []string{"foo", "bar"},
StartTs: confs[1].From.Time.Add(5 * time.Minute).Time(),
EndTs: now.Time(),
Path: "foo",
},
numExpectedShards: 3,
},
{
name: "series query touching just the prev schema config",
req: &LokiSeriesRequest{
Match: []string{"foo", "bar"},
StartTs: confs[0].From.Time.Time(),
EndTs: confs[0].From.Time.Add(time.Hour).Time(),
Path: "foo",
},
numExpectedShards: 2,
},
{
name: "series query covering both schemas",
req: &LokiSeriesRequest{
Match: []string{"foo", "bar"},
StartTs: confs[0].From.Time.Time(),
EndTs: now.Time(),
Path: "foo",
}, numExpectedShards: 1,
},
} {
t.Run(tc.name, func(t *testing.T) {
ctx := user.InjectOrgID(context.Background(), "1")
var lock sync.Mutex
called := 0
mware := NewSeriesQueryShardMiddleware(
log.NewNopLogger(),
confs,
queryrangebase.NewInstrumentMiddlewareMetrics(nil, constants.Loki),
nilShardingMetrics,
fakeLimits{
maxQueryParallelism: 10,
},
DefaultCodec,
)
_, err := mware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
_, ok := r.(*LokiSeriesRequest)
if !ok {
return nil, errors.New("not a series call")
}
lock.Lock()
defer lock.Unlock()
called++
return &LokiSeriesResponse{
Status: "success",
Version: 1,
Data: []logproto.SeriesIdentifier{},
}, nil
})).Do(ctx, tc.req)
require.NoError(t, err)
require.Equal(t, tc.numExpectedShards, called)
})
}
}
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
func Test_ASTMapper_MaxLookBackPeriod(t *testing.T) {
engineOpts := testEngineOpts
engineOpts.MaxLookBackPeriod = 1 * time.Hour
queryHandler := queryrangebase.HandlerFunc(func(_ context.Context, _ queryrangebase.Request) (queryrangebase.Response, error) {
return &LokiResponse{}, nil
})
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
statsHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
// This is the actual check that we're testing.
require.Equal(t, testTime.Add(-engineOpts.MaxLookBackPeriod).UnixMilli(), req.GetStart().UnixMilli())
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
return &IndexStatsResponse{
Response: &logproto.IndexStatsResponse{
Bytes: 1 << 10,
},
}, nil
})
mware := newASTMapperware(
testSchemasTSDB,
engineOpts,
queryHandler,
queryHandler,
statsHandler,
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, tsdbMaxQueryParallelism: 1, queryTimeout: time.Second},
0,
[]string{},
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
)
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
q := `{cluster="dev-us-central-0"}`
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
lokiReq := &LokiInstantRequest{
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
Query: q,
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
Limit: 1000,
TimeTs: testTime,
Direction: logproto.FORWARD,
Path: "/loki/api/v1/query",
Send query plan to querier. (#11246) **What this PR does / why we need it**: Following https://github.com/grafana/loki/pull/11123 and in order to enable https://github.com/grafana/loki/pull/10417 the query frontend should send the serialized LogQL AST instead of the query string to the queriers. This enables the frontend to change the AST and inject expressions that are not expressible in LogQL. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
2 years ago
Plan: &plan.QueryPlan{
AST: syntax.MustParseExpr(q),
},
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
}
ctx := user.InjectOrgID(context.Background(), "foo")
_, err := mware.Do(ctx, lokiReq)
require.NoError(t, err)
}
func Test_ConstantShardingDefaultIndexType(t *testing.T) {
engineOpts := testEngineOpts
queryHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
req.(*LokiInstantRequest).Plan.AST = syntax.MustParseExpr(`{cluster="dev-us-central-0"}`)
shards, _, err := logql.ParseShards(req.(*LokiInstantRequest).Shards)
require.NoError(t, err)
require.Equal(t, 1, len(shards))
require.Equal(t, logql.PowerOfTwoVersion, shards[0].Variant())
require.Equal(t, uint32(32), shards[0].PowerOfTwo.Of)
return &LokiResponse{}, nil
})
statsHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
// This is the actual check that we're testing.
require.Equal(t, testTime.Add(-engineOpts.MaxLookBackPeriod).UnixMilli(), req.GetStart().UnixMilli())
return &IndexStatsResponse{
Response: &logproto.IndexStatsResponse{
Bytes: 1 << 10,
},
}, nil
})
mware := newASTMapperware(
ShardingConfigs{
{
From: config.DayTime{Time: model.Now().Add(-2 * 24 * time.Hour)},
RowShards: 2,
IndexType: "tsdb",
},
{
From: config.DayTime{Time: model.Now().Add(-1 * 24 * time.Hour)},
RowShards: 32,
},
},
engineOpts,
queryHandler,
queryHandler,
statsHandler,
log.NewNopLogger(),
nilShardingMetrics,
fakeLimits{maxSeries: math.MaxInt32, tsdbMaxQueryParallelism: 1, queryTimeout: time.Second},
0,
[]string{},
)
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
q := `{cluster="dev-us-central-0"}`
lokiReq := &LokiInstantRequest{
Query: q,
Limit: 1000,
TimeTs: model.Now().Add(-1 * time.Hour).Time(),
Direction: logproto.FORWARD,
Path: "/loki/api/v1/query",
Plan: &plan.QueryPlan{
AST: syntax.MustParseExpr(q),
},
}
ctx := user.InjectOrgID(context.Background(), "foo")
_, err := mware.Do(ctx, lokiReq)
require.NoError(t, err)
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `CHANGELOG.md` updated - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` [1]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
}