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/logql/shardmapper_test.go

1283 lines
38 KiB

package logql
import (
"testing"
"time"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/querier/astmapper"
)
func TestShardedStringer(t *testing.T) {
for _, tc := range []struct {
in syntax.Expr
out string
}{
{
in: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
LogSelectorExpr: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
},
next: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
LogSelectorExpr: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
},
next: nil,
},
},
out: `downstream<{foo="bar"}, shard=0_of_2> ++ downstream<{foo="bar"}, shard=1_of_2>`,
},
} {
t.Run(tc.out, func(t *testing.T) {
require.Equal(t, tc.out, tc.in.String())
})
}
}
func TestMapSampleExpr(t *testing.T) {
m := NewShardMapper(ConstantShards(2), nilShardMetrics)
for _, tc := range []struct {
in syntax.SampleExpr
out syntax.SampleExpr
}{
{
in: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: time.Minute,
},
},
out: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: time.Minute,
},
},
},
next: nil,
},
},
},
} {
t.Run(tc.in.String(), func(t *testing.T) {
mapped, err := m.mapSampleExpr(tc.in, nilShardMetrics.downstreamRecorder())
require.Nil(t, err)
require.Equal(t, tc.out, mapped)
})
}
}
func TestMappingStrings(t *testing.T) {
m := NewShardMapper(ConstantShards(2), nilShardMetrics)
for _, tc := range []struct {
in string
out 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
{
in: `{foo="bar"}`,
out: `downstream<{foo="bar"}, shard=0_of_2>
++ downstream<{foo="bar"}, shard=1_of_2>`,
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
},
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
{
in: `{foo="bar"} |= "foo" |~ "bar" | json | latency >= 10s or foo<5 and bar="t" | line_format "b{{.blip}}"`,
out: `downstream<{foo="bar"} |="foo" |~"bar" | json | (latency>=10s or (foo<5,bar="t")) | line_format "b{{.blip}}", shard=0_of_2>
++downstream<{foo="bar"} |="foo" |~"bar" | json | (latency>=10s or (foo<5, bar="t")) | line_format "b{{.blip}}", shard=1_of_2>`,
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
},
{
in: `sum(rate({foo="bar"}[1m]))`,
out: `sum(
downstream<sum(rate({foo="bar"}[1m])), shard=0_of_2>
++ downstream<sum(rate({foo="bar"}[1m])), shard=1_of_2>
)`,
},
{
in: `max(count(rate({foo="bar"}[5m]))) / 2`,
out: `(max(
sum(
downstream<count(rate({foo="bar"}[5m])), shard=0_of_2>
++ downstream<count(rate({foo="bar"}[5m])), shard=1_of_2>)
) / 2
)`,
},
{
in: `topk(3, rate({foo="bar"}[5m]))`,
out: `topk(3,
downstream<rate({foo="bar"}[5m]), shard=0_of_2>
++ downstream<rate({foo="bar"}[5m]), shard=1_of_2>
)`,
},
{
in: `sum(max(rate({foo="bar"}[5m])))`,
out: `sum(max(
downstream<rate({foo="bar"}[5m]), shard=0_of_2>
++ downstream<rate({foo="bar"}[5m]), shard=1_of_2>
))`,
},
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
{
in: `sum(max(rate({foo="bar"} | json | label_format foo=bar [5m])))`,
out: `sum(max(rate({foo="bar"} | json | label_format foo=bar [5m])))`,
},
{
in: `rate({foo="bar"} | json | label_format foo=bar [5m])`,
out: `rate({foo="bar"} | json | label_format foo=bar [5m])`,
},
{
in: `count(rate({foo="bar"} | json [5m]))`,
out: `count(
downstream<rate({foo="bar"} | json [5m]), shard=0_of_2>
++ downstream<rate({foo="bar"} | json [5m]), shard=1_of_2>
)`,
},
{
in: `avg(rate({foo="bar"} | json [5m]))`,
out: `avg(
downstream<rate({foo="bar"} | json [5m]), shard=0_of_2>
++ downstream<rate({foo="bar"} | json [5m]), shard=1_of_2>
)`,
},
{
in: `{foo="bar"} |= "id=123"`,
out: `downstream<{foo="bar"}|="id=123", shard=0_of_2>
++ downstream<{foo="bar"}|="id=123", shard=1_of_2>`,
},
{
in: `sum by (cluster) (rate({foo="bar"} |= "id=123" [5m]))`,
out: `sum by (cluster) (
downstream<sum by(cluster)(rate({foo="bar"}|="id=123"[5m])), shard=0_of_2>
++ downstream<sum by(cluster)(rate({foo="bar"}|="id=123"[5m])), shard=1_of_2>
)`,
},
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
{
in: `sum by (cluster) (sum_over_time({foo="bar"} |= "id=123" | logfmt | unwrap latency [5m]))`,
out: `sum by (cluster) (
downstream<sum by(cluster)(sum_over_time({foo="bar"}|="id=123"| logfmt | unwrap latency[5m])), shard=0_of_2>
++ downstream<sum by(cluster)(sum_over_time({foo="bar"}|="id=123"| logfmt | unwrap latency[5m])), shard=1_of_2>
)`,
LogQL: Labels and Metrics Extraction (#2769) * Adds logfmt, regexp and json logql parser Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook the ast with parsers. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with memchunk. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with the storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * hook parser with ingesters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixes all tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactor to pipeline and implement ast parsing. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the lexer for duration and range Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes all tests and add some for label filters Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add label and line format. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests for fmt label and line with validations. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Polishing parsers and add some more test cases Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Finish the unwrap parser, still need to add more tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Indent this hell. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Moar tests and it works. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests which lead me to find a bug in the lexer Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add more tests and fix all engine tests Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes match stage in promtail pipelines. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hook Pipeline into ingester, tailer and storage. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Correctly setup sharding for logqlv2 Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes precedences issue with label filters and add moar tests :v: * Adds quantile_over_time, grouping for non associate range aggregation parsing and moar tests * Extract with grouping * Adds parsing duration on unwrap * Improve the lexer to support more common identifier as functions. Also add duration convertion for unwrap. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes the frontend logs to include org_id. The auth middleware was happening after the stats one and so org_id was not set :facepalm:. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Support byte sizes in label filters. This patch extends the duration label filter with support for byte sizes such as `1kB` and `42MiB`. * Wip on error handling. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes json parser with prometheus label name rules. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * fixup! Support byte sizes in label filters. * Wip error handling, commit before big refactoring. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactoring in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Work in progress. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Got something that builds and throw __error__ labels properly now. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add error handling + fixes groupins and post filtering. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * 400 on pipeline errors. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes a races in the log pipeline. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Unsure the key is parsable and valid. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Cleanup and code documentation. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Lint. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes frontend handler. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fixes old test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Fix go1.15 local failing test. Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Karsten Jeschkies <k@jeschkies.xyz>
5 years ago
},
{
in: `sum by (cluster) (stddev_over_time({foo="bar"} |= "id=123" | logfmt | unwrap latency [5m]))`,
out: `sum by (cluster) (stddev_over_time({foo="bar"} |= "id=123" | logfmt | unwrap latency [5m]))`,
},
{
in: `sum without (a) (
label_replace(
sum without (b) (
rate({foo="bar"}[5m])
),
"baz", "buz", "foo", "(.*)"
)
)`,
out: `sum without(a) (
label_replace(
sum without (b) (
downstream<sum without (b)(rate({foo="bar"}[5m])), shard=0_of_2>
++downstream<sum without(b)(rate({foo="bar"}[5m])), shard=1_of_2>
),
"baz", "buz", "foo", "(.*)"
)
)`,
},
{
// Ensure we don't try to shard expressions that include label reformatting.
in: `sum(count_over_time({foo="bar"} | logfmt | label_format bar=baz | bar="buz" [5m]))`,
out: `sum(count_over_time({foo="bar"} | logfmt | label_format bar=baz | bar="buz" [5m]))`,
},
{
in: `sum by (cluster) (rate({foo="bar"} [5m])) + ignoring(machine) sum by (cluster,machine) (rate({foo="bar"} [5m]))`,
out: `(
sum by (cluster) (
downstream<sum by (cluster) (rate({foo="bar"}[5m])), shard=0_of_2>
++ downstream<sum by (cluster) (rate({foo="bar"}[5m])), shard=1_of_2>
)
+ ignoring(machine) sum by (cluster, machine) (
downstream<sum by (cluster, machine) (rate({foo="bar"}[5m])), shard=0_of_2>
++ downstream<sum by (cluster, machine) (rate({foo="bar"}[5m])), shard=1_of_2>
)
)`,
},
{
in: `sum by (cluster) (sum by (cluster) (rate({foo="bar"} [5m])) + ignoring(machine) sum by (cluster,machine) (rate({foo="bar"} [5m])))`,
out: `sum by (cluster) (
(
sum by (cluster) (
downstream<sum by (cluster) (rate({foo="bar"}[5m])), shard=0_of_2>
++ downstream<sum by (cluster) (rate({foo="bar"}[5m])), shard=1_of_2>
)
+ ignoring(machine) sum by (cluster, machine)(
downstream<sum by (cluster, machine) (rate({foo="bar"}[5m])), shard=0_of_2>
++ downstream<sum by (cluster, machine) (rate({foo="bar"}[5m])), shard=1_of_2>
)
)
)`,
},
{
in: `avg(avg_over_time({job=~"myapps.*"} |= "stats" | json busy="utilization" | unwrap busy [5m]))`,
out: `avg(avg_over_time({job=~"myapps.*"} |= "stats" | json busy="utilization" | unwrap busy [5m]))`,
},
{
in: `avg_over_time({job=~"myapps.*"} |= "stats" | json busy="utilization" | unwrap busy [5m])`,
out: `avg_over_time({job=~"myapps.*"} |= "stats" | json busy="utilization" | unwrap busy [5m])`,
},
// should be noop if VectorExpr
{
in: `vector(0)`,
out: `vector(0.000000)`,
},
{
// or exprs aren't shardable
in: `count_over_time({a=~".+"}[1s]) or count_over_time({a=~".+"}[1s])`,
out: `(downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>ordownstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>)`,
},
{
// vector() exprs aren't shardable
in: `sum(count_over_time({a=~".+"}[1s]) + vector(1))`,
out: `sum((downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>+vector(1.000000)))`,
},
{
// on() is never shardable as it can mutate labels
in: `sum(count_over_time({a=~".+"}[1s]) * on () count_over_time({a=~".+"}[1s]))`,
out: `sum((downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>*on()downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>))`,
},
{
// ignoring(<non-empty-labels>) is never shardable as it can mutate labels
in: `sum(count_over_time({a=~".+"}[1s]) * ignoring (foo) count_over_time({a=~".+"}[1s]))`,
out: `sum((downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>*ignoring(foo)downstream<count_over_time({a=~".+"}[1s]),shard=0_of_2>++downstream<count_over_time({a=~".+"}[1s]),shard=1_of_2>))`,
},
{
// ignoring () doesn't mutate labels and therefore can be shardable
// as long as the operation is shardable
in: `sum(count_over_time({a=~".+"}[1s]) * ignoring () count_over_time({a=~".+"}[1s]))`,
out: `sum(downstream<sum((count_over_time({a=~".+"}[1s])*count_over_time({a=~".+"}[1s]))),shard=0_of_2>++downstream<sum((count_over_time({a=~".+"}[1s])*count_over_time({a=~".+"}[1s]))),shard=1_of_2>)`,
},
} {
t.Run(tc.in, func(t *testing.T) {
ast, err := syntax.ParseExpr(tc.in)
require.Nil(t, err)
mapped, err := m.Map(ast, nilShardMetrics.downstreamRecorder())
require.Nil(t, err)
require.Equal(t, removeWhiteSpace(tc.out), removeWhiteSpace(mapped.String()))
})
}
}
func TestMapping(t *testing.T) {
m := NewShardMapper(ConstantShards(2), nilShardMetrics)
for _, tc := range []struct {
in string
expr syntax.Expr
err error
}{
{
in: `{foo="bar"}`,
expr: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
LogSelectorExpr: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
},
next: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
LogSelectorExpr: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
},
next: nil,
},
},
},
{
in: `{foo="bar"} |= "error"`,
expr: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
LogSelectorExpr: &syntax.PipelineExpr{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
MultiStages: syntax.MultiStageExpr{
&syntax.LineFilterExpr{
Ty: labels.MatchEqual,
Match: "error",
Op: "",
},
},
},
},
next: &ConcatLogSelectorExpr{
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
DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
LogSelectorExpr: &syntax.PipelineExpr{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
MultiStages: syntax.MultiStageExpr{
&syntax.LineFilterExpr{
Ty: labels.MatchEqual,
Match: "error",
Op: "",
},
},
},
},
next: nil,
},
},
},
{
in: `rate({foo="bar"}[5m])`,
expr: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: nil,
},
},
},
{
in: `count_over_time({foo="bar"}[5m])`,
expr: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeCount,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeCount,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: nil,
},
},
},
{
in: `sum(rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
{
in: `topk(3, rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Params: 3,
Operation: syntax.OpTypeTopK,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: nil,
},
},
},
},
{
in: `max without (env) (rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Without: true,
Groups: []string{"env"},
},
Operation: syntax.OpTypeMax,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: nil,
},
},
},
},
{
in: `count(rate({foo="bar"}[5m]))`,
expr: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
{
in: `avg(rate({foo="bar"}[5m]))`,
expr: &syntax.BinOpExpr{
Op: syntax.OpTypeDiv,
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
RHS: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
{
in: `1 + sum by (cluster) (rate({foo="bar"}[5m]))`,
expr: &syntax.BinOpExpr{
Op: syntax.OpTypeAdd,
Opts: &syntax.BinOpOptions{
ReturnBool: false,
VectorMatching: &syntax.VectorMatching{Card: syntax.CardOneToOne},
},
SampleExpr: &syntax.LiteralExpr{Val: 1},
RHS: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
[bugfix] logql: vectorExpr run fail when query_range: parallelise_shardable_queries: true (#7045) …hardable_queries: true <!-- Thanks for sending a pull request! Before submitting: 1. Read our CONTRIBUTING.md guide 2. Name your PR as `<Feature Area>: Describe your change`. a. Do not end the title with punctuation. It will be added in the changelog. b. Start with an imperative verb. Example: Fix the latency between System A and System B. c. Use sentence case, not title case. d. Use a complete phrase or sentence. The PR title will appear in a changelog, so help other people understand what your change will be. 3. Rebase your PR if it gets out of sync with main --> **What this PR does / why we need it**: A grafana error occurred when I deployed the vector(0) function to our online loki cluster. It was finally found because online loki has a configuration `unexpected expr type (*syntax.VectorExpr) for ASTMapper type (logql.ShardMapper) ` ```yaml query_range: parallelise_shardable_queries: true ``` **Which issue(s) this PR fixes**: Fixes # **Special notes for your reviewer**: error snapssnapshot ![image](https://user-images.githubusercontent.com/9583245/188357957-e573e589-a789-4e5e-a1c3-b91c019ceb62.png) with this pr,success snapshot ![image](https://user-images.githubusercontent.com/9583245/188358036-5ca0d9be-c2b6-4516-807c-c027445ee62d.png) <!-- Note about CHANGELOG entries, if a change adds: * an important feature * fixes an issue present in a previous release, * causes a change in operation that would be useful for an operator of Loki to know then please add a CHANGELOG entry. For documentation changes, build changes, simple fixes etc please skip this step. We are attempting to curate a changelog of the most relevant and important changes to be easier to ingest by end users of Loki. Note about the upgrade guide, if this changes: * default configuration values * metric names or label names * changes existing log lines such as the metrics.go query output line * configuration parameters * anything to do with any API * any other change that would require special attention or extra steps to upgrade Please document clearly what changed AND what needs to be done in the upgrade guide. --> **Checklist** - [ ] Documentation added - [ ] Tests updated - [ ] Is this an important fix or new feature? Add an entry in the `CHANGELOG.md`. - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md`
3 years ago
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
{
in: `vector(0) or sum (rate({foo="bar"}[5m]))`,
expr: &syntax.BinOpExpr{
Op: syntax.OpTypeOr,
Opts: &syntax.BinOpOptions{
ReturnBool: false,
VectorMatching: &syntax.VectorMatching{Card: syntax.CardOneToOne},
},
SampleExpr: &syntax.VectorExpr{Val: 0},
RHS: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
// sum(max) should not shard the maxes
{
in: `sum(max(rate({foo="bar"}[5m])))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeSum,
Left: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeMax,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
next: nil,
},
},
},
},
},
// max(count) should shard the count, but not the max
{
in: `max(count(rate({foo="bar"}[5m])))`,
expr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeMax,
Left: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
{
in: `max(sum by (cluster) (rate({foo="bar"}[5m]))) / count(rate({foo="bar"}[5m]))`,
expr: &syntax.BinOpExpr{
Op: syntax.OpTypeDiv,
Opts: &syntax.BinOpOptions{
ReturnBool: false,
VectorMatching: &syntax.VectorMatching{Card: syntax.CardOneToOne},
},
SampleExpr: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeMax,
Grouping: &syntax.Grouping{},
Left: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
RHS: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
{
in: `sum by (cluster) (rate({foo="bar"}[5m])) / ignoring(cluster) count(rate({foo="bar"}[5m]))`,
expr: &syntax.BinOpExpr{
Op: syntax.OpTypeDiv,
Opts: &syntax.BinOpOptions{
ReturnBool: false,
VectorMatching: &syntax.VectorMatching{
On: false,
MatchingLabels: []string{"cluster"},
},
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{
Groups: []string{"cluster"},
},
Operation: syntax.OpTypeSum,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
RHS: &syntax.VectorAggregationExpr{
Operation: syntax.OpTypeSum,
Grouping: &syntax.Grouping{},
Left: &ConcatSampleExpr{
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 0,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: &ConcatSampleExpr{
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
DownstreamSampleExpr: DownstreamSampleExpr{
shard: &astmapper.ShardAnnotation{
Shard: 1,
Of: 2,
},
SampleExpr: &syntax.VectorAggregationExpr{
Grouping: &syntax.Grouping{},
Operation: syntax.OpTypeCount,
Left: &syntax.RangeAggregationExpr{
Operation: syntax.OpRangeTypeRate,
Left: &syntax.LogRange{
Left: &syntax.MatchersExpr{
Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")},
},
Interval: 5 * time.Minute,
},
},
},
},
next: nil,
},
},
},
},
},
} {
t.Run(tc.in, func(t *testing.T) {
ast, err := syntax.ParseExpr(tc.in)
require.Equal(t, tc.err, err)
mapped, err := m.Map(ast, nilShardMetrics.downstreamRecorder())
require.Equal(t, tc.err, err)
require.Equal(t, tc.expr.String(), mapped.String())
require.Equal(t, tc.expr, mapped)
})
}
}
// nolint unparam
func mustNewMatcher(t labels.MatchType, n, v string) *labels.Matcher {
m, err := labels.NewMatcher(t, n, v)
if err != nil {
panic(logqlmodel.NewParseError(err.Error(), 0, 0))
}
return m
}
func TestStringTrimming(t *testing.T) {
for _, tc := range []struct {
expr string
expected string
shards int
}{
{
// sample expr in entirety for low shard count
shards: 2,
expr: `count_over_time({app="foo"}[1m])`,
expected: `
downstream<count_over_time({app="foo"}[1m]),shard=0_of_2> ++
downstream<count_over_time({app="foo"}[1m]),shard=1_of_2>
`,
},
{
// sample expr doesnt display infinite shards
shards: 5,
expr: `count_over_time({app="foo"}[1m])`,
expected: `
downstream<count_over_time({app="foo"}[1m]),shard=0_of_5> ++
downstream<count_over_time({app="foo"}[1m]),shard=1_of_5> ++
downstream<count_over_time({app="foo"}[1m]),shard=2_of_5> ++
downstream<count_over_time({app="foo"}[1m]),shard=3_of_5> ++
...
`,
},
{
// log selector expr in entirety for low shard count
shards: 2,
expr: `{app="foo"}`,
expected: `
downstream<{app="foo"},shard=0_of_2> ++
downstream<{app="foo"},shard=1_of_2>
`,
},
{
// log selector expr doesnt display infinite shards
shards: 5,
expr: `{app="foo"}`,
expected: `
downstream<{app="foo"},shard=0_of_5> ++
downstream<{app="foo"},shard=1_of_5> ++
downstream<{app="foo"},shard=2_of_5> ++
downstream<{app="foo"},shard=3_of_5> ++
...
`,
},
} {
t.Run(tc.expr, func(t *testing.T) {
m := NewShardMapper(ConstantShards(tc.shards), nilShardMetrics)
_, mappedExpr, err := m.Parse(tc.expr)
require.Nil(t, err)
require.Equal(t, removeWhiteSpace(tc.expected), removeWhiteSpace(mappedExpr.String()))
})
}
}