From f00ec35963fcefa23d7c119764e1641a728809ba Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 5 Feb 2020 20:59:43 -0500 Subject: [PATCH 01/67] [wip] sharding evaluator/ast --- pkg/logql/astmapper.go | 11 +++ pkg/logql/properties.go | 40 ++++++++++ pkg/logql/sharding.go | 170 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 221 insertions(+) create mode 100644 pkg/logql/astmapper.go create mode 100644 pkg/logql/properties.go create mode 100644 pkg/logql/sharding.go diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go new file mode 100644 index 0000000000000..4e1a91324fd89 --- /dev/null +++ b/pkg/logql/astmapper.go @@ -0,0 +1,11 @@ +package logql + +// ASTMapper is the exported interface for mapping between multiple AST representations +type ASTMapper interface { + Map(Expr) (Expr, error) +} + +// CloneExpr is a helper function to clone a node. +func CloneExpr(expr Expr) (Expr, error) { + return ParseExpr(expr.String()) +} diff --git a/pkg/logql/properties.go b/pkg/logql/properties.go new file mode 100644 index 0000000000000..39977f922fbca --- /dev/null +++ b/pkg/logql/properties.go @@ -0,0 +1,40 @@ +package logql + +// technically, std{dev,var} are also parallelizable if there is no cross-shard merging +// in descendent nodes in the AST. This optimization is currently avoided for simplicity. +var parallelOperations = map[string]bool{ + OpTypeSum: true, + OpTypeAvg: true, + OpTypeMax: true, + OpTypeMin: true, + OpTypeCount: true, + OpTypeBottomK: true, + OpTypeTopK: true, + OpTypeCountOverTime: true, + OpTypeRate: true, +} + +// PropertyExpr is an expression which can determine certain properties of an expression +// and also impls ASTMapper +type PropertyExpr interface { + Expr + CanParallel() bool // Whether this expression can be parallelized + +} + +type propertyExpr struct { + Expr +} + +func (e propertyExpr) CanParallel() bool { + switch expr := e.Expr.(type) { + case *matchersExpr, *filterExpr: + return true + case *rangeAggregationExpr: + return parallelOperations[expr.operation] + case *vectorAggregationExpr: + return parallelOperations[expr.operation] && propertyExpr{expr.left}.CanParallel() + default: + return false + } +} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go new file mode 100644 index 0000000000000..76e3393ba8de0 --- /dev/null +++ b/pkg/logql/sharding.go @@ -0,0 +1,170 @@ +package logql + +import ( + "context" + "errors" + + "github.com/grafana/loki/pkg/iter" + "github.com/prometheus/prometheus/pkg/labels" +) + +// type shardedLogSelectorExpr struct { +// LogSelectorExpr +// shard int +// } + +// type shardedSampleExpr struct { +// SampleExpr +// shard int +// } + +// type ShardMapper struct { +// shards int +// } + +// func (m ShardMapper) Map(expr Expr) (Expr, error) { +// cloned, err := CloneExpr(expr) +// if err != nil { +// return nil, err +// } + +// if (propertyExpr{cloned}).CanParallel() { +// return m.parallelize(cloned) +// } + +// switch e := cloned.(type) { +// case *rangeAggregationExpr: +// mapped, err := m.Map(e.left.left) +// if err != nil { +// return nil, err +// } +// e.left.left = mapped.(LogSelectorExpr) +// return e, nil +// case *vectorAggregationExpr: +// mapped, err := m.Map(e.left) +// if err != nil { +// return nil, err +// } +// e.left = mapped.(SampleExpr) +// return e, nil +// default: +// return nil, errors.Errorf("unexpected expr marked as not parallelizable: %+v", expr) +// } +// } + +// func (m ShardMapper) parallelize(expr Expr) (Expr, error) { +// switch e := expr.(type) { +// case *matchersExpr: +// case *filterExpr: +// case *rangeAggregationExpr: +// case *vectorAggregationExpr: +// default: +// return nil, errors.Errorf("unexpected expr: %+v", expr) +// } +// } + +// DownstreamExpr impls both LogSelectorExpr and SampleExpr in order to transparently +// wrap an expr and signal that it should be executed on a downstream querier. +type DownstreamExpr struct { + shard *int + Expr +} + +func (e DownstreamExpr) Selector() LogSelectorExpr { + return e.Expr.(SampleExpr).Selector() +} + +func (e DownstreamExpr) Filter() (Filter, error) { + return e.Expr.(LogSelectorExpr).Filter() +} + +func (e DownstreamExpr) Matchers() []*labels.Matcher { + return e.Expr.(LogSelectorExpr).Matchers() +} + +// ConcatSampleExpr is a sample expr which is used to signal a list of +// SampleExprs which should be joined +type ConcatSampleExpr struct { + SampleExpr + next *ConcatSampleExpr +} + +// ConcatLogSelectorExpr is a sample expr which is used to signal a list of +// LogSelectorExprs which should be joined +type ConcatLogSelectorExpr struct { + LogSelectorExpr + next *ConcatLogSelectorExpr +} + +// shardedEvaluator is an evaluator which handles shard aware AST nodes +// and embeds a default evaluator otherwise +type shardedEvaluator struct { + shards int + evaluator *defaultEvaluator +} + +// Evaluator returns a StepEvaluator for a given SampleExpr +func (ev *shardedEvaluator) Evaluator( + ctx context.Context, + expr SampleExpr, + params Params, +) (StepEvaluator, error) { + switch e := expr.(type) { + case DownstreamExpr: + // TODO(owen): downstream this and present as StepEvaluator + return nil, errors.New("unimplemented") + case ConcatSampleExpr: + var evaluators []StepEvaluator + for { + eval, err := ev.Evaluator(ctx, e.SampleExpr, params) + if err != nil { + return nil, err + } + evaluators = append(evaluators, eval) + if e.next != nil { + break + } + e = *e.next + } + return ConcatEvaluator(evaluators) + default: + return ev.evaluator.Evaluator(ctx, expr, params) + } +} + +// Iterator returns the iter.EntryIterator for a given LogSelectorExpr +func (ev *shardedEvaluator) Iterator( + ctx context.Context, + expr LogSelectorExpr, + params Params, +) (iter.EntryIterator, error) { + switch e := expr.(type) { + case DownstreamExpr: + // TODO(owen): downstream this and present as iter.EntryIterator + return nil, errors.New("unimplemented") + case ConcatLogSelectorExpr: + var iters []iter.EntryIterator + + for { + iter, err := ev.Iterator(ctx, e.LogSelectorExpr, params) + // TODO(owen): close these iters? + if err != nil { + return nil, err + } + iters = append(iters, iter) + if e.next == nil { + break + } + e = *e.next + } + return iter.NewHeapIterator(ctx, iters, params.Direction()) + default: + return nil, errors.Errorf("unexpected type (%T): %v", e, e) + } +} + +/* +map :: AST -> AST + + +*/ From 406ce2ba8691fcdea5c67b0cd8788a500f0fe372 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 6 Feb 2020 10:55:15 -0500 Subject: [PATCH 02/67] [wip] continues experimenting with ast mapping --- pkg/logql/evaluator.go | 26 +++++++++++++++ pkg/logql/properties.go | 18 ++--------- pkg/logql/sharding.go | 70 ++++++++++++++++++----------------------- 3 files changed, 59 insertions(+), 55 deletions(-) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 1846bad12168f..f5cb2fce5fe69 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -592,3 +592,29 @@ func (ev *defaultEvaluator) literalEvaluator( eval.Close, ) } + +// ConcatEvaluator joins multiple StepEvaluators. +// Contract: They must be of identical start, end, and step values. +func ConcatEvaluator(evaluators []StepEvaluator) StepEvaluator { + return newStepEvaluator( + func() (done bool, ts int64, vec promql.Vector) { + var cur promql.Vector + for { + for _, eval := range evaluators { + done, ts, cur = eval.Next() + vec = append(vec, cur...) + } + } + return done, ts, vec + + }, + func() (lastErr error) { + for _, eval := range evaluators { + if err := eval.Close(); err != nil { + lastErr = err + } + } + return lastErr + }, + ) +} diff --git a/pkg/logql/properties.go b/pkg/logql/properties.go index 39977f922fbca..421779e398eb2 100644 --- a/pkg/logql/properties.go +++ b/pkg/logql/properties.go @@ -14,26 +14,14 @@ var parallelOperations = map[string]bool{ OpTypeRate: true, } -// PropertyExpr is an expression which can determine certain properties of an expression -// and also impls ASTMapper -type PropertyExpr interface { - Expr - CanParallel() bool // Whether this expression can be parallelized - -} - -type propertyExpr struct { - Expr -} - -func (e propertyExpr) CanParallel() bool { - switch expr := e.Expr.(type) { +func CanParallel(e Expr) bool { + switch expr := e.(type) { case *matchersExpr, *filterExpr: return true case *rangeAggregationExpr: return parallelOperations[expr.operation] case *vectorAggregationExpr: - return parallelOperations[expr.operation] && propertyExpr{expr.left}.CanParallel() + return parallelOperations[expr.operation] && CanParallel(expr.Left) default: return false } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 76e3393ba8de0..d92e7154e9267 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -8,49 +8,39 @@ import ( "github.com/prometheus/prometheus/pkg/labels" ) -// type shardedLogSelectorExpr struct { -// LogSelectorExpr -// shard int -// } - -// type shardedSampleExpr struct { -// SampleExpr -// shard int -// } - -// type ShardMapper struct { -// shards int -// } +type ShardMapper struct { + shards int +} -// func (m ShardMapper) Map(expr Expr) (Expr, error) { -// cloned, err := CloneExpr(expr) -// if err != nil { -// return nil, err -// } +func (m ShardMapper) Map(expr Expr) (Expr, error) { + cloned, err := CloneExpr(expr) + if err != nil { + return nil, err + } -// if (propertyExpr{cloned}).CanParallel() { -// return m.parallelize(cloned) -// } + if CanParallel(cloned) { + return m.parallelize(cloned) + } -// switch e := cloned.(type) { -// case *rangeAggregationExpr: -// mapped, err := m.Map(e.left.left) -// if err != nil { -// return nil, err -// } -// e.left.left = mapped.(LogSelectorExpr) -// return e, nil -// case *vectorAggregationExpr: -// mapped, err := m.Map(e.left) -// if err != nil { -// return nil, err -// } -// e.left = mapped.(SampleExpr) -// return e, nil -// default: -// return nil, errors.Errorf("unexpected expr marked as not parallelizable: %+v", expr) -// } -// } + switch e := cloned.(type) { + case *rangeAggregationExpr: + mapped, err := m.Map(e.left.left) + if err != nil { + return nil, err + } + e.left.left = mapped.(LogSelectorExpr) + return e, nil + case *vectorAggregationExpr: + mapped, err := m.Map(e.left) + if err != nil { + return nil, err + } + e.left = mapped.(SampleExpr) + return e, nil + default: + return nil, errors.Errorf("unexpected expr marked as not parallelizable: %+v", expr) + } +} // func (m ShardMapper) parallelize(expr Expr) (Expr, error) { // switch e := expr.(type) { From d58aad6a80a7f80b418e39a951fde2e6fe51ad7d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 14 Feb 2020 08:08:43 -0500 Subject: [PATCH 03/67] refactoring in preparation for binops --- pkg/logql/properties.go | 2 +- pkg/logql/sharding.go | 160 -------------------------------- pkg/logql/sharding_evaluator.go | 71 ++++++++++++++ pkg/logql/sharding_exprs.go | 25 +++++ pkg/logql/sharding_mapper.go | 5 + 5 files changed, 102 insertions(+), 161 deletions(-) delete mode 100644 pkg/logql/sharding.go create mode 100644 pkg/logql/sharding_evaluator.go create mode 100644 pkg/logql/sharding_exprs.go create mode 100644 pkg/logql/sharding_mapper.go diff --git a/pkg/logql/properties.go b/pkg/logql/properties.go index 421779e398eb2..96cd220409242 100644 --- a/pkg/logql/properties.go +++ b/pkg/logql/properties.go @@ -21,7 +21,7 @@ func CanParallel(e Expr) bool { case *rangeAggregationExpr: return parallelOperations[expr.operation] case *vectorAggregationExpr: - return parallelOperations[expr.operation] && CanParallel(expr.Left) + return parallelOperations[expr.operation] && CanParallel(expr.left) default: return false } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go deleted file mode 100644 index d92e7154e9267..0000000000000 --- a/pkg/logql/sharding.go +++ /dev/null @@ -1,160 +0,0 @@ -package logql - -import ( - "context" - "errors" - - "github.com/grafana/loki/pkg/iter" - "github.com/prometheus/prometheus/pkg/labels" -) - -type ShardMapper struct { - shards int -} - -func (m ShardMapper) Map(expr Expr) (Expr, error) { - cloned, err := CloneExpr(expr) - if err != nil { - return nil, err - } - - if CanParallel(cloned) { - return m.parallelize(cloned) - } - - switch e := cloned.(type) { - case *rangeAggregationExpr: - mapped, err := m.Map(e.left.left) - if err != nil { - return nil, err - } - e.left.left = mapped.(LogSelectorExpr) - return e, nil - case *vectorAggregationExpr: - mapped, err := m.Map(e.left) - if err != nil { - return nil, err - } - e.left = mapped.(SampleExpr) - return e, nil - default: - return nil, errors.Errorf("unexpected expr marked as not parallelizable: %+v", expr) - } -} - -// func (m ShardMapper) parallelize(expr Expr) (Expr, error) { -// switch e := expr.(type) { -// case *matchersExpr: -// case *filterExpr: -// case *rangeAggregationExpr: -// case *vectorAggregationExpr: -// default: -// return nil, errors.Errorf("unexpected expr: %+v", expr) -// } -// } - -// DownstreamExpr impls both LogSelectorExpr and SampleExpr in order to transparently -// wrap an expr and signal that it should be executed on a downstream querier. -type DownstreamExpr struct { - shard *int - Expr -} - -func (e DownstreamExpr) Selector() LogSelectorExpr { - return e.Expr.(SampleExpr).Selector() -} - -func (e DownstreamExpr) Filter() (Filter, error) { - return e.Expr.(LogSelectorExpr).Filter() -} - -func (e DownstreamExpr) Matchers() []*labels.Matcher { - return e.Expr.(LogSelectorExpr).Matchers() -} - -// ConcatSampleExpr is a sample expr which is used to signal a list of -// SampleExprs which should be joined -type ConcatSampleExpr struct { - SampleExpr - next *ConcatSampleExpr -} - -// ConcatLogSelectorExpr is a sample expr which is used to signal a list of -// LogSelectorExprs which should be joined -type ConcatLogSelectorExpr struct { - LogSelectorExpr - next *ConcatLogSelectorExpr -} - -// shardedEvaluator is an evaluator which handles shard aware AST nodes -// and embeds a default evaluator otherwise -type shardedEvaluator struct { - shards int - evaluator *defaultEvaluator -} - -// Evaluator returns a StepEvaluator for a given SampleExpr -func (ev *shardedEvaluator) Evaluator( - ctx context.Context, - expr SampleExpr, - params Params, -) (StepEvaluator, error) { - switch e := expr.(type) { - case DownstreamExpr: - // TODO(owen): downstream this and present as StepEvaluator - return nil, errors.New("unimplemented") - case ConcatSampleExpr: - var evaluators []StepEvaluator - for { - eval, err := ev.Evaluator(ctx, e.SampleExpr, params) - if err != nil { - return nil, err - } - evaluators = append(evaluators, eval) - if e.next != nil { - break - } - e = *e.next - } - return ConcatEvaluator(evaluators) - default: - return ev.evaluator.Evaluator(ctx, expr, params) - } -} - -// Iterator returns the iter.EntryIterator for a given LogSelectorExpr -func (ev *shardedEvaluator) Iterator( - ctx context.Context, - expr LogSelectorExpr, - params Params, -) (iter.EntryIterator, error) { - switch e := expr.(type) { - case DownstreamExpr: - // TODO(owen): downstream this and present as iter.EntryIterator - return nil, errors.New("unimplemented") - case ConcatLogSelectorExpr: - var iters []iter.EntryIterator - - for { - iter, err := ev.Iterator(ctx, e.LogSelectorExpr, params) - // TODO(owen): close these iters? - if err != nil { - return nil, err - } - iters = append(iters, iter) - if e.next == nil { - break - } - e = *e.next - } - return iter.NewHeapIterator(ctx, iters, params.Direction()) - default: - return nil, errors.Errorf("unexpected type (%T): %v", e, e) - } -} - -/* -map :: AST -> AST - - -*/ diff --git a/pkg/logql/sharding_evaluator.go b/pkg/logql/sharding_evaluator.go new file mode 100644 index 0000000000000..c5452863c860b --- /dev/null +++ b/pkg/logql/sharding_evaluator.go @@ -0,0 +1,71 @@ +package logql + +import ( + "context" + "fmt" + + "github.com/grafana/loki/pkg/iter" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/promql" +) + +// downstreamEvaluator is an evaluator which handles shard aware AST nodes +// and embeds a default evaluator otherwise +type downstreamEvaluator struct { + shards int + evaluator *defaultEvaluator +} + +// Evaluator returns a StepEvaluator for a given SampleExpr +func (ev *downstreamEvaluator) Evaluator( + ctx context.Context, + expr SampleExpr, + params Params, +) (StepEvaluator, error) { + switch e := expr.(type) { + case DownstreamSampleExpr: + // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier + return nil, errors.New("unimplemented") + case ConcatSampleExpr: + // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat + return nil, errors.New("unimplemented") + default: + // used for aggregating downstreamed exprs, literalExprs + return ev.evaluator.Evaluator(ctx, expr, params) + } +} + +// Iterator returns the iter.EntryIterator for a given LogSelectorExpr +func (ev *downstreamEvaluator) Iterator( + _ context.Context, + expr LogSelectorExpr, + _ Params, +) (iter.EntryIterator, error) { + return nil, fmt.Errorf("downstreamEvaluator does not implement Iterator, called with expr: %+v", expr) +} + +// ConcatEvaluator joins multiple StepEvaluators. +// Contract: They must be of identical start, end, and step values. +func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { + return newStepEvaluator( + func() (done bool, ts int64, vec promql.Vector) { + var cur promql.Vector + for { + for _, eval := range evaluators { + done, ts, cur = eval.Next() + vec = append(vec, cur...) + } + } + return done, ts, vec + + }, + func() (lastErr error) { + for _, eval := range evaluators { + if err := eval.Close(); err != nil { + lastErr = err + } + } + return lastErr + }, + ) +} diff --git a/pkg/logql/sharding_exprs.go b/pkg/logql/sharding_exprs.go new file mode 100644 index 0000000000000..b37164920137d --- /dev/null +++ b/pkg/logql/sharding_exprs.go @@ -0,0 +1,25 @@ +package logql + +// DownstreamSampleExpr is a SampleExpr which signals downstream computation +type DownstreamSampleExpr struct { + shard *int + SampleExpr +} + +// DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation +type DownstreamLogSelectorExpr struct { + shard *int + LogSelectorExpr +} + +// ConcatSampleExpr is an expr for concatenating multiple SampleExpr +type ConcatSampleExpr struct { + SampleExpr + next *ConcatSampleExpr +} + +// ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr +type ConcatLogSelectorExpr struct { + LogSelectorExpr + next *ConcatLogSelectorExpr +} diff --git a/pkg/logql/sharding_mapper.go b/pkg/logql/sharding_mapper.go new file mode 100644 index 0000000000000..bd63cb5167ee2 --- /dev/null +++ b/pkg/logql/sharding_mapper.go @@ -0,0 +1,5 @@ +package logql + +type ShardMapper struct { + shards int +} From a2dca1763cc13901ab381a948c33d7a1a733295e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 19 Mar 2020 15:06:56 -0400 Subject: [PATCH 04/67] evaluators can pass state to other evaluators --- pkg/logql/ast.go | 2 +- pkg/logql/evaluator.go | 87 +++++++++++++++++++++++++----------------- 2 files changed, 52 insertions(+), 37 deletions(-) diff --git a/pkg/logql/ast.go b/pkg/logql/ast.go index efd6c2f522223..91e3c8c5ade60 100644 --- a/pkg/logql/ast.go +++ b/pkg/logql/ast.go @@ -386,7 +386,7 @@ func mustNewBinOpExpr(op string, lhs, rhs Expr) SampleExpr { // This is because literals need match all labels, which is currently difficult to encode into StepEvaluators. // Therefore, we ensure a binop can be reduced/simplified, maintaining the invariant that it does not have two literal legs. func reduceBinOp(op string, left, right *literalExpr) *literalExpr { - merged := (&defaultEvaluator{}).mergeBinOp( + merged := mergeBinOp( op, &promql.Sample{Point: promql.Point{V: left.value}}, &promql.Sample{Point: promql.Point{V: right.value}}, diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index f5cb2fce5fe69..c25bd5f87edb7 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -69,8 +69,9 @@ func GetRangeType(q Params) QueryRangeType { // Evaluator is an interface for iterating over data at different nodes in the AST type Evaluator interface { - // Evaluator returns a StepEvaluator for a given SampleExpr - Evaluator(context.Context, SampleExpr, Params) (StepEvaluator, error) + // Evaluator returns a StepEvaluator for a given SampleExpr. It's explicitly passed another Evaluator// in order to enable arbitrary compuation of embedded expressions. This allows more modular & extensible + // Evaluator implementations which can be composed. + Evaluator(ctx context.Context, nextEvaluator Evaluator, expr SampleExpr, p Params) (StepEvaluator, error) // Iterator returns the iter.EntryIterator for a given LogSelectorExpr Iterator(context.Context, LogSelectorExpr, Params) (iter.EntryIterator, error) } @@ -99,21 +100,43 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, } -func (ev *defaultEvaluator) Evaluator(ctx context.Context, expr SampleExpr, q Params) (StepEvaluator, error) { +func (ev *defaultEvaluator) Evaluator( + ctx context.Context, + nextEv Evaluator, + expr SampleExpr, + q Params, +) (StepEvaluator, error) { switch e := expr.(type) { case *vectorAggregationExpr: - return ev.vectorAggEvaluator(ctx, e, q) + return vectorAggEvaluator(ctx, nextEv, e, q) case *rangeAggregationExpr: - return ev.rangeAggEvaluator(ctx, e, q) + entryIter, err := ev.querier.Select(ctx, SelectParams{ + &logproto.QueryRequest{ + Start: q.Start().Add(-e.left.interval), + End: q.End(), + Limit: 0, + Direction: logproto.FORWARD, + Selector: expr.Selector().String(), + }, + }) + if err != nil { + return nil, err + } + return rangeAggEvaluator(ctx, entryIter, e, q) case *binOpExpr: - return ev.binOpEvaluator(ctx, e, q) + return binOpStepEvaluator(ctx, nextEv, e, q) default: return nil, errors.Errorf("unexpected type (%T): %v", e, e) } } -func (ev *defaultEvaluator) vectorAggEvaluator(ctx context.Context, expr *vectorAggregationExpr, q Params) (StepEvaluator, error) { - nextEvaluator, err := ev.Evaluator(ctx, expr.left, q) +func vectorAggEvaluator( + ctx context.Context, + ev Evaluator, + expr *vectorAggregationExpr, + q Params, +) (StepEvaluator, error) { + nextEvaluator, err := ev.Evaluator(ctx, ev, expr.left, q) if err != nil { return nil, err } @@ -302,21 +325,12 @@ func (ev *defaultEvaluator) vectorAggEvaluator(ctx context.Context, expr *vector }, nextEvaluator.Close) } -func (ev *defaultEvaluator) rangeAggEvaluator(ctx context.Context, expr *rangeAggregationExpr, q Params) (StepEvaluator, error) { - entryIter, err := ev.querier.Select(ctx, SelectParams{ - &logproto.QueryRequest{ - Start: q.Start().Add(-expr.left.interval), - End: q.End(), - Limit: 0, - Direction: logproto.FORWARD, - Selector: expr.Selector().String(), - }, - }) - - if err != nil { - return nil, err - } - +func rangeAggEvaluator( + ctx context.Context, + entryIter iter.EntryIterator, + expr *rangeAggregationExpr, + q Params, +) (StepEvaluator, error) { vecIter := newRangeVectorIterator(entryIter, expr.left.interval.Nanoseconds(), q.Step().Nanoseconds(), q.Start().UnixNano(), q.End().UnixNano()) @@ -341,8 +355,9 @@ func (ev *defaultEvaluator) rangeAggEvaluator(ctx context.Context, expr *rangeAg // binOpExpr explicly does not handle when both legs are literals as // it makes the type system simpler and these are reduced in mustNewBinOpExpr -func (ev *defaultEvaluator) binOpEvaluator( +func binOpStepEvaluator( ctx context.Context, + ev Evaluator, expr *binOpExpr, q Params, ) (StepEvaluator, error) { @@ -352,26 +367,26 @@ func (ev *defaultEvaluator) binOpEvaluator( // match a literal expr with all labels in the other leg if lOk { - rhs, err := ev.Evaluator(ctx, expr.RHS, q) + rhs, err := ev.Evaluator(ctx, ev, expr.RHS, q) if err != nil { return nil, err } - return ev.literalEvaluator(expr.op, leftLit, rhs, false) + return literalStepEvaluator(expr.op, leftLit, rhs, false) } if rOk { - lhs, err := ev.Evaluator(ctx, expr.SampleExpr, q) + lhs, err := ev.Evaluator(ctx, ev, expr.SampleExpr, q) if err != nil { return nil, err } - return ev.literalEvaluator(expr.op, rightLit, lhs, true) + return literalStepEvaluator(expr.op, rightLit, lhs, true) } // we have two non literal legs - lhs, err := ev.Evaluator(ctx, expr.SampleExpr, q) + lhs, err := ev.Evaluator(ctx, ev, expr.SampleExpr, q) if err != nil { return nil, err } - rhs, err := ev.Evaluator(ctx, expr.RHS, q) + rhs, err := ev.Evaluator(ctx, ev, expr.RHS, q) if err != nil { return nil, err } @@ -409,7 +424,7 @@ func (ev *defaultEvaluator) binOpEvaluator( for _, pair := range pairs { // merge - if merged := ev.mergeBinOp(expr.op, pair[0], pair[1]); merged != nil { + if merged := mergeBinOp(expr.op, pair[0], pair[1]); merged != nil { results = append(results, *merged) } } @@ -425,7 +440,7 @@ func (ev *defaultEvaluator) binOpEvaluator( }) } -func (ev *defaultEvaluator) mergeBinOp(op string, left, right *promql.Sample) *promql.Sample { +func mergeBinOp(op string, left, right *promql.Sample) *promql.Sample { var merger func(left, right *promql.Sample) *promql.Sample switch op { @@ -554,9 +569,9 @@ func (ev *defaultEvaluator) mergeBinOp(op string, left, right *promql.Sample) *p } -// literalEvaluator merges a literal with a StepEvaluator. Since order matters in +// literalStepEvaluator merges a literal with a StepEvaluator. Since order matters in // non commutative operations, inverted should be true when the literalExpr is not the left argument. -func (ev *defaultEvaluator) literalEvaluator( +func literalStepEvaluator( op string, lit *literalExpr, eval StepEvaluator, @@ -578,7 +593,7 @@ func (ev *defaultEvaluator) literalEvaluator( left, right = right, left } - if merged := ev.mergeBinOp( + if merged := mergeBinOp( op, left, right, @@ -595,7 +610,7 @@ func (ev *defaultEvaluator) literalEvaluator( // ConcatEvaluator joins multiple StepEvaluators. // Contract: They must be of identical start, end, and step values. -func ConcatEvaluator(evaluators []StepEvaluator) StepEvaluator { +func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { return newStepEvaluator( func() (done bool, ts int64, vec promql.Vector) { var cur promql.Vector From 161016cfb4d9ac3613f415d308f766504a330a78 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 19 Mar 2020 17:13:15 -0400 Subject: [PATCH 05/67] compiler alignment --- pkg/logql/engine.go | 2 +- pkg/logql/evaluator_test.go | 5 ++--- pkg/logql/sharding_evaluator.go | 35 ++++----------------------------- 3 files changed, 7 insertions(+), 35 deletions(-) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 0753173148368..444f11775ced5 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -212,7 +212,7 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr return ng.evalLiteral(ctx, lit, q) } - stepEvaluator, err := ng.evaluator.Evaluator(ctx, expr, q) + stepEvaluator, err := ng.evaluator.Evaluator(ctx, ng.evaluator, expr, q) if err != nil { return nil, err } diff --git a/pkg/logql/evaluator_test.go b/pkg/logql/evaluator_test.go index 23ef3d2f71c94..a25a72e6ae85e 100644 --- a/pkg/logql/evaluator_test.go +++ b/pkg/logql/evaluator_test.go @@ -9,9 +9,8 @@ import ( ) func TestDefaultEvaluator_DivideByZero(t *testing.T) { - ev := &defaultEvaluator{} - require.Equal(t, true, math.IsNaN(ev.mergeBinOp(OpTypeDiv, + require.Equal(t, true, math.IsNaN(mergeBinOp(OpTypeDiv, &promql.Sample{ Point: promql.Point{T: 1, V: 1}, }, @@ -20,7 +19,7 @@ func TestDefaultEvaluator_DivideByZero(t *testing.T) { }, ).Point.V)) - require.Equal(t, true, math.IsNaN(ev.mergeBinOp(OpTypeMod, + require.Equal(t, true, math.IsNaN(mergeBinOp(OpTypeMod, &promql.Sample{ Point: promql.Point{T: 1, V: 1}, }, diff --git a/pkg/logql/sharding_evaluator.go b/pkg/logql/sharding_evaluator.go index c5452863c860b..e5d7dd9958758 100644 --- a/pkg/logql/sharding_evaluator.go +++ b/pkg/logql/sharding_evaluator.go @@ -6,23 +6,22 @@ import ( "github.com/grafana/loki/pkg/iter" "github.com/pkg/errors" - "github.com/prometheus/prometheus/promql" ) // downstreamEvaluator is an evaluator which handles shard aware AST nodes // and embeds a default evaluator otherwise type downstreamEvaluator struct { - shards int - evaluator *defaultEvaluator + shards int } // Evaluator returns a StepEvaluator for a given SampleExpr func (ev *downstreamEvaluator) Evaluator( ctx context.Context, + nextEv Evaluator, expr SampleExpr, params Params, ) (StepEvaluator, error) { - switch e := expr.(type) { + switch expr.(type) { case DownstreamSampleExpr: // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier return nil, errors.New("unimplemented") @@ -31,7 +30,7 @@ func (ev *downstreamEvaluator) Evaluator( return nil, errors.New("unimplemented") default: // used for aggregating downstreamed exprs, literalExprs - return ev.evaluator.Evaluator(ctx, expr, params) + return nextEv.Evaluator(ctx, nextEv, expr, params) } } @@ -43,29 +42,3 @@ func (ev *downstreamEvaluator) Iterator( ) (iter.EntryIterator, error) { return nil, fmt.Errorf("downstreamEvaluator does not implement Iterator, called with expr: %+v", expr) } - -// ConcatEvaluator joins multiple StepEvaluators. -// Contract: They must be of identical start, end, and step values. -func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { - return newStepEvaluator( - func() (done bool, ts int64, vec promql.Vector) { - var cur promql.Vector - for { - for _, eval := range evaluators { - done, ts, cur = eval.Next() - vec = append(vec, cur...) - } - } - return done, ts, vec - - }, - func() (lastErr error) { - for _, eval := range evaluators { - if err := eval.Close(); err != nil { - lastErr = err - } - } - return lastErr - }, - ) -} From f6f47f79fff0c68f808e6bca2bb690994401a5d8 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 19 Mar 2020 17:31:08 -0400 Subject: [PATCH 06/67] Evaluator method renamed to StepEvaluator --- pkg/logql/engine.go | 2 +- pkg/logql/evaluator.go | 20 ++++++++++---------- pkg/logql/sharding_evaluator.go | 4 ++-- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 444f11775ced5..39d2618abee40 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -212,7 +212,7 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr return ng.evalLiteral(ctx, lit, q) } - stepEvaluator, err := ng.evaluator.Evaluator(ctx, ng.evaluator, expr, q) + stepEvaluator, err := ng.evaluator.StepEvaluator(ctx, ng.evaluator, expr, q) if err != nil { return nil, err } diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index c25bd5f87edb7..57165302d06c7 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -67,11 +67,11 @@ func GetRangeType(q Params) QueryRangeType { return RangeType } -// Evaluator is an interface for iterating over data at different nodes in the AST +// StepEvaluator is an interface for iterating over data at different nodes in the AST type Evaluator interface { - // Evaluator returns a StepEvaluator for a given SampleExpr. It's explicitly passed another Evaluator// in order to enable arbitrary compuation of embedded expressions. This allows more modular & extensible - // Evaluator implementations which can be composed. - Evaluator(ctx context.Context, nextEvaluator Evaluator, expr SampleExpr, p Params) (StepEvaluator, error) + // StepEvaluator returns a StepEvaluator for a given SampleExpr. It's explicitly passed another StepEvaluator// in order to enable arbitrary compuation of embedded expressions. This allows more modular & extensible + // StepEvaluator implementations which can be composed. + StepEvaluator(ctx context.Context, nextEvaluator Evaluator, expr SampleExpr, p Params) (StepEvaluator, error) // Iterator returns the iter.EntryIterator for a given LogSelectorExpr Iterator(context.Context, LogSelectorExpr, Params) (iter.EntryIterator, error) } @@ -100,7 +100,7 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, } -func (ev *defaultEvaluator) Evaluator( +func (ev *defaultEvaluator) StepEvaluator( ctx context.Context, nextEv Evaluator, expr SampleExpr, @@ -136,7 +136,7 @@ func vectorAggEvaluator( expr *vectorAggregationExpr, q Params, ) (StepEvaluator, error) { - nextEvaluator, err := ev.Evaluator(ctx, ev, expr.left, q) + nextEvaluator, err := ev.StepEvaluator(ctx, ev, expr.left, q) if err != nil { return nil, err } @@ -367,14 +367,14 @@ func binOpStepEvaluator( // match a literal expr with all labels in the other leg if lOk { - rhs, err := ev.Evaluator(ctx, ev, expr.RHS, q) + rhs, err := ev.StepEvaluator(ctx, ev, expr.RHS, q) if err != nil { return nil, err } return literalStepEvaluator(expr.op, leftLit, rhs, false) } if rOk { - lhs, err := ev.Evaluator(ctx, ev, expr.SampleExpr, q) + lhs, err := ev.StepEvaluator(ctx, ev, expr.SampleExpr, q) if err != nil { return nil, err } @@ -382,11 +382,11 @@ func binOpStepEvaluator( } // we have two non literal legs - lhs, err := ev.Evaluator(ctx, ev, expr.SampleExpr, q) + lhs, err := ev.StepEvaluator(ctx, ev, expr.SampleExpr, q) if err != nil { return nil, err } - rhs, err := ev.Evaluator(ctx, ev, expr.RHS, q) + rhs, err := ev.StepEvaluator(ctx, ev, expr.RHS, q) if err != nil { return nil, err } diff --git a/pkg/logql/sharding_evaluator.go b/pkg/logql/sharding_evaluator.go index e5d7dd9958758..9cfe2b2558d4b 100644 --- a/pkg/logql/sharding_evaluator.go +++ b/pkg/logql/sharding_evaluator.go @@ -15,7 +15,7 @@ type downstreamEvaluator struct { } // Evaluator returns a StepEvaluator for a given SampleExpr -func (ev *downstreamEvaluator) Evaluator( +func (ev *downstreamEvaluator) StepEvaluator( ctx context.Context, nextEv Evaluator, expr SampleExpr, @@ -30,7 +30,7 @@ func (ev *downstreamEvaluator) Evaluator( return nil, errors.New("unimplemented") default: // used for aggregating downstreamed exprs, literalExprs - return nextEv.Evaluator(ctx, nextEv, expr, params) + return nextEv.StepEvaluator(ctx, nextEv, expr, params) } } From 3147f1e354810538b3c23d6c73c1f5307e26326c Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 19 Mar 2020 18:16:56 -0400 Subject: [PATCH 07/67] chained evaluator impl --- pkg/logql/chainedevaluator.go | 53 +++++++++++++++++++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 pkg/logql/chainedevaluator.go diff --git a/pkg/logql/chainedevaluator.go b/pkg/logql/chainedevaluator.go new file mode 100644 index 0000000000000..346dd6992cb4f --- /dev/null +++ b/pkg/logql/chainedevaluator.go @@ -0,0 +1,53 @@ +package logql + +import ( + "context" + + "github.com/grafana/loki/pkg/iter" + "github.com/pkg/errors" +) + +// ChainedEvaluator is an evaluator which chains multiple other evaluators, +// deferring to the first successful one. +type ChainedEvaluator struct { + evaluators []Evaluator +} + +// StepEvaluator attempts the embedded evaluators until one succeeds or they all error. +func (c *ChainedEvaluator) StepEvaluator( + ctx context.Context, + nextEvaluator Evaluator, + expr SampleExpr, + p Params, +) (stepper StepEvaluator, err error) { + for _, eval := range c.evaluators { + if stepper, err = eval.StepEvaluator(ctx, nextEvaluator, expr, p); err == nil { + return stepper, nil + } + } + return nil, err +} + +// Iterator attempts the embedded evaluators until one succeeds or they all error. +func (c *ChainedEvaluator) Iterator( + ctx context.Context, + expr LogSelectorExpr, + p Params, +) (iterator iter.EntryIterator, err error) { + for _, eval := range c.evaluators { + if iterator, err = eval.Iterator(ctx, expr, p); err == nil { + return iterator, nil + } + } + return nil, err +} + +// NewChainedEvaluator constructs a ChainedEvaluator from one or more Evaluators +func NewChainedEvaluator(evals ...Evaluator) (*ChainedEvaluator, error) { + if len(evals) == 0 { + return nil, errors.New("must supply an Evaluator") + } + return &ChainedEvaluator{ + evaluators: evals, + }, nil +} From 66efd8cf588d51cffa8064c47ea9900a4157df8e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 20 Mar 2020 15:29:29 -0400 Subject: [PATCH 08/67] tidying up sharding code --- pkg/logql/evaluator.go | 33 +++--------- pkg/logql/sharding.go | 93 +++++++++++++++++++++++++++++++++ pkg/logql/sharding_evaluator.go | 44 ---------------- pkg/logql/sharding_exprs.go | 25 --------- pkg/logql/sharding_mapper.go | 5 -- 5 files changed, 99 insertions(+), 101 deletions(-) create mode 100644 pkg/logql/sharding.go delete mode 100644 pkg/logql/sharding_evaluator.go delete mode 100644 pkg/logql/sharding_exprs.go delete mode 100644 pkg/logql/sharding_mapper.go diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 57165302d06c7..f0e39d5238866 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -76,6 +76,11 @@ type Evaluator interface { Iterator(context.Context, LogSelectorExpr, Params) (iter.EntryIterator, error) } +// EvaluatorUnsupportedType is a helper for signaling that an evaluator does not support an Expr type +func EvaluatorUnsupportedType(expr Expr, ev Evaluator) error { + return errors.Errorf("unexpected expr type (%T) for Evaluator type (%T) ", expr, ev) +} + type defaultEvaluator struct { maxLookBackPeriod time.Duration querier Querier @@ -126,7 +131,7 @@ func (ev *defaultEvaluator) StepEvaluator( case *binOpExpr: return binOpStepEvaluator(ctx, nextEv, e, q) default: - return nil, errors.Errorf("unexpected type (%T): %v", e, e) + return nil, EvaluatorUnsupportedType(e, ev) } } @@ -607,29 +612,3 @@ func literalStepEvaluator( eval.Close, ) } - -// ConcatEvaluator joins multiple StepEvaluators. -// Contract: They must be of identical start, end, and step values. -func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { - return newStepEvaluator( - func() (done bool, ts int64, vec promql.Vector) { - var cur promql.Vector - for { - for _, eval := range evaluators { - done, ts, cur = eval.Next() - vec = append(vec, cur...) - } - } - return done, ts, vec - - }, - func() (lastErr error) { - for _, eval := range evaluators { - if err := eval.Close(); err != nil { - lastErr = err - } - } - return lastErr - }, - ) -} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go new file mode 100644 index 0000000000000..a28a611d1b754 --- /dev/null +++ b/pkg/logql/sharding.go @@ -0,0 +1,93 @@ +package logql + +import ( + "context" + "fmt" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/iter" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/promql" +) + +// DownstreamSampleExpr is a SampleExpr which signals downstream computation +type DownstreamSampleExpr struct { + shard *astmapper.ShardAnnotation + SampleExpr +} + +// DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation +type DownstreamLogSelectorExpr struct { + shard *astmapper.ShardAnnotation + LogSelectorExpr +} + +// ConcatSampleExpr is an expr for concatenating multiple SampleExpr +type ConcatSampleExpr struct { + SampleExpr + next *ConcatSampleExpr +} + +// ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr +type ConcatLogSelectorExpr struct { + LogSelectorExpr + next *ConcatLogSelectorExpr +} + +// downstreamEvaluator is an evaluator which handles shard aware AST nodes +// and embeds a default evaluator otherwise +type downstreamEvaluator struct { + shards int +} + +// Evaluator returns a StepEvaluator for a given SampleExpr +func (ev *downstreamEvaluator) StepEvaluator( + ctx context.Context, + nextEv Evaluator, + expr SampleExpr, + params Params, +) (StepEvaluator, error) { + switch expr.(type) { + case DownstreamSampleExpr: + // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier + return nil, errors.New("unimplemented") + case ConcatSampleExpr: + // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat + return nil, errors.New("unimplemented") + default: + return nil, EvaluatorUnsupportedType(expr, ev) + } +} + +// Iterator returns the iter.EntryIterator for a given LogSelectorExpr +func (ev *downstreamEvaluator) Iterator( + _ context.Context, + expr LogSelectorExpr, + _ Params, +) (iter.EntryIterator, error) { + return nil, fmt.Errorf("downstreamEvaluator does not implement Iterator, called with expr: %+v", expr) +} + +// ConcatEvaluator joins multiple StepEvaluators. +// Contract: They must be of identical start, end, and step values. +func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { + return newStepEvaluator( + func() (done bool, ts int64, vec promql.Vector) { + var cur promql.Vector + for _, eval := range evaluators { + done, ts, cur = eval.Next() + vec = append(vec, cur...) + } + return done, ts, vec + + }, + func() (lastErr error) { + for _, eval := range evaluators { + if err := eval.Close(); err != nil { + lastErr = err + } + } + return lastErr + }, + ) +} diff --git a/pkg/logql/sharding_evaluator.go b/pkg/logql/sharding_evaluator.go deleted file mode 100644 index 9cfe2b2558d4b..0000000000000 --- a/pkg/logql/sharding_evaluator.go +++ /dev/null @@ -1,44 +0,0 @@ -package logql - -import ( - "context" - "fmt" - - "github.com/grafana/loki/pkg/iter" - "github.com/pkg/errors" -) - -// downstreamEvaluator is an evaluator which handles shard aware AST nodes -// and embeds a default evaluator otherwise -type downstreamEvaluator struct { - shards int -} - -// Evaluator returns a StepEvaluator for a given SampleExpr -func (ev *downstreamEvaluator) StepEvaluator( - ctx context.Context, - nextEv Evaluator, - expr SampleExpr, - params Params, -) (StepEvaluator, error) { - switch expr.(type) { - case DownstreamSampleExpr: - // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier - return nil, errors.New("unimplemented") - case ConcatSampleExpr: - // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat - return nil, errors.New("unimplemented") - default: - // used for aggregating downstreamed exprs, literalExprs - return nextEv.StepEvaluator(ctx, nextEv, expr, params) - } -} - -// Iterator returns the iter.EntryIterator for a given LogSelectorExpr -func (ev *downstreamEvaluator) Iterator( - _ context.Context, - expr LogSelectorExpr, - _ Params, -) (iter.EntryIterator, error) { - return nil, fmt.Errorf("downstreamEvaluator does not implement Iterator, called with expr: %+v", expr) -} diff --git a/pkg/logql/sharding_exprs.go b/pkg/logql/sharding_exprs.go deleted file mode 100644 index b37164920137d..0000000000000 --- a/pkg/logql/sharding_exprs.go +++ /dev/null @@ -1,25 +0,0 @@ -package logql - -// DownstreamSampleExpr is a SampleExpr which signals downstream computation -type DownstreamSampleExpr struct { - shard *int - SampleExpr -} - -// DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation -type DownstreamLogSelectorExpr struct { - shard *int - LogSelectorExpr -} - -// ConcatSampleExpr is an expr for concatenating multiple SampleExpr -type ConcatSampleExpr struct { - SampleExpr - next *ConcatSampleExpr -} - -// ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr -type ConcatLogSelectorExpr struct { - LogSelectorExpr - next *ConcatLogSelectorExpr -} diff --git a/pkg/logql/sharding_mapper.go b/pkg/logql/sharding_mapper.go deleted file mode 100644 index bd63cb5167ee2..0000000000000 --- a/pkg/logql/sharding_mapper.go +++ /dev/null @@ -1,5 +0,0 @@ -package logql - -type ShardMapper struct { - shards int -} From 738dd17fbd0b4e5d246b9922fc1972d5123372dc Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 10:54:22 -0400 Subject: [PATCH 09/67] handling for ConcatSampleExpr --- pkg/logql/sharding.go | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index a28a611d1b754..e5815d4928581 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -35,10 +35,7 @@ type ConcatLogSelectorExpr struct { } // downstreamEvaluator is an evaluator which handles shard aware AST nodes -// and embeds a default evaluator otherwise -type downstreamEvaluator struct { - shards int -} +type downstreamEvaluator struct{} // Evaluator returns a StepEvaluator for a given SampleExpr func (ev *downstreamEvaluator) StepEvaluator( @@ -47,13 +44,27 @@ func (ev *downstreamEvaluator) StepEvaluator( expr SampleExpr, params Params, ) (StepEvaluator, error) { - switch expr.(type) { + switch e := expr.(type) { case DownstreamSampleExpr: // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier return nil, errors.New("unimplemented") + case ConcatSampleExpr: // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat - return nil, errors.New("unimplemented") + var xs []StepEvaluator + cur := &e + + for cur != nil { + eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) + if err != nil { + return nil, err + } + xs = append(xs, eval) + cur = cur.next + } + + return ConcatEvaluator(xs) + default: return nil, EvaluatorUnsupportedType(expr, ev) } From 16d70163c747bd12ec8c3f4a5f15ef59b9181bd4 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 14:22:00 -0400 Subject: [PATCH 10/67] downstream iterator --- pkg/logql/sharding.go | 23 ++++++++++++++++++----- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index e5815d4928581..88ebb3ad4c11b 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -2,7 +2,6 @@ package logql import ( "context" - "fmt" "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/grafana/loki/pkg/iter" @@ -46,7 +45,7 @@ func (ev *downstreamEvaluator) StepEvaluator( ) (StepEvaluator, error) { switch e := expr.(type) { case DownstreamSampleExpr: - // determine type (SampleExpr, LogSelectorExpr) and downstream to a querier + // downstream to a querier return nil, errors.New("unimplemented") case ConcatSampleExpr: @@ -72,11 +71,25 @@ func (ev *downstreamEvaluator) StepEvaluator( // Iterator returns the iter.EntryIterator for a given LogSelectorExpr func (ev *downstreamEvaluator) Iterator( - _ context.Context, + ctx context.Context, expr LogSelectorExpr, - _ Params, + params Params, ) (iter.EntryIterator, error) { - return nil, fmt.Errorf("downstreamEvaluator does not implement Iterator, called with expr: %+v", expr) + switch e := expr.(type) { + case DownstreamLogSelectorExpr: + case ConcatLogSelectorExpr: + var iters []iter.EntryIterator + cur := &e + for cur != nil { + iterator, err := ev.Iterator(ctx, e, params) + if err != nil { + return nil, err + } + iters = append(iters, iterator) + } + return iter.NewHeapIterator(ctx, iters, params.Direction()), nil + } + return nil, errors.New("unimplemented") } // ConcatEvaluator joins multiple StepEvaluators. From 67385fc4701adde8b361c3be6c9ba5a81c307049 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 14:57:32 -0400 Subject: [PATCH 11/67] structure for downstreaming asts --- pkg/logql/downstreamer.go | 7 +++++++ pkg/logql/sharding.go | 12 +++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 pkg/logql/downstreamer.go diff --git a/pkg/logql/downstreamer.go b/pkg/logql/downstreamer.go new file mode 100644 index 0000000000000..96fdef69dc919 --- /dev/null +++ b/pkg/logql/downstreamer.go @@ -0,0 +1,7 @@ +package logql + +// Downstreamer is an interface for deferring responsibility for query execution. +// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. +type Downstreamer interface { + // Downstream(*LokiRequest) (*LokiResponse, error) +} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 88ebb3ad4c11b..1af369cb7653d 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -34,7 +34,7 @@ type ConcatLogSelectorExpr struct { } // downstreamEvaluator is an evaluator which handles shard aware AST nodes -type downstreamEvaluator struct{} +type downstreamEvaluator struct{ Downstreamer } // Evaluator returns a StepEvaluator for a given SampleExpr func (ev *downstreamEvaluator) StepEvaluator( @@ -56,6 +56,10 @@ func (ev *downstreamEvaluator) StepEvaluator( for cur != nil { eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) if err != nil { + // Close previously opened StepEvaluators + for _, x := range xs { + x.Close() + } return nil, err } xs = append(xs, eval) @@ -77,12 +81,18 @@ func (ev *downstreamEvaluator) Iterator( ) (iter.EntryIterator, error) { switch e := expr.(type) { case DownstreamLogSelectorExpr: + // downstream to a querier + return nil, errors.New("unimplemented") case ConcatLogSelectorExpr: var iters []iter.EntryIterator cur := &e for cur != nil { iterator, err := ev.Iterator(ctx, e, params) if err != nil { + // Close previously opened StepEvaluators + for _, x := range iters { + x.Close() + } return nil, err } iters = append(iters, iterator) From 1271dfc0525afd61195a140c44bbc51a3fc32775 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 16:49:38 -0400 Subject: [PATCH 12/67] outlines sharding optimizations --- pkg/logql/astmapper.go | 33 +++++++++++++++++++++++++++++ pkg/logql/properties.go | 28 ------------------------ pkg/logql/sharding_optimizations.go | 19 +++++++++++++++++ 3 files changed, 52 insertions(+), 28 deletions(-) delete mode 100644 pkg/logql/properties.go create mode 100644 pkg/logql/sharding_optimizations.go diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go index 4e1a91324fd89..3f4693d0970c0 100644 --- a/pkg/logql/astmapper.go +++ b/pkg/logql/astmapper.go @@ -9,3 +9,36 @@ type ASTMapper interface { func CloneExpr(expr Expr) (Expr, error) { return ParseExpr(expr.String()) } + +type ShardMapper struct { + shards int +} + +func (m ShardMapper) Map(expr Expr) (Expr, error) { return nil, nil } + +// technically, std{dev,var} are also parallelizable if there is no cross-shard merging +// in descendent nodes in the AST. This optimization is currently avoided for simplicity. +var parallelOperations = map[string]bool{ + OpTypeSum: true, + OpTypeAvg: true, + OpTypeMax: true, + OpTypeMin: true, + OpTypeCount: true, + OpTypeBottomK: true, + OpTypeTopK: true, + OpTypeCountOverTime: true, + OpTypeRate: true, +} + +func CanParallel(e Expr) bool { + switch expr := e.(type) { + case *matchersExpr, *filterExpr: + return true + case *rangeAggregationExpr: + return parallelOperations[expr.operation] + case *vectorAggregationExpr: + return parallelOperations[expr.operation] && CanParallel(expr.left) + default: + return false + } +} diff --git a/pkg/logql/properties.go b/pkg/logql/properties.go deleted file mode 100644 index 96cd220409242..0000000000000 --- a/pkg/logql/properties.go +++ /dev/null @@ -1,28 +0,0 @@ -package logql - -// technically, std{dev,var} are also parallelizable if there is no cross-shard merging -// in descendent nodes in the AST. This optimization is currently avoided for simplicity. -var parallelOperations = map[string]bool{ - OpTypeSum: true, - OpTypeAvg: true, - OpTypeMax: true, - OpTypeMin: true, - OpTypeCount: true, - OpTypeBottomK: true, - OpTypeTopK: true, - OpTypeCountOverTime: true, - OpTypeRate: true, -} - -func CanParallel(e Expr) bool { - switch expr := e.(type) { - case *matchersExpr, *filterExpr: - return true - case *rangeAggregationExpr: - return parallelOperations[expr.operation] - case *vectorAggregationExpr: - return parallelOperations[expr.operation] && CanParallel(expr.left) - default: - return false - } -} diff --git a/pkg/logql/sharding_optimizations.go b/pkg/logql/sharding_optimizations.go new file mode 100644 index 0000000000000..2d045ac32907d --- /dev/null +++ b/pkg/logql/sharding_optimizations.go @@ -0,0 +1,19 @@ +package logql + +// avg(x) -> sum(x)/count(x) + +// sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) + +// max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) + +// min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) + +// count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) + +// topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) + +// botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) + +// rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... + +// count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... From 84fe72cb462e856897883a0c1d0478a6eb30b8a3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 22:23:47 -0400 Subject: [PATCH 13/67] work on sharding mapper --- pkg/logql/astmapper.go | 106 ++++++++++++++++++++++++++++++++--------- 1 file changed, 84 insertions(+), 22 deletions(-) diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go index 3f4693d0970c0..c3a4834327fb3 100644 --- a/pkg/logql/astmapper.go +++ b/pkg/logql/astmapper.go @@ -1,5 +1,12 @@ package logql +import ( + "fmt" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/pkg/errors" +) + // ASTMapper is the exported interface for mapping between multiple AST representations type ASTMapper interface { Map(Expr) (Expr, error) @@ -14,31 +21,86 @@ type ShardMapper struct { shards int } -func (m ShardMapper) Map(expr Expr) (Expr, error) { return nil, nil } +func (m ShardMapper) Map(expr Expr) (Expr, error) { + switch e := expr.(type) { + case *literalExpr: + return e, nil + case *matchersExpr, *filterExpr: + return m.mapLogSelectorExpr(e.(LogSelectorExpr)), nil + case *vectorAggregationExpr: + return m.mapVectorAggregationExpr(e) + case *rangeAggregationExpr: + return m.mapRangeAggregationExpr(e) + case *binOpExpr: + lhsMapped, err := m.Map(e.SampleExpr) + if err != nil { + return nil, err + } + rhsMapped, err := m.Map(e.SampleExpr) + if err != nil { + return nil, err + } + lhsSampleExpr, ok := lhsMapped.(SampleExpr) + if !ok { + return nil, badASTMapping("SampleExpr", lhsMapped) + } + rhsSampleExpr, ok := rhsMapped.(SampleExpr) + if !ok { + return nil, badASTMapping("SampleExpr", rhsMapped) + } + e.SampleExpr = lhsSampleExpr + e.RHS = rhsSampleExpr + return e, nil + default: + return nil, MapperUnsupportedType(expr, m) + } +} + +func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { + var head *ConcatLogSelectorExpr + for i := m.shards - 1; i >= 0; i-- { + head = &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + LogSelectorExpr: expr, + shard: &astmapper.ShardAnnotation{Shard: i, Of: m.shards}, + }, + next: head, + } + } + + return head +} // technically, std{dev,var} are also parallelizable if there is no cross-shard merging // in descendent nodes in the AST. This optimization is currently avoided for simplicity. -var parallelOperations = map[string]bool{ - OpTypeSum: true, - OpTypeAvg: true, - OpTypeMax: true, - OpTypeMin: true, - OpTypeCount: true, - OpTypeBottomK: true, - OpTypeTopK: true, - OpTypeCountOverTime: true, - OpTypeRate: true, -} - -func CanParallel(e Expr) bool { - switch expr := e.(type) { - case *matchersExpr, *filterExpr: - return true - case *rangeAggregationExpr: - return parallelOperations[expr.operation] - case *vectorAggregationExpr: - return parallelOperations[expr.operation] && CanParallel(expr.left) +func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Expr, error) { + switch expr.operation { + case OpTypeSum: + case OpTypeAvg: + case OpTypeMax: + case OpTypeMin: + case OpTypeCount: + case OpTypeBottomK: + case OpTypeTopK: default: - return false + return expr, nil } } + +func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) (Expr, error) { + switch expr.operation { + case OpTypeCountOverTime: + case OpTypeRate: + default: + return expr, nil + } +} + +func badASTMapping(expected string, got Expr) error { + return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) +} + +// MapperUnsuportedType is a helper for signaling that an evaluator does not support an Expr type +func MapperUnsupportedType(expr Expr, m ASTMapper) error { + return errors.Errorf("unexpected expr type (%T) for ASTMapper type (%T) ", expr, m) +} From 1c0d630673c91937c65afef7612f1e67f63c923e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sun, 22 Mar 2020 23:16:46 -0400 Subject: [PATCH 14/67] ast sharding optimizations --- pkg/logql/astmapper.go | 85 +++++++++++++++++++++++++---- pkg/logql/sharding_optimizations.go | 19 ------- 2 files changed, 73 insertions(+), 31 deletions(-) delete mode 100644 pkg/logql/sharding_optimizations.go diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go index c3a4834327fb3..9db674378a3fc 100644 --- a/pkg/logql/astmapper.go +++ b/pkg/logql/astmapper.go @@ -30,7 +30,7 @@ func (m ShardMapper) Map(expr Expr) (Expr, error) { case *vectorAggregationExpr: return m.mapVectorAggregationExpr(e) case *rangeAggregationExpr: - return m.mapRangeAggregationExpr(e) + return m.mapRangeAggregationExpr(e), nil case *binOpExpr: lhsMapped, err := m.Map(e.SampleExpr) if err != nil { @@ -62,7 +62,27 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { head = &ConcatLogSelectorExpr{ LogSelectorExpr: DownstreamLogSelectorExpr{ LogSelectorExpr: expr, - shard: &astmapper.ShardAnnotation{Shard: i, Of: m.shards}, + shard: &astmapper.ShardAnnotation{ + Shard: i, + Of: m.shards, + }, + }, + next: head, + } + } + + return head +} + +func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { + var head *ConcatSampleExpr + for i := m.shards - 1; i >= 0; i-- { + head = &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: i, + Of: m.shards, + }, }, next: head, } @@ -73,26 +93,67 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { // technically, std{dev,var} are also parallelizable if there is no cross-shard merging // in descendent nodes in the AST. This optimization is currently avoided for simplicity. -func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Expr, error) { +func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) { switch expr.operation { - case OpTypeSum: + // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) + // max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) + // min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) + // topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) + // botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) + case OpTypeSum, OpTypeMax, OpTypeMin, OpTypeTopK, OpTypeBottomK: + return &vectorAggregationExpr{ + left: m.mapSampleExpr(expr), + grouping: expr.grouping, + params: expr.params, + operation: expr.operation, + }, nil + case OpTypeAvg: - case OpTypeMax: - case OpTypeMin: + // avg(x) -> sum(x)/count(x) + lhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ + left: expr.left, + grouping: expr.grouping, + operation: OpTypeSum, + }) + if err != nil { + return nil, err + } + rhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ + left: expr.left, + grouping: expr.grouping, + operation: OpTypeCount, + }) + if err != nil { + return nil, err + } + + return &binOpExpr{ + SampleExpr: lhs, + RHS: rhs, + op: OpTypeDiv, + }, nil + case OpTypeCount: - case OpTypeBottomK: - case OpTypeTopK: + // count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) + sharded := m.mapSampleExpr(expr) + return &vectorAggregationExpr{ + left: sharded, + grouping: expr.grouping, + operation: OpTypeSum, + }, nil default: return expr, nil } } -func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) (Expr, error) { +func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleExpr { switch expr.operation { - case OpTypeCountOverTime: - case OpTypeRate: + case OpTypeCountOverTime, OpTypeRate: + // count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... + // rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... + return m.mapSampleExpr(expr) default: - return expr, nil + return expr } } diff --git a/pkg/logql/sharding_optimizations.go b/pkg/logql/sharding_optimizations.go deleted file mode 100644 index 2d045ac32907d..0000000000000 --- a/pkg/logql/sharding_optimizations.go +++ /dev/null @@ -1,19 +0,0 @@ -package logql - -// avg(x) -> sum(x)/count(x) - -// sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) - -// max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) - -// min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) - -// count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) - -// topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) - -// botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) - -// rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... - -// count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... From 654618f14b33e1bfb3b13baf98d3a7c3144b7caf Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 23 Mar 2020 09:34:48 -0400 Subject: [PATCH 15/67] test for different logrange positions --- pkg/logql/parser_test.go | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/pkg/logql/parser_test.go b/pkg/logql/parser_test.go index 13a38536fb99c..2687249513c2d 100644 --- a/pkg/logql/parser_test.go +++ b/pkg/logql/parser_test.go @@ -20,6 +20,44 @@ func TestParse(t *testing.T) { exp Expr err error }{ + { + // test [12h] before filter expr + in: `count_over_time({foo="bar"}[12h] |= "error")`, + exp: &rangeAggregationExpr{ + operation: "count_over_time", + left: &logRange{ + left: &filterExpr{ + ty: labels.MatchEqual, + match: "error", + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + interval: 12 * time.Hour, + }, + }, + }, + { + // test [12h] after filter expr + in: `count_over_time({foo="bar"} |= "error" [12h])`, + exp: &rangeAggregationExpr{ + operation: "count_over_time", + left: &logRange{ + left: &filterExpr{ + ty: labels.MatchEqual, + match: "error", + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + interval: 12 * time.Hour, + }, + }, + }, { in: `{foo="bar"}`, exp: &matchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}}, From 0f6715589251628d36e1fc0623c1c51f830e919e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 23 Mar 2020 10:33:05 -0400 Subject: [PATCH 16/67] shard mapper tests --- pkg/logql/astmapper.go | 141 ----------------------- pkg/logql/shardmapper.go | 155 +++++++++++++++++++++++++ pkg/logql/shardmapper_test.go | 209 ++++++++++++++++++++++++++++++++++ 3 files changed, 364 insertions(+), 141 deletions(-) create mode 100644 pkg/logql/shardmapper.go create mode 100644 pkg/logql/shardmapper_test.go diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go index 9db674378a3fc..6067fbeb93e94 100644 --- a/pkg/logql/astmapper.go +++ b/pkg/logql/astmapper.go @@ -3,7 +3,6 @@ package logql import ( "fmt" - "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/pkg/errors" ) @@ -17,146 +16,6 @@ func CloneExpr(expr Expr) (Expr, error) { return ParseExpr(expr.String()) } -type ShardMapper struct { - shards int -} - -func (m ShardMapper) Map(expr Expr) (Expr, error) { - switch e := expr.(type) { - case *literalExpr: - return e, nil - case *matchersExpr, *filterExpr: - return m.mapLogSelectorExpr(e.(LogSelectorExpr)), nil - case *vectorAggregationExpr: - return m.mapVectorAggregationExpr(e) - case *rangeAggregationExpr: - return m.mapRangeAggregationExpr(e), nil - case *binOpExpr: - lhsMapped, err := m.Map(e.SampleExpr) - if err != nil { - return nil, err - } - rhsMapped, err := m.Map(e.SampleExpr) - if err != nil { - return nil, err - } - lhsSampleExpr, ok := lhsMapped.(SampleExpr) - if !ok { - return nil, badASTMapping("SampleExpr", lhsMapped) - } - rhsSampleExpr, ok := rhsMapped.(SampleExpr) - if !ok { - return nil, badASTMapping("SampleExpr", rhsMapped) - } - e.SampleExpr = lhsSampleExpr - e.RHS = rhsSampleExpr - return e, nil - default: - return nil, MapperUnsupportedType(expr, m) - } -} - -func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { - var head *ConcatLogSelectorExpr - for i := m.shards - 1; i >= 0; i-- { - head = &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ - LogSelectorExpr: expr, - shard: &astmapper.ShardAnnotation{ - Shard: i, - Of: m.shards, - }, - }, - next: head, - } - } - - return head -} - -func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { - var head *ConcatSampleExpr - for i := m.shards - 1; i >= 0; i-- { - head = &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ - shard: &astmapper.ShardAnnotation{ - Shard: i, - Of: m.shards, - }, - }, - next: head, - } - } - - return head -} - -// technically, std{dev,var} are also parallelizable if there is no cross-shard merging -// in descendent nodes in the AST. This optimization is currently avoided for simplicity. -func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) { - switch expr.operation { - // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) - // max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) - // min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) - // topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) - // botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) - case OpTypeSum, OpTypeMax, OpTypeMin, OpTypeTopK, OpTypeBottomK: - return &vectorAggregationExpr{ - left: m.mapSampleExpr(expr), - grouping: expr.grouping, - params: expr.params, - operation: expr.operation, - }, nil - - case OpTypeAvg: - // avg(x) -> sum(x)/count(x) - lhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ - left: expr.left, - grouping: expr.grouping, - operation: OpTypeSum, - }) - if err != nil { - return nil, err - } - rhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ - left: expr.left, - grouping: expr.grouping, - operation: OpTypeCount, - }) - if err != nil { - return nil, err - } - - return &binOpExpr{ - SampleExpr: lhs, - RHS: rhs, - op: OpTypeDiv, - }, nil - - case OpTypeCount: - // count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) - sharded := m.mapSampleExpr(expr) - return &vectorAggregationExpr{ - left: sharded, - grouping: expr.grouping, - operation: OpTypeSum, - }, nil - default: - return expr, nil - } -} - -func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleExpr { - switch expr.operation { - case OpTypeCountOverTime, OpTypeRate: - // count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... - // rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... - return m.mapSampleExpr(expr) - default: - return expr - } -} - func badASTMapping(expected string, got Expr) error { return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) } diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go new file mode 100644 index 0000000000000..dbfbd483dcf95 --- /dev/null +++ b/pkg/logql/shardmapper.go @@ -0,0 +1,155 @@ +package logql + +import ( + "fmt" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" +) + +func NewShardMapper(shards int) (ShardMapper, error) { + if shards < 2 { + return ShardMapper{}, fmt.Errorf("Cannot create ShardMapper with <2 shards. Received %d", shards) + } + return ShardMapper{shards}, nil +} + +type ShardMapper struct { + shards int +} + +func (m ShardMapper) Map(expr Expr) (Expr, error) { + switch e := expr.(type) { + case *literalExpr: + return e, nil + case *matchersExpr, *filterExpr: + return m.mapLogSelectorExpr(e.(LogSelectorExpr)), nil + case *vectorAggregationExpr: + return m.mapVectorAggregationExpr(e) + case *rangeAggregationExpr: + return m.mapRangeAggregationExpr(e), nil + case *binOpExpr: + lhsMapped, err := m.Map(e.SampleExpr) + if err != nil { + return nil, err + } + rhsMapped, err := m.Map(e.SampleExpr) + if err != nil { + return nil, err + } + lhsSampleExpr, ok := lhsMapped.(SampleExpr) + if !ok { + return nil, badASTMapping("SampleExpr", lhsMapped) + } + rhsSampleExpr, ok := rhsMapped.(SampleExpr) + if !ok { + return nil, badASTMapping("SampleExpr", rhsMapped) + } + e.SampleExpr = lhsSampleExpr + e.RHS = rhsSampleExpr + return e, nil + default: + return nil, MapperUnsupportedType(expr, m) + } +} + +func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { + var head *ConcatLogSelectorExpr + for i := m.shards - 1; i >= 0; i-- { + head = &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: i, + Of: m.shards, + }, + LogSelectorExpr: expr, + }, + next: head, + } + } + + return head +} + +func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { + var head *ConcatSampleExpr + for i := m.shards - 1; i >= 0; i-- { + head = &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: i, + Of: m.shards, + }, + SampleExpr: expr, + }, + next: head, + } + } + + return head +} + +// technically, std{dev,var} are also parallelizable if there is no cross-shard merging +// in descendent nodes in the AST. This optimization is currently avoided for simplicity. +func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) { + switch expr.operation { + // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) + // max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) + // min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) + // topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) + // botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) + case OpTypeSum, OpTypeMax, OpTypeMin, OpTypeTopK, OpTypeBottomK: + return &vectorAggregationExpr{ + left: m.mapSampleExpr(expr), + grouping: expr.grouping, + params: expr.params, + operation: expr.operation, + }, nil + + case OpTypeAvg: + // avg(x) -> sum(x)/count(x) + lhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ + left: expr.left, + grouping: expr.grouping, + operation: OpTypeSum, + }) + if err != nil { + return nil, err + } + rhs, err := m.mapVectorAggregationExpr(&vectorAggregationExpr{ + left: expr.left, + grouping: expr.grouping, + operation: OpTypeCount, + }) + if err != nil { + return nil, err + } + + return &binOpExpr{ + SampleExpr: lhs, + RHS: rhs, + op: OpTypeDiv, + }, nil + + case OpTypeCount: + // count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) + sharded := m.mapSampleExpr(expr) + return &vectorAggregationExpr{ + left: sharded, + grouping: expr.grouping, + operation: OpTypeSum, + }, nil + default: + return expr, nil + } +} + +func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleExpr { + switch expr.operation { + case OpTypeCountOverTime, OpTypeRate: + // count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... + // rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... + return m.mapSampleExpr(expr) + default: + return expr + } +} diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go new file mode 100644 index 0000000000000..ccd73f4be3a96 --- /dev/null +++ b/pkg/logql/shardmapper_test.go @@ -0,0 +1,209 @@ +package logql + +import ( + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" +) + +func TestMapSampleExpr(t *testing.T) { + m, err := NewShardMapper(2) + require.Nil(t, err) + + for _, tc := range []struct { + in SampleExpr + out SampleExpr + }{ + { + in: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: time.Minute, + }, + }, + out: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + } { + t.Run(tc.in.String(), func(t *testing.T) { + require.Equal(t, tc.out, m.mapSampleExpr(tc.in)) + }) + + } +} + +func TestMapping(t *testing.T) { + m, err := NewShardMapper(2) + require.Nil(t, err) + + for _, tc := range []struct { + in string + expr Expr + err error + }{ + { + in: `{foo="bar"}`, + expr: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + LogSelectorExpr: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + next: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + LogSelectorExpr: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + next: nil, + }, + }, + }, + { + in: `{foo="bar"} |= "error"`, + expr: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + LogSelectorExpr: &filterExpr{ + match: "error", + ty: labels.MatchEqual, + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + }, + next: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + LogSelectorExpr: &filterExpr{ + match: "error", + ty: labels.MatchEqual, + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + }, + next: nil, + }, + }, + }, + { + in: `rate({foo="bar"}[5m])`, + expr: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + } { + t.Run(tc.in, func(t *testing.T) { + ast, err := ParseExpr(tc.in) + require.Equal(t, tc.err, err) + + mapped, err := m.Map(ast) + require.Equal(t, tc.err, err) + require.Equal(t, tc.expr, mapped) + }) + } +} From 648cd4c94c61e7c754125f481680e55987cca9e1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 23 Mar 2020 18:54:25 -0400 Subject: [PATCH 17/67] stronger ast sharding & tests --- pkg/logql/ast.go | 40 +- pkg/logql/sharding.go | 27 ++ pkg/logql/shardmapper.go | 83 +++- pkg/logql/shardmapper_test.go | 690 ++++++++++++++++++++++++++++++++++ 4 files changed, 824 insertions(+), 16 deletions(-) diff --git a/pkg/logql/ast.go b/pkg/logql/ast.go index 91e3c8c5ade60..73b5cea25bbab 100644 --- a/pkg/logql/ast.go +++ b/pkg/logql/ast.go @@ -182,15 +182,18 @@ func addFilterToLogRangeExpr(left *logRange, ty labels.MatchType, match string) } const ( - OpTypeSum = "sum" - OpTypeAvg = "avg" - OpTypeMax = "max" - OpTypeMin = "min" - OpTypeCount = "count" - OpTypeStddev = "stddev" - OpTypeStdvar = "stdvar" - OpTypeBottomK = "bottomk" - OpTypeTopK = "topk" + // vector ops + OpTypeSum = "sum" + OpTypeAvg = "avg" + OpTypeMax = "max" + OpTypeMin = "min" + OpTypeCount = "count" + OpTypeStddev = "stddev" + OpTypeStdvar = "stdvar" + OpTypeBottomK = "bottomk" + OpTypeTopK = "topk" + + // range vector ops OpTypeCountOverTime = "count_over_time" OpTypeRate = "rate" @@ -217,6 +220,8 @@ func IsLogicalBinOp(op string) bool { type SampleExpr interface { // Selector is the LogQL selector to apply when retrieving logs. Selector() LogSelectorExpr + // Operations returns the list of operations used in this SampleExpr + Operations() []string Expr } @@ -244,6 +249,11 @@ func (e *rangeAggregationExpr) String() string { return formatOperation(e.operation, nil, e.left.String()) } +// impl SampleExpr +func (e *rangeAggregationExpr) Operations() []string { + return []string{e.operation} +} + type grouping struct { groups []string without bool @@ -320,6 +330,11 @@ func (e *vectorAggregationExpr) String() string { return formatOperation(e.operation, e.grouping, params...) } +// impl SampleExpr +func (e *vectorAggregationExpr) Operations() []string { + return append(e.left.Operations(), e.operation) +} + type binOpExpr struct { SampleExpr RHS SampleExpr @@ -330,6 +345,12 @@ func (e *binOpExpr) String() string { return fmt.Sprintf("%s %s %s", e.SampleExpr.String(), e.op, e.RHS.String()) } +// impl SampleExpr +func (e *binOpExpr) Operations() []string { + ops := append(e.SampleExpr.Operations(), e.RHS.Operations()...) + return append(ops, e.op) +} + func mustNewBinOpExpr(op string, lhs, rhs Expr) SampleExpr { left, ok := lhs.(SampleExpr) if !ok { @@ -423,6 +444,7 @@ func (e *literalExpr) String() string { // to facilitate sum types. We'll be type switching when evaluating them anyways // and they will only be present in binary operation legs. func (e *literalExpr) Selector() LogSelectorExpr { return e } +func (e *literalExpr) Operations() []string { return nil } func (e *literalExpr) Filter() (LineFilter, error) { return nil, nil } func (e *literalExpr) Matchers() []*labels.Matcher { return nil } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 1af369cb7653d..d8ae72ea3e8d6 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -2,6 +2,7 @@ package logql import ( "context" + "fmt" "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/grafana/loki/pkg/iter" @@ -15,24 +16,50 @@ type DownstreamSampleExpr struct { SampleExpr } +func (d DownstreamSampleExpr) String() string { + return fmt.Sprintf("downstream<%s, shard=%s>", d.SampleExpr.String(), d.shard) +} + // DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation type DownstreamLogSelectorExpr struct { shard *astmapper.ShardAnnotation LogSelectorExpr } +func (d DownstreamLogSelectorExpr) String() string { + return fmt.Sprintf("downstream<%s, shard=%s>", d.LogSelectorExpr.String(), d.shard) +} + // ConcatSampleExpr is an expr for concatenating multiple SampleExpr +// Contract: The embedded SampleExprs within a linked list of ConcatSampleExprs must be of the +// same structure. This makes special implementations of SampleExpr.Associative() unnecessary. type ConcatSampleExpr struct { SampleExpr next *ConcatSampleExpr } +func (c ConcatSampleExpr) String() string { + if c.next == nil { + return c.SampleExpr.String() + } + + return fmt.Sprintf("%s ++ %s", c.SampleExpr.String(), c.next.String()) +} + // ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr type ConcatLogSelectorExpr struct { LogSelectorExpr next *ConcatLogSelectorExpr } +func (c ConcatLogSelectorExpr) String() string { + if c.next == nil { + return c.LogSelectorExpr.String() + } + + return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) +} + // downstreamEvaluator is an evaluator which handles shard aware AST nodes type downstreamEvaluator struct{ Downstreamer } diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index dbfbd483dcf95..79303365a96d0 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -4,6 +4,8 @@ import ( "fmt" "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/cortexproject/cortex/pkg/util" + "github.com/go-kit/kit/log/level" ) func NewShardMapper(shards int) (ShardMapper, error) { @@ -32,7 +34,7 @@ func (m ShardMapper) Map(expr Expr) (Expr, error) { if err != nil { return nil, err } - rhsMapped, err := m.Map(e.SampleExpr) + rhsMapped, err := m.Map(e.RHS) if err != nil { return nil, err } @@ -91,13 +93,31 @@ func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { // technically, std{dev,var} are also parallelizable if there is no cross-shard merging // in descendent nodes in the AST. This optimization is currently avoided for simplicity. func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) { + + // if this AST contains unshardable operations, don't shard this at this level, + // but attempt to shard a child node. + if shardable := isShardable(expr.Operations()); !shardable { + subMapped, err := m.Map(expr.left) + if err != nil { + return nil, err + } + sampleExpr, ok := subMapped.(SampleExpr) + if !ok { + return nil, badASTMapping("SampleExpr", subMapped) + } + + return &vectorAggregationExpr{ + left: sampleExpr, + grouping: expr.grouping, + params: expr.params, + operation: expr.operation, + }, nil + + } + switch expr.operation { - // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) - // max(x) -> max(max(x, shard=1) ++ max(x, shard=2)...) - // min(x) -> min(min(x, shard=1) ++ min(x, shard=2)...) - // topk(x) -> topk(topk(x, shard=1) ++ topk(x, shard=2)...) - // botk(x) -> botk(botk(x, shard=1) ++ botk(x, shard=2)...) - case OpTypeSum, OpTypeMax, OpTypeMin, OpTypeTopK, OpTypeBottomK: + case OpTypeSum: + // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) return &vectorAggregationExpr{ left: m.mapSampleExpr(expr), grouping: expr.grouping, @@ -139,6 +159,12 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp operation: OpTypeSum, }, nil default: + // this should not be reachable. If an operation is shardable it should + // have an optimization listed. + level.Warn(util.Logger).Log( + "msg", "unexpected operation which appears shardable, ignoring", + "operation", expr.operation, + ) return expr, nil } } @@ -153,3 +179,46 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleE return expr } } + +// isShardable returns false if any of the listed operation types are not shardable and true otherwise +func isShardable(ops []string) bool { + for _, op := range ops { + if shardable := shardableOps[op]; !shardable { + return false + } + } + return true +} + +// shardableOps lists the operations which may be sharded. +// topk, botk, max, & min all must be concatenated and then evaluated in order to avoid +// potential data loss due to series distribution across shards. +// For example, grouping by `cluster` for a `max` operation may yield +// 2 results on the first shard and 10 results on the second. If we prematurely +// calculated `max`s on each shard, the shard/label combination with `2` may be +// discarded and some other combination with `11` may be reported falsely as the max. +// +// Explanation: this is my (owen-d) best understanding. +// +// For an operation to be shardable, first the sample-operation itself must be associative like (+, *) but not (%, /, ^). +// Secondly, if the operation is part of a vector aggregation expression or utilizes logical/set binary ops, +// the vector operation must be distributive over the sample-operation. +// This ensures that the vector merging operation can be applied repeatedly to data in different shards. +// references: +// https://en.wikipedia.org/wiki/Associative_property +// https://en.wikipedia.org/wiki/Distributive_property +var shardableOps = map[string]bool{ + // vector ops + OpTypeSum: true, + // avg is only marked as shardable because we remap it into sum/count. + OpTypeAvg: true, + OpTypeCount: true, + + // range vector ops + OpTypeCountOverTime: true, + OpTypeRate: true, + + // binops - arith + OpTypeAdd: true, + OpTypeMul: true, +} diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index ccd73f4be3a96..9929c9e929d01 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -9,6 +9,48 @@ import ( "github.com/stretchr/testify/require" ) +func TestStringer(t *testing.T) { + for _, tc := range []struct { + in Expr + out string + }{ + { + in: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + LogSelectorExpr: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + }, + next: &ConcatLogSelectorExpr{ + LogSelectorExpr: DownstreamLogSelectorExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + LogSelectorExpr: &matchersExpr{ + matchers: []*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, err := NewShardMapper(2) require.Nil(t, err) @@ -196,13 +238,661 @@ func TestMapping(t *testing.T) { }, }, }, + { + in: `count_over_time({foo="bar"}[5m])`, + expr: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeCountOverTime, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeCountOverTime, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + { + in: `sum(rate({foo="bar"}[5m]))`, + expr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + }, + { + in: `topk(3, rate({foo="bar"}[5m]))`, + expr: &vectorAggregationExpr{ + grouping: &grouping{}, + params: 3, + operation: OpTypeTopK, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + }, + { + in: `max without (env) (rate({foo="bar"}[5m]))`, + expr: &vectorAggregationExpr{ + grouping: &grouping{ + without: true, + groups: []string{"env"}, + }, + operation: OpTypeMax, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + }, + { + in: `count(rate({foo="bar"}[5m]))`, + expr: &vectorAggregationExpr{ + operation: OpTypeSum, + grouping: &grouping{}, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + }, + { + in: `avg(rate({foo="bar"}[5m]))`, + expr: &binOpExpr{ + op: OpTypeDiv, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + RHS: &vectorAggregationExpr{ + operation: OpTypeSum, + grouping: &grouping{}, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + }, + }, + { + in: `1 + sum by (cluster) (rate({foo="bar"}[5m]))`, + expr: &binOpExpr{ + op: OpTypeAdd, + SampleExpr: &literalExpr{1}, + RHS: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*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: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeSum, + left: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeMax, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*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: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeMax, + left: &vectorAggregationExpr{ + operation: OpTypeSum, + grouping: &grouping{}, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*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: &binOpExpr{ + op: OpTypeDiv, + SampleExpr: &vectorAggregationExpr{ + operation: OpTypeMax, + grouping: &grouping{}, + left: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{ + groups: []string{"cluster"}, + }, + operation: OpTypeSum, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + }, + RHS: &vectorAggregationExpr{ + operation: OpTypeSum, + grouping: &grouping{}, + left: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: &ConcatSampleExpr{ + SampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 2, + }, + SampleExpr: &vectorAggregationExpr{ + grouping: &grouping{}, + operation: OpTypeCount, + left: &rangeAggregationExpr{ + operation: OpTypeRate, + left: &logRange{ + left: &matchersExpr{ + matchers: []*labels.Matcher{ + mustNewMatcher(labels.MatchEqual, "foo", "bar"), + }, + }, + interval: 5 * time.Minute, + }, + }, + }, + }, + next: nil, + }, + }, + }, + }, + }, } { t.Run(tc.in, func(t *testing.T) { ast, err := ParseExpr(tc.in) require.Equal(t, tc.err, err) mapped, err := m.Map(ast) + require.Equal(t, tc.err, err) + require.Equal(t, tc.expr.String(), mapped.String()) require.Equal(t, tc.expr, mapped) }) } From 101ef57ea26f5e10c44f8cb5d9c354524fbba628 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 24 Mar 2020 10:01:06 -0400 Subject: [PATCH 18/67] shardmapper tests for string->string --- pkg/logql/shardmapper_test.go | 45 +++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 9929c9e929d01..f02107b7adfe0 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -119,6 +119,51 @@ func TestMapSampleExpr(t *testing.T) { } } +func TestMappingStrings(t *testing.T) { + m, err := NewShardMapper(2) + require.Nil(t, err) + for _, tc := range []struct { + in string + out string + }{ + { + in: `sum(rate({foo="bar"}[1m]))`, + out: `sum(downstream ++ downstream)`, + }, + { + in: `max(count(rate({foo="bar"}[5m]))) / 2`, + out: `max(sum(downstream ++ downstream)) / 2.000000`, + }, + { + in: `topk(3, rate({foo="bar"}[5m]))`, + out: `topk(3,downstream ++ downstream)`, + }, + { + in: `sum(max(rate({foo="bar"}[5m])))`, + out: `sum(max(downstream ++ downstream))`, + }, + { + 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 ++ downstream)`, + }, + } { + t.Run(tc.in, func(t *testing.T) { + ast, err := ParseExpr(tc.in) + require.Nil(t, err) + + mapped, err := m.Map(ast) + require.Nil(t, err) + + require.Equal(t, tc.out, mapped.String()) + + }) + } +} + func TestMapping(t *testing.T) { m, err := NewShardMapper(2) require.Nil(t, err) From 55d41b9519da9496e9471f13a5048d903ea04aaa Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 24 Mar 2020 10:22:55 -0400 Subject: [PATCH 19/67] removes sharding evaluator code --- pkg/logql/chainedevaluator.go | 53 ------------------- pkg/logql/downstreamer.go | 7 --- pkg/logql/sharding.go | 97 ----------------------------------- 3 files changed, 157 deletions(-) delete mode 100644 pkg/logql/chainedevaluator.go delete mode 100644 pkg/logql/downstreamer.go diff --git a/pkg/logql/chainedevaluator.go b/pkg/logql/chainedevaluator.go deleted file mode 100644 index 346dd6992cb4f..0000000000000 --- a/pkg/logql/chainedevaluator.go +++ /dev/null @@ -1,53 +0,0 @@ -package logql - -import ( - "context" - - "github.com/grafana/loki/pkg/iter" - "github.com/pkg/errors" -) - -// ChainedEvaluator is an evaluator which chains multiple other evaluators, -// deferring to the first successful one. -type ChainedEvaluator struct { - evaluators []Evaluator -} - -// StepEvaluator attempts the embedded evaluators until one succeeds or they all error. -func (c *ChainedEvaluator) StepEvaluator( - ctx context.Context, - nextEvaluator Evaluator, - expr SampleExpr, - p Params, -) (stepper StepEvaluator, err error) { - for _, eval := range c.evaluators { - if stepper, err = eval.StepEvaluator(ctx, nextEvaluator, expr, p); err == nil { - return stepper, nil - } - } - return nil, err -} - -// Iterator attempts the embedded evaluators until one succeeds or they all error. -func (c *ChainedEvaluator) Iterator( - ctx context.Context, - expr LogSelectorExpr, - p Params, -) (iterator iter.EntryIterator, err error) { - for _, eval := range c.evaluators { - if iterator, err = eval.Iterator(ctx, expr, p); err == nil { - return iterator, nil - } - } - return nil, err -} - -// NewChainedEvaluator constructs a ChainedEvaluator from one or more Evaluators -func NewChainedEvaluator(evals ...Evaluator) (*ChainedEvaluator, error) { - if len(evals) == 0 { - return nil, errors.New("must supply an Evaluator") - } - return &ChainedEvaluator{ - evaluators: evals, - }, nil -} diff --git a/pkg/logql/downstreamer.go b/pkg/logql/downstreamer.go deleted file mode 100644 index 96fdef69dc919..0000000000000 --- a/pkg/logql/downstreamer.go +++ /dev/null @@ -1,7 +0,0 @@ -package logql - -// Downstreamer is an interface for deferring responsibility for query execution. -// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. -type Downstreamer interface { - // Downstream(*LokiRequest) (*LokiResponse, error) -} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index d8ae72ea3e8d6..5141a160d3a36 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -1,13 +1,9 @@ package logql import ( - "context" "fmt" "github.com/cortexproject/cortex/pkg/querier/astmapper" - "github.com/grafana/loki/pkg/iter" - "github.com/pkg/errors" - "github.com/prometheus/prometheus/promql" ) // DownstreamSampleExpr is a SampleExpr which signals downstream computation @@ -59,96 +55,3 @@ func (c ConcatLogSelectorExpr) String() string { return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) } - -// downstreamEvaluator is an evaluator which handles shard aware AST nodes -type downstreamEvaluator struct{ Downstreamer } - -// Evaluator returns a StepEvaluator for a given SampleExpr -func (ev *downstreamEvaluator) StepEvaluator( - ctx context.Context, - nextEv Evaluator, - expr SampleExpr, - params Params, -) (StepEvaluator, error) { - switch e := expr.(type) { - case DownstreamSampleExpr: - // downstream to a querier - return nil, errors.New("unimplemented") - - case ConcatSampleExpr: - // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat - var xs []StepEvaluator - cur := &e - - for cur != nil { - eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) - if err != nil { - // Close previously opened StepEvaluators - for _, x := range xs { - x.Close() - } - return nil, err - } - xs = append(xs, eval) - cur = cur.next - } - - return ConcatEvaluator(xs) - - default: - return nil, EvaluatorUnsupportedType(expr, ev) - } -} - -// Iterator returns the iter.EntryIterator for a given LogSelectorExpr -func (ev *downstreamEvaluator) Iterator( - ctx context.Context, - expr LogSelectorExpr, - params Params, -) (iter.EntryIterator, error) { - switch e := expr.(type) { - case DownstreamLogSelectorExpr: - // downstream to a querier - return nil, errors.New("unimplemented") - case ConcatLogSelectorExpr: - var iters []iter.EntryIterator - cur := &e - for cur != nil { - iterator, err := ev.Iterator(ctx, e, params) - if err != nil { - // Close previously opened StepEvaluators - for _, x := range iters { - x.Close() - } - return nil, err - } - iters = append(iters, iterator) - } - return iter.NewHeapIterator(ctx, iters, params.Direction()), nil - } - return nil, errors.New("unimplemented") -} - -// ConcatEvaluator joins multiple StepEvaluators. -// Contract: They must be of identical start, end, and step values. -func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { - return newStepEvaluator( - func() (done bool, ts int64, vec promql.Vector) { - var cur promql.Vector - for _, eval := range evaluators { - done, ts, cur = eval.Next() - vec = append(vec, cur...) - } - return done, ts, vec - - }, - func() (lastErr error) { - for _, eval := range evaluators { - if err := eval.Close(); err != nil { - lastErr = err - } - } - return lastErr - }, - ) -} From 3bf5df174cb8b8c130906b82ade7929669957e9e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 24 Mar 2020 10:37:35 -0400 Subject: [PATCH 20/67] removes unused ctx arg --- pkg/logql/evaluator.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index f0e39d5238866..01b2e35f65c80 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -127,7 +127,7 @@ func (ev *defaultEvaluator) StepEvaluator( if err != nil { return nil, err } - return rangeAggEvaluator(ctx, entryIter, e, q) + return rangeAggEvaluator(entryIter, e, q) case *binOpExpr: return binOpStepEvaluator(ctx, nextEv, e, q) default: @@ -331,7 +331,6 @@ func vectorAggEvaluator( } func rangeAggEvaluator( - ctx context.Context, entryIter iter.EntryIterator, expr *rangeAggregationExpr, q Params, From 951f035e3738f2acb6fe63bc361abf49ca7da335 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 24 Mar 2020 12:15:46 -0400 Subject: [PATCH 21/67] Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. --- pkg/logql/chainedevaluator.go | 53 +++++++++++++++++++ pkg/logql/downstreamer.go | 7 +++ pkg/logql/sharding.go | 97 +++++++++++++++++++++++++++++++++++ 3 files changed, 157 insertions(+) create mode 100644 pkg/logql/chainedevaluator.go create mode 100644 pkg/logql/downstreamer.go diff --git a/pkg/logql/chainedevaluator.go b/pkg/logql/chainedevaluator.go new file mode 100644 index 0000000000000..346dd6992cb4f --- /dev/null +++ b/pkg/logql/chainedevaluator.go @@ -0,0 +1,53 @@ +package logql + +import ( + "context" + + "github.com/grafana/loki/pkg/iter" + "github.com/pkg/errors" +) + +// ChainedEvaluator is an evaluator which chains multiple other evaluators, +// deferring to the first successful one. +type ChainedEvaluator struct { + evaluators []Evaluator +} + +// StepEvaluator attempts the embedded evaluators until one succeeds or they all error. +func (c *ChainedEvaluator) StepEvaluator( + ctx context.Context, + nextEvaluator Evaluator, + expr SampleExpr, + p Params, +) (stepper StepEvaluator, err error) { + for _, eval := range c.evaluators { + if stepper, err = eval.StepEvaluator(ctx, nextEvaluator, expr, p); err == nil { + return stepper, nil + } + } + return nil, err +} + +// Iterator attempts the embedded evaluators until one succeeds or they all error. +func (c *ChainedEvaluator) Iterator( + ctx context.Context, + expr LogSelectorExpr, + p Params, +) (iterator iter.EntryIterator, err error) { + for _, eval := range c.evaluators { + if iterator, err = eval.Iterator(ctx, expr, p); err == nil { + return iterator, nil + } + } + return nil, err +} + +// NewChainedEvaluator constructs a ChainedEvaluator from one or more Evaluators +func NewChainedEvaluator(evals ...Evaluator) (*ChainedEvaluator, error) { + if len(evals) == 0 { + return nil, errors.New("must supply an Evaluator") + } + return &ChainedEvaluator{ + evaluators: evals, + }, nil +} diff --git a/pkg/logql/downstreamer.go b/pkg/logql/downstreamer.go new file mode 100644 index 0000000000000..96fdef69dc919 --- /dev/null +++ b/pkg/logql/downstreamer.go @@ -0,0 +1,7 @@ +package logql + +// Downstreamer is an interface for deferring responsibility for query execution. +// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. +type Downstreamer interface { + // Downstream(*LokiRequest) (*LokiResponse, error) +} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 5141a160d3a36..d8ae72ea3e8d6 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -1,9 +1,13 @@ package logql import ( + "context" "fmt" "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/iter" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/promql" ) // DownstreamSampleExpr is a SampleExpr which signals downstream computation @@ -55,3 +59,96 @@ func (c ConcatLogSelectorExpr) String() string { return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) } + +// downstreamEvaluator is an evaluator which handles shard aware AST nodes +type downstreamEvaluator struct{ Downstreamer } + +// Evaluator returns a StepEvaluator for a given SampleExpr +func (ev *downstreamEvaluator) StepEvaluator( + ctx context.Context, + nextEv Evaluator, + expr SampleExpr, + params Params, +) (StepEvaluator, error) { + switch e := expr.(type) { + case DownstreamSampleExpr: + // downstream to a querier + return nil, errors.New("unimplemented") + + case ConcatSampleExpr: + // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat + var xs []StepEvaluator + cur := &e + + for cur != nil { + eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) + if err != nil { + // Close previously opened StepEvaluators + for _, x := range xs { + x.Close() + } + return nil, err + } + xs = append(xs, eval) + cur = cur.next + } + + return ConcatEvaluator(xs) + + default: + return nil, EvaluatorUnsupportedType(expr, ev) + } +} + +// Iterator returns the iter.EntryIterator for a given LogSelectorExpr +func (ev *downstreamEvaluator) Iterator( + ctx context.Context, + expr LogSelectorExpr, + params Params, +) (iter.EntryIterator, error) { + switch e := expr.(type) { + case DownstreamLogSelectorExpr: + // downstream to a querier + return nil, errors.New("unimplemented") + case ConcatLogSelectorExpr: + var iters []iter.EntryIterator + cur := &e + for cur != nil { + iterator, err := ev.Iterator(ctx, e, params) + if err != nil { + // Close previously opened StepEvaluators + for _, x := range iters { + x.Close() + } + return nil, err + } + iters = append(iters, iterator) + } + return iter.NewHeapIterator(ctx, iters, params.Direction()), nil + } + return nil, errors.New("unimplemented") +} + +// ConcatEvaluator joins multiple StepEvaluators. +// Contract: They must be of identical start, end, and step values. +func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { + return newStepEvaluator( + func() (done bool, ts int64, vec promql.Vector) { + var cur promql.Vector + for _, eval := range evaluators { + done, ts, cur = eval.Next() + vec = append(vec, cur...) + } + return done, ts, vec + + }, + func() (lastErr error) { + for _, eval := range evaluators { + if err := eval.Close(); err != nil { + lastErr = err + } + } + return lastErr + }, + ) +} From 47ac2a4e6c3740728a15ab46b38f71f763797bf1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 25 Mar 2020 16:54:04 -0400 Subject: [PATCH 22/67] interfaces for downstreaming, type conversions --- pkg/logql/downstreamer.go | 7 --- pkg/logql/sharding.go | 100 ++++++++++++++++++++++++++++++++++++-- 2 files changed, 96 insertions(+), 11 deletions(-) delete mode 100644 pkg/logql/downstreamer.go diff --git a/pkg/logql/downstreamer.go b/pkg/logql/downstreamer.go deleted file mode 100644 index 96fdef69dc919..0000000000000 --- a/pkg/logql/downstreamer.go +++ /dev/null @@ -1,7 +0,0 @@ -package logql - -// Downstreamer is an interface for deferring responsibility for query execution. -// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. -type Downstreamer interface { - // Downstream(*LokiRequest) (*LokiResponse, error) -} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index d8ae72ea3e8d6..fd3f4a9a6db16 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -3,10 +3,10 @@ package logql import ( "context" "fmt" + "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/grafana/loki/pkg/iter" - "github.com/pkg/errors" "github.com/prometheus/prometheus/promql" ) @@ -60,6 +60,12 @@ func (c ConcatLogSelectorExpr) String() string { return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) } +// Downstreamer is an interface for deferring responsibility for query execution. +// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. +type Downstreamer interface { + Downstream(Expr, Params, *astmapper.ShardAnnotation) Query +} + // downstreamEvaluator is an evaluator which handles shard aware AST nodes type downstreamEvaluator struct{ Downstreamer } @@ -73,7 +79,12 @@ func (ev *downstreamEvaluator) StepEvaluator( switch e := expr.(type) { case DownstreamSampleExpr: // downstream to a querier - return nil, errors.New("unimplemented") + qry := ev.Downstream(e.SampleExpr, params, e.shard) + res, err := qry.Exec(ctx) + if err != nil { + return nil, err + } + return ResultStepEvaluator(res, params) case ConcatSampleExpr: // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat @@ -109,7 +120,13 @@ func (ev *downstreamEvaluator) Iterator( switch e := expr.(type) { case DownstreamLogSelectorExpr: // downstream to a querier - return nil, errors.New("unimplemented") + qry := ev.Downstream(e.LogSelectorExpr, params, e.shard) + res, err := qry.Exec(ctx) + if err != nil { + return nil, err + } + return ResultIterator(res, params) + case ConcatLogSelectorExpr: var iters []iter.EntryIterator cur := &e @@ -125,8 +142,10 @@ func (ev *downstreamEvaluator) Iterator( iters = append(iters, iterator) } return iter.NewHeapIterator(ctx, iters, params.Direction()), nil + + default: + return nil, EvaluatorUnsupportedType(expr, ev) } - return nil, errors.New("unimplemented") } // ConcatEvaluator joins multiple StepEvaluators. @@ -152,3 +171,76 @@ func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { }, ) } + +// ResultStepEvaluator coerces a downstream vector or matrix into a StepEvaluator +func ResultStepEvaluator(res Result, params Params) (StepEvaluator, error) { + var ( + end = params.End() + step = params.Step() + ts = params.Start() + increment = func() { + ts = ts.Add(step) + } + ) + + switch data := res.Data.(type) { + case promql.Vector: + var exhausted bool + return newStepEvaluator(func() (bool, int64, promql.Vector) { + if !exhausted { + exhausted = true + return true, ts.UnixNano() / int64(time.Millisecond), data + } + return false, 0, nil + }, nil) + case promql.Matrix: + var i int + var maxLn int + if len(data) > 0 { + maxLn = len(data[0].Points) + } + return newStepEvaluator(func() (bool, int64, promql.Vector) { + defer increment() + if ts.After(end) { + return false, 0, nil + } + + tsInt := ts.UnixNano() / int64(time.Millisecond) + + // Ensure that the resulting StepEvaluator maintains + // the same shape that the parameters expect. For example, + // it's possible that a downstream query returns matches no + // log streams and thus returns an empty matrix. + // However, we still need to ensure that it can be merged effectively + // with another leg that may match series. + // Therefore, we determine our steps from the parameters + // and not the underlying Matrix. + if i >= maxLn { + return true, tsInt, nil + } + + vec := make(promql.Vector, 0, len(data)) + for j := 0; j < len(data); j++ { + series := data[j] + vec = append(vec, promql.Sample{ + Point: series.Points[i], + Metric: series.Metric, + }) + } + i++ + return true, tsInt, vec + }, nil) + default: + return nil, fmt.Errorf("unexpected type (%s) uncoercible to StepEvaluator", data.Type()) + } +} + +// ResultIterator coerces a downstream streams result into an iter.EntryIterator +func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { + streams, ok := res.Data.(Streams) + if !ok { + return nil, fmt.Errorf("Unexpected type (%s) for ResultIterator; expected %s", res.Data.Type(), ValueTypeStreams) + } + return iter.NewStreamsIterator(context.Background(), streams, params.Direction()), nil + +} From 7498d6f68b1ac487830f684b9a1926f6d5830782 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 27 Mar 2020 14:15:39 -0400 Subject: [PATCH 23/67] sharding plumbing on frontend --- pkg/logql/engine.go | 14 +- pkg/logql/engine_test.go | 25 ++- pkg/logql/evaluator.go | 9 + pkg/logql/sharding.go | 8 +- pkg/logql/shardmapper_test.go | 4 + pkg/querier/querier.go | 5 +- pkg/querier/queryrange/queryrange.pb.go | 154 ++++++++++---- pkg/querier/queryrange/queryrange.proto | 1 + pkg/querier/queryrange/querysharding.go | 257 ++++++++++++++++++++++++ 9 files changed, 412 insertions(+), 65 deletions(-) create mode 100644 pkg/querier/queryrange/querysharding.go diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 39d2618abee40..08d5e9c5f95fd 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -79,19 +79,11 @@ type engine struct { } // NewEngine creates a new LogQL engine. -func NewEngine(opts EngineOpts, q Querier) Engine { - if q == nil { - panic("nil Querier") - } - +func NewEngine(opts EngineOpts, mkEvaluator func(EngineOpts) Evaluator) Engine { opts.applyDefault() - return &engine{ - timeout: opts.Timeout, - evaluator: &defaultEvaluator{ - querier: q, - maxLookBackPeriod: opts.MaxLookBackPeriod, - }, + timeout: opts.Timeout, + evaluator: mkEvaluator(opts), } } diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index c82245d987562..4f8a2961ba6c4 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -324,7 +324,9 @@ func TestEngine_NewInstantQuery(t *testing.T) { t.Run(fmt.Sprintf("%s %s", test.qs, test.direction), func(t *testing.T) { t.Parallel() - eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) + eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { + return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) + }) q := eng.NewInstantQuery(test.qs, test.ts, test.direction, test.limit) res, err := q.Exec(context.Background()) if err != nil { @@ -1074,7 +1076,9 @@ func TestEngine_NewRangeQuery(t *testing.T) { t.Run(fmt.Sprintf("%s %s", test.qs, test.direction), func(t *testing.T) { t.Parallel() - eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) + eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { + return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) + }) q := eng.NewRangeQuery(test.qs, test.start, test.end, test.step, test.direction, test.limit) res, err := q.Exec(context.Background()) @@ -1087,11 +1091,14 @@ func TestEngine_NewRangeQuery(t *testing.T) { } func TestEngine_Stats(t *testing.T) { - eng := NewEngine(EngineOpts{}, QuerierFunc(func(ctx context.Context, sp SelectParams) (iter.EntryIterator, error) { - st := stats.GetChunkData(ctx) - st.DecompressedBytes++ - return iter.NoopIterator, nil - })) + eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { + return NewDefaultEvaluator(QuerierFunc(func(ctx context.Context, sp SelectParams) (iter.EntryIterator, error) { + st := stats.GetChunkData(ctx) + st.DecompressedBytes++ + return iter.NoopIterator, nil + }), opts.MaxLookBackPeriod) + }) + q := eng.NewInstantQuery(`{foo="bar"}`, time.Now(), logproto.BACKWARD, 1000) r, err := q.Exec(context.Background()) require.NoError(t, err) @@ -1117,7 +1124,9 @@ var result promql.Value func benchmarkRangeQuery(testsize int64, b *testing.B) { b.ReportAllocs() - eng := NewEngine(EngineOpts{}, getLocalQuerier(testsize)) + eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { + return NewDefaultEvaluator(getLocalQuerier(testsize), opts.MaxLookBackPeriod) + }) start := time.Unix(0, 0) end := time.Unix(testsize, 0) b.ResetTimer() diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 01b2e35f65c80..ca3ed23061efc 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -86,6 +86,15 @@ type defaultEvaluator struct { querier Querier } +// NewDefaultEvaluator constructs a defaultEvaluator +func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) Evaluator { + return &defaultEvaluator{ + querier: querier, + maxLookBackPeriod: maxLookBackPeriod, + } + +} + func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) { params := SelectParams{ QueryRequest: &logproto.QueryRequest{ diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index fd3f4a9a6db16..8ab0963fd2402 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -66,11 +66,11 @@ type Downstreamer interface { Downstream(Expr, Params, *astmapper.ShardAnnotation) Query } -// downstreamEvaluator is an evaluator which handles shard aware AST nodes -type downstreamEvaluator struct{ Downstreamer } +// DownstreamEvaluator is an evaluator which handles shard aware AST nodes +type DownstreamEvaluator struct{ Downstreamer } // Evaluator returns a StepEvaluator for a given SampleExpr -func (ev *downstreamEvaluator) StepEvaluator( +func (ev *DownstreamEvaluator) StepEvaluator( ctx context.Context, nextEv Evaluator, expr SampleExpr, @@ -112,7 +112,7 @@ func (ev *downstreamEvaluator) StepEvaluator( } // Iterator returns the iter.EntryIterator for a given LogSelectorExpr -func (ev *downstreamEvaluator) Iterator( +func (ev *DownstreamEvaluator) Iterator( ctx context.Context, expr LogSelectorExpr, params Params, diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index f02107b7adfe0..edd779f4cb0ec 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -126,6 +126,10 @@ func TestMappingStrings(t *testing.T) { in string out string }{ + { + in: `{foo="bar"}`, + out: `downstream<{foo="bar"}, shard=0_of_2> ++ downstream<{foo="bar"}, shard=1_of_2>`, + }, { in: `sum(rate({foo="bar"}[1m]))`, out: `sum(downstream ++ downstream)`, diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 8b9a04a656b24..c8b35abfbfb17 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -85,7 +85,10 @@ func newQuerier(cfg Config, clientCfg client.Config, clientFactory cortex_client store: store, limits: limits, } - querier.engine = logql.NewEngine(cfg.Engine, &querier) + + querier.engine = logql.NewEngine(cfg.Engine, func(opts logql.EngineOpts) logql.Evaluator { + return logql.NewDefaultEvaluator(&querier, opts.MaxLookBackPeriod) + }) err := services.StartAndAwaitRunning(context.Background(), querier.pool) if err != nil { return nil, errors.Wrap(err, "querier pool") diff --git a/pkg/querier/queryrange/queryrange.pb.go b/pkg/querier/queryrange/queryrange.pb.go index aac3f3690df3c..07f675eca1fbd 100644 --- a/pkg/querier/queryrange/queryrange.pb.go +++ b/pkg/querier/queryrange/queryrange.pb.go @@ -39,6 +39,7 @@ type LokiRequest struct { EndTs time.Time `protobuf:"bytes,5,opt,name=endTs,proto3,stdtime" json:"endTs"` Direction logproto.Direction `protobuf:"varint,6,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"` Path string `protobuf:"bytes,7,opt,name=path,proto3" json:"path,omitempty"` + Shards []string `protobuf:"bytes,8,rep,name=shards,proto3" json:"shards"` } func (m *LokiRequest) Reset() { *m = LokiRequest{} } @@ -122,6 +123,13 @@ func (m *LokiRequest) GetPath() string { return "" } +func (m *LokiRequest) GetShards() []string { + if m != nil { + return m.Shards + } + return nil +} + type LokiResponse struct { Status string `protobuf:"bytes,1,opt,name=Status,json=status,proto3" json:"status"` Data LokiData `protobuf:"bytes,2,opt,name=Data,json=data,proto3" json:"data,omitempty"` @@ -336,47 +344,48 @@ func init() { } var fileDescriptor_51b9d53b40d11902 = []byte{ - // 642 bytes of a gzipped FileDescriptorProto + // 658 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x92, 0xcd, 0x6e, 0xd3, 0x4a, - 0x14, 0xc7, 0x3d, 0x69, 0xe2, 0x24, 0x93, 0xdb, 0xdc, 0x6a, 0x5a, 0xdd, 0x6b, 0xe5, 0x4a, 0xe3, - 0x28, 0x9b, 0x1b, 0x24, 0x70, 0x44, 0x0a, 0x12, 0x62, 0x81, 0x8a, 0x55, 0xc4, 0x86, 0x05, 0x72, - 0xf3, 0x02, 0x6e, 0x3a, 0xb8, 0xa6, 0x71, 0xc6, 0x9d, 0x39, 0x46, 0x74, 0xc7, 0x96, 0x5d, 0x1f, - 0x83, 0x47, 0xe9, 0xb2, 0xcb, 0xae, 0x02, 0x75, 0x37, 0x28, 0xab, 0x3e, 0x00, 0x0b, 0x34, 0x33, - 0x76, 0xe2, 0x22, 0x16, 0x65, 0xe3, 0x39, 0x5f, 0xff, 0x39, 0x67, 0x7e, 0xc7, 0xf8, 0xff, 0xf4, - 0x24, 0x1a, 0x9d, 0x66, 0x4c, 0xc4, 0x4c, 0xe8, 0xf3, 0x4c, 0x84, 0xf3, 0x88, 0x55, 0x4c, 0x2f, - 0x15, 0x1c, 0x38, 0xc1, 0xeb, 0x48, 0xef, 0x51, 0x14, 0xc3, 0x71, 0x76, 0xe8, 0x4d, 0x79, 0x32, - 0x8a, 0x78, 0xc4, 0x47, 0xba, 0xe4, 0x30, 0x7b, 0xa7, 0x3d, 0xed, 0x68, 0xcb, 0x48, 0x7b, 0xff, - 0xa9, 0x1e, 0x33, 0x1e, 0x99, 0x44, 0x69, 0xfc, 0x92, 0x3c, 0x9d, 0x8d, 0x24, 0x84, 0x20, 0xcd, - 0xb7, 0x48, 0xbe, 0xae, 0x34, 0x9a, 0x72, 0x01, 0xec, 0x63, 0x2a, 0xf8, 0x7b, 0x36, 0x85, 0xc2, - 0x1b, 0xdd, 0x73, 0xfa, 0x9e, 0x1b, 0x71, 0x1e, 0xcd, 0xd8, 0x7a, 0x50, 0x88, 0x13, 0x26, 0x21, - 0x4c, 0x52, 0x53, 0x30, 0x38, 0xaf, 0xe1, 0xce, 0x1b, 0x7e, 0x12, 0x07, 0xec, 0x34, 0x63, 0x12, - 0xc8, 0x0e, 0x6e, 0xe8, 0x4b, 0x1c, 0xd4, 0x47, 0xc3, 0x76, 0x60, 0x1c, 0x15, 0x9d, 0xc5, 0x49, - 0x0c, 0x4e, 0xad, 0x8f, 0x86, 0x9b, 0x81, 0x71, 0x08, 0xc1, 0x75, 0x09, 0x2c, 0x75, 0x36, 0xfa, - 0x68, 0xb8, 0x11, 0x68, 0x9b, 0xbc, 0xc0, 0x4d, 0x09, 0xa1, 0x80, 0x89, 0x74, 0xea, 0x7d, 0x34, - 0xec, 0x8c, 0x7b, 0x9e, 0x19, 0xc1, 0x2b, 0x47, 0xf0, 0x26, 0xe5, 0x08, 0x7e, 0xeb, 0x62, 0xe1, - 0x5a, 0xe7, 0x5f, 0x5d, 0x14, 0x94, 0x22, 0xf2, 0x1c, 0x37, 0xd8, 0xfc, 0x68, 0x22, 0x9d, 0xc6, - 0x1f, 0xa8, 0x8d, 0x84, 0x3c, 0xc6, 0xed, 0xa3, 0x58, 0xb0, 0x29, 0xc4, 0x7c, 0xee, 0xd8, 0x7d, - 0x34, 0xec, 0x8e, 0xb7, 0xbd, 0x15, 0xf6, 0xfd, 0x32, 0x15, 0xac, 0xab, 0xd4, 0x13, 0xd2, 0x10, - 0x8e, 0x9d, 0xa6, 0x7e, 0xad, 0xb6, 0x07, 0x3f, 0x6a, 0xf8, 0x2f, 0x83, 0x44, 0xa6, 0x7c, 0x2e, - 0x19, 0x19, 0x60, 0xfb, 0x00, 0x42, 0xc8, 0xa4, 0x81, 0xe2, 0xe3, 0xe5, 0xc2, 0xb5, 0xa5, 0x8e, - 0x04, 0xc5, 0x49, 0xf6, 0x70, 0x7d, 0x3f, 0x84, 0x50, 0x03, 0xea, 0x8c, 0x77, 0xbc, 0xca, 0x26, - 0xd4, 0x5d, 0x2a, 0xe7, 0xff, 0xa3, 0x06, 0x5e, 0x2e, 0xdc, 0xee, 0x51, 0x08, 0xe1, 0x43, 0x9e, - 0xc4, 0xc0, 0x92, 0x14, 0xce, 0x82, 0xba, 0xf2, 0xc9, 0x53, 0xdc, 0x7e, 0x25, 0x04, 0x17, 0x93, - 0xb3, 0x94, 0x69, 0xa4, 0x6d, 0xff, 0xdf, 0xe5, 0xc2, 0xdd, 0x66, 0x65, 0xb0, 0xa2, 0x68, 0xaf, - 0x82, 0xe4, 0x01, 0x6e, 0x68, 0x99, 0xc6, 0xdd, 0xf6, 0xb7, 0x97, 0x0b, 0xf7, 0x6f, 0x9d, 0xad, - 0x94, 0x37, 0x74, 0xe0, 0x2e, 0x9f, 0xc6, 0xbd, 0xf8, 0xac, 0x16, 0x6f, 0x57, 0x17, 0xef, 0xe0, - 0xe6, 0x07, 0x26, 0xa4, 0xba, 0xa6, 0xa9, 0xe3, 0xa5, 0x4b, 0x5e, 0x62, 0xac, 0x80, 0xc4, 0x12, - 0xe2, 0xa9, 0x74, 0x5a, 0x1a, 0xc6, 0xa6, 0x67, 0x7e, 0xed, 0x80, 0xc9, 0x6c, 0x06, 0x3e, 0x29, - 0x28, 0x54, 0x0a, 0x83, 0x8a, 0x3d, 0x00, 0xdc, 0x2a, 0x89, 0x11, 0x0f, 0x63, 0xa3, 0xd2, 0x50, - 0x0c, 0xfd, 0xae, 0xd2, 0x8a, 0x55, 0x34, 0xa8, 0xd8, 0xe4, 0x19, 0xb6, 0x4d, 0xbd, 0x53, 0xeb, - 0x6f, 0x0c, 0x3b, 0xe3, 0xad, 0xf5, 0xf3, 0x0e, 0x40, 0xb0, 0x30, 0xf1, 0xbb, 0x45, 0x77, 0xdb, - 0xa8, 0x82, 0xe2, 0x1c, 0x7c, 0x46, 0x78, 0x4b, 0xb5, 0x7d, 0x2b, 0x78, 0xb2, 0x5a, 0xfc, 0x1e, - 0x6e, 0x89, 0xc2, 0xd6, 0xcd, 0x3b, 0x63, 0x5a, 0x5d, 0xac, 0xaa, 0x65, 0x70, 0xcc, 0x32, 0x59, - 0x2a, 0xfc, 0xfa, 0xc5, 0xc2, 0x45, 0xc1, 0x4a, 0x45, 0x76, 0xef, 0xf0, 0xa8, 0xfd, 0x8e, 0x87, - 0x92, 0x58, 0x55, 0x02, 0xfe, 0x93, 0xcb, 0x6b, 0x6a, 0x5d, 0x5d, 0x53, 0xeb, 0xf6, 0x9a, 0xa2, - 0x4f, 0x39, 0x45, 0x5f, 0x72, 0x8a, 0x2e, 0x72, 0x8a, 0x2e, 0x73, 0x8a, 0xbe, 0xe5, 0x14, 0x7d, - 0xcf, 0xa9, 0x75, 0x9b, 0x53, 0x74, 0x7e, 0x43, 0xad, 0xcb, 0x1b, 0x6a, 0x5d, 0xdd, 0x50, 0xeb, - 0xd0, 0xd6, 0xef, 0xdc, 0xfd, 0x19, 0x00, 0x00, 0xff, 0xff, 0x64, 0x10, 0x10, 0x5f, 0xda, 0x04, + 0x14, 0xc7, 0x3d, 0xf9, 0xce, 0xe4, 0x36, 0xb7, 0x9a, 0x56, 0xf7, 0x5a, 0x41, 0x1a, 0x47, 0xd9, + 0x60, 0x24, 0x70, 0x44, 0x0a, 0x12, 0x62, 0x81, 0x8a, 0x55, 0xc4, 0x86, 0x05, 0x9a, 0xe6, 0x05, + 0xdc, 0x64, 0x70, 0x4c, 0xe3, 0x8c, 0x3b, 0x33, 0x46, 0x74, 0xc7, 0x96, 0x5d, 0x1f, 0x83, 0x07, + 0xe0, 0x21, 0xba, 0xec, 0xb2, 0xab, 0x40, 0xdd, 0x0d, 0xca, 0xaa, 0x0f, 0xc0, 0x02, 0xcd, 0x8c, + 0x9d, 0xb8, 0x88, 0x45, 0xd9, 0x78, 0xce, 0xd7, 0xdf, 0xe7, 0x9c, 0xdf, 0x0c, 0xbc, 0x9f, 0x1c, + 0x87, 0xc3, 0x93, 0x94, 0xf2, 0x88, 0x72, 0x7d, 0x9e, 0xf2, 0x60, 0x11, 0xd2, 0x92, 0xe9, 0x25, + 0x9c, 0x49, 0x86, 0xe0, 0x26, 0xd2, 0x7b, 0x14, 0x46, 0x72, 0x96, 0x1e, 0x79, 0x13, 0x16, 0x0f, + 0x43, 0x16, 0xb2, 0xa1, 0x2e, 0x39, 0x4a, 0xdf, 0x69, 0x4f, 0x3b, 0xda, 0x32, 0xd2, 0xde, 0x3d, + 0xd5, 0x63, 0xce, 0x42, 0x93, 0x28, 0x8c, 0xdf, 0x92, 0x27, 0xf3, 0xa1, 0x90, 0x81, 0x14, 0xe6, + 0x9b, 0x27, 0x5f, 0x97, 0x1a, 0x4d, 0x18, 0x97, 0xf4, 0x63, 0xc2, 0xd9, 0x7b, 0x3a, 0x91, 0xb9, + 0x37, 0xbc, 0xe3, 0xf4, 0x3d, 0x27, 0x64, 0x2c, 0x9c, 0xd3, 0xcd, 0xa0, 0x32, 0x8a, 0xa9, 0x90, + 0x41, 0x9c, 0x98, 0x82, 0xc1, 0xd7, 0x0a, 0xec, 0xbc, 0x61, 0xc7, 0x11, 0xa1, 0x27, 0x29, 0x15, + 0x12, 0xed, 0xc2, 0xba, 0xfe, 0x89, 0x0d, 0xfa, 0xc0, 0x6d, 0x13, 0xe3, 0xa8, 0xe8, 0x3c, 0x8a, + 0x23, 0x69, 0x57, 0xfa, 0xc0, 0xdd, 0x22, 0xc6, 0x41, 0x08, 0xd6, 0x84, 0xa4, 0x89, 0x5d, 0xed, + 0x03, 0xb7, 0x4a, 0xb4, 0x8d, 0x5e, 0xc0, 0xa6, 0x90, 0x01, 0x97, 0x63, 0x61, 0xd7, 0xfa, 0xc0, + 0xed, 0x8c, 0x7a, 0x9e, 0x19, 0xc1, 0x2b, 0x46, 0xf0, 0xc6, 0xc5, 0x08, 0x7e, 0xeb, 0x7c, 0xe9, + 0x58, 0x67, 0xdf, 0x1c, 0x40, 0x0a, 0x11, 0x7a, 0x0e, 0xeb, 0x74, 0x31, 0x1d, 0x0b, 0xbb, 0xfe, + 0x17, 0x6a, 0x23, 0x41, 0x8f, 0x61, 0x7b, 0x1a, 0x71, 0x3a, 0x91, 0x11, 0x5b, 0xd8, 0x8d, 0x3e, + 0x70, 0xbb, 0xa3, 0x1d, 0x6f, 0x8d, 0xfd, 0xa0, 0x48, 0x91, 0x4d, 0x95, 0x5a, 0x21, 0x09, 0xe4, + 0xcc, 0x6e, 0xea, 0x6d, 0xb5, 0x8d, 0x06, 0xb0, 0x21, 0x66, 0x01, 0x9f, 0x0a, 0xbb, 0xd5, 0xaf, + 0xba, 0x6d, 0x1f, 0xae, 0x96, 0x4e, 0x1e, 0x21, 0xf9, 0x39, 0xf8, 0x59, 0x81, 0xff, 0x18, 0x6c, + 0x22, 0x61, 0x0b, 0x41, 0x95, 0xe8, 0x50, 0x06, 0x32, 0x15, 0x06, 0x5c, 0x2e, 0xd2, 0x11, 0x92, + 0x9f, 0x68, 0x1f, 0xd6, 0x0e, 0x02, 0x19, 0x68, 0x88, 0x9d, 0xd1, 0xae, 0x57, 0xba, 0x2d, 0xf5, + 0x2f, 0x95, 0xf3, 0xff, 0x53, 0x4b, 0xad, 0x96, 0x4e, 0x77, 0x1a, 0xc8, 0xe0, 0x21, 0x8b, 0x23, + 0x49, 0xe3, 0x44, 0x9e, 0x92, 0x9a, 0xf2, 0xd1, 0x53, 0xd8, 0x7e, 0xc5, 0x39, 0xe3, 0xe3, 0xd3, + 0x84, 0x6a, 0xec, 0x6d, 0xff, 0xff, 0xd5, 0xd2, 0xd9, 0xa1, 0x45, 0xb0, 0xa4, 0x68, 0xaf, 0x83, + 0xe8, 0x01, 0xac, 0x6b, 0x99, 0xbe, 0x92, 0xb6, 0xbf, 0xb3, 0x5a, 0x3a, 0xff, 0xea, 0x6c, 0xa9, + 0xbc, 0xae, 0x03, 0xb7, 0x19, 0xd6, 0xef, 0xc4, 0x70, 0xfd, 0x38, 0x1a, 0xe5, 0xc7, 0x61, 0xc3, + 0xe6, 0x07, 0xca, 0x85, 0xfa, 0x4d, 0x53, 0xc7, 0x0b, 0x17, 0xbd, 0x84, 0x50, 0x01, 0x89, 0x84, + 0x8c, 0x26, 0x8a, 0xb1, 0x82, 0xb1, 0xe5, 0x99, 0xe7, 0x4f, 0xa8, 0x48, 0xe7, 0xd2, 0x47, 0x39, + 0x85, 0x52, 0x21, 0x29, 0xd9, 0x03, 0x09, 0x5b, 0x05, 0x31, 0xe4, 0x41, 0x68, 0x54, 0x1a, 0x8a, + 0xa1, 0xdf, 0x55, 0x5a, 0xbe, 0x8e, 0x92, 0x92, 0x8d, 0x9e, 0xc1, 0x86, 0xa9, 0xb7, 0x2b, 0xfd, + 0xaa, 0xdb, 0x19, 0x6d, 0x6f, 0xd6, 0x3b, 0x94, 0x9c, 0x06, 0xb1, 0xdf, 0xcd, 0xbb, 0x37, 0x8c, + 0x8a, 0xe4, 0xe7, 0xe0, 0x33, 0x80, 0xdb, 0xaa, 0xed, 0x5b, 0xce, 0xe2, 0xf5, 0xc5, 0xef, 0xc3, + 0x16, 0xcf, 0x6d, 0xdd, 0xbc, 0x33, 0xc2, 0xe5, 0x8b, 0x55, 0xb5, 0x54, 0xce, 0x68, 0x2a, 0x0a, + 0x85, 0x5f, 0x3b, 0x5f, 0x3a, 0x80, 0xac, 0x55, 0x68, 0xef, 0x16, 0x8f, 0xca, 0x9f, 0x78, 0x28, + 0x89, 0x55, 0x26, 0xe0, 0x3f, 0xb9, 0xb8, 0xc2, 0xd6, 0xe5, 0x15, 0xb6, 0x6e, 0xae, 0x30, 0xf8, + 0x94, 0x61, 0xf0, 0x25, 0xc3, 0xe0, 0x3c, 0xc3, 0xe0, 0x22, 0xc3, 0xe0, 0x7b, 0x86, 0xc1, 0x8f, + 0x0c, 0x5b, 0x37, 0x19, 0x06, 0x67, 0xd7, 0xd8, 0xba, 0xb8, 0xc6, 0xd6, 0xe5, 0x35, 0xb6, 0x8e, + 0x1a, 0x7a, 0xcf, 0xbd, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x52, 0x1c, 0xc6, 0xa2, 0xfe, 0x04, 0x00, 0x00, } @@ -420,6 +429,14 @@ func (this *LokiRequest) Equal(that interface{}) bool { if this.Path != that1.Path { return false } + if len(this.Shards) != len(that1.Shards) { + return false + } + for i := range this.Shards { + if this.Shards[i] != that1.Shards[i] { + return false + } + } return true } func (this *LokiResponse) Equal(that interface{}) bool { @@ -530,7 +547,7 @@ func (this *LokiRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&queryrange.LokiRequest{") s = append(s, "Query: "+fmt.Sprintf("%#v", this.Query)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") @@ -539,6 +556,7 @@ func (this *LokiRequest) GoString() string { s = append(s, "EndTs: "+fmt.Sprintf("%#v", this.EndTs)+",\n") s = append(s, "Direction: "+fmt.Sprintf("%#v", this.Direction)+",\n") s = append(s, "Path: "+fmt.Sprintf("%#v", this.Path)+",\n") + s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -655,6 +673,21 @@ func (m *LokiRequest) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintQueryrange(dAtA, i, uint64(len(m.Path))) i += copy(dAtA[i:], m.Path) } + if len(m.Shards) > 0 { + for _, s := range m.Shards { + dAtA[i] = 0x42 + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } return i, nil } @@ -833,6 +866,12 @@ func (m *LokiRequest) Size() (n int) { if l > 0 { n += 1 + l + sovQueryrange(uint64(l)) } + if len(m.Shards) > 0 { + for _, s := range m.Shards { + l = len(s) + n += 1 + l + sovQueryrange(uint64(l)) + } + } return n } @@ -929,6 +968,7 @@ func (this *LokiRequest) String() string { `EndTs:` + strings.Replace(strings.Replace(this.EndTs.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, `Direction:` + fmt.Sprintf("%v", this.Direction) + `,`, `Path:` + fmt.Sprintf("%v", this.Path) + `,`, + `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `}`, }, "") return s @@ -1196,6 +1236,38 @@ func (m *LokiRequest) Unmarshal(dAtA []byte) error { } m.Path = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQueryrange + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthQueryrange + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthQueryrange + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipQueryrange(dAtA[iNdEx:]) diff --git a/pkg/querier/queryrange/queryrange.proto b/pkg/querier/queryrange/queryrange.proto index f3a5e6f92ba72..95dd3a42a9674 100644 --- a/pkg/querier/queryrange/queryrange.proto +++ b/pkg/querier/queryrange/queryrange.proto @@ -19,6 +19,7 @@ message LokiRequest { google.protobuf.Timestamp endTs = 5 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false]; logproto.Direction direction = 6; string path = 7; + repeated string shards = 8 [(gogoproto.jsontag) = "shards"]; } message LokiResponse { diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go new file mode 100644 index 0000000000000..8b91484f66921 --- /dev/null +++ b/pkg/querier/queryrange/querysharding.go @@ -0,0 +1,257 @@ +package queryrange + +import ( + "context" + "fmt" + "time" + + "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/go-kit/kit/log" + "github.com/go-kit/kit/log/level" + "github.com/grafana/loki/pkg/loghttp" + "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/marshal" + "github.com/prometheus/prometheus/promql" +) + +var nanosecondsInMillisecond = int64(time.Millisecond / time.Nanosecond) + +// NewQueryShardMiddleware creates a middleware which downstreams queries after AST mapping and query encoding. +func NewQueryShardMiddleware( + logger log.Logger, + engine logql.Engine, + confs queryrange.ShardingConfigs, + minShardingLookback time.Duration, + metrics *queryrange.InstrumentMiddlewareMetrics, +) queryrange.Middleware { + + noshards := !hasShards(confs) + + if noshards { + level.Warn(logger).Log( + "middleware", "QueryShard", + "msg", "no configuration with shard found", + "confs", fmt.Sprintf("%+v", confs), + ) + return queryrange.PassthroughMiddleware + } + + mapperware := queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { + return newASTMapperware(confs, next, logger) + }) + + return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { + return &shardSplitter{ + MinShardingLookback: minShardingLookback, + shardingware: queryrange.MergeMiddlewares( + queryrange.InstrumentMiddleware("shardingware", metrics), + mapperware, + ).Wrap(next), + now: time.Now, + next: queryrange.InstrumentMiddleware("sharding-bypass", metrics).Wrap(next), + } + }) + +} + +func newASTMapperware(confs queryrange.ShardingConfigs, next queryrange.Handler, logger log.Logger) *astMapperware { + + return &astMapperware{ + confs: confs, + logger: log.With(logger, "middleware", "QueryShard.astMapperware"), + next: next, + engine: logql.NewEngine(logql.EngineOpts{}, func(_ logql.EngineOpts) logql.Evaluator { + return &logql.DownstreamEvaluator{nil} + }), + } +} + +type astMapperware struct { + confs queryrange.ShardingConfigs + logger log.Logger + engine logql.Engine + next queryrange.Handler +} + +func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) { + conf, err := ast.confs.GetConf(r) + // cannot shard with this timerange + if err != nil { + level.Warn(ast.logger).Log("err", err.Error(), "msg", "skipped AST mapper for request") + return ast.next.Do(ctx, r) + } + + shardMapper, err := logql.NewShardMapper(int(conf.RowShards)) + if err != nil { + return nil, err + } + + strQuery := r.GetQuery() + parsed, err := logql.ParseExpr(strQuery) + if err != nil { + return nil, err + } + + mappedQuery, err := shardMapper.Map(parsed) + if err != nil { + return nil, err + } + + req, ok := r.(*LokiRequest) + if !ok { + return nil, fmt.Errorf("expected *LokiRequest, got (%T)", r) + } + + strMappedQuery := mappedQuery.String() + level.Debug(ast.logger).Log("msg", "mapped query", "original", strQuery, "mapped", strMappedQuery) + + query := ast.engine.NewRangeQuery( + mappedQuery.String(), + req.GetStartTs(), + req.GetEndTs(), + time.Duration(r.GetStep())*time.Millisecond, + req.GetDirection(), + req.GetLimit(), + ) + + res, err := query.Exec(ctx) + if err != nil { + return nil, err + } + + value, err := marshal.NewResultValue(res.Data) + if err != nil { + return nil, err + } + + switch res.Data.Type() { + case promql.ValueTypeMatrix: + return &LokiPromResponse{ + Response: &queryrange.PrometheusResponse{ + Status: loghttp.QueryStatusSuccess, + Data: queryrange.PrometheusData{ + ResultType: loghttp.ResultTypeMatrix, + Result: toProto(value.(loghttp.Matrix)), + }, + }, + Statistics: res.Statistics, + }, nil + case logql.ValueTypeStreams: + return &LokiResponse{ + Status: loghttp.QueryStatusSuccess, + Direction: req.Direction, + Limit: req.Limit, + Version: uint32(loghttp.GetVersion(req.Path)), + Statistics: res.Statistics, + Data: LokiData{ + ResultType: loghttp.ResultTypeStream, + Result: value.(loghttp.Streams).ToProto(), + }, + }, nil + default: + return nil, fmt.Errorf("unexpected downstream response type (%T)", res.Data) + } +} + +// shardSplitter middleware will only shard appropriate requests that do not extend past the MinShardingLookback interval. +// This is used to send nonsharded requests to the ingesters in order to not overload them. +// TODO(owen-d): export in cortex so we don't duplicate code +type shardSplitter struct { + MinShardingLookback time.Duration // delimiter for splitting sharded vs non-sharded queries + shardingware queryrange.Handler // handler for sharded queries + next queryrange.Handler // handler for non-sharded queries + now func() time.Time // injectable time.Now +} + +func (splitter *shardSplitter) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) { + cutoff := splitter.now().Add(-splitter.MinShardingLookback) + sharded, nonsharded := partitionRequest(r, cutoff) + + return splitter.parallel(ctx, sharded, nonsharded) + +} + +func (splitter *shardSplitter) parallel(ctx context.Context, sharded, nonsharded queryrange.Request) (queryrange.Response, error) { + if sharded == nil { + return splitter.next.Do(ctx, nonsharded) + } + + if nonsharded == nil { + return splitter.shardingware.Do(ctx, sharded) + } + + nonshardCh := make(chan queryrange.Response, 1) + shardCh := make(chan queryrange.Response, 1) + errCh := make(chan error, 2) + + go func() { + res, err := splitter.next.Do(ctx, nonsharded) + if err != nil { + errCh <- err + return + } + nonshardCh <- res + + }() + + go func() { + res, err := splitter.shardingware.Do(ctx, sharded) + if err != nil { + errCh <- err + return + } + shardCh <- res + }() + + resps := make([]queryrange.Response, 0, 2) + for i := 0; i < 2; i++ { + select { + case r := <-nonshardCh: + resps = append(resps, r) + case r := <-shardCh: + resps = append(resps, r) + case err := <-errCh: + return nil, err + case <-ctx.Done(): + return nil, ctx.Err() + } + + } + + return lokiCodec.MergeResponse(resps...) +} + +// TODO(owen-d): export in cortex so we don't duplicate code +func hasShards(confs queryrange.ShardingConfigs) bool { + for _, conf := range confs { + if conf.RowShards > 0 { + return true + } + } + return false +} + +// partitionRequet splits a request into potentially multiple requests, one including the request's time range +// [0,t). The other will include [t,inf) +// TODO(owen-d): export in cortex so we don't duplicate code +func partitionRequest(r queryrange.Request, t time.Time) (before queryrange.Request, after queryrange.Request) { + boundary := TimeToMillis(t) + if r.GetStart() >= boundary { + return nil, r + } + + if r.GetEnd() < boundary { + return r, nil + } + + return r.WithStartEnd(r.GetStart(), boundary), r.WithStartEnd(boundary, r.GetEnd()) +} + +// TimeFromMillis is a helper to turn milliseconds -> time.Time +func TimeFromMillis(ms int64) time.Time { + return time.Unix(0, ms*nanosecondsInMillisecond) +} + +func TimeToMillis(t time.Time) int64 { + return t.UnixNano() / nanosecondsInMillisecond +} From 170d2c7b40109ddcd9e558362495d719e0d92e7e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 28 Mar 2020 10:13:27 -0400 Subject: [PATCH 24/67] type alignment in queryrange to downstream sharded queriers --- pkg/logql/engine.go | 2 +- pkg/logql/sharding.go | 22 ++++- pkg/querier/queryrange/codec.go | 10 +++ pkg/querier/queryrange/downstreamer.go | 104 ++++++++++++++++++++++++ pkg/querier/queryrange/querysharding.go | 2 +- 5 files changed, 135 insertions(+), 5 deletions(-) create mode 100644 pkg/querier/queryrange/downstreamer.go diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 08d5e9c5f95fd..b45968a3f687f 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -275,7 +275,7 @@ func PopulateMatrixFromScalar(data promql.Scalar, params LiteralParams) promql.M []promql.Point, 0, // allocate enough space for all needed entries - int(params.End().Sub(params.Start())/params.Step())+1, + int(end.Sub(start)/step)+1, ), } ) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 8ab0963fd2402..7e3a664876630 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -63,7 +63,7 @@ func (c ConcatLogSelectorExpr) String() string { // Downstreamer is an interface for deferring responsibility for query execution. // It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type Downstreamer interface { - Downstream(Expr, Params, *astmapper.ShardAnnotation) Query + Downstream(Expr, Params, []astmapper.ShardAnnotation) (Query, error) } // DownstreamEvaluator is an evaluator which handles shard aware AST nodes @@ -79,7 +79,15 @@ func (ev *DownstreamEvaluator) StepEvaluator( switch e := expr.(type) { case DownstreamSampleExpr: // downstream to a querier - qry := ev.Downstream(e.SampleExpr, params, e.shard) + var shards []astmapper.ShardAnnotation + if e.shard != nil { + shards = append(shards, *e.shard) + } + qry, err := ev.Downstream(e.SampleExpr, params, shards) + if err != nil { + return nil, err + } + res, err := qry.Exec(ctx) if err != nil { return nil, err @@ -120,7 +128,15 @@ func (ev *DownstreamEvaluator) Iterator( switch e := expr.(type) { case DownstreamLogSelectorExpr: // downstream to a querier - qry := ev.Downstream(e.LogSelectorExpr, params, e.shard) + var shards []astmapper.ShardAnnotation + if e.shard != nil { + shards = append(shards, *e.shard) + } + qry, err := ev.Downstream(e.LogSelectorExpr, params, shards) + if err != nil { + return nil, err + } + res, err := qry.Exec(ctx) if err != nil { return nil, err diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 89b0af7e9a875..8faf35322b417 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -13,6 +13,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/querier/queryrange" json "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -52,6 +53,15 @@ func (r *LokiRequest) WithQuery(query string) queryrange.Request { return &new } +func (r *LokiRequest) WithShards(shards []astmapper.ShardAnnotation) *LokiRequest { + new := *r + new.Shards = make([]string, 0, len(shards)) + for _, shard := range shards { + new.Shards = append(new.Shards, shard.String()) + } + return &new +} + func (codec) DecodeRequest(_ context.Context, r *http.Request) (queryrange.Request, error) { if err := r.ParseForm(); err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go new file mode 100644 index 0000000000000..b10a77d48556b --- /dev/null +++ b/pkg/querier/queryrange/downstreamer.go @@ -0,0 +1,104 @@ +package queryrange + +import ( + "context" + "fmt" + "time" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/grafana/loki/pkg/logql" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/promql" +) + +type DownstreamHandler struct { + next queryrange.Handler +} + +type QuerierFunc func(context.Context) (logql.Result, error) + +func (fn QuerierFunc) Exec(ctx context.Context) (logql.Result, error) { + return fn(ctx) +} + +func ParamsToLokiRequest(params logql.Params) *LokiRequest { + return &LokiRequest{ + Query: params.Query(), + Limit: params.Limit(), + Step: int64(params.Step() / time.Millisecond), + StartTs: params.Start(), + EndTs: params.End(), + Direction: params.Direction(), + Path: "/loki/api/v1/query_range", // TODO(owen-d): make this derivable + } +} + +func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shards []astmapper.ShardAnnotation) (logql.Query, error) { + req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()) + + return QuerierFunc(func(ctx context.Context) (logql.Result, error) { + res, err := h.next.Do(ctx, req) + if err != nil { + return logql.Result{}, err + } + return ResponseToResult(res) + }), nil +} + +// convert to matrix +func sampleStreamToMatrix(streams []queryrange.SampleStream) promql.Value { + xs := make(promql.Matrix, 0, len(streams)) + for _, stream := range streams { + x := promql.Series{} + x.Metric = make(labels.Labels, 0, len(stream.Labels)) + for _, l := range stream.Labels { + x.Metric = append(x.Metric, labels.Label(l)) + + } + + x.Points = make([]promql.Point, 0, len(stream.Samples)) + for _, sample := range stream.Samples { + x.Points = append(x.Points, promql.Point{ + T: sample.TimestampMs, + V: sample.Value, + }) + } + + xs = append(xs, x) + } + return xs +} + +func ResponseToResult(resp queryrange.Response) (logql.Result, error) { + switch r := resp.(type) { + case *LokiResponse: + if r.Error != "" { + return logql.Result{}, fmt.Errorf("%s: %s", r.ErrorType, r.Error) + } + + streams := make(logql.Streams, 0, len(r.Data.Result)) + + for _, stream := range r.Data.Result { + streams = append(streams, &stream) + } + + return logql.Result{ + Statistics: r.Statistics, + Data: streams, + }, nil + + case *LokiPromResponse: + if r.Response.Error != "" { + return logql.Result{}, fmt.Errorf("%s: %s", r.Response.ErrorType, r.Response.Error) + } + + return logql.Result{ + Statistics: r.Statistics, + Data: sampleStreamToMatrix(r.Response.Data.Result), + }, nil + + default: + return logql.Result{}, fmt.Errorf("cannot decode (%T)", resp) + } +} diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 8b91484f66921..be9c698f7435b 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -61,7 +61,7 @@ func newASTMapperware(confs queryrange.ShardingConfigs, next queryrange.Handler, logger: log.With(logger, "middleware", "QueryShard.astMapperware"), next: next, engine: logql.NewEngine(logql.EngineOpts{}, func(_ logql.EngineOpts) logql.Evaluator { - return &logql.DownstreamEvaluator{nil} + return &logql.DownstreamEvaluator{DownstreamHandler{next}} }), } } From 0240587ac312182f902e400f21956a57c3db8ebb Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 28 Mar 2020 13:57:29 -0400 Subject: [PATCH 25/67] downstreaming support for sharding incl storage code --- pkg/loghttp/params.go | 4 + pkg/loghttp/query.go | 3 + pkg/logproto/logproto.pb.go | 217 ++++++++++++++++-------- pkg/logproto/logproto.proto | 2 + pkg/logql/chainedevaluator.go | 3 +- pkg/logql/engine.go | 44 ++--- pkg/logql/engine_test.go | 34 +++- pkg/logql/evaluator.go | 26 +++ pkg/logql/sharding.go | 20 ++- pkg/querier/http.go | 34 +++- pkg/querier/queryrange/codec.go | 11 +- pkg/querier/queryrange/downstreamer.go | 3 +- pkg/querier/queryrange/querysharding.go | 13 +- pkg/storage/iterator.go | 17 +- pkg/storage/store.go | 23 +++ 15 files changed, 323 insertions(+), 131 deletions(-) diff --git a/pkg/loghttp/params.go b/pkg/loghttp/params.go index 7649a27804726..7d2fd19dd8eb3 100644 --- a/pkg/loghttp/params.go +++ b/pkg/loghttp/params.go @@ -44,6 +44,10 @@ func direction(r *http.Request) (logproto.Direction, error) { return parseDirection(r.Form.Get("direction"), logproto.BACKWARD) } +func shards(r *http.Request) []string { + return r.Form["shards"] +} + func bounds(r *http.Request) (time.Time, time.Time, error) { now := time.Now() start, err := parseTimestamp(r.Form.Get("start"), now.Add(-defaultSince)) diff --git a/pkg/loghttp/query.go b/pkg/loghttp/query.go index a75783e98d03c..132390f77c310 100644 --- a/pkg/loghttp/query.go +++ b/pkg/loghttp/query.go @@ -242,6 +242,7 @@ type RangeQuery struct { Query string Direction logproto.Direction Limit uint32 + Shards []string } // ParseRangeQuery parses a RangeQuery request from an http request. @@ -278,6 +279,8 @@ func ParseRangeQuery(r *http.Request) (*RangeQuery, error) { return nil, errNegativeStep } + result.Shards = shards(r) + // For safety, limit the number of returned points per timeseries. // This is sufficient for 60s resolution for a week or 1h resolution for a year. if (result.End.Sub(result.Start) / result.Step) > 11000 { diff --git a/pkg/logproto/logproto.pb.go b/pkg/logproto/logproto.pb.go index 52428bf978b03..72005993dfcca 100644 --- a/pkg/logproto/logproto.pb.go +++ b/pkg/logproto/logproto.pb.go @@ -138,6 +138,7 @@ type QueryRequest struct { Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` End time.Time `protobuf:"bytes,4,opt,name=end,proto3,stdtime" json:"end"` Direction Direction `protobuf:"varint,5,opt,name=direction,proto3,enum=logproto.Direction" json:"direction,omitempty"` + Shards []string `protobuf:"bytes,7,rep,name=shards,proto3" json:"shards,omitempty"` } func (m *QueryRequest) Reset() { *m = QueryRequest{} } @@ -207,6 +208,13 @@ func (m *QueryRequest) GetDirection() Direction { return FORWARD } +func (m *QueryRequest) GetShards() []string { + if m != nil { + return m.Shards + } + return nil +} + type QueryResponse struct { Streams []*Stream `protobuf:"bytes,1,rep,name=streams,proto3" json:"streams,omitempty"` } @@ -1086,77 +1094,79 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/logproto.proto", fileDescriptor_c28a5f14f1f4c79a) } var fileDescriptor_c28a5f14f1f4c79a = []byte{ - // 1113 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0x5f, 0x6f, 0x1b, 0x45, - 0x10, 0xbf, 0xb5, 0xcf, 0x67, 0x7b, 0xfc, 0xa7, 0xd6, 0x36, 0x4d, 0x8c, 0x0b, 0x67, 0xeb, 0x84, - 0x5a, 0xab, 0x80, 0x0d, 0xe1, 0x4f, 0xd3, 0xf0, 0x4f, 0x71, 0x4a, 0x44, 0x02, 0x52, 0xdb, 0x4b, - 0x24, 0xa4, 0x4a, 0xa8, 0xba, 0xc4, 0x6b, 0xe7, 0x14, 0xfb, 0xce, 0xbd, 0x5d, 0x23, 0xe5, 0x8d, - 0x2f, 0x80, 0xd4, 0x37, 0x1e, 0xf8, 0x02, 0x88, 0x07, 0x3e, 0x47, 0x9f, 0x50, 0x1e, 0xfb, 0x14, - 0x88, 0xf3, 0x82, 0x22, 0x21, 0xf5, 0x23, 0xa0, 0xfd, 0x73, 0xe7, 0xb5, 0x93, 0x82, 0xdc, 0x17, - 0x7b, 0x67, 0x76, 0x66, 0x77, 0xe6, 0x37, 0xbf, 0x99, 0x3d, 0xb8, 0x39, 0x3a, 0xea, 0xb7, 0x07, - 0x61, 0x7f, 0x14, 0x85, 0x2c, 0x4c, 0x16, 0x2d, 0xf1, 0x8b, 0x73, 0xb1, 0x5c, 0xab, 0xf7, 0xc3, - 0xb0, 0x3f, 0x20, 0x6d, 0x21, 0xed, 0x8f, 0x7b, 0x6d, 0xe6, 0x0f, 0x09, 0x65, 0xde, 0x70, 0x24, - 0x4d, 0x6b, 0xef, 0xf5, 0x7d, 0x76, 0x38, 0xde, 0x6f, 0x1d, 0x84, 0xc3, 0x76, 0x3f, 0xec, 0x87, - 0x53, 0x4b, 0x2e, 0xc9, 0xd3, 0xf9, 0x4a, 0x9a, 0x3b, 0x5b, 0x50, 0x78, 0x38, 0xa6, 0x87, 0x2e, - 0x79, 0x3a, 0x26, 0x94, 0xe1, 0xbb, 0x90, 0xa5, 0x2c, 0x22, 0xde, 0x90, 0x56, 0x51, 0x23, 0xdd, - 0x2c, 0xac, 0x56, 0x5a, 0x49, 0x28, 0xbb, 0x62, 0xa3, 0x53, 0xb8, 0x38, 0xad, 0xc7, 0x46, 0x6e, - 0xbc, 0x70, 0xca, 0x50, 0x94, 0xe7, 0xd0, 0x51, 0x18, 0x50, 0xe2, 0xfc, 0x83, 0xa0, 0xf8, 0x68, - 0x4c, 0xa2, 0xe3, 0xf8, 0xe4, 0x1a, 0xe4, 0x28, 0x19, 0x90, 0x03, 0x16, 0x46, 0x55, 0xd4, 0x40, - 0xcd, 0xbc, 0x9b, 0xc8, 0x78, 0x09, 0x32, 0x03, 0x7f, 0xe8, 0xb3, 0x6a, 0xaa, 0x81, 0x9a, 0x25, - 0x57, 0x0a, 0x78, 0x1d, 0x32, 0x94, 0x79, 0x11, 0xab, 0xa6, 0x1b, 0xa8, 0x59, 0x58, 0xad, 0xb5, - 0x64, 0xea, 0xad, 0x38, 0xa1, 0xd6, 0x5e, 0x9c, 0x7a, 0x27, 0xf7, 0xfc, 0xb4, 0x6e, 0x3c, 0xfb, - 0xb3, 0x8e, 0x5c, 0xe9, 0x82, 0x3f, 0x81, 0x34, 0x09, 0xba, 0x55, 0x73, 0x01, 0x4f, 0xee, 0x80, - 0x3f, 0x80, 0x7c, 0xd7, 0x8f, 0xc8, 0x01, 0xf3, 0xc3, 0xa0, 0x9a, 0x69, 0xa0, 0x66, 0x79, 0xf5, - 0xfa, 0x14, 0x81, 0xfb, 0xf1, 0x96, 0x3b, 0xb5, 0xda, 0x31, 0x73, 0x56, 0x25, 0xeb, 0x7c, 0x0a, - 0x25, 0x95, 0xae, 0x04, 0x00, 0xdf, 0xf9, 0x5f, 0x24, 0xa7, 0xe0, 0xfd, 0x8e, 0xa0, 0xf8, 0xad, - 0xb7, 0x4f, 0x06, 0x31, 0x58, 0x18, 0xcc, 0xc0, 0x1b, 0x12, 0x05, 0x94, 0x58, 0xe3, 0x65, 0xb0, - 0x7e, 0xf0, 0x06, 0x63, 0x42, 0x05, 0x4a, 0x39, 0x57, 0x49, 0x8b, 0xc2, 0x84, 0x5e, 0x1b, 0x26, - 0x94, 0xc0, 0xe4, 0xdc, 0x86, 0x92, 0x8a, 0x57, 0x65, 0x3b, 0x0d, 0x8e, 0x27, 0x9b, 0x8f, 0x83, - 0x73, 0x0e, 0xc1, 0x92, 0xc9, 0x62, 0x07, 0xac, 0x01, 0x77, 0xa1, 0x32, 0xa9, 0x0e, 0x5c, 0x9c, - 0xd6, 0x95, 0xc6, 0x55, 0xff, 0x78, 0x1d, 0xb2, 0x24, 0x60, 0x91, 0x2f, 0x72, 0xe4, 0x98, 0x5d, - 0x9b, 0x62, 0xf6, 0x55, 0xc0, 0xa2, 0xe3, 0xce, 0x35, 0x5e, 0x2e, 0x4e, 0x40, 0x65, 0xe7, 0xc6, - 0x0b, 0x27, 0x84, 0x8c, 0x30, 0xc1, 0x5f, 0x43, 0x3e, 0xe9, 0x09, 0x71, 0xd7, 0x7f, 0x67, 0x56, - 0x56, 0x27, 0xa6, 0x18, 0x15, 0xf9, 0x4d, 0x9d, 0xf1, 0x9b, 0x60, 0x0e, 0xfc, 0x80, 0x08, 0xbc, - 0xf3, 0x9d, 0xdc, 0xc5, 0x69, 0x5d, 0xc8, 0xae, 0xf8, 0x75, 0x7e, 0x46, 0x50, 0xd8, 0xf3, 0xfc, - 0xa4, 0x66, 0x4b, 0x90, 0x79, 0xca, 0x19, 0xa0, 0x8a, 0x26, 0x05, 0x4e, 0xfb, 0x2e, 0x19, 0x78, - 0xc7, 0x5b, 0x61, 0x24, 0x0a, 0x54, 0x72, 0x13, 0x79, 0x4a, 0x7b, 0xf3, 0x4a, 0xda, 0x67, 0x16, - 0xa6, 0xfd, 0x8e, 0x99, 0x4b, 0x55, 0xd2, 0xce, 0x31, 0x14, 0x65, 0x60, 0xaa, 0x38, 0x4d, 0xb0, - 0x24, 0xd3, 0x14, 0x1c, 0x97, 0x99, 0xa8, 0xf6, 0xf1, 0x97, 0x50, 0xee, 0x46, 0xe1, 0x68, 0x44, - 0xba, 0xbb, 0x8a, 0xbb, 0xb2, 0x0e, 0x2b, 0x5a, 0x0f, 0xe8, 0xfb, 0xee, 0x9c, 0xb9, 0xf3, 0x0b, - 0x82, 0xd2, 0x2e, 0x11, 0x95, 0x51, 0xb0, 0x24, 0xe9, 0xa0, 0xd7, 0xee, 0xe2, 0xd4, 0xa2, 0x5d, - 0xbc, 0x0c, 0x56, 0x3f, 0x0a, 0xc7, 0x23, 0x5a, 0x4d, 0x4b, 0x36, 0x4a, 0xc9, 0xd9, 0x81, 0x72, - 0x1c, 0x9c, 0x82, 0x66, 0x0d, 0x2c, 0x2a, 0x34, 0xaa, 0x49, 0x6b, 0x1a, 0x34, 0x42, 0xbf, 0xdd, - 0x25, 0x01, 0xf3, 0x7b, 0x3e, 0x89, 0x3a, 0x26, 0xbf, 0xc4, 0x55, 0xf6, 0xce, 0x4f, 0x08, 0x2a, - 0xf3, 0x26, 0xf8, 0x0b, 0x8d, 0xe4, 0xfc, 0xb8, 0x5b, 0xaf, 0x3e, 0xae, 0x25, 0x1a, 0x88, 0x0a, - 0xce, 0xc6, 0x0d, 0x50, 0xbb, 0x07, 0x05, 0x4d, 0x8d, 0x2b, 0x90, 0x3e, 0x22, 0x31, 0xa1, 0xf8, - 0x92, 0x53, 0x46, 0x74, 0x96, 0xe4, 0xa4, 0x2b, 0x85, 0xf5, 0xd4, 0x1a, 0xe2, 0x74, 0x2c, 0xcd, - 0xd4, 0x06, 0xaf, 0x81, 0xd9, 0x8b, 0xc2, 0xe1, 0x42, 0xc0, 0x0b, 0x0f, 0xfc, 0x11, 0xa4, 0x58, - 0xb8, 0x10, 0xec, 0x29, 0x16, 0x72, 0xd4, 0x55, 0xf2, 0x69, 0x11, 0x9c, 0x92, 0x9c, 0xdf, 0x10, - 0x5c, 0xe3, 0x3e, 0x12, 0x81, 0xcd, 0xc3, 0x71, 0x70, 0x84, 0x9b, 0x50, 0xe1, 0x37, 0x3d, 0xf1, - 0x83, 0x3e, 0xa1, 0x8c, 0x44, 0x4f, 0xfc, 0xae, 0x4a, 0xb3, 0xcc, 0xf5, 0xdb, 0x4a, 0xbd, 0xdd, - 0xc5, 0x2b, 0x90, 0x1d, 0x53, 0x69, 0x20, 0x73, 0xb6, 0xb8, 0xb8, 0xdd, 0xc5, 0xef, 0x68, 0xd7, - 0x71, 0xac, 0xb5, 0x39, 0x2d, 0x30, 0x7c, 0xe8, 0xf9, 0x51, 0x32, 0x59, 0x6e, 0x83, 0x75, 0xc0, - 0x2f, 0xa6, 0x55, 0x73, 0x7e, 0xb0, 0x88, 0x80, 0x5c, 0xb5, 0xed, 0x7c, 0x0c, 0xf9, 0xc4, 0xfb, - 0xca, 0x31, 0x7c, 0x65, 0x05, 0x9c, 0x9b, 0x90, 0x91, 0x89, 0x61, 0x30, 0xbb, 0x1e, 0xf3, 0x84, - 0x4b, 0xd1, 0x15, 0x6b, 0xa7, 0x0a, 0xcb, 0x7b, 0x91, 0x17, 0xd0, 0x1e, 0x89, 0x84, 0x51, 0x42, - 0x3f, 0xe7, 0x06, 0x5c, 0xe7, 0x9d, 0x4a, 0x22, 0xba, 0x19, 0x8e, 0x03, 0xa6, 0x7a, 0xc6, 0x79, - 0x17, 0x96, 0x66, 0xd5, 0x8a, 0xad, 0x4b, 0x90, 0x39, 0xe0, 0x0a, 0x71, 0x7a, 0xc9, 0x95, 0xc2, - 0x9d, 0x5b, 0x90, 0x4f, 0x1e, 0x26, 0x5c, 0x80, 0xec, 0xd6, 0x03, 0xf7, 0xbb, 0x0d, 0xf7, 0x7e, - 0xc5, 0xc0, 0x45, 0xc8, 0x75, 0x36, 0x36, 0xbf, 0x11, 0x12, 0x5a, 0xdd, 0x00, 0x8b, 0x3f, 0xd1, - 0x24, 0xc2, 0x77, 0xc1, 0xe4, 0x2b, 0x7c, 0x63, 0x8a, 0x82, 0xf6, 0x11, 0x50, 0x5b, 0x9e, 0x57, - 0xab, 0x68, 0x8d, 0xd5, 0x3f, 0x52, 0x90, 0xe5, 0xcf, 0x1c, 0xe7, 0xfa, 0x67, 0x90, 0x11, 0x2f, - 0x1e, 0xd6, 0xcc, 0xf5, 0x17, 0xbf, 0xb6, 0x72, 0x49, 0x1f, 0x9f, 0xf3, 0x3e, 0xe2, 0x63, 0x41, - 0xe0, 0xac, 0x7b, 0xeb, 0x4f, 0xa0, 0xee, 0x3d, 0xf3, 0xd4, 0x38, 0x06, 0xbe, 0x07, 0x26, 0x87, - 0x47, 0x0f, 0x5f, 0x1b, 0xc4, 0x7a, 0xf8, 0xfa, 0x18, 0x14, 0xd7, 0x7e, 0x0e, 0x96, 0xa4, 0x21, - 0x5e, 0x99, 0x6f, 0xcd, 0xd8, 0xbd, 0x7a, 0x79, 0x23, 0xb9, 0xf9, 0x81, 0x9c, 0xac, 0x71, 0x61, - 0xf0, 0x5b, 0xb3, 0x57, 0xcd, 0xd5, 0xb1, 0x66, 0xbf, 0x6a, 0x3b, 0x01, 0xf4, 0x7b, 0xc8, 0xc5, - 0x5c, 0xc7, 0x8f, 0xa0, 0x3c, 0x4b, 0x13, 0xfc, 0x86, 0xe6, 0x3f, 0xdb, 0x40, 0xb5, 0x86, 0xb6, - 0x75, 0x35, 0xb7, 0x8c, 0x26, 0xea, 0x3c, 0x3e, 0x39, 0xb3, 0x8d, 0x17, 0x67, 0xb6, 0xf1, 0xf2, - 0xcc, 0x46, 0x3f, 0x4e, 0x6c, 0xf4, 0xeb, 0xc4, 0x46, 0xcf, 0x27, 0x36, 0x3a, 0x99, 0xd8, 0xe8, - 0xaf, 0x89, 0x8d, 0xfe, 0x9e, 0xd8, 0xc6, 0xcb, 0x89, 0x8d, 0x9e, 0x9d, 0xdb, 0xc6, 0xc9, 0xb9, - 0x6d, 0xbc, 0x38, 0xb7, 0x8d, 0xc7, 0x6f, 0xeb, 0x9f, 0x90, 0x91, 0xd7, 0xf3, 0x02, 0xaf, 0x3d, - 0x08, 0x8f, 0xfc, 0xb6, 0xfe, 0x89, 0xba, 0x6f, 0x89, 0xbf, 0x0f, 0xff, 0x0d, 0x00, 0x00, 0xff, - 0xff, 0x89, 0x7a, 0x2c, 0xbd, 0xb9, 0x0a, 0x00, 0x00, + // 1144 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xcd, 0x6e, 0xdb, 0xc6, + 0x13, 0xe7, 0x4a, 0x14, 0x25, 0x8d, 0x3e, 0x2c, 0x6c, 0x1c, 0x9b, 0x7f, 0xe5, 0x5f, 0x4a, 0x20, + 0x8a, 0x44, 0x48, 0x5d, 0xa9, 0x75, 0x3f, 0xe2, 0xb8, 0x5f, 0xb0, 0x9c, 0x1a, 0xb5, 0x5b, 0x20, + 0x09, 0x6d, 0xa0, 0x40, 0x80, 0x22, 0xa0, 0xcd, 0xb5, 0x4c, 0x58, 0x22, 0x15, 0xee, 0xaa, 0x80, + 0x6f, 0x7d, 0x81, 0x02, 0xb9, 0xf5, 0x90, 0x17, 0x28, 0x7a, 0xe8, 0x73, 0xe4, 0x54, 0xf8, 0x98, + 0x93, 0x5b, 0xcb, 0x97, 0xc2, 0xa7, 0x3c, 0x42, 0xb1, 0x1f, 0xa4, 0x28, 0xd9, 0x69, 0xa1, 0x5c, + 0xa4, 0x9d, 0xd9, 0x99, 0xdd, 0x99, 0xdf, 0xfc, 0x66, 0x96, 0x70, 0x6b, 0x78, 0xdc, 0xeb, 0xf4, + 0xc3, 0xde, 0x30, 0x0a, 0x59, 0x98, 0x2c, 0xda, 0xe2, 0x17, 0x17, 0x62, 0xb9, 0xde, 0xe8, 0x85, + 0x61, 0xaf, 0x4f, 0x3a, 0x42, 0xda, 0x1f, 0x1d, 0x76, 0x98, 0x3f, 0x20, 0x94, 0xb9, 0x83, 0xa1, + 0x34, 0xad, 0xbf, 0xdf, 0xf3, 0xd9, 0xd1, 0x68, 0xbf, 0x7d, 0x10, 0x0e, 0x3a, 0xbd, 0xb0, 0x17, + 0x4e, 0x2c, 0xb9, 0x24, 0x4f, 0xe7, 0x2b, 0x69, 0x6e, 0x6f, 0x41, 0xe9, 0xd1, 0x88, 0x1e, 0x39, + 0xe4, 0xd9, 0x88, 0x50, 0x86, 0xef, 0x41, 0x9e, 0xb2, 0x88, 0xb8, 0x03, 0x6a, 0xa2, 0x66, 0xb6, + 0x55, 0x5a, 0xad, 0xb5, 0x93, 0x50, 0x76, 0xc5, 0x46, 0xb7, 0x74, 0x79, 0xd6, 0x88, 0x8d, 0x9c, + 0x78, 0x61, 0x57, 0xa1, 0x2c, 0xcf, 0xa1, 0xc3, 0x30, 0xa0, 0xc4, 0x7e, 0x91, 0x81, 0xf2, 0xe3, + 0x11, 0x89, 0x4e, 0xe2, 0x93, 0xeb, 0x50, 0xa0, 0xa4, 0x4f, 0x0e, 0x58, 0x18, 0x99, 0xa8, 0x89, + 0x5a, 0x45, 0x27, 0x91, 0xf1, 0x22, 0xe4, 0xfa, 0xfe, 0xc0, 0x67, 0x66, 0xa6, 0x89, 0x5a, 0x15, + 0x47, 0x0a, 0x78, 0x1d, 0x72, 0x94, 0xb9, 0x11, 0x33, 0xb3, 0x4d, 0xd4, 0x2a, 0xad, 0xd6, 0xdb, + 0x32, 0xf5, 0x76, 0x9c, 0x50, 0x7b, 0x2f, 0x4e, 0xbd, 0x5b, 0x78, 0x79, 0xd6, 0xd0, 0x9e, 0xff, + 0xd9, 0x40, 0x8e, 0x74, 0xc1, 0x9f, 0x42, 0x96, 0x04, 0x9e, 0xa9, 0xcf, 0xe1, 0xc9, 0x1d, 0xf0, + 0x87, 0x50, 0xf4, 0xfc, 0x88, 0x1c, 0x30, 0x3f, 0x0c, 0xcc, 0x5c, 0x13, 0xb5, 0xaa, 0xab, 0x37, + 0x26, 0x08, 0x3c, 0x88, 0xb7, 0x9c, 0x89, 0x15, 0x5e, 0x01, 0x83, 0x1e, 0xb9, 0x91, 0x47, 0xcd, + 0x7c, 0x33, 0xdb, 0x2a, 0x76, 0x17, 0x2f, 0xcf, 0x1a, 0x35, 0xa9, 0x59, 0x09, 0x07, 0x3e, 0x23, + 0x83, 0x21, 0x3b, 0x71, 0x94, 0xcd, 0x8e, 0x5e, 0x30, 0x6a, 0x79, 0xfb, 0x33, 0xa8, 0x28, 0x70, + 0x24, 0x5c, 0xf8, 0xee, 0x7f, 0xe2, 0x3e, 0x81, 0xfa, 0x77, 0x04, 0xe5, 0xef, 0xdc, 0x7d, 0xd2, + 0x8f, 0xa1, 0xc5, 0xa0, 0x07, 0xee, 0x80, 0x28, 0x58, 0xc5, 0x1a, 0x2f, 0x81, 0xf1, 0xa3, 0xdb, + 0x1f, 0x11, 0x2a, 0x30, 0x2d, 0x38, 0x4a, 0x9a, 0x17, 0x54, 0xf4, 0xd6, 0xa0, 0xa2, 0x04, 0x54, + 0xfb, 0x0e, 0x54, 0x54, 0xbc, 0x2a, 0xdb, 0x49, 0x70, 0x3c, 0xd9, 0x62, 0x1c, 0x9c, 0x7d, 0x04, + 0x86, 0x4c, 0x16, 0xdb, 0x60, 0xf4, 0xb9, 0x0b, 0x95, 0x49, 0x75, 0xe1, 0xf2, 0xac, 0xa1, 0x34, + 0x8e, 0xfa, 0xc7, 0xeb, 0x90, 0x27, 0x01, 0x8b, 0x7c, 0x91, 0x23, 0xc7, 0x6c, 0x61, 0x82, 0xd9, + 0xd7, 0x01, 0x8b, 0x4e, 0xba, 0x0b, 0xbc, 0xb8, 0x9c, 0xae, 0xca, 0xce, 0x89, 0x17, 0x76, 0x08, + 0x39, 0x61, 0x82, 0xbf, 0x81, 0x62, 0xd2, 0x41, 0xe2, 0xae, 0x7f, 0xcf, 0xac, 0xaa, 0x4e, 0xcc, + 0x30, 0x2a, 0xf2, 0x9b, 0x38, 0xe3, 0xff, 0x83, 0xde, 0xf7, 0x03, 0x22, 0xf0, 0x2e, 0x76, 0x0b, + 0x97, 0x67, 0x0d, 0x21, 0x3b, 0xe2, 0xd7, 0xfe, 0x05, 0x41, 0x69, 0xcf, 0xf5, 0x93, 0x9a, 0x2d, + 0x42, 0xee, 0x19, 0x67, 0x80, 0x2a, 0x9a, 0x14, 0x78, 0x93, 0x78, 0xa4, 0xef, 0x9e, 0x6c, 0x85, + 0x91, 0x28, 0x50, 0xc5, 0x49, 0xe4, 0x49, 0x93, 0xe8, 0xd7, 0x36, 0x49, 0x6e, 0xee, 0x26, 0xd9, + 0xd1, 0x0b, 0x99, 0x5a, 0xd6, 0x3e, 0x81, 0xb2, 0x0c, 0x4c, 0x15, 0xa7, 0x05, 0x86, 0x64, 0x9a, + 0x82, 0xe3, 0x2a, 0x13, 0xd5, 0x3e, 0xfe, 0x0a, 0xaa, 0x5e, 0x14, 0x0e, 0x87, 0xc4, 0xdb, 0x55, + 0xdc, 0x95, 0x75, 0x58, 0x4e, 0x75, 0x4c, 0x7a, 0xdf, 0x99, 0x31, 0xb7, 0x5f, 0x20, 0xa8, 0xec, + 0x12, 0x51, 0x19, 0x05, 0x4b, 0x92, 0x0e, 0x7a, 0xeb, 0x9e, 0xcf, 0xcc, 0xdb, 0xf3, 0x4b, 0x60, + 0xf4, 0xa2, 0x70, 0x34, 0xa4, 0x66, 0x56, 0xb2, 0x51, 0x4a, 0xf6, 0x0e, 0x54, 0xe3, 0xe0, 0x14, + 0x34, 0x6b, 0x60, 0x50, 0xa1, 0x51, 0x4d, 0x5a, 0x4f, 0x41, 0x23, 0xf4, 0xdb, 0x1e, 0x09, 0x98, + 0x7f, 0xe8, 0x93, 0xa8, 0xab, 0xf3, 0x4b, 0x1c, 0x65, 0x6f, 0xff, 0x8c, 0xa0, 0x36, 0x6b, 0x82, + 0xbf, 0x4c, 0x91, 0x9c, 0x1f, 0x77, 0xfb, 0xcd, 0xc7, 0xb5, 0x45, 0x03, 0x51, 0xc1, 0xd9, 0xb8, + 0x01, 0xea, 0xf7, 0xa1, 0x94, 0x52, 0xe3, 0x1a, 0x64, 0x8f, 0x49, 0x4c, 0x28, 0xbe, 0xe4, 0x94, + 0x11, 0x9d, 0x25, 0x39, 0xe9, 0x48, 0x61, 0x3d, 0xb3, 0x86, 0x38, 0x1d, 0x2b, 0x53, 0xb5, 0xc1, + 0x6b, 0xa0, 0x1f, 0x46, 0xe1, 0x60, 0x2e, 0xe0, 0x85, 0x07, 0xfe, 0x18, 0x32, 0x2c, 0x9c, 0x0b, + 0xf6, 0x0c, 0x0b, 0x39, 0xea, 0x2a, 0xf9, 0xac, 0x08, 0x4e, 0x49, 0xf6, 0x6f, 0x08, 0x16, 0xb8, + 0x8f, 0x44, 0x60, 0xf3, 0x68, 0x14, 0x1c, 0xe3, 0x16, 0xd4, 0xf8, 0x4d, 0x4f, 0xfd, 0xa0, 0x47, + 0x28, 0x23, 0xd1, 0x53, 0xdf, 0x53, 0x69, 0x56, 0xb9, 0x7e, 0x5b, 0xa9, 0xb7, 0x3d, 0xbc, 0x0c, + 0xf9, 0x11, 0x95, 0x06, 0x32, 0x67, 0x83, 0x8b, 0xdb, 0x1e, 0x7e, 0x2f, 0x75, 0x1d, 0xc7, 0x3a, + 0x35, 0xd5, 0x05, 0x86, 0x8f, 0x5c, 0x3f, 0x4a, 0x26, 0xcb, 0x1d, 0x30, 0x0e, 0xf8, 0xc5, 0xd4, + 0xd4, 0x67, 0x07, 0x8b, 0x08, 0xc8, 0x51, 0xdb, 0xf6, 0x27, 0x50, 0x4c, 0xbc, 0xaf, 0x1d, 0xc3, + 0xd7, 0x56, 0xc0, 0xbe, 0x05, 0x39, 0x99, 0x18, 0x06, 0xdd, 0x73, 0x99, 0x2b, 0x5c, 0xca, 0x8e, + 0x58, 0xdb, 0x26, 0x2c, 0xed, 0x45, 0x6e, 0x40, 0x0f, 0x49, 0x24, 0x8c, 0x12, 0xfa, 0xd9, 0x37, + 0xe1, 0x06, 0xef, 0x54, 0x12, 0xd1, 0xcd, 0x70, 0x14, 0x30, 0xd5, 0x33, 0xf6, 0x0a, 0x2c, 0x4e, + 0xab, 0x15, 0x5b, 0x17, 0x21, 0x77, 0xc0, 0x15, 0xe2, 0xf4, 0x8a, 0x23, 0x85, 0xbb, 0xb7, 0xa1, + 0x98, 0x3c, 0x63, 0xb8, 0x04, 0xf9, 0xad, 0x87, 0xce, 0xf7, 0x1b, 0xce, 0x83, 0x9a, 0x86, 0xcb, + 0x50, 0xe8, 0x6e, 0x6c, 0x7e, 0x2b, 0x24, 0xb4, 0xba, 0x01, 0x06, 0x7f, 0xd0, 0x49, 0x84, 0xef, + 0x81, 0xce, 0x57, 0xf8, 0xe6, 0x04, 0x85, 0xd4, 0x27, 0x43, 0x7d, 0x69, 0x56, 0xad, 0xa2, 0xd5, + 0x56, 0xff, 0xc8, 0x40, 0x9e, 0x3f, 0x73, 0x9c, 0xeb, 0x9f, 0x43, 0x4e, 0xbc, 0x78, 0x38, 0x65, + 0x9e, 0xfe, 0x3e, 0xa8, 0x2f, 0x5f, 0xd1, 0xc7, 0xe7, 0x7c, 0x80, 0xf8, 0x58, 0x10, 0x38, 0xa7, + 0xbd, 0xd3, 0x4f, 0x60, 0xda, 0x7b, 0xea, 0xa9, 0xb1, 0x35, 0x7c, 0x1f, 0x74, 0x0e, 0x4f, 0x3a, + 0xfc, 0xd4, 0x20, 0x4e, 0x87, 0x9f, 0x1e, 0x83, 0xe2, 0xda, 0x2f, 0xc0, 0x90, 0x34, 0xc4, 0xcb, + 0xb3, 0xad, 0x19, 0xbb, 0x9b, 0x57, 0x37, 0x92, 0x9b, 0x1f, 0xca, 0xc9, 0x1a, 0x17, 0x06, 0xbf, + 0x33, 0x7d, 0xd5, 0x4c, 0x1d, 0xeb, 0xd6, 0x9b, 0xb6, 0x13, 0x40, 0x7f, 0x80, 0x42, 0xcc, 0x75, + 0xfc, 0x18, 0xaa, 0xd3, 0x34, 0xc1, 0xff, 0x4b, 0xf9, 0x4f, 0x37, 0x50, 0xbd, 0x99, 0xda, 0xba, + 0x9e, 0x5b, 0x5a, 0x0b, 0x75, 0x9f, 0x9c, 0x9e, 0x5b, 0xda, 0xab, 0x73, 0x4b, 0x7b, 0x7d, 0x6e, + 0xa1, 0x9f, 0xc6, 0x16, 0xfa, 0x75, 0x6c, 0xa1, 0x97, 0x63, 0x0b, 0x9d, 0x8e, 0x2d, 0xf4, 0xd7, + 0xd8, 0x42, 0x7f, 0x8f, 0x2d, 0xed, 0xf5, 0xd8, 0x42, 0xcf, 0x2f, 0x2c, 0xed, 0xf4, 0xc2, 0xd2, + 0x5e, 0x5d, 0x58, 0xda, 0x93, 0x77, 0xd3, 0x1f, 0x9c, 0x91, 0x7b, 0xe8, 0x06, 0x6e, 0xa7, 0x1f, + 0x1e, 0xfb, 0x9d, 0xf4, 0x07, 0xed, 0xbe, 0x21, 0xfe, 0x3e, 0xfa, 0x27, 0x00, 0x00, 0xff, 0xff, + 0xae, 0x30, 0x1e, 0x07, 0xe7, 0x0a, 0x00, 0x00, } func (x Direction) String() string { @@ -1250,6 +1260,14 @@ func (this *QueryRequest) Equal(that interface{}) bool { if this.Direction != that1.Direction { return false } + if len(this.Shards) != len(that1.Shards) { + return false + } + for i := range this.Shards { + if this.Shards[i] != that1.Shards[i] { + return false + } + } return true } func (this *QueryResponse) Equal(that interface{}) bool { @@ -1783,13 +1801,14 @@ func (this *QueryRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 10) s = append(s, "&logproto.QueryRequest{") s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n") s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") s = append(s, "Start: "+fmt.Sprintf("%#v", this.Start)+",\n") s = append(s, "End: "+fmt.Sprintf("%#v", this.End)+",\n") s = append(s, "Direction: "+fmt.Sprintf("%#v", this.Direction)+",\n") + s = append(s, "Shards: "+fmt.Sprintf("%#v", this.Shards)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -2537,6 +2556,21 @@ func (m *QueryRequest) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintLogproto(dAtA, i, uint64(m.Direction)) } + if len(m.Shards) > 0 { + for _, s := range m.Shards { + dAtA[i] = 0x3a + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } return i, nil } @@ -3181,6 +3215,12 @@ func (m *QueryRequest) Size() (n int) { if m.Direction != 0 { n += 1 + sovLogproto(uint64(m.Direction)) } + if len(m.Shards) > 0 { + for _, s := range m.Shards { + l = len(s) + n += 1 + l + sovLogproto(uint64(l)) + } + } return n } @@ -3511,6 +3551,7 @@ func (this *QueryRequest) String() string { `Start:` + strings.Replace(strings.Replace(this.Start.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, `End:` + strings.Replace(strings.Replace(this.End.String(), "Timestamp", "types.Timestamp", 1), `&`, ``, 1) + `,`, `Direction:` + fmt.Sprintf("%v", this.Direction) + `,`, + `Shards:` + fmt.Sprintf("%v", this.Shards) + `,`, `}`, }, "") return s @@ -4023,6 +4064,38 @@ func (m *QueryRequest) Unmarshal(dAtA []byte) error { break } } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogproto + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthLogproto + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthLogproto + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogproto(dAtA[iNdEx:]) diff --git a/pkg/logproto/logproto.proto b/pkg/logproto/logproto.proto index 241c819471cd9..ddafb1398c1ec 100644 --- a/pkg/logproto/logproto.proto +++ b/pkg/logproto/logproto.proto @@ -37,6 +37,8 @@ message QueryRequest { google.protobuf.Timestamp end = 4 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false]; Direction direction = 5; reserved 6; + repeated string shards = 7 [(gogoproto.jsontag) = "shards,omitempty"]; + } enum Direction { diff --git a/pkg/logql/chainedevaluator.go b/pkg/logql/chainedevaluator.go index 346dd6992cb4f..9d2707bafd398 100644 --- a/pkg/logql/chainedevaluator.go +++ b/pkg/logql/chainedevaluator.go @@ -3,8 +3,9 @@ package logql import ( "context" - "github.com/grafana/loki/pkg/iter" "github.com/pkg/errors" + + "github.com/grafana/loki/pkg/iter" ) // ChainedEvaluator is an evaluator which chains multiple other evaluators, diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index b45968a3f687f..73a904b688892 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -68,8 +68,8 @@ func (opts *EngineOpts) applyDefault() { // Engine interface used to construct queries type Engine interface { - NewRangeQuery(qs string, start, end time.Time, step time.Duration, direction logproto.Direction, limit uint32) Query - NewInstantQuery(qs string, ts time.Time, direction logproto.Direction, limit uint32) Query + NewRangeQuery(Params) Query + NewInstantQuery(Params) Query } // engine is the LogQL engine. @@ -94,7 +94,7 @@ type Query interface { } type query struct { - LiteralParams + Params ng *engine } @@ -134,38 +134,18 @@ func (q *query) Exec(ctx context.Context) (Result, error) { } // NewRangeQuery creates a new LogQL range query. -func (ng *engine) NewRangeQuery( - qs string, - start, end time.Time, step time.Duration, - direction logproto.Direction, limit uint32) Query { +func (ng *engine) NewRangeQuery(params Params) Query { return &query{ - LiteralParams: LiteralParams{ - qs: qs, - start: start, - end: end, - step: step, - direction: direction, - limit: limit, - }, - ng: ng, + Params: params, + ng: ng, } } // NewInstantQuery creates a new LogQL instant query. -func (ng *engine) NewInstantQuery( - qs string, - ts time.Time, - direction logproto.Direction, limit uint32) Query { +func (ng *engine) NewInstantQuery(params Params) Query { return &query{ - LiteralParams: LiteralParams{ - qs: qs, - start: ts, - end: ts, - step: 0, - direction: direction, - limit: limit, - }, - ng: ng, + Params: params, + ng: ng, } } @@ -191,7 +171,7 @@ func (ng *engine) exec(ctx context.Context, q *query) (promql.Value, error) { return nil, err } defer helpers.LogError("closing iterator", iter.Close) - streams, err := readStreams(iter, q.limit) + streams, err := readStreams(iter, q.Limit()) return streams, err } @@ -261,11 +241,11 @@ func (ng *engine) evalLiteral(_ context.Context, expr *literalExpr, q *query) (p return s, nil } - return PopulateMatrixFromScalar(s, q.LiteralParams), nil + return PopulateMatrixFromScalar(s, q.Params), nil } -func PopulateMatrixFromScalar(data promql.Scalar, params LiteralParams) promql.Matrix { +func PopulateMatrixFromScalar(data promql.Scalar, params Params) promql.Matrix { var ( start = params.Start() end = params.End() diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index 4f8a2961ba6c4..5966d0565d2f7 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -327,7 +327,13 @@ func TestEngine_NewInstantQuery(t *testing.T) { eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) }) - q := eng.NewInstantQuery(test.qs, test.ts, test.direction, test.limit) + q := eng.NewInstantQuery(LiteralParams{ + qs: test.qs, + start: test.ts, + end: test.ts, + direction: test.direction, + limit: test.limit, + }) res, err := q.Exec(context.Background()) if err != nil { t.Fatal(err) @@ -1080,7 +1086,14 @@ func TestEngine_NewRangeQuery(t *testing.T) { return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) }) - q := eng.NewRangeQuery(test.qs, test.start, test.end, test.step, test.direction, test.limit) + q := eng.NewRangeQuery(LiteralParams{ + qs: test.qs, + start: test.start, + end: test.end, + step: test.step, + direction: test.direction, + limit: test.limit, + }) res, err := q.Exec(context.Background()) if err != nil { t.Fatal(err) @@ -1099,7 +1112,13 @@ func TestEngine_Stats(t *testing.T) { }), opts.MaxLookBackPeriod) }) - q := eng.NewInstantQuery(`{foo="bar"}`, time.Now(), logproto.BACKWARD, 1000) + q := eng.NewInstantQuery(LiteralParams{ + qs: `{foo="bar"}`, + start: time.Now(), + end: time.Now(), + direction: logproto.BACKWARD, + limit: 1000, + }) r, err := q.Exec(context.Background()) require.NoError(t, err) require.Equal(t, int64(1), r.Statistics.Store.DecompressedBytes) @@ -1157,7 +1176,14 @@ func benchmarkRangeQuery(testsize int64, b *testing.B) { {`bottomk(2,rate(({app=~"foo|bar"} |~".+bar")[1m]))`, logproto.FORWARD}, {`bottomk(3,rate(({app=~"foo|bar"} |~".+bar")[1m])) without (app)`, logproto.FORWARD}, } { - q := eng.NewRangeQuery(test.qs, start, end, 60*time.Second, test.direction, 1000) + q := eng.NewRangeQuery(LiteralParams{ + qs: test.qs, + start: start, + end: end, + step: 60 * time.Second, + direction: test.direction, + limit: 1000, + }) res, err := q.Exec(context.Background()) if err != nil { b.Fatal(err) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index ca3ed23061efc..6afe1086eca92 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -30,6 +30,26 @@ type Params interface { Step() time.Duration Limit() uint32 Direction() logproto.Direction + Shards() []string +} + +func NewLiteralParams( + qs string, + start, end time.Time, + step time.Duration, + direction logproto.Direction, + limit uint32, + shards []string, +) LiteralParams { + return LiteralParams{ + qs: qs, + start: start, + end: end, + step: step, + direction: direction, + limit: limit, + shards: shards, + } } // LiteralParams impls Params @@ -39,6 +59,7 @@ type LiteralParams struct { step time.Duration direction logproto.Direction limit uint32 + shards []string } // String impls Params @@ -59,6 +80,9 @@ func (p LiteralParams) Limit() uint32 { return p.limit } // Direction impls Params func (p LiteralParams) Direction() logproto.Direction { return p.direction } +// Shards impls Params +func (p LiteralParams) Shards() []string { return p.shards } + // GetRangeType returns whether a query is an instant query or range query func GetRangeType(q Params) QueryRangeType { if q.Start() == q.End() && q.Step() == 0 { @@ -103,6 +127,7 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, Limit: q.Limit(), Direction: q.Direction(), Selector: expr.String(), + Shards: q.Shards(), }, } @@ -131,6 +156,7 @@ func (ev *defaultEvaluator) StepEvaluator( Limit: 0, Direction: logproto.FORWARD, Selector: expr.Selector().String(), + Shards: q.Shards(), }, }) if err != nil { diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 7e3a664876630..d457be8239b45 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -6,8 +6,9 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" - "github.com/grafana/loki/pkg/iter" "github.com/prometheus/prometheus/promql" + + "github.com/grafana/loki/pkg/iter" ) // DownstreamSampleExpr is a SampleExpr which signals downstream computation @@ -260,3 +261,20 @@ func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { return iter.NewStreamsIterator(context.Background(), streams, params.Direction()), nil } + +// ParseShards parses a list of string encoded shards +func ParseShards(strs []string) ([]astmapper.ShardAnnotation, error) { + if len(strs) == 0 { + return nil, nil + } + shards := make([]astmapper.ShardAnnotation, 0, len(strs)) + + for _, str := range strs { + shard, err := astmapper.ParseShard(str) + if err != nil { + return nil, err + } + shards = append(shards, shard) + } + return shards, nil +} diff --git a/pkg/querier/http.go b/pkg/querier/http.go index 73b4a12089393..7fac8049fe736 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -50,7 +50,16 @@ func (q *Querier) RangeQueryHandler(w http.ResponseWriter, r *http.Request) { return } - query := q.engine.NewRangeQuery(request.Query, request.Start, request.End, request.Step, request.Direction, request.Limit) + params := logql.NewLiteralParams( + request.Query, + request.Start, + request.End, + request.Step, + request.Direction, + request.Limit, + request.Shards, + ) + query := q.engine.NewRangeQuery(params) result, err := query.Exec(ctx) if err != nil { writeError(err, w) @@ -80,7 +89,16 @@ func (q *Querier) InstantQueryHandler(w http.ResponseWriter, r *http.Request) { return } - query := q.engine.NewInstantQuery(request.Query, request.Ts, request.Direction, request.Limit) + params := logql.NewLiteralParams( + request.Query, + request.Ts, + request.Ts, + 0, + request.Direction, + request.Limit, + nil, + ) + query := q.engine.NewInstantQuery(params) result, err := query.Exec(ctx) if err != nil { writeError(err, w) @@ -127,7 +145,17 @@ func (q *Querier) LogQueryHandler(w http.ResponseWriter, r *http.Request) { return } - query := q.engine.NewRangeQuery(request.Query, request.Start, request.End, request.Step, request.Direction, request.Limit) + params := logql.NewLiteralParams( + request.Query, + request.Start, + request.End, + request.Step, + request.Direction, + request.Limit, + request.Shards, + ) + query := q.engine.NewRangeQuery(params) + result, err := query.Exec(ctx) if err != nil { writeError(err, w) diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index 8faf35322b417..c94f8b4b7f16b 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -77,8 +77,9 @@ func (codec) DecodeRequest(_ context.Context, r *http.Request) (queryrange.Reque StartTs: req.Start.UTC(), EndTs: req.End.UTC(), // GetStep must return milliseconds - Step: int64(req.Step) / 1e6, - Path: r.URL.Path, + Step: int64(req.Step) / 1e6, + Path: r.URL.Path, + Shards: req.Shards, }, nil } @@ -94,6 +95,9 @@ func (codec) EncodeRequest(ctx context.Context, r queryrange.Request) (*http.Req "direction": []string{lokiReq.Direction.String()}, "limit": []string{fmt.Sprintf("%d", lokiReq.Limit)}, } + if len(lokiReq.Shards) > 0 { + params["shards"] = lokiReq.Shards + } if lokiReq.Step != 0 { params["step"] = []string{fmt.Sprintf("%f", float64(lokiReq.Step)/float64(1e3))} } @@ -413,3 +417,6 @@ func (p paramsWrapper) Limit() uint32 { func (p paramsWrapper) Direction() logproto.Direction { return p.LokiRequest.Direction } +func (p paramsWrapper) Shards() []string { + return p.LokiRequest.Shards +} diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index b10a77d48556b..1906e6ecd84f2 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -7,9 +7,10 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/querier/queryrange" - "github.com/grafana/loki/pkg/logql" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql" + + "github.com/grafana/loki/pkg/logql" ) type DownstreamHandler struct { diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index be9c698f7435b..e6b6f82c0311a 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -8,10 +8,11 @@ import ( "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" + "github.com/prometheus/prometheus/promql" + "github.com/grafana/loki/pkg/loghttp" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/logql/marshal" - "github.com/prometheus/prometheus/promql" ) var nanosecondsInMillisecond = int64(time.Millisecond / time.Nanosecond) @@ -105,14 +106,8 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra strMappedQuery := mappedQuery.String() level.Debug(ast.logger).Log("msg", "mapped query", "original", strQuery, "mapped", strMappedQuery) - query := ast.engine.NewRangeQuery( - mappedQuery.String(), - req.GetStartTs(), - req.GetEndTs(), - time.Duration(r.GetStep())*time.Millisecond, - req.GetDirection(), - req.GetLimit(), - ) + params := paramsFromRequest(req.WithQuery(strMappedQuery)) + query := ast.engine.NewRangeQuery(params) res, err := query.Exec(ctx) if err != nil { diff --git a/pkg/storage/iterator.go b/pkg/storage/iterator.go index 847a4873c12cf..9c33e583bfd75 100644 --- a/pkg/storage/iterator.go +++ b/pkg/storage/iterator.go @@ -6,6 +6,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/go-kit/kit/log/level" @@ -85,14 +86,18 @@ type batchChunkIterator struct { // newBatchChunkIterator creates a new batch iterator with the given batchSize. func newBatchChunkIterator(ctx context.Context, chunks []*chunkenc.LazyChunk, batchSize int, matchers []*labels.Matcher, filter logql.LineFilter, req *logproto.QueryRequest) *batchChunkIterator { - // __name__ is not something we filter by because it's a constant in loki and only used for upstream compatibility. - // Therefore remove it - for i := range matchers { - if matchers[i].Name == labels.MetricName { - matchers = append(matchers[:i], matchers[i+1:]...) - break + // __name__ is not something we filter by because it's a constant in loki + // and only used for upstream compatibility; therefore remove it. + // The same applies to the sharding label which is injected by the cortex storage code. + for _, omit := range []string{labels.MetricName, astmapper.ShardLabel} { + for i := range matchers { + if matchers[i].Name == omit { + matchers = append(matchers[:i], matchers[i+1:]...) + break + } } } + ctx, cancel := context.WithCancel(ctx) res := &batchChunkIterator{ batchSize: batchSize, diff --git a/pkg/storage/store.go b/pkg/storage/store.go index aea3cc0b2a3d0..4418fa0ab6531 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -10,6 +10,7 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/storage" + "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/iter" @@ -80,6 +81,28 @@ func (s *store) LazyQuery(ctx context.Context, req logql.SelectParams) (iter.Ent } matchers = append(matchers, nameLabelMatcher) + if shards := req.GetShards(); shards != nil { + parsed, err := logql.ParseShards(shards) + if err != nil { + return nil, err + } + for _, s := range parsed { + shardMatcher, err := labels.NewMatcher( + labels.MatchEqual, + astmapper.ShardLabel, + s.String(), + ) + if err != nil { + return nil, err + } + matchers = append(matchers, shardMatcher) + + // TODO(owen-d): passing more than one shard will require + // a refactor to cortex to support it. We're leaving this codepath in + // preparation of that but will not pass more than one until it's supported. + break // nolint:staticcheck + } + } from, through := util.RoundToMilliseconds(req.Start, req.End) chks, fetchers, err := s.GetChunkRefs(ctx, userID, from, through, matchers...) if err != nil { From b1c8fafb7f4ad5c16924ce8ada6d7c5dc08e1ef7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 8 Apr 2020 09:35:04 -0400 Subject: [PATCH 26/67] removes chainedevaluator --- pkg/logql/chainedevaluator.go | 54 ----------------------------------- 1 file changed, 54 deletions(-) delete mode 100644 pkg/logql/chainedevaluator.go diff --git a/pkg/logql/chainedevaluator.go b/pkg/logql/chainedevaluator.go deleted file mode 100644 index 9d2707bafd398..0000000000000 --- a/pkg/logql/chainedevaluator.go +++ /dev/null @@ -1,54 +0,0 @@ -package logql - -import ( - "context" - - "github.com/pkg/errors" - - "github.com/grafana/loki/pkg/iter" -) - -// ChainedEvaluator is an evaluator which chains multiple other evaluators, -// deferring to the first successful one. -type ChainedEvaluator struct { - evaluators []Evaluator -} - -// StepEvaluator attempts the embedded evaluators until one succeeds or they all error. -func (c *ChainedEvaluator) StepEvaluator( - ctx context.Context, - nextEvaluator Evaluator, - expr SampleExpr, - p Params, -) (stepper StepEvaluator, err error) { - for _, eval := range c.evaluators { - if stepper, err = eval.StepEvaluator(ctx, nextEvaluator, expr, p); err == nil { - return stepper, nil - } - } - return nil, err -} - -// Iterator attempts the embedded evaluators until one succeeds or they all error. -func (c *ChainedEvaluator) Iterator( - ctx context.Context, - expr LogSelectorExpr, - p Params, -) (iterator iter.EntryIterator, err error) { - for _, eval := range c.evaluators { - if iterator, err = eval.Iterator(ctx, expr, p); err == nil { - return iterator, nil - } - } - return nil, err -} - -// NewChainedEvaluator constructs a ChainedEvaluator from one or more Evaluators -func NewChainedEvaluator(evals ...Evaluator) (*ChainedEvaluator, error) { - if len(evals) == 0 { - return nil, errors.New("must supply an Evaluator") - } - return &ChainedEvaluator{ - evaluators: evals, - }, nil -} From 2af43fffa9d4000562d23055648184707bf4dae3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 8 Apr 2020 16:52:05 -0400 Subject: [PATCH 27/67] comment alignment --- pkg/logql/evaluator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 6afe1086eca92..543247602a2ac 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -91,7 +91,7 @@ func GetRangeType(q Params) QueryRangeType { return RangeType } -// StepEvaluator is an interface for iterating over data at different nodes in the AST +// Evaluator is an interface for iterating over data at different nodes in the AST type Evaluator interface { // StepEvaluator returns a StepEvaluator for a given SampleExpr. It's explicitly passed another StepEvaluator// in order to enable arbitrary compuation of embedded expressions. This allows more modular & extensible // StepEvaluator implementations which can be composed. From f87553bbf088ed6b80cdf9d380750972e885a627 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 8 Apr 2020 17:50:25 -0400 Subject: [PATCH 28/67] storage shard injection --- pkg/storage/iterator_test.go | 2 +- pkg/storage/store_test.go | 63 +++++++++++++++++++++++++++++++----- pkg/storage/util_test.go | 9 ++++-- 3 files changed, 63 insertions(+), 11 deletions(-) diff --git a/pkg/storage/iterator_test.go b/pkg/storage/iterator_test.go index 2a42251b50f39..561a326d976db 100644 --- a/pkg/storage/iterator_test.go +++ b/pkg/storage/iterator_test.go @@ -545,7 +545,7 @@ func Test_newBatchChunkIterator(t *testing.T) { for name, tt := range tests { tt := tt t.Run(name, func(t *testing.T) { - it := newBatchChunkIterator(context.Background(), tt.chunks, tt.batchSize, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction)) + it := newBatchChunkIterator(context.Background(), tt.chunks, tt.batchSize, newMatchers(tt.matchers), nil, newQuery("", tt.start, tt.end, tt.direction, nil)) streams, _, err := iter.ReadBatch(it, 1000) _ = it.Close() if err != nil { diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 33cd9105c19d7..7576ea34aac9a 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -10,12 +10,14 @@ import ( "time" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" "github.com/weaveworks/common/user" "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/local" "github.com/cortexproject/cortex/pkg/chunk/storage" + "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/grafana/loki/pkg/iter" "github.com/grafana/loki/pkg/logproto" @@ -182,7 +184,7 @@ func Test_store_LazyQuery(t *testing.T) { }{ { "all", - newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []*logproto.Stream{ { Labels: "{foo=\"bar\"}", @@ -250,7 +252,7 @@ func Test_store_LazyQuery(t *testing.T) { }, { "filter regex", - newQuery("{foo=~\"ba.*\"} |~ \"1|2|3\" !~ \"2|3\"", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"ba.*\"} |~ \"1|2|3\" !~ \"2|3\"", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []*logproto.Stream{ { Labels: "{foo=\"bar\"}", @@ -274,7 +276,7 @@ func Test_store_LazyQuery(t *testing.T) { }, { "filter matcher", - newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []*logproto.Stream{ { Labels: "{foo=\"bar\"}", @@ -311,7 +313,7 @@ func Test_store_LazyQuery(t *testing.T) { }, { "filter time", - newQuery("{foo=~\"ba.*\"}", from, from.Add(time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"ba.*\"}", from, from.Add(time.Millisecond), logproto.FORWARD, nil), []*logproto.Stream{ { Labels: "{foo=\"bar\"}", @@ -368,7 +370,7 @@ func Test_store_GetSeries(t *testing.T) { }{ { "all", - newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []logproto.SeriesIdentifier{ {Labels: mustParseLabels("{foo=\"bar\"}")}, {Labels: mustParseLabels("{foo=\"bazz\"}")}, @@ -377,7 +379,7 @@ func Test_store_GetSeries(t *testing.T) { }, { "all-single-batch", - newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []logproto.SeriesIdentifier{ {Labels: mustParseLabels("{foo=\"bar\"}")}, {Labels: mustParseLabels("{foo=\"bazz\"}")}, @@ -386,7 +388,7 @@ func Test_store_GetSeries(t *testing.T) { }, { "regexp filter (post chunk fetching)", - newQuery("{foo=~\"bar.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=~\"bar.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []logproto.SeriesIdentifier{ {Labels: mustParseLabels("{foo=\"bar\"}")}, }, @@ -394,7 +396,7 @@ func Test_store_GetSeries(t *testing.T) { }, { "filter matcher", - newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD), + newQuery("{foo=\"bar\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), []logproto.SeriesIdentifier{ {Labels: mustParseLabels("{foo=\"bar\"}")}, }, @@ -420,6 +422,51 @@ func Test_store_GetSeries(t *testing.T) { } } +func Test_store_decodeReq_Matchers(t *testing.T) { + tests := []struct { + name string + req *logproto.QueryRequest + matchers []*labels.Matcher + }{ + { + "unsharded", + newQuery("{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, nil), + []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.*"), + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "logs"), + }, + }, + { + "unsharded", + newQuery( + "{foo=~\"ba.*\"}", from, from.Add(6*time.Millisecond), logproto.FORWARD, + []astmapper.ShardAnnotation{ + {Shard: 1, Of: 2}, + }, + ), + []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.*"), + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "logs"), + labels.MustNewMatcher( + labels.MatchEqual, + astmapper.ShardLabel, + astmapper.ShardAnnotation{Shard: 1, Of: 2}.String(), + ), + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + ms, _, _, _, err := decodeReq(logql.SelectParams{QueryRequest: tt.req}) + if err != nil { + t.Errorf("store.GetSeries() error = %v", err) + return + } + require.Equal(t, tt.matchers, ms) + }) + } +} + func mustParseLabels(s string) map[string]string { l, err := marshal.NewLabelSet(s) diff --git a/pkg/storage/util_test.go b/pkg/storage/util_test.go index 70849ac9d064a..69bd94f82e9f4 100644 --- a/pkg/storage/util_test.go +++ b/pkg/storage/util_test.go @@ -9,6 +9,7 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/cache" "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/davecgh/go-spew/spew" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" @@ -92,14 +93,18 @@ func newMatchers(matchers string) []*labels.Matcher { return res } -func newQuery(query string, start, end time.Time, direction logproto.Direction) *logproto.QueryRequest { - return &logproto.QueryRequest{ +func newQuery(query string, start, end time.Time, direction logproto.Direction, shards []astmapper.ShardAnnotation) *logproto.QueryRequest { + req := &logproto.QueryRequest{ Selector: query, Start: start, Limit: 1000, End: end, Direction: direction, } + for _, shard := range shards { + req.Shards = append(req.Shards, shard.String()) + } + return req } type mockChunkStore struct { From 66b217c88d45eda703d54fe841d2611aea1750a6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 8 Apr 2020 21:40:14 -0400 Subject: [PATCH 29/67] speccing out testware for sharding equivalence --- pkg/logql/sharding_test.go | 16 +++++ pkg/logql/test_utils.go | 135 +++++++++++++++++++++++++++++++++++++ pkg/storage/store.go | 8 +-- 3 files changed, 155 insertions(+), 4 deletions(-) create mode 100644 pkg/logql/sharding_test.go create mode 100644 pkg/logql/test_utils.go diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go new file mode 100644 index 0000000000000..74e93fab1b34c --- /dev/null +++ b/pkg/logql/sharding_test.go @@ -0,0 +1,16 @@ +package logql + +import "testing" + +func TestMappingEquivalence(t *testing.T) { + for _, tc := range []struct { + query string + }{} { + q := NewMockQuerier( + 16, + randomStreams(500, 200, 16, []string{"a", "b", "c", "d"}), + ) + t.Run(tc.query, func(t *testing.T) { + }) + } +} diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go new file mode 100644 index 0000000000000..931fef854cdb9 --- /dev/null +++ b/pkg/logql/test_utils.go @@ -0,0 +1,135 @@ +package logql + +import ( + "context" + "fmt" + "log" + "time" + + "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/grafana/loki/pkg/iter" + "github.com/grafana/loki/pkg/logproto" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/promql" +) + +func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier { + return MockQuerier{ + shards: shards, + streams: streams, + } +} + +// Shard aware mock querier +type MockQuerier struct { + shards int + streams []logproto.Stream +} + +func (q MockQuerier) Select(_ context.Context, req SelectParams) (iter.EntryIterator, error) { + expr, err := req.LogSelector() + if err != nil { + return nil, err + } + filter, err := expr.Filter() + if err != nil { + return nil, err + } + + matchers := expr.Matchers() + + var matched []*logproto.Stream + +outer: + for _, stream := range q.streams { + ls := mustParseLabels(stream.Labels) + for _, matcher := range matchers { + if matcher.Name == astmapper.ShardLabel { + shard, err := astmapper.ParseShard(matcher.Value) + if err != nil { + return nil, err + } + + if !(ls.Hash()%uint64(q.shards) == uint64(shard.Shard)) { + continue outer + } + } else if !matcher.Matches(ls.Get(matcher.Name)) { + continue outer + } + } + matched = append(matched, &stream) + } + + // apply the LineFilter + filtered := make([]*logproto.Stream, 0, len(matched)) + if filter == TrueFilter { + filtered = matched + } else { + for _, s := range matched { + var entries []logproto.Entry + for _, entry := range s.Entries { + if filter.Filter([]byte(entry.Line)) { + entries = append(entries, entry) + } + } + + if len(entries) > 0 { + filtered = append(filtered, &logproto.Stream{ + Labels: s.Labels, + Entries: entries, + }) + } + } + + } + + return iter.NewTimeRangedIterator( + iter.NewStreamsIterator(context.Background(), filtered, req.Direction), + req.Start, + req.End, + ), nil +} + +// create nStreams of nEntries with labelNames each where each label value +// with the exception of the "index" label is modulo'd into a shard +func randomStreams(nStreams, nEntries, nShards int, labelNames []string) (streams []*logproto.Stream) { + for i := 0; i < nStreams; i++ { + // labels + stream := &logproto.Stream{} + ls := labels.Labels{{Name: "index", Value: fmt.Sprintf("%d", i)}} + + for _, lName := range labelNames { + // I needed a way to hash something to uint64 + // in order to get some form of random label distribution + shard := append(ls, labels.Label{ + Name: lName, + Value: fmt.Sprintf("%d", i), + }).Hash() % uint64(nShards) + + ls = append(ls, labels.Label{ + Name: lName, + Value: fmt.Sprintf("%d", shard), + }) + } + for j := 0; j < nEntries; j++ { + stream.Entries = append(stream.Entries, logproto.Entry{ + Timestamp: time.Unix(0, int64(j*int(time.Millisecond))), + Line: fmt.Sprintf("line number: %d", j), + }) + } + + stream.Labels = ls.String() + streams = append(streams, stream) + } + return streams + +} + +func mustParseLabels(s string) labels.Labels { + labels, err := promql.ParseMetric(s) + if err != nil { + log.Fatalf("Failed to parse %s", s) + } + + return labels +} diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 6f4030d6285bf..e1a95fac7d4a9 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -57,9 +57,9 @@ func NewStore(cfg Config, storeCfg chunk.StoreConfig, schemaCfg chunk.SchemaConf }, nil } -// decodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher, +// DecodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher, // and adds the "__cortex_shard__" label if this is a sharded query. -func decodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) { +func DecodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) { expr, err := req.LogSelector() if err != nil { return nil, nil, 0, 0, err @@ -135,7 +135,7 @@ func (s *store) lazyChunks(ctx context.Context, matchers []*labels.Matcher, from } func (s *store) GetSeries(ctx context.Context, req logql.SelectParams) ([]logproto.SeriesIdentifier, error) { - matchers, _, from, through, err := decodeReq(req) + matchers, _, from, through, err := DecodeReq(req) if err != nil { return nil, err } @@ -202,7 +202,7 @@ func (s *store) GetSeries(ctx context.Context, req logql.SelectParams) ([]logpro // LazyQuery returns an iterator that will query the store for more chunks while iterating instead of fetching all chunks upfront // for that request. func (s *store) LazyQuery(ctx context.Context, req logql.SelectParams) (iter.EntryIterator, error) { - matchers, filter, from, through, err := decodeReq(req) + matchers, filter, from, through, err := DecodeReq(req) if err != nil { return nil, err } From 7fffd35539ead97198056a64be443042f2bd9d33 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 9 Apr 2020 11:04:28 -0400 Subject: [PATCH 30/67] [wip] shared engine refactor --- pkg/logql/engine.go | 88 +++++++++++++++++---------------- pkg/logql/engine_test.go | 24 +++------ pkg/logql/evaluator.go | 2 + pkg/logql/sharding.go | 81 ++++++++++++++++++++++++------ pkg/logql/sharding_test.go | 77 +++++++++++++++++++++++------ pkg/logql/test_utils.go | 23 +++++++-- pkg/querier/querier.go | 4 +- pkg/querier/queryrange/codec.go | 8 +-- 8 files changed, 209 insertions(+), 98 deletions(-) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 73a904b688892..6a89560f3ae51 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -2,6 +2,7 @@ package logql import ( "context" + "errors" "sort" "time" @@ -79,11 +80,11 @@ type engine struct { } // NewEngine creates a new LogQL engine. -func NewEngine(opts EngineOpts, mkEvaluator func(EngineOpts) Evaluator) Engine { +func NewEngine(opts EngineOpts, q Querier) Engine { opts.applyDefault() return &engine{ timeout: opts.Timeout, - evaluator: mkEvaluator(opts), + evaluator: NewDefaultEvaluator(q, opts.MaxLookBackPeriod), } } @@ -94,17 +95,18 @@ type Query interface { } type query struct { - Params - - ng *engine + timeout time.Duration + params Params + parse func(string) (Expr, error) + evaluator Evaluator } -// Exec Implements `Query` +// Exec Implements `Query`. It handles instrumentation & defers to Eval. func (q *query) Exec(ctx context.Context) (Result, error) { log, ctx := spanlogger.New(ctx, "Engine.Exec") defer log.Finish() - rangeType := GetRangeType(q) + rangeType := GetRangeType(q.params) timer := prometheus.NewTimer(queryTime.WithLabelValues(string(rangeType))) defer timer.ObserveDuration() @@ -113,7 +115,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) { start := time.Now() ctx = stats.NewContext(ctx) - data, err := q.ng.exec(ctx, q) + data, err := q.Eval(ctx) statResult = stats.Snapshot(ctx, time.Since(start)) statResult.Log(level.Debug(log)) @@ -125,7 +127,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) { status = "400" } } - RecordMetrics(ctx, q, status, statResult) + RecordMetrics(ctx, q.params, status, statResult) return Result{ Data: data, @@ -133,58 +135,60 @@ func (q *query) Exec(ctx context.Context) (Result, error) { }, err } -// NewRangeQuery creates a new LogQL range query. -func (ng *engine) NewRangeQuery(params Params) Query { - return &query{ - Params: params, - ng: ng, - } -} - -// NewInstantQuery creates a new LogQL instant query. -func (ng *engine) NewInstantQuery(params Params) Query { - return &query{ - Params: params, - ng: ng, - } -} - -func (ng *engine) exec(ctx context.Context, q *query) (promql.Value, error) { - ctx, cancel := context.WithTimeout(ctx, ng.timeout) +func (q *query) Eval(ctx context.Context) (promql.Value, error) { + ctx, cancel := context.WithTimeout(ctx, q.timeout) defer cancel() - qs := q.Query() - - expr, err := ParseExpr(qs) + expr, err := q.parse(q.params.Query()) if err != nil { return nil, err } switch e := expr.(type) { case SampleExpr: - value, err := ng.evalSample(ctx, e, q) + value, err := q.evalSample(ctx, e) return value, err case LogSelectorExpr: - iter, err := ng.evaluator.Iterator(ctx, e, q) + iter, err := q.evaluator.Iterator(ctx, e, q.params) if err != nil { return nil, err } defer helpers.LogError("closing iterator", iter.Close) - streams, err := readStreams(iter, q.Limit()) + streams, err := readStreams(iter, q.params.Limit()) return streams, err + default: + return nil, errors.New("Unexpected type (%T): cannot evaluate") } +} - return nil, nil +// NewRangeQuery creates a new LogQL range query. +func (ng *engine) NewRangeQuery(params Params) Query { + return &query{ + timeout: ng.timeout, + params: params, + evaluator: ng.evaluator, + parse: ParseExpr, + } +} + +// NewInstantQuery creates a new LogQL instant query. +func (ng *engine) NewInstantQuery(params Params) Query { + return &query{ + timeout: ng.timeout, + params: params, + evaluator: ng.evaluator, + parse: ParseExpr, + } } // evalSample evaluate a sampleExpr -func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (promql.Value, error) { +func (q *query) evalSample(ctx context.Context, expr SampleExpr) (promql.Value, error) { if lit, ok := expr.(*literalExpr); ok { - return ng.evalLiteral(ctx, lit, q) + return q.evalLiteral(ctx, lit) } - stepEvaluator, err := ng.evaluator.StepEvaluator(ctx, ng.evaluator, expr, q) + stepEvaluator, err := q.evaluator.StepEvaluator(ctx, q.evaluator, expr, q.params) if err != nil { return nil, err } @@ -193,7 +197,7 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr seriesIndex := map[uint64]*promql.Series{} next, ts, vec := stepEvaluator.Next() - if GetRangeType(q) == InstantType { + if GetRangeType(q.params) == InstantType { sort.Slice(vec, func(i, j int) bool { return labels.Compare(vec[i].Metric, vec[j].Metric) < 0 }) return vec, nil } @@ -231,17 +235,17 @@ func (ng *engine) evalSample(ctx context.Context, expr SampleExpr, q *query) (pr return result, nil } -func (ng *engine) evalLiteral(_ context.Context, expr *literalExpr, q *query) (promql.Value, error) { +func (q *query) evalLiteral(_ context.Context, expr *literalExpr) (promql.Value, error) { s := promql.Scalar{ - T: q.Start().UnixNano() / int64(time.Millisecond), + T: q.params.Start().UnixNano() / int64(time.Millisecond), V: expr.value, } - if GetRangeType(q) == InstantType { + if GetRangeType(q.params) == InstantType { return s, nil } - return PopulateMatrixFromScalar(s, q.Params), nil + return PopulateMatrixFromScalar(s, q.params), nil } diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index 5966d0565d2f7..f35da48cdc73c 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -324,9 +324,7 @@ func TestEngine_NewInstantQuery(t *testing.T) { t.Run(fmt.Sprintf("%s %s", test.qs, test.direction), func(t *testing.T) { t.Parallel() - eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { - return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) - }) + eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) q := eng.NewInstantQuery(LiteralParams{ qs: test.qs, start: test.ts, @@ -1082,9 +1080,7 @@ func TestEngine_NewRangeQuery(t *testing.T) { t.Run(fmt.Sprintf("%s %s", test.qs, test.direction), func(t *testing.T) { t.Parallel() - eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { - return NewDefaultEvaluator(newQuerierRecorder(test.streams, test.params), opts.MaxLookBackPeriod) - }) + eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) q := eng.NewRangeQuery(LiteralParams{ qs: test.qs, @@ -1104,13 +1100,11 @@ func TestEngine_NewRangeQuery(t *testing.T) { } func TestEngine_Stats(t *testing.T) { - eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { - return NewDefaultEvaluator(QuerierFunc(func(ctx context.Context, sp SelectParams) (iter.EntryIterator, error) { - st := stats.GetChunkData(ctx) - st.DecompressedBytes++ - return iter.NoopIterator, nil - }), opts.MaxLookBackPeriod) - }) + eng := NewEngine(EngineOpts{}, QuerierFunc(func(ctx context.Context, sp SelectParams) (iter.EntryIterator, error) { + st := stats.GetChunkData(ctx) + st.DecompressedBytes++ + return iter.NoopIterator, nil + })) q := eng.NewInstantQuery(LiteralParams{ qs: `{foo="bar"}`, @@ -1143,9 +1137,7 @@ var result promql.Value func benchmarkRangeQuery(testsize int64, b *testing.B) { b.ReportAllocs() - eng := NewEngine(EngineOpts{}, func(opts EngineOpts) Evaluator { - return NewDefaultEvaluator(getLocalQuerier(testsize), opts.MaxLookBackPeriod) - }) + eng := NewEngine(EngineOpts{}, getLocalQuerier(testsize)) start := time.Unix(0, 0) end := time.Unix(testsize, 0) b.ResetTimer() diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 543247602a2ac..66e22bf3873f4 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -62,6 +62,8 @@ type LiteralParams struct { shards []string } +func (p LiteralParams) Copy() LiteralParams { return p } + // String impls Params func (p LiteralParams) Query() string { return p.qs } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index d457be8239b45..349f13f907cd4 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -61,10 +61,37 @@ func (c ConcatLogSelectorExpr) String() string { return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) } +type Shards []astmapper.ShardAnnotation + +func (xs Shards) Encode() (encoded []string) { + for _, shard := range xs { + encoded = append(encoded, shard.String()) + } + + return encoded +} + +// ParseShards parses a list of string encoded shards +func ParseShards(strs []string) (Shards, error) { + if len(strs) == 0 { + return nil, nil + } + shards := make([]astmapper.ShardAnnotation, 0, len(strs)) + + for _, str := range strs { + shard, err := astmapper.ParseShard(str) + if err != nil { + return nil, err + } + shards = append(shards, shard) + } + return shards, nil +} + // Downstreamer is an interface for deferring responsibility for query execution. // It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type Downstreamer interface { - Downstream(Expr, Params, []astmapper.ShardAnnotation) (Query, error) + Downstream(Expr, Params, Shards) (Query, error) } // DownstreamEvaluator is an evaluator which handles shard aware AST nodes @@ -129,7 +156,7 @@ func (ev *DownstreamEvaluator) Iterator( switch e := expr.(type) { case DownstreamLogSelectorExpr: // downstream to a querier - var shards []astmapper.ShardAnnotation + var shards Shards if e.shard != nil { shards = append(shards, *e.shard) } @@ -262,19 +289,45 @@ func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { } -// ParseShards parses a list of string encoded shards -func ParseShards(strs []string) ([]astmapper.ShardAnnotation, error) { - if len(strs) == 0 { - return nil, nil +type shardedEngine struct { + timeout time.Duration + mapper ASTMapper + evaluator Evaluator +} + +func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer) (Engine, error) { + mapper, err := NewShardMapper(shards) + if err != nil { + return nil, err } - shards := make([]astmapper.ShardAnnotation, 0, len(strs)) - for _, str := range strs { - shard, err := astmapper.ParseShard(str) - if err != nil { - return nil, err - } - shards = append(shards, shard) + return &shardedEngine{ + mapper: mapper, + evaluator: &DownstreamEvaluator{downstreamer}, + }, nil + +} + +func (ng *shardedEngine) query(p Params) Query { + return &query{ + timeout: ng.timeout, + params: p, + evaluator: ng.evaluator, + parse: func(query string) (Expr, error) { + parsed, err := ParseExpr(query) + if err != nil { + return nil, err + } + + return ng.mapper.Map(parsed) + }, } - return shards, nil +} + +func (ng *shardedEngine) NewRangeQuery(p Params) Query { + return ng.query(p) +} + +func (ng *shardedEngine) NewInstantQuery(p Params) Query { + return ng.query(p) } diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 74e93fab1b34c..52fc38f7f5e19 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -1,16 +1,65 @@ package logql -import "testing" - -func TestMappingEquivalence(t *testing.T) { - for _, tc := range []struct { - query string - }{} { - q := NewMockQuerier( - 16, - randomStreams(500, 200, 16, []string{"a", "b", "c", "d"}), - ) - t.Run(tc.query, func(t *testing.T) { - }) - } -} +// func TestMappingEquivalence(t *testing.T) { +// var ( +// shards = 16 +// nStreams = 500 +// rounds = 200 +// streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"}) +// start = time.Unix(0, 0) +// end = time.Unix(0, int64(time.Millisecond*time.Duration(rounds))) +// limit = 100 +// ) + +// for _, tc := range []struct { +// query string +// }{ +// {`{foo="bar"}`}, +// } { +// q := NewMockQuerier( +// shards, +// streams, +// ) + +// opts := EngineOpts{} +// regular := NewEngine(opts, q) +// sharded := regular +// // sharded := NewEngine(opts, func(_ EngineOpts) Evaluator { +// // return &DownstreamEvaluator{ +// // MockDownstreamer{regular}, // downstream to the regular engine +// // } +// // }) + +// shardMapper, err := NewShardMapper(int(shards)) +// require.Nil(t, err) + +// t.Run(tc.query, func(t *testing.T) { +// params := NewLiteralParams( +// tc.query, +// start, +// end, +// time.Millisecond*10, +// logproto.FORWARD, +// uint32(limit), +// nil, +// ) +// shardedParams := params.Copy() + +// parsed, err := ParseExpr(tc.query) +// require.Nil(t, err) +// shardedQuery, err := shardMapper.Map(parsed) +// require.Nil(t, err) +// shardedParams.qs = shardedQuery.String() + +// qry := regular.NewRangeQuery(params) +// shardedQry := sharded.NewRangeQuery(shardedParams) + +// res, err := qry.Exec(context.Background()) +// require.Nil(t, err) +// shardedRes, err := shardedQry.Exec(context.Background()) +// require.Nil(t, err) + +// require.Equal(t, res, shardedRes) +// }) +// } +// } diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 931fef854cdb9..68c907af4698d 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -13,7 +13,7 @@ import ( "github.com/prometheus/prometheus/promql" ) -func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier { +func NewMockQuerier(shards int, streams []*logproto.Stream) MockQuerier { return MockQuerier{ shards: shards, streams: streams, @@ -23,7 +23,7 @@ func NewMockQuerier(shards int, streams []logproto.Stream) MockQuerier { // Shard aware mock querier type MockQuerier struct { shards int - streams []logproto.Stream + streams []*logproto.Stream } func (q MockQuerier) Select(_ context.Context, req SelectParams) (iter.EntryIterator, error) { @@ -57,7 +57,7 @@ outer: continue outer } } - matched = append(matched, &stream) + matched = append(matched, stream) } // apply the LineFilter @@ -90,6 +90,23 @@ outer: ), nil } +type MockDownstreamer struct { + Engine +} + +func (d MockDownstreamer) Downstream(expr Expr, p Params, shards Shards) (Query, error) { + params := NewLiteralParams( + expr.String(), + p.Start(), + p.End(), + p.Step(), + p.Direction(), + p.Limit(), + shards.Encode(), + ) + return d.NewRangeQuery(params), nil +} + // create nStreams of nEntries with labelNames each where each label value // with the exception of the "index" label is modulo'd into a shard func randomStreams(nStreams, nEntries, nShards int, labelNames []string) (streams []*logproto.Stream) { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index f77a944f141f4..04ccabc58f795 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -85,9 +85,7 @@ func newQuerier(cfg Config, clientCfg client.Config, clientFactory cortex_client limits: limits, } - querier.engine = logql.NewEngine(cfg.Engine, func(opts logql.EngineOpts) logql.Evaluator { - return logql.NewDefaultEvaluator(&querier, opts.MaxLookBackPeriod) - }) + querier.engine = logql.NewEngine(cfg.Engine, querier) err := services.StartAndAwaitRunning(context.Background(), querier.pool) if err != nil { return nil, errors.Wrap(err, "querier pool") diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index c94f8b4b7f16b..13999df4c0fa3 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -13,7 +13,6 @@ import ( "time" "github.com/cortexproject/cortex/pkg/ingester/client" - "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/querier/queryrange" json "github.com/json-iterator/go" "github.com/opentracing/opentracing-go" @@ -53,12 +52,9 @@ func (r *LokiRequest) WithQuery(query string) queryrange.Request { return &new } -func (r *LokiRequest) WithShards(shards []astmapper.ShardAnnotation) *LokiRequest { +func (r *LokiRequest) WithShards(shards logql.Shards) *LokiRequest { new := *r - new.Shards = make([]string, 0, len(shards)) - for _, shard := range shards { - new.Shards = append(new.Shards, shard.String()) - } + new.Shards = shards.Encode() return &new } From 1dcd308163e4819fdb3befcc7694a241d067efc0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 9 Apr 2020 12:40:13 -0400 Subject: [PATCH 31/67] sorting streams, sharding eval fixes --- pkg/logql/engine.go | 9 ++- pkg/logql/sharding.go | 13 ++-- pkg/logql/sharding_test.go | 122 ++++++++++++++++++------------------- pkg/logql/test_utils.go | 2 +- 4 files changed, 76 insertions(+), 70 deletions(-) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 6a89560f3ae51..b470efd98f8d8 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -35,6 +35,12 @@ const ValueTypeStreams = "streams" // Streams is promql.Value type Streams []*logproto.Stream +func (streams Streams) Len() int { return len(streams) } +func (streams Streams) Swap(i, j int) { streams[i], streams[j] = streams[j], streams[i] } +func (streams Streams) Less(i, j int) bool { + return streams[i].Labels <= streams[j].Labels +} + // Type implements `promql.Value` func (Streams) Type() promql.ValueType { return ValueTypeStreams } @@ -288,10 +294,11 @@ func readStreams(i iter.EntryIterator, size uint32) (Streams, error) { stream.Entries = append(stream.Entries, entry) } - result := make([]*logproto.Stream, 0, len(streams)) + result := make(Streams, 0, len(streams)) for _, stream := range streams { result = append(result, stream) } + sort.Sort(result) return result, i.Error() } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 349f13f907cd4..e064b544cbf28 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -122,10 +122,10 @@ func (ev *DownstreamEvaluator) StepEvaluator( } return ResultStepEvaluator(res, params) - case ConcatSampleExpr: + case *ConcatSampleExpr: // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat var xs []StepEvaluator - cur := &e + cur := e for cur != nil { eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) @@ -171,11 +171,11 @@ func (ev *DownstreamEvaluator) Iterator( } return ResultIterator(res, params) - case ConcatLogSelectorExpr: + case *ConcatLogSelectorExpr: var iters []iter.EntryIterator - cur := &e + cur := e for cur != nil { - iterator, err := ev.Iterator(ctx, e, params) + iterator, err := ev.Iterator(ctx, cur.LogSelectorExpr, params) if err != nil { // Close previously opened StepEvaluators for _, x := range iters { @@ -184,6 +184,7 @@ func (ev *DownstreamEvaluator) Iterator( return nil, err } iters = append(iters, iterator) + cur = cur.next } return iter.NewHeapIterator(ctx, iters, params.Direction()), nil @@ -296,12 +297,14 @@ type shardedEngine struct { } func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer) (Engine, error) { + opts.applyDefault() mapper, err := NewShardMapper(shards) if err != nil { return nil, err } return &shardedEngine{ + timeout: opts.Timeout, mapper: mapper, evaluator: &DownstreamEvaluator{downstreamer}, }, nil diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 52fc38f7f5e19..f1fe4f01ce0fd 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -1,65 +1,61 @@ package logql -// func TestMappingEquivalence(t *testing.T) { -// var ( -// shards = 16 -// nStreams = 500 -// rounds = 200 -// streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"}) -// start = time.Unix(0, 0) -// end = time.Unix(0, int64(time.Millisecond*time.Duration(rounds))) -// limit = 100 -// ) - -// for _, tc := range []struct { -// query string -// }{ -// {`{foo="bar"}`}, -// } { -// q := NewMockQuerier( -// shards, -// streams, -// ) - -// opts := EngineOpts{} -// regular := NewEngine(opts, q) -// sharded := regular -// // sharded := NewEngine(opts, func(_ EngineOpts) Evaluator { -// // return &DownstreamEvaluator{ -// // MockDownstreamer{regular}, // downstream to the regular engine -// // } -// // }) - -// shardMapper, err := NewShardMapper(int(shards)) -// require.Nil(t, err) - -// t.Run(tc.query, func(t *testing.T) { -// params := NewLiteralParams( -// tc.query, -// start, -// end, -// time.Millisecond*10, -// logproto.FORWARD, -// uint32(limit), -// nil, -// ) -// shardedParams := params.Copy() - -// parsed, err := ParseExpr(tc.query) -// require.Nil(t, err) -// shardedQuery, err := shardMapper.Map(parsed) -// require.Nil(t, err) -// shardedParams.qs = shardedQuery.String() - -// qry := regular.NewRangeQuery(params) -// shardedQry := sharded.NewRangeQuery(shardedParams) - -// res, err := qry.Exec(context.Background()) -// require.Nil(t, err) -// shardedRes, err := shardedQry.Exec(context.Background()) -// require.Nil(t, err) - -// require.Equal(t, res, shardedRes) -// }) -// } -// } +import ( + "context" + "testing" + "time" + + "github.com/grafana/loki/pkg/logproto" + "github.com/stretchr/testify/require" +) + +func TestMappingEquivalence(t *testing.T) { + var ( + shards = 3 + nStreams = 10 + rounds = 20 + streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"}) + start = time.Unix(0, 0) + end = time.Unix(0, int64(time.Millisecond*time.Duration(rounds))) + limit = 100 + ) + + for _, tc := range []struct { + query string + }{ + {`{a="1"}`}, + {`{a="1"} |= "number: 10"`}, + } { + q := NewMockQuerier( + shards, + streams, + ) + + opts := EngineOpts{} + regular := NewEngine(opts, q) + sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}) + require.Nil(t, err) + + t.Run(tc.query, func(t *testing.T) { + params := NewLiteralParams( + tc.query, + start, + end, + time.Millisecond*10, + logproto.FORWARD, + uint32(limit), + nil, + ) + qry := regular.NewRangeQuery(params) + shardedQry := sharded.NewRangeQuery(params) + + res, err := qry.Exec(context.Background()) + require.Nil(t, err) + shardedRes, err := shardedQry.Exec(context.Background()) + require.Nil(t, err) + + require.Equal(t, res.Data, shardedRes.Data) + + }) + } +} diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 68c907af4698d..4a849b2a7b5e8 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -62,7 +62,7 @@ outer: // apply the LineFilter filtered := make([]*logproto.Stream, 0, len(matched)) - if filter == TrueFilter { + if filter == nil || filter == TrueFilter { filtered = matched } else { for _, s := range matched { From 90dcc1772cbea0226efcc3f8944ee9257cb252a8 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 9 Apr 2020 17:21:33 -0400 Subject: [PATCH 32/67] downstream evaluator embeds defaultevaluator --- pkg/logql/evaluator.go | 12 ++++---- pkg/logql/sharding.go | 24 ++++++++++++++-- pkg/logql/sharding_test.go | 59 ++++++++++++++++++++++++++++++++------ pkg/logql/test_utils.go | 28 +++++++++++------- 4 files changed, 96 insertions(+), 27 deletions(-) diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index 66e22bf3873f4..5c8928e580076 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -107,21 +107,21 @@ func EvaluatorUnsupportedType(expr Expr, ev Evaluator) error { return errors.Errorf("unexpected expr type (%T) for Evaluator type (%T) ", expr, ev) } -type defaultEvaluator struct { +type DefaultEvaluator struct { maxLookBackPeriod time.Duration querier Querier } -// NewDefaultEvaluator constructs a defaultEvaluator -func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) Evaluator { - return &defaultEvaluator{ +// NewDefaultEvaluator constructs a DefaultEvaluator +func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) *DefaultEvaluator { + return &DefaultEvaluator{ querier: querier, maxLookBackPeriod: maxLookBackPeriod, } } -func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) { +func (ev *DefaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) { params := SelectParams{ QueryRequest: &logproto.QueryRequest{ Start: q.Start(), @@ -141,7 +141,7 @@ func (ev *defaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, } -func (ev *defaultEvaluator) StepEvaluator( +func (ev *DefaultEvaluator) StepEvaluator( ctx context.Context, nextEv Evaluator, expr SampleExpr, diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index e064b544cbf28..91ff7e4c0b6dc 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -2,6 +2,7 @@ package logql import ( "context" + "errors" "fmt" "time" @@ -95,7 +96,23 @@ type Downstreamer interface { } // DownstreamEvaluator is an evaluator which handles shard aware AST nodes -type DownstreamEvaluator struct{ Downstreamer } +type DownstreamEvaluator struct { + Downstreamer + defaultEvaluator *DefaultEvaluator +} + +func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator { + return &DownstreamEvaluator{ + Downstreamer: downstreamer, + defaultEvaluator: NewDefaultEvaluator( + QuerierFunc(func(_ context.Context, p SelectParams) (iter.EntryIterator, error) { + // TODO(owen-d): add metric here, this should never happen. + return nil, errors.New("Unimplemented") + }), + 0, + ), + } +} // Evaluator returns a StepEvaluator for a given SampleExpr func (ev *DownstreamEvaluator) StepEvaluator( @@ -142,6 +159,9 @@ func (ev *DownstreamEvaluator) StepEvaluator( return ConcatEvaluator(xs) + case *vectorAggregationExpr, *binOpExpr: + return ev.defaultEvaluator.StepEvaluator(ctx, nextEv, e, params) + default: return nil, EvaluatorUnsupportedType(expr, ev) } @@ -306,7 +326,7 @@ func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer) (E return &shardedEngine{ timeout: opts.Timeout, mapper: mapper, - evaluator: &DownstreamEvaluator{downstreamer}, + evaluator: NewDownstreamEvaluator(downstreamer), }, nil } diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index f1fe4f01ce0fd..6df41cb27b1a6 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -2,29 +2,49 @@ package logql import ( "context" + "math" "testing" "time" "github.com/grafana/loki/pkg/logproto" + "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/require" ) func TestMappingEquivalence(t *testing.T) { var ( shards = 3 - nStreams = 10 + nStreams = 60 rounds = 20 streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"}) start = time.Unix(0, 0) - end = time.Unix(0, int64(time.Millisecond*time.Duration(rounds))) + end = time.Unix(0, int64(time.Second*time.Duration(rounds))) + step = time.Second limit = 100 ) for _, tc := range []struct { - query string + query string + approximate bool }{ - {`{a="1"}`}, - {`{a="1"} |= "number: 10"`}, + {`1`, false}, + {`1 + 1`, false}, + {`{a="1"}`, false}, + {`{a="1"} |= "number: 10"`, false}, + {`rate({a=~".*"}[1s])`, false}, + {`sum by (a) (rate({a=~".*"}[1s]))`, false}, + {`max without (a) (rate({a=~".*"}[1s]))`, false}, + {`count(rate({a=~".*"}[1s]))`, false}, + {`avg(rate({a=~".*"}[1s]))`, true}, + {`1 + sum by (cluster) (rate({a=~".*"}[1s]))`, false}, + {`sum(max(rate({a=~".*"}[1s])))`, false}, + {`max(count(rate({a=~".*"}[1s])))`, false}, + {`max(sum by (cluster) (rate({a=~".*"}[1s]))) / count(rate({a=~".*"}[1s]))`, false}, + // topk prefers already-seen values in tiebreakers. Since the test data generates + // the same log lines for each series & the resulting promql.Vectors aren't deterministically + // sorted by labels, we don't expect this to pass. + // We could sort them as stated, but it doesn't seem worth the performance hit. + // {`topk(3, rate({a=~".*"}[1s]))`, false}, } { q := NewMockQuerier( shards, @@ -41,7 +61,7 @@ func TestMappingEquivalence(t *testing.T) { tc.query, start, end, - time.Millisecond*10, + step, logproto.FORWARD, uint32(limit), nil, @@ -54,8 +74,31 @@ func TestMappingEquivalence(t *testing.T) { shardedRes, err := shardedQry.Exec(context.Background()) require.Nil(t, err) - require.Equal(t, res.Data, shardedRes.Data) - + if tc.approximate { + approximatelyEquals(t, res.Data.(promql.Matrix), shardedRes.Data.(promql.Matrix)) + } else { + require.Equal(t, res.Data, shardedRes.Data) + } }) } } + +// approximatelyEquals ensures two responses are approximately equal, up to 6 decimals precision per sample +func approximatelyEquals(t *testing.T, as, bs promql.Matrix) { + require.Equal(t, len(as), len(bs)) + + for i := 0; i < len(as); i++ { + a := as[i] + b := bs[i] + require.Equal(t, a.Metric, b.Metric) + require.Equal(t, len(a.Points), len(b.Points)) + + for j := 0; j < len(a.Points); j++ { + aSample := &a.Points[j] + aSample.V = math.Round(aSample.V*1e6) / 1e6 + bSample := &b.Points[j] + bSample.V = math.Round(bSample.V*1e6) / 1e6 + } + require.Equal(t, a, b) + } +} diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 4a849b2a7b5e8..11c4387ebc663 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -38,22 +38,28 @@ func (q MockQuerier) Select(_ context.Context, req SelectParams) (iter.EntryIter matchers := expr.Matchers() + var shard *astmapper.ShardAnnotation + if len(req.Shards) > 0 { + shards, err := ParseShards(req.Shards) + if err != nil { + return nil, err + } + shard = &shards[0] + } + var matched []*logproto.Stream outer: for _, stream := range q.streams { ls := mustParseLabels(stream.Labels) - for _, matcher := range matchers { - if matcher.Name == astmapper.ShardLabel { - shard, err := astmapper.ParseShard(matcher.Value) - if err != nil { - return nil, err - } - if !(ls.Hash()%uint64(q.shards) == uint64(shard.Shard)) { - continue outer - } - } else if !matcher.Matches(ls.Get(matcher.Name)) { + // filter by shard if requested + if shard != nil && ls.Hash()%uint64(shard.Of) != uint64(shard.Shard) { + continue + } + + for _, matcher := range matchers { + if !matcher.Matches(ls.Get(matcher.Name)) { continue outer } } @@ -130,7 +136,7 @@ func randomStreams(nStreams, nEntries, nShards int, labelNames []string) (stream } for j := 0; j < nEntries; j++ { stream.Entries = append(stream.Entries, logproto.Entry{ - Timestamp: time.Unix(0, int64(j*int(time.Millisecond))), + Timestamp: time.Unix(0, int64(j*int(time.Second))), Line: fmt.Sprintf("line number: %d", j), }) } From 1793dcee9520c7f365f81393a520ab4c9a29c4df Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 9 Apr 2020 17:57:51 -0400 Subject: [PATCH 33/67] other pkgs adopt logql changes --- pkg/logcli/query/query.go | 20 ++++++++++++++++-- pkg/logql/sharding_test.go | 3 ++- pkg/logql/test_utils.go | 5 +++-- pkg/querier/querier.go | 2 +- pkg/querier/queryrange/downstreamer.go | 3 +-- pkg/querier/queryrange/querysharding.go | 27 +++++-------------------- pkg/storage/store_test.go | 2 +- 7 files changed, 31 insertions(+), 31 deletions(-) diff --git a/pkg/logcli/query/query.go b/pkg/logcli/query/query.go index b7569c4eebe97..be0484cd76a5f 100644 --- a/pkg/logcli/query/query.go +++ b/pkg/logcli/query/query.go @@ -118,9 +118,25 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string eng := logql.NewEngine(conf.Querier.Engine, querier) var query logql.Query if q.isInstant() { - query = eng.NewInstantQuery(q.QueryString, q.Start, q.resultsDirection(), uint32(q.Limit)) + query = eng.NewInstantQuery(logql.NewLiteralParams( + q.QueryString, + q.Start, + q.Start, + 0, + q.resultsDirection(), + uint32(q.Limit), + nil, + )) } else { - query = eng.NewRangeQuery(q.QueryString, q.Start, q.End, q.Step, q.resultsDirection(), uint32(q.Limit)) + query = eng.NewRangeQuery(logql.NewLiteralParams( + q.QueryString, + q.Start, + q.End, + q.Step, + q.resultsDirection(), + uint32(q.Limit), + nil, + )) } // execute the query diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 6df41cb27b1a6..b784e4f13eda4 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -6,9 +6,10 @@ import ( "testing" "time" - "github.com/grafana/loki/pkg/logproto" "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/logproto" ) func TestMappingEquivalence(t *testing.T) { diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 11c4387ebc663..c8d3e23ae0ba1 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -7,10 +7,11 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" - "github.com/grafana/loki/pkg/iter" - "github.com/grafana/loki/pkg/logproto" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql" + + "github.com/grafana/loki/pkg/iter" + "github.com/grafana/loki/pkg/logproto" ) func NewMockQuerier(shards int, streams []*logproto.Stream) MockQuerier { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 04ccabc58f795..45d08a54e4b08 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -85,7 +85,7 @@ func newQuerier(cfg Config, clientCfg client.Config, clientFactory cortex_client limits: limits, } - querier.engine = logql.NewEngine(cfg.Engine, querier) + querier.engine = logql.NewEngine(cfg.Engine, &querier) err := services.StartAndAwaitRunning(context.Background(), querier.pool) if err != nil { return nil, errors.Wrap(err, "querier pool") diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index 1906e6ecd84f2..eca128ef0c763 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -5,7 +5,6 @@ import ( "fmt" "time" - "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql" @@ -35,7 +34,7 @@ func ParamsToLokiRequest(params logql.Params) *LokiRequest { } } -func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shards []astmapper.ShardAnnotation) (logql.Query, error) { +func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shards logql.Shards) (logql.Query, error) { req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()) return QuerierFunc(func(ctx context.Context) (logql.Result, error) { diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index e6b6f82c0311a..b342c37783d2f 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -61,16 +61,12 @@ func newASTMapperware(confs queryrange.ShardingConfigs, next queryrange.Handler, confs: confs, logger: log.With(logger, "middleware", "QueryShard.astMapperware"), next: next, - engine: logql.NewEngine(logql.EngineOpts{}, func(_ logql.EngineOpts) logql.Evaluator { - return &logql.DownstreamEvaluator{DownstreamHandler{next}} - }), } } type astMapperware struct { confs queryrange.ShardingConfigs logger log.Logger - engine logql.Engine next queryrange.Handler } @@ -82,32 +78,19 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra return ast.next.Do(ctx, r) } - shardMapper, err := logql.NewShardMapper(int(conf.RowShards)) - if err != nil { - return nil, err - } + ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}) - strQuery := r.GetQuery() - parsed, err := logql.ParseExpr(strQuery) if err != nil { - return nil, err - } - - mappedQuery, err := shardMapper.Map(parsed) - if err != nil { - return nil, err + level.Warn(ast.logger).Log("err", err.Error(), "msg", "failed to create sharded engine") + return ast.next.Do(ctx, r) } req, ok := r.(*LokiRequest) if !ok { return nil, fmt.Errorf("expected *LokiRequest, got (%T)", r) } - - strMappedQuery := mappedQuery.String() - level.Debug(ast.logger).Log("msg", "mapped query", "original", strQuery, "mapped", strMappedQuery) - - params := paramsFromRequest(req.WithQuery(strMappedQuery)) - query := ast.engine.NewRangeQuery(params) + params := paramsFromRequest(req) + query := ng.NewRangeQuery(params) res, err := query.Exec(ctx) if err != nil { diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 7576ea34aac9a..7eba2b0d5ed09 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -457,7 +457,7 @@ func Test_store_decodeReq_Matchers(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - ms, _, _, _, err := decodeReq(logql.SelectParams{QueryRequest: tt.req}) + ms, _, _, _, err := DecodeReq(logql.SelectParams{QueryRequest: tt.req}) if err != nil { t.Errorf("store.GetSeries() error = %v", err) return From 03ae942d3cd24d2e55405f38ea6813c8cac86b13 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 10 Apr 2020 11:35:36 -0400 Subject: [PATCH 34/67] metrics & logs use same middleware instantiation process --- pkg/querier/queryrange/querysharding.go | 1 - pkg/querier/queryrange/roundtrip.go | 176 +++++++++--------------- 2 files changed, 68 insertions(+), 109 deletions(-) diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index b342c37783d2f..0dc7c6204aa33 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -20,7 +20,6 @@ var nanosecondsInMillisecond = int64(time.Millisecond / time.Nanosecond) // NewQueryShardMiddleware creates a middleware which downstreams queries after AST mapping and query encoding. func NewQueryShardMiddleware( logger log.Logger, - engine logql.Engine, confs queryrange.ShardingConfigs, minShardingLookback time.Duration, metrics *queryrange.InstrumentMiddlewareMetrics, diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 11ccf90446335..893e1d33c3805 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -44,17 +44,76 @@ func NewTripperware(cfg Config, log log.Logger, limits Limits, registerer promet instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer) retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer) - metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics) - if err != nil { - return nil, nil, err + metricsWare := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} + logFilterWare := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} + + // ----Alignment---- + + if cfg.AlignQueriesWithStep { + metricsWare = append( + metricsWare, + queryrange.InstrumentMiddleware("step_align", instrumentMetrics), + queryrange.StepAlignMiddleware, + ) } - logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, lokiCodec, instrumentMetrics, retryMetrics) - if err != nil { - return nil, nil, err + + // ----Splitting---- + + // SplitQueriesByDay is deprecated use SplitQueriesByInterval. + if cfg.SplitQueriesByDay { + level.Warn(log).Log("msg", "flag querier.split-queries-by-day (or config split_queries_by_day) is deprecated, use querier.split-queries-by-interval instead.") + } + + if cfg.SplitQueriesByInterval != 0 { + + splitWare := queryrange.MergeMiddlewares( + queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), + SplitByIntervalMiddleware(limits, lokiCodec), + ) + + metricsWare = append(metricsWare, splitWare) + logFilterWare = append(logFilterWare, splitWare) + } + + // ----Caching---- + + var c cache.Cache + if cfg.CacheResults { + queryCacheMiddleware, cache, err := queryrange.NewResultsCacheMiddleware( + log, + cfg.ResultsCacheConfig, + cacheKeyLimits{limits}, + limits, + lokiCodec, + prometheusResponseExtractor, + ) + if err != nil { + return nil, nil, err + } + c = cache + metricsWare = append( + metricsWare, + queryrange.InstrumentMiddleware("results_cache", instrumentMetrics), + queryCacheMiddleware, + ) + } + + // ----Sharding---- + + // ----Retries---- + + if cfg.MaxRetries > 0 { + retryWare := queryrange.MergeMiddlewares( + queryrange.InstrumentMiddleware("retry", instrumentMetrics), + queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMetrics), + ) + logFilterWare = append(logFilterWare, retryWare) + metricsWare = append(metricsWare, retryWare) } + return func(next http.RoundTripper) http.RoundTripper { - metricRT := metricsTripperware(next) - logFilterRT := logFilterTripperware(next) + metricRT := queryrange.NewRoundTripper(next, lokiCodec, metricsWare...) + logFilterRT := queryrange.NewRoundTripper(next, lokiCodec, logFilterWare...) return frontend.RoundTripFunc(func(req *http.Request) (*http.Response, error) { if !strings.HasSuffix(req.URL.Path, "/query_range") && !strings.HasSuffix(req.URL.Path, "/prom/query") { return next.RoundTrip(req) @@ -92,7 +151,7 @@ func NewTripperware(cfg Config, log log.Logger, limits Limits, registerer promet } return next.RoundTrip(req) }) - }, cache, nil + }, c, nil } // validates log entries limits @@ -114,102 +173,3 @@ func validateLimits(req *http.Request, params url.Values, limits Limits) error { } return nil } - -// NewLogFilterTripperware creates a new frontend tripperware responsible for handling log requests with regex. -func NewLogFilterTripperware( - cfg Config, - log log.Logger, - limits Limits, - codec queryrange.Codec, - instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, - retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, -) (frontend.Tripperware, error) { - queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} - if cfg.SplitQueriesByInterval != 0 { - queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec)) - } - if cfg.MaxRetries > 0 { - queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("retry", instrumentMetrics), queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMiddlewareMetrics)) - } - - return func(next http.RoundTripper) http.RoundTripper { - if len(queryRangeMiddleware) > 0 { - return queryrange.NewRoundTripper(next, codec, queryRangeMiddleware...) - } - return next - }, nil -} - -// NewMetricTripperware creates a new frontend tripperware responsible for handling metric queries -func NewMetricTripperware( - cfg Config, - log log.Logger, - limits Limits, - codec queryrange.Codec, - extractor queryrange.Extractor, - instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, - retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, -) (frontend.Tripperware, Stopper, error) { - queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} - if cfg.AlignQueriesWithStep { - queryRangeMiddleware = append( - queryRangeMiddleware, - queryrange.InstrumentMiddleware("step_align", instrumentMetrics), - queryrange.StepAlignMiddleware, - ) - } - - // SplitQueriesByDay is deprecated use SplitQueriesByInterval. - if cfg.SplitQueriesByDay { - level.Warn(log).Log("msg", "flag querier.split-queries-by-day (or config split_queries_by_day) is deprecated, use querier.split-queries-by-interval instead.") - } - - queryRangeMiddleware = append( - queryRangeMiddleware, - queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), - SplitByIntervalMiddleware(limits, codec), - ) - - var c cache.Cache - if cfg.CacheResults { - queryCacheMiddleware, cache, err := queryrange.NewResultsCacheMiddleware( - log, - cfg.ResultsCacheConfig, - cacheKeyLimits{limits}, - limits, - codec, - extractor, - ) - if err != nil { - return nil, nil, err - } - c = cache - queryRangeMiddleware = append( - queryRangeMiddleware, - queryrange.InstrumentMiddleware("results_cache", instrumentMetrics), - queryCacheMiddleware, - ) - } - - if cfg.MaxRetries > 0 { - queryRangeMiddleware = append( - queryRangeMiddleware, - queryrange.InstrumentMiddleware("retry", instrumentMetrics), - queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMiddlewareMetrics), - ) - } - - return func(next http.RoundTripper) http.RoundTripper { - // Finally, if the user selected any query range middleware, stitch it in. - if len(queryRangeMiddleware) > 0 { - rt := queryrange.NewRoundTripper(next, codec, queryRangeMiddleware...) - return frontend.RoundTripFunc(func(r *http.Request) (*http.Response, error) { - if !strings.HasSuffix(r.URL.Path, "/query_range") { - return next.RoundTrip(r) - } - return rt.RoundTrip(r) - }) - } - return next - }, c, nil -} From b43d995dd2247bfbbd40de37bf5a636b30c7926e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 10 Apr 2020 11:59:47 -0400 Subject: [PATCH 35/67] wires up shardingware --- pkg/loki/modules.go | 9 +++++++- pkg/querier/queryrange/roundtrip.go | 28 +++++++++++++++++++++++- pkg/querier/queryrange/roundtrip_test.go | 15 +++++++------ 3 files changed, 43 insertions(+), 9 deletions(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 86f00005e53d4..f5fdff8ce6cf8 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -296,7 +296,14 @@ func (t *Loki) initQueryFrontend() (err error) { "config", fmt.Sprintf("%+v", t.cfg.QueryRange), "limits", fmt.Sprintf("%+v", t.cfg.LimitsConfig), ) - tripperware, stopper, err := queryrange.NewTripperware(t.cfg.QueryRange, util.Logger, t.overrides, prometheus.DefaultRegisterer) + tripperware, stopper, err := queryrange.NewTripperware( + t.cfg.QueryRange, + util.Logger, + t.overrides, + t.cfg.SchemaConfig, + t.cfg.Querier.IngesterMaxQueryLookback, + prometheus.DefaultRegisterer, + ) if err != nil { return err } diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 893e1d33c3805..1c50b64ff223f 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -1,12 +1,15 @@ package queryrange import ( + "errors" "flag" "net/http" "net/url" "strconv" "strings" + "time" + "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/cache" "github.com/cortexproject/cortex/pkg/querier/frontend" "github.com/cortexproject/cortex/pkg/querier/queryrange" @@ -36,7 +39,14 @@ type Stopper interface { } // NewTripperware returns a Tripperware configured with middlewares to align, split and cache requests. -func NewTripperware(cfg Config, log log.Logger, limits Limits, registerer prometheus.Registerer) (frontend.Tripperware, Stopper, error) { +func NewTripperware( + cfg Config, + log log.Logger, + limits Limits, + schema chunk.SchemaConfig, + minShardingLookback time.Duration, + registerer prometheus.Registerer, +) (frontend.Tripperware, Stopper, error) { // Ensure that QuerySplitDuration uses configuration defaults. // This avoids divide by zero errors when determining cache keys where user specific overrides don't exist. limits = WithDefaultLimits(limits, cfg.Config) @@ -100,6 +110,22 @@ func NewTripperware(cfg Config, log log.Logger, limits Limits, registerer promet // ----Sharding---- + if cfg.ShardedQueries { + if minShardingLookback == 0 { + return nil, nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled") + } + + shardingware := NewQueryShardMiddleware( + log, + schema.Configs, + minShardingLookback, + instrumentMetrics, // instrumentation is included in the sharding middleware + ) + + metricsWare = append(metricsWare, shardingware) + logFilterWare = append(metricsWare, shardingware) + } + // ----Retries---- if cfg.MaxRetries > 0 { diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index 809145aa28496..6b575b3a48f7f 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -9,6 +9,7 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/cache" "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/cortexproject/cortex/pkg/util" @@ -76,7 +77,7 @@ var ( // those tests are mostly for testing the glue between all component and make sure they activate correctly. func TestMetricsTripperware(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -140,7 +141,7 @@ func TestMetricsTripperware(t *testing.T) { func TestLogFilterTripperware(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -187,7 +188,7 @@ func TestLogFilterTripperware(t *testing.T) { } func TestLogNoRegex(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -221,7 +222,7 @@ func TestLogNoRegex(t *testing.T) { } func TestUnhandledPath(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -245,7 +246,7 @@ func TestUnhandledPath(t *testing.T) { } func TestRegexpParamsSupport(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -288,7 +289,7 @@ func TestRegexpParamsSupport(t *testing.T) { } func TestEntriesLimitsTripperware(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } @@ -319,7 +320,7 @@ func TestEntriesLimitsTripperware(t *testing.T) { } func TestEntriesLimitWithZeroTripperware(t *testing.T) { - tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, nil) + tpw, stopper, err := NewTripperware(testConfig, util.Logger, fakeLimits{}, chunk.SchemaConfig{}, 0, nil) if stopper != nil { defer stopper.Stop() } From 78de015e2304f0e46153af23e7f9b0099ea8f7d3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 10 Apr 2020 15:47:17 -0400 Subject: [PATCH 36/67] middleware per metrics/logfilter --- pkg/querier/queryrange/roundtrip.go | 224 +++++++++++++++++----------- 1 file changed, 141 insertions(+), 83 deletions(-) diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 1c50b64ff223f..a3a038aa98623 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -54,92 +54,18 @@ func NewTripperware( instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer) retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer) - metricsWare := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} - logFilterWare := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} - - // ----Alignment---- - - if cfg.AlignQueriesWithStep { - metricsWare = append( - metricsWare, - queryrange.InstrumentMiddleware("step_align", instrumentMetrics), - queryrange.StepAlignMiddleware, - ) - } - - // ----Splitting---- - - // SplitQueriesByDay is deprecated use SplitQueriesByInterval. - if cfg.SplitQueriesByDay { - level.Warn(log).Log("msg", "flag querier.split-queries-by-day (or config split_queries_by_day) is deprecated, use querier.split-queries-by-interval instead.") - } - - if cfg.SplitQueriesByInterval != 0 { - - splitWare := queryrange.MergeMiddlewares( - queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), - SplitByIntervalMiddleware(limits, lokiCodec), - ) - - metricsWare = append(metricsWare, splitWare) - logFilterWare = append(logFilterWare, splitWare) - } - - // ----Caching---- - - var c cache.Cache - if cfg.CacheResults { - queryCacheMiddleware, cache, err := queryrange.NewResultsCacheMiddleware( - log, - cfg.ResultsCacheConfig, - cacheKeyLimits{limits}, - limits, - lokiCodec, - prometheusResponseExtractor, - ) - if err != nil { - return nil, nil, err - } - c = cache - metricsWare = append( - metricsWare, - queryrange.InstrumentMiddleware("results_cache", instrumentMetrics), - queryCacheMiddleware, - ) - } - - // ----Sharding---- - - if cfg.ShardedQueries { - if minShardingLookback == 0 { - return nil, nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled") - } - - shardingware := NewQueryShardMiddleware( - log, - schema.Configs, - minShardingLookback, - instrumentMetrics, // instrumentation is included in the sharding middleware - ) - - metricsWare = append(metricsWare, shardingware) - logFilterWare = append(metricsWare, shardingware) + metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics) + if err != nil { + return nil, nil, err } - - // ----Retries---- - - if cfg.MaxRetries > 0 { - retryWare := queryrange.MergeMiddlewares( - queryrange.InstrumentMiddleware("retry", instrumentMetrics), - queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMetrics), - ) - logFilterWare = append(logFilterWare, retryWare) - metricsWare = append(metricsWare, retryWare) + logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics) + if err != nil { + return nil, nil, err } return func(next http.RoundTripper) http.RoundTripper { - metricRT := queryrange.NewRoundTripper(next, lokiCodec, metricsWare...) - logFilterRT := queryrange.NewRoundTripper(next, lokiCodec, logFilterWare...) + metricRT := metricsTripperware(next) + logFilterRT := logFilterTripperware(next) return frontend.RoundTripFunc(func(req *http.Request) (*http.Response, error) { if !strings.HasSuffix(req.URL.Path, "/query_range") && !strings.HasSuffix(req.URL.Path, "/prom/query") { return next.RoundTrip(req) @@ -177,7 +103,7 @@ func NewTripperware( } return next.RoundTrip(req) }) - }, c, nil + }, cache, nil } // validates log entries limits @@ -199,3 +125,135 @@ func validateLimits(req *http.Request, params url.Values, limits Limits) error { } return nil } + +// NewLogFilterTripperware creates a new frontend tripperware responsible for handling log requests with regex. +func NewLogFilterTripperware( + cfg Config, + log log.Logger, + limits Limits, + schema chunk.SchemaConfig, + minShardingLookback time.Duration, + codec queryrange.Codec, + instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, + retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, +) (frontend.Tripperware, error) { + queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} + if cfg.SplitQueriesByInterval != 0 { + queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec)) + } + + if cfg.ShardedQueries { + if minShardingLookback == 0 { + return nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled") + } + queryRangeMiddleware = append(queryRangeMiddleware, + NewQueryShardMiddleware( + log, + schema.Configs, + minShardingLookback, + instrumentMetrics, // instrumentation is included in the sharding middleware + ), + ) + } + + if cfg.MaxRetries > 0 { + queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("retry", instrumentMetrics), queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMiddlewareMetrics)) + } + + return func(next http.RoundTripper) http.RoundTripper { + if len(queryRangeMiddleware) > 0 { + return queryrange.NewRoundTripper(next, codec, queryRangeMiddleware...) + } + return next + }, nil +} + +// NewMetricTripperware creates a new frontend tripperware responsible for handling metric queries +func NewMetricTripperware( + cfg Config, + log log.Logger, + limits Limits, + schema chunk.SchemaConfig, + minShardingLookback time.Duration, + codec queryrange.Codec, + extractor queryrange.Extractor, + instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, + retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, +) (frontend.Tripperware, Stopper, error) { + queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} + if cfg.AlignQueriesWithStep { + queryRangeMiddleware = append( + queryRangeMiddleware, + queryrange.InstrumentMiddleware("step_align", instrumentMetrics), + queryrange.StepAlignMiddleware, + ) + } + + // SplitQueriesByDay is deprecated use SplitQueriesByInterval. + if cfg.SplitQueriesByDay { + level.Warn(log).Log("msg", "flag querier.split-queries-by-day (or config split_queries_by_day) is deprecated, use querier.split-queries-by-interval instead.") + } + + queryRangeMiddleware = append( + queryRangeMiddleware, + queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), + SplitByIntervalMiddleware(limits, codec), + ) + + var c cache.Cache + if cfg.CacheResults { + queryCacheMiddleware, cache, err := queryrange.NewResultsCacheMiddleware( + log, + cfg.ResultsCacheConfig, + cacheKeyLimits{limits}, + limits, + codec, + extractor, + ) + if err != nil { + return nil, nil, err + } + c = cache + queryRangeMiddleware = append( + queryRangeMiddleware, + queryrange.InstrumentMiddleware("results_cache", instrumentMetrics), + queryCacheMiddleware, + ) + } + + if cfg.ShardedQueries { + if minShardingLookback == 0 { + return nil, nil, errors.New("a non-zero value is required for querier.query-ingesters-within when -querier.parallelise-shardable-queries is enabled") + } + queryRangeMiddleware = append(queryRangeMiddleware, + NewQueryShardMiddleware( + log, + schema.Configs, + minShardingLookback, + instrumentMetrics, // instrumentation is included in the sharding middleware + ), + ) + } + + if cfg.MaxRetries > 0 { + queryRangeMiddleware = append( + queryRangeMiddleware, + queryrange.InstrumentMiddleware("retry", instrumentMetrics), + queryrange.NewRetryMiddleware(log, cfg.MaxRetries, retryMiddlewareMetrics), + ) + } + + return func(next http.RoundTripper) http.RoundTripper { + // Finally, if the user selected any query range middleware, stitch it in. + if len(queryRangeMiddleware) > 0 { + rt := queryrange.NewRoundTripper(next, codec, queryRangeMiddleware...) + return frontend.RoundTripFunc(func(r *http.Request) (*http.Response, error) { + if !strings.HasSuffix(r.URL.Path, "/query_range") { + return next.RoundTrip(r) + } + return rt.RoundTrip(r) + }) + } + return next + }, c, nil +} From 16881d3a0d245d8749508ea91f22be4bce891eca Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 15 Apr 2020 16:37:32 -0400 Subject: [PATCH 37/67] empty step populating StepEvaluator promql.Matrix adapter --- pkg/logql/matrix.go | 67 +++++++++++++++++++++++ pkg/logql/matrix_test.go | 115 +++++++++++++++++++++++++++++++++++++++ pkg/logql/sharding.go | 48 ++-------------- 3 files changed, 187 insertions(+), 43 deletions(-) create mode 100644 pkg/logql/matrix.go create mode 100644 pkg/logql/matrix_test.go diff --git a/pkg/logql/matrix.go b/pkg/logql/matrix.go new file mode 100644 index 0000000000000..59fcf348027a8 --- /dev/null +++ b/pkg/logql/matrix.go @@ -0,0 +1,67 @@ +package logql + +import ( + "time" + + "github.com/prometheus/prometheus/promql" +) + +// MatrixStepper exposes a promql.Matrix as a StepEvaluator. +// Ensure that the resulting StepEvaluator maintains +// the same shape that the parameters expect. For example, +// it's possible that a downstream query returns matches no +// log streams and thus returns an empty matrix. +// However, we still need to ensure that it can be merged effectively +// with another leg that may match series. +// Therefore, we determine our steps from the parameters +// and not the underlying Matrix. +type MatrixStepper struct { + start, end, ts time.Time + step time.Duration + m promql.Matrix +} + +func NewMatrixStepper(start, end time.Time, step time.Duration, m promql.Matrix) *MatrixStepper { + return &MatrixStepper{ + start: start, + end: end, + ts: start.Add(-step), // will be corrected on first Next() call + step: step, + m: m, + } +} + +func (m *MatrixStepper) Next() (bool, int64, promql.Vector) { + m.ts = m.ts.Add(m.step) + if !m.ts.Before(m.end) { + return false, 0, nil + } + + ts := m.ts.UnixNano() / int64(time.Millisecond) + vec := make(promql.Vector, 0, len(m.m)) + + for i, series := range m.m { + ln := len(series.Points) + + if ln == 0 || series.Points[0].T != ts { + vec = append(vec, promql.Sample{ + Point: promql.Point{ + T: ts, + V: 0, + }, + Metric: series.Metric, + }) + continue + } + + vec = append(vec, promql.Sample{ + Point: series.Points[0], + Metric: series.Metric, + }) + m.m[i].Points = m.m[i].Points[1:] + } + + return true, ts, vec +} + +func (m *MatrixStepper) Close() error { return nil } diff --git a/pkg/logql/matrix_test.go b/pkg/logql/matrix_test.go new file mode 100644 index 0000000000000..ddd0389c5c35a --- /dev/null +++ b/pkg/logql/matrix_test.go @@ -0,0 +1,115 @@ +package logql + +import ( + "testing" + "time" + + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/promql" + "github.com/stretchr/testify/require" +) + +func TestMatrixStepper(t *testing.T) { + var ( + start = time.Unix(0, 0) + end = time.Unix(6, 0) + step = time.Second + ) + + m := promql.Matrix{ + promql.Series{ + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + Points: []promql.Point{ + {T: start.UnixNano() / int64(step), V: 0}, + {T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 1}, + {T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2}, + {T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 3}, + {T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4}, + {T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 5}, + }, + }, + promql.Series{ + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + Points: []promql.Point{ + {T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2}, + {T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4}, + }, + }, + } + + s := NewMatrixStepper(start, end, step, m) + + expected := []promql.Vector{ + { + promql.Sample{ + Point: promql.Point{T: start.UnixNano() / int64(step), V: 0}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.UnixNano() / int64(step), V: 0}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + { + promql.Sample{ + Point: promql.Point{T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 1}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.Add(step).UnixNano() / int64(time.Millisecond), V: 0}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + { + promql.Sample{ + Point: promql.Point{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.Add(2*step).UnixNano() / int64(time.Millisecond), V: 2}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + { + promql.Sample{ + Point: promql.Point{T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 3}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.Add(3*step).UnixNano() / int64(time.Millisecond), V: 0}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + { + promql.Sample{ + Point: promql.Point{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.Add(4*step).UnixNano() / int64(time.Millisecond), V: 4}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + { + promql.Sample{ + Point: promql.Point{T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 5}, + Metric: labels.Labels{{Name: "foo", Value: "bar"}}, + }, + promql.Sample{ + Point: promql.Point{T: start.Add(5*step).UnixNano() / int64(time.Millisecond), V: 0}, + Metric: labels.Labels{{Name: "bazz", Value: "buzz"}}, + }, + }, + } + + for i := 0; i < int(end.Sub(start)/step); i++ { + ok, ts, vec := s.Next() + require.Equal(t, ok, true) + require.Equal(t, start.Add(step*time.Duration(i)).UnixNano()/int64(time.Millisecond), ts) + require.Equal(t, expected[i], vec) + } + + ok, _, _ := s.Next() + + require.Equal(t, ok, false) +} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 91ff7e4c0b6dc..9d3c8968e221a 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -240,12 +240,9 @@ func ConcatEvaluator(evaluators []StepEvaluator) (StepEvaluator, error) { // ResultStepEvaluator coerces a downstream vector or matrix into a StepEvaluator func ResultStepEvaluator(res Result, params Params) (StepEvaluator, error) { var ( - end = params.End() - step = params.Step() - ts = params.Start() - increment = func() { - ts = ts.Add(step) - } + start = params.Start() + end = params.End() + step = params.Step() ) switch data := res.Data.(type) { @@ -254,47 +251,12 @@ func ResultStepEvaluator(res Result, params Params) (StepEvaluator, error) { return newStepEvaluator(func() (bool, int64, promql.Vector) { if !exhausted { exhausted = true - return true, ts.UnixNano() / int64(time.Millisecond), data + return true, start.UnixNano() / int64(time.Millisecond), data } return false, 0, nil }, nil) case promql.Matrix: - var i int - var maxLn int - if len(data) > 0 { - maxLn = len(data[0].Points) - } - return newStepEvaluator(func() (bool, int64, promql.Vector) { - defer increment() - if ts.After(end) { - return false, 0, nil - } - - tsInt := ts.UnixNano() / int64(time.Millisecond) - - // Ensure that the resulting StepEvaluator maintains - // the same shape that the parameters expect. For example, - // it's possible that a downstream query returns matches no - // log streams and thus returns an empty matrix. - // However, we still need to ensure that it can be merged effectively - // with another leg that may match series. - // Therefore, we determine our steps from the parameters - // and not the underlying Matrix. - if i >= maxLn { - return true, tsInt, nil - } - - vec := make(promql.Vector, 0, len(data)) - for j := 0; j < len(data); j++ { - series := data[j] - vec = append(vec, promql.Sample{ - Point: series.Points[i], - Metric: series.Metric, - }) - } - i++ - return true, tsInt, vec - }, nil) + return NewMatrixStepper(start, end, step, data), nil default: return nil, fmt.Errorf("unexpected type (%s) uncoercible to StepEvaluator", data.Type()) } From 6219b8ae6a7baa96eb1c5147d46c1c8abf6a3f54 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 16 Apr 2020 12:32:11 -0400 Subject: [PATCH 38/67] sharding metrics --- pkg/logql/astmapper.go | 26 ----- pkg/logql/sharding.go | 17 +-- pkg/logql/sharding_test.go | 4 +- pkg/logql/shardmapper.go | 140 ++++++++++++++++++++---- pkg/logql/shardmapper_test.go | 12 +- pkg/querier/queryrange/querysharding.go | 29 +++-- pkg/querier/queryrange/roundtrip.go | 8 +- 7 files changed, 161 insertions(+), 75 deletions(-) delete mode 100644 pkg/logql/astmapper.go diff --git a/pkg/logql/astmapper.go b/pkg/logql/astmapper.go deleted file mode 100644 index 6067fbeb93e94..0000000000000 --- a/pkg/logql/astmapper.go +++ /dev/null @@ -1,26 +0,0 @@ -package logql - -import ( - "fmt" - - "github.com/pkg/errors" -) - -// ASTMapper is the exported interface for mapping between multiple AST representations -type ASTMapper interface { - Map(Expr) (Expr, error) -} - -// CloneExpr is a helper function to clone a node. -func CloneExpr(expr Expr) (Expr, error) { - return ParseExpr(expr.String()) -} - -func badASTMapping(expected string, got Expr) error { - return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) -} - -// MapperUnsuportedType is a helper for signaling that an evaluator does not support an Expr type -func MapperUnsupportedType(expr Expr, m ASTMapper) error { - return errors.Errorf("unexpected expr type (%T) for ASTMapper type (%T) ", expr, m) -} diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 9d3c8968e221a..c7320d5060d56 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -274,13 +274,14 @@ func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { type shardedEngine struct { timeout time.Duration - mapper ASTMapper + mapper ShardMapper evaluator Evaluator + metrics *ShardingMetrics } -func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer) (Engine, error) { +func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer, metrics *ShardingMetrics) (Engine, error) { opts.applyDefault() - mapper, err := NewShardMapper(shards) + mapper, err := NewShardMapper(shards, metrics) if err != nil { return nil, err } @@ -289,6 +290,7 @@ func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer) (E timeout: opts.Timeout, mapper: mapper, evaluator: NewDownstreamEvaluator(downstreamer), + metrics: metrics, }, nil } @@ -298,14 +300,7 @@ func (ng *shardedEngine) query(p Params) Query { timeout: ng.timeout, params: p, evaluator: ng.evaluator, - parse: func(query string) (Expr, error) { - parsed, err := ParseExpr(query) - if err != nil { - return nil, err - } - - return ng.mapper.Map(parsed) - }, + parse: ng.mapper.Parse, } } diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index b784e4f13eda4..0aa56ccd38187 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -12,6 +12,8 @@ import ( "github.com/grafana/loki/pkg/logproto" ) +var nilMetrics = NewShardingMetrics(nil) + func TestMappingEquivalence(t *testing.T) { var ( shards = 3 @@ -54,7 +56,7 @@ func TestMappingEquivalence(t *testing.T) { opts := EngineOpts{} regular := NewEngine(opts, q) - sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}) + sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}, nilMetrics) require.Nil(t, err) t.Run(tc.query, func(t *testing.T) { diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 79303365a96d0..0c62bfad3825f 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -6,35 +6,133 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/util" "github.com/go-kit/kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" ) -func NewShardMapper(shards int) (ShardMapper, error) { +// keys used in metrics +const ( + StreamsKey = "streams" + MetricsKey = "metrics" + SuccessKey = "success" + FailureKey = "failure" + NoopKey = "noop" +) + +// ShardingMetrics is the metrics wrapper used in shard mapping +type ShardingMetrics struct { + shards *prometheus.CounterVec // sharded queries total, partitioned by (streams/metric) + parsed *prometheus.CounterVec // parsed ASTs total, partitioned by (success/failure/noop) + shardFactor prometheus.Histogram // per request shard factor +} + +func NewShardingMetrics(registerer prometheus.Registerer) *ShardingMetrics { + + return &ShardingMetrics{ + shards: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "query_frontend_shards_total", + }, []string{"type"}), + parsed: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki", + Name: "query_frontend_sharding_parsed_queries_total", + }, []string{"type"}), + shardFactor: promauto.With(registerer).NewHistogram(prometheus.HistogramOpts{ + Namespace: "loki", + Name: "query_frontend_shard_factor", + Help: "Number of shards per request", + Buckets: prometheus.LinearBuckets(0, 16, 4), // 16 is the default shard factor for later schemas + }), + } +} + +// ShardRecorder constructs a recorder using the underlying metrics. +func (m *ShardingMetrics) ShardRecorder() *ShardRecorder { + return &ShardRecorder{ + ShardingMetrics: m, + } +} + +// ShardRecorder wraps a vector & histogram, providing an easy way to increment sharding counts. +// and unify them into histogram entries. +// NOT SAFE FOR CONCURRENT USE! We avoid introducing mutex locking here +// because AST mapping is single threaded. +type ShardRecorder struct { + done bool + total int + *ShardingMetrics +} + +// Add increments both the shard count and tracks it for the eventual histogram entry. +func (r *ShardRecorder) Add(x int, key string) { + r.total += x + r.shards.WithLabelValues(key).Add(float64(x)) +} + +// Finish idemptotently records a histogram entry with the total shard factor. +func (r *ShardRecorder) Finish() { + if !r.done { + r.done = true + r.shardFactor.Observe(float64(r.total)) + } +} + +func badASTMapping(expected string, got Expr) error { + return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) +} + +func NewShardMapper(shards int, metrics *ShardingMetrics) (ShardMapper, error) { if shards < 2 { return ShardMapper{}, fmt.Errorf("Cannot create ShardMapper with <2 shards. Received %d", shards) } - return ShardMapper{shards}, nil + return ShardMapper{ + shards: shards, + metrics: metrics, + }, nil } type ShardMapper struct { - shards int + shards int + metrics *ShardingMetrics +} + +func (m ShardMapper) Parse(query string) (Expr, error) { + parsed, err := ParseExpr(query) + if err != nil { + return nil, err + } + + recorder := m.metrics.ShardRecorder() + defer recorder.Finish() + + mapped, err := m.Map(parsed, recorder) + if err != nil { + m.metrics.parsed.WithLabelValues(FailureKey).Inc() + } else if parsed.String() == mapped.String() { + m.metrics.parsed.WithLabelValues(NoopKey).Inc() + } else { + m.metrics.parsed.WithLabelValues(SuccessKey).Inc() + } + return mapped, err } -func (m ShardMapper) Map(expr Expr) (Expr, error) { +func (m ShardMapper) Map(expr Expr, r *ShardRecorder) (Expr, error) { switch e := expr.(type) { case *literalExpr: return e, nil case *matchersExpr, *filterExpr: - return m.mapLogSelectorExpr(e.(LogSelectorExpr)), nil + return m.mapLogSelectorExpr(e.(LogSelectorExpr), r), nil case *vectorAggregationExpr: - return m.mapVectorAggregationExpr(e) + return m.mapVectorAggregationExpr(e, r) case *rangeAggregationExpr: - return m.mapRangeAggregationExpr(e), nil + return m.mapRangeAggregationExpr(e, r), nil case *binOpExpr: - lhsMapped, err := m.Map(e.SampleExpr) + lhsMapped, err := m.Map(e.SampleExpr, r) if err != nil { return nil, err } - rhsMapped, err := m.Map(e.RHS) + rhsMapped, err := m.Map(e.RHS, r) if err != nil { return nil, err } @@ -50,11 +148,11 @@ func (m ShardMapper) Map(expr Expr) (Expr, error) { e.RHS = rhsSampleExpr return e, nil default: - return nil, MapperUnsupportedType(expr, m) + return nil, errors.Errorf("unexpected expr type (%T) for ASTMapper type (%T) ", expr, m) } } -func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { +func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *ShardRecorder) LogSelectorExpr { var head *ConcatLogSelectorExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatLogSelectorExpr{ @@ -68,11 +166,12 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr) LogSelectorExpr { next: head, } } + r.Add(m.shards, StreamsKey) return head } -func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { +func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *ShardRecorder) SampleExpr { var head *ConcatSampleExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatSampleExpr{ @@ -86,18 +185,19 @@ func (m ShardMapper) mapSampleExpr(expr SampleExpr) SampleExpr { next: head, } } + r.Add(m.shards, MetricsKey) return head } // technically, std{dev,var} are also parallelizable if there is no cross-shard merging // in descendent nodes in the AST. This optimization is currently avoided for simplicity. -func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (SampleExpr, error) { +func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr, r *ShardRecorder) (SampleExpr, error) { // if this AST contains unshardable operations, don't shard this at this level, // but attempt to shard a child node. if shardable := isShardable(expr.Operations()); !shardable { - subMapped, err := m.Map(expr.left) + subMapped, err := m.Map(expr.left, r) if err != nil { return nil, err } @@ -119,7 +219,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp case OpTypeSum: // sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...) return &vectorAggregationExpr{ - left: m.mapSampleExpr(expr), + left: m.mapSampleExpr(expr, r), grouping: expr.grouping, params: expr.params, operation: expr.operation, @@ -131,7 +231,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp left: expr.left, grouping: expr.grouping, operation: OpTypeSum, - }) + }, r) if err != nil { return nil, err } @@ -139,7 +239,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp left: expr.left, grouping: expr.grouping, operation: OpTypeCount, - }) + }, r) if err != nil { return nil, err } @@ -152,7 +252,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp case OpTypeCount: // count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...) - sharded := m.mapSampleExpr(expr) + sharded := m.mapSampleExpr(expr, r) return &vectorAggregationExpr{ left: sharded, grouping: expr.grouping, @@ -169,12 +269,12 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr) (Samp } } -func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr) SampleExpr { +func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr, r *ShardRecorder) SampleExpr { switch expr.operation { case OpTypeCountOverTime, OpTypeRate: // count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... // rate(x) -> rate(x, shard=1) ++ rate(x, shard=2)... - return m.mapSampleExpr(expr) + return m.mapSampleExpr(expr, r) default: return expr } diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index edd779f4cb0ec..656d7f2c64bd3 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -52,7 +52,7 @@ func TestStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - m, err := NewShardMapper(2) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { @@ -113,14 +113,14 @@ func TestMapSampleExpr(t *testing.T) { }, } { t.Run(tc.in.String(), func(t *testing.T) { - require.Equal(t, tc.out, m.mapSampleExpr(tc.in)) + require.Equal(t, tc.out, m.mapSampleExpr(tc.in, nilMetrics.ShardRecorder())) }) } } func TestMappingStrings(t *testing.T) { - m, err := NewShardMapper(2) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { in string @@ -159,7 +159,7 @@ func TestMappingStrings(t *testing.T) { ast, err := ParseExpr(tc.in) require.Nil(t, err) - mapped, err := m.Map(ast) + mapped, err := m.Map(ast, nilMetrics.ShardRecorder()) require.Nil(t, err) require.Equal(t, tc.out, mapped.String()) @@ -169,7 +169,7 @@ func TestMappingStrings(t *testing.T) { } func TestMapping(t *testing.T) { - m, err := NewShardMapper(2) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { @@ -938,7 +938,7 @@ func TestMapping(t *testing.T) { ast, err := ParseExpr(tc.in) require.Equal(t, tc.err, err) - mapped, err := m.Map(ast) + mapped, err := m.Map(ast, nilMetrics.ShardRecorder()) require.Equal(t, tc.err, err) require.Equal(t, tc.expr.String(), mapped.String()) diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 0dc7c6204aa33..d13e7c93a6728 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -8,6 +8,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/promql" "github.com/grafana/loki/pkg/loghttp" @@ -23,6 +24,7 @@ func NewQueryShardMiddleware( confs queryrange.ShardingConfigs, minShardingLookback time.Duration, metrics *queryrange.InstrumentMiddlewareMetrics, + r prometheus.Registerer, ) queryrange.Middleware { noshards := !hasShards(confs) @@ -36,8 +38,10 @@ func NewQueryShardMiddleware( return queryrange.PassthroughMiddleware } + shardingMetrics := logql.NewShardingMetrics(r) + mapperware := queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { - return newASTMapperware(confs, next, logger) + return newASTMapperware(confs, next, logger, shardingMetrics) }) return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { @@ -54,19 +58,26 @@ func NewQueryShardMiddleware( } -func newASTMapperware(confs queryrange.ShardingConfigs, next queryrange.Handler, logger log.Logger) *astMapperware { +func newASTMapperware( + confs queryrange.ShardingConfigs, + next queryrange.Handler, + logger log.Logger, + metrics *logql.ShardingMetrics, +) *astMapperware { return &astMapperware{ - confs: confs, - logger: log.With(logger, "middleware", "QueryShard.astMapperware"), - next: next, + confs: confs, + logger: log.With(logger, "middleware", "QueryShard.astMapperware"), + next: next, + metrics: metrics, } } type astMapperware struct { - confs queryrange.ShardingConfigs - logger log.Logger - next queryrange.Handler + confs queryrange.ShardingConfigs + logger log.Logger + next queryrange.Handler + metrics *logql.ShardingMetrics } func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) { @@ -77,7 +88,7 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra return ast.next.Do(ctx, r) } - ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}) + ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}, ast.metrics) if err != nil { level.Warn(ast.logger).Log("err", err.Error(), "msg", "failed to create sharded engine") diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index a3a038aa98623..8554ef510c5c6 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -54,11 +54,11 @@ func NewTripperware( instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer) retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer) - metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics) + metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics, registerer) if err != nil { return nil, nil, err } - logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics) + logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, registerer) if err != nil { return nil, nil, err } @@ -136,6 +136,7 @@ func NewLogFilterTripperware( codec queryrange.Codec, instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, + registerer prometheus.Registerer, ) (frontend.Tripperware, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.SplitQueriesByInterval != 0 { @@ -152,6 +153,7 @@ func NewLogFilterTripperware( schema.Configs, minShardingLookback, instrumentMetrics, // instrumentation is included in the sharding middleware + registerer, ), ) } @@ -179,6 +181,7 @@ func NewMetricTripperware( extractor queryrange.Extractor, instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, + registerer prometheus.Registerer, ) (frontend.Tripperware, Stopper, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.AlignQueriesWithStep { @@ -231,6 +234,7 @@ func NewMetricTripperware( schema.Configs, minShardingLookback, instrumentMetrics, // instrumentation is included in the sharding middleware + registerer, ), ) } From 4996c3a30f1cb83bcadc75ff9a48628136313e8b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 16 Apr 2020 13:17:58 -0400 Subject: [PATCH 39/67] log/span injection into sharded engine --- pkg/logql/sharding.go | 7 +++++-- pkg/logql/sharding_test.go | 3 ++- pkg/logql/shardmapper.go | 19 ++++++++++++++++--- pkg/logql/shardmapper_test.go | 7 ++++--- pkg/querier/queryrange/downstreamer.go | 1 + pkg/querier/queryrange/querysharding.go | 6 +++++- 6 files changed, 33 insertions(+), 10 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index c7320d5060d56..128dacd5e5996 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -7,6 +7,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/promql" "github.com/grafana/loki/pkg/iter" @@ -277,11 +278,12 @@ type shardedEngine struct { mapper ShardMapper evaluator Evaluator metrics *ShardingMetrics + logger log.Logger } -func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer, metrics *ShardingMetrics) (Engine, error) { +func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer, metrics *ShardingMetrics, log log.Logger) (Engine, error) { opts.applyDefault() - mapper, err := NewShardMapper(shards, metrics) + mapper, err := NewShardMapper(shards, metrics, log) if err != nil { return nil, err } @@ -291,6 +293,7 @@ func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer, me mapper: mapper, evaluator: NewDownstreamEvaluator(downstreamer), metrics: metrics, + logger: log, }, nil } diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 0aa56ccd38187..6828b60bb60fb 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/require" @@ -56,7 +57,7 @@ func TestMappingEquivalence(t *testing.T) { opts := EngineOpts{} regular := NewEngine(opts, q) - sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}, nilMetrics) + sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}, nilMetrics, log.NewNopLogger()) require.Nil(t, err) t.Run(tc.query, func(t *testing.T) { diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 0c62bfad3825f..36e757bf60e52 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -5,6 +5,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/util" + "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -82,19 +83,21 @@ func badASTMapping(expected string, got Expr) error { return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) } -func NewShardMapper(shards int, metrics *ShardingMetrics) (ShardMapper, error) { +func NewShardMapper(shards int, metrics *ShardingMetrics, log log.Logger) (ShardMapper, error) { if shards < 2 { return ShardMapper{}, fmt.Errorf("Cannot create ShardMapper with <2 shards. Received %d", shards) } return ShardMapper{ shards: shards, metrics: metrics, + logger: log, }, nil } type ShardMapper struct { shards int metrics *ShardingMetrics + logger log.Logger } func (m ShardMapper) Parse(query string) (Expr, error) { @@ -104,16 +107,26 @@ func (m ShardMapper) Parse(query string) (Expr, error) { } recorder := m.metrics.ShardRecorder() - defer recorder.Finish() mapped, err := m.Map(parsed, recorder) if err != nil { + level.Warn(m.logger).Log("msg", "failed mapping AST", "err", err.Error()) m.metrics.parsed.WithLabelValues(FailureKey).Inc() - } else if parsed.String() == mapped.String() { + return nil, err + } + + mappedStr := mapped.String() + originalStr := parsed.String() + noop := originalStr == mappedStr + if noop { m.metrics.parsed.WithLabelValues(NoopKey).Inc() } else { m.metrics.parsed.WithLabelValues(SuccessKey).Inc() } + + recorder.Finish() + level.Debug(m.logger).Log("mapped", !noop, "query", mappedStr, "original", originalStr) + return mapped, err } diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 656d7f2c64bd3..5da086544980d 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -5,6 +5,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" ) @@ -52,7 +53,7 @@ func TestStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics) + m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) require.Nil(t, err) for _, tc := range []struct { @@ -120,7 +121,7 @@ func TestMapSampleExpr(t *testing.T) { } func TestMappingStrings(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics) + m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) require.Nil(t, err) for _, tc := range []struct { in string @@ -169,7 +170,7 @@ func TestMappingStrings(t *testing.T) { } func TestMapping(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics) + m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) require.Nil(t, err) for _, tc := range []struct { diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index eca128ef0c763..6f10b11929297 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -38,6 +38,7 @@ func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shar req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()) return QuerierFunc(func(ctx context.Context) (logql.Result, error) { + res, err := h.next.Do(ctx, req) if err != nil { return logql.Result{}, err diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index d13e7c93a6728..033a7ea0365fa 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -6,6 +6,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/prometheus/client_golang/prometheus" @@ -88,7 +89,10 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra return ast.next.Do(ctx, r) } - ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}, ast.metrics) + shardedLog, ctx := spanlogger.New(ctx, "shardedEngine") + defer shardedLog.Finish() + + ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}, ast.metrics, shardedLog) if err != nil { level.Warn(ast.logger).Log("err", err.Error(), "msg", "failed to create sharded engine") From 41881162b16774f2b95c2d2ca01dc9b5c88850ca Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 16 Apr 2020 14:11:15 -0400 Subject: [PATCH 40/67] sharding metrics avoids multiple instantiation --- pkg/querier/queryrange/querysharding.go | 11 ++++------- pkg/querier/queryrange/roundtrip.go | 13 +++++++------ 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 033a7ea0365fa..d8ed92585df0b 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -9,7 +9,6 @@ import ( "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/promql" "github.com/grafana/loki/pkg/loghttp" @@ -24,8 +23,8 @@ func NewQueryShardMiddleware( logger log.Logger, confs queryrange.ShardingConfigs, minShardingLookback time.Duration, - metrics *queryrange.InstrumentMiddlewareMetrics, - r prometheus.Registerer, + middlewareMetrics *queryrange.InstrumentMiddlewareMetrics, + shardingMetrics *logql.ShardingMetrics, ) queryrange.Middleware { noshards := !hasShards(confs) @@ -39,8 +38,6 @@ func NewQueryShardMiddleware( return queryrange.PassthroughMiddleware } - shardingMetrics := logql.NewShardingMetrics(r) - mapperware := queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { return newASTMapperware(confs, next, logger, shardingMetrics) }) @@ -49,11 +46,11 @@ func NewQueryShardMiddleware( return &shardSplitter{ MinShardingLookback: minShardingLookback, shardingware: queryrange.MergeMiddlewares( - queryrange.InstrumentMiddleware("shardingware", metrics), + queryrange.InstrumentMiddleware("shardingware", middlewareMetrics), mapperware, ).Wrap(next), now: time.Now, - next: queryrange.InstrumentMiddleware("sharding-bypass", metrics).Wrap(next), + next: queryrange.InstrumentMiddleware("sharding-bypass", middlewareMetrics).Wrap(next), } }) diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 8554ef510c5c6..611970f986b38 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -53,12 +53,13 @@ func NewTripperware( instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer) retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer) + shardingMetrics := logql.NewShardingMetrics(registerer) - metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics, registerer) + metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics, shardingMetrics) if err != nil { return nil, nil, err } - logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, registerer) + logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, shardingMetrics) if err != nil { return nil, nil, err } @@ -136,7 +137,7 @@ func NewLogFilterTripperware( codec queryrange.Codec, instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, - registerer prometheus.Registerer, + shardingMetrics *logql.ShardingMetrics, ) (frontend.Tripperware, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.SplitQueriesByInterval != 0 { @@ -153,7 +154,7 @@ func NewLogFilterTripperware( schema.Configs, minShardingLookback, instrumentMetrics, // instrumentation is included in the sharding middleware - registerer, + shardingMetrics, ), ) } @@ -181,7 +182,7 @@ func NewMetricTripperware( extractor queryrange.Extractor, instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, - registerer prometheus.Registerer, + shardingMetrics *logql.ShardingMetrics, ) (frontend.Tripperware, Stopper, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.AlignQueriesWithStep { @@ -234,7 +235,7 @@ func NewMetricTripperware( schema.Configs, minShardingLookback, instrumentMetrics, // instrumentation is included in the sharding middleware - registerer, + shardingMetrics, ), ) } From 77d09d12b8cb2bdfd73cea031880cd4e590470f6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 16 Apr 2020 14:33:53 -0400 Subject: [PATCH 41/67] downstreamhandler tracing --- pkg/querier/queryrange/downstreamer.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index 6f10b11929297..bf048c8904f97 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -6,6 +6,8 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/cortexproject/cortex/pkg/util/spanlogger" + "github.com/go-kit/kit/log/level" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql" @@ -35,10 +37,14 @@ func ParamsToLokiRequest(params logql.Params) *LokiRequest { } func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shards logql.Shards) (logql.Query, error) { - req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()) + req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()).(*LokiRequest) return QuerierFunc(func(ctx context.Context) (logql.Result, error) { + logger, ctx := spanlogger.New(ctx, "DownstreamHandler") + defer logger.Finish() + level.Debug(logger).Log("shards", req.Shards, "query", req.Query) + res, err := h.next.Do(ctx, req) if err != nil { return logql.Result{}, err From c0897f8dc2ee6b1100ae5558071e123f596fc483 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 17 Apr 2020 10:53:19 -0400 Subject: [PATCH 42/67] sharding parameterized libsonnet --- production/ksonnet/loki/config.libsonnet | 47 +++++++++++++++++-- production/ksonnet/loki/querier.libsonnet | 5 +- .../ksonnet/loki/query-frontend.libsonnet | 14 ++++-- 3 files changed, 56 insertions(+), 10 deletions(-) diff --git a/production/ksonnet/loki/config.libsonnet b/production/ksonnet/loki/config.libsonnet index 24faefb3b2320..9050e3b37fcdc 100644 --- a/production/ksonnet/loki/config.libsonnet +++ b/production/ksonnet/loki/config.libsonnet @@ -7,9 +7,33 @@ replication_factor: 3, memcached_replicas: 3, - querierConcurrency: 16, grpc_server_max_msg_size: 100 << 20, // 100MB + + // The expectation is that if sharding is enabled, we can force more (smaller) + // queries on the queriers. However this can't be extended too far because most queries + // concern recent (ingester) data, which isn't sharded. Therefore, we must strike a balance + // which allows us to process more sharded queries in parallel when requested, but not overload + // queriers during normal queries. + querier: { + replicas: if $._config.queryFrontend.sharded_queries_enabled then 6 else 3, + concurrency: 16, + }, + + queryFrontend: { + replicas: 2, + shard_factor: 16, // v10 schema shard factor + sharded_queries_enabled: false, + // Queries can technically be sharded an arbitrary number of times. Thus query_split_factor is used + // as a coefficient to multiply the frontend tenant queues by. The idea is that this + // yields a bit of headroom so tenant queues aren't underprovisioned. Therefore the split factor + // should represent the highest reasonable split factor for a query. If too low, a long query + // (i.e. 30d) with a high split factor (i.e. 5) would result in + // (day_splits * shard_factor * split_factor) or 30 * 16 * 5 = 2400 sharded queries, which may be + // more than the max queue size and thus would always error. + query_split_factor:: 3, + }, + // Default to GCS and Bigtable for chunk and index store storage_backend: 'bigtable,gcs', @@ -104,12 +128,19 @@ }, frontend: { compress_responses: true, + } + if $._config.queryFrontend.sharded_queries_enabled then { + // In process tenant queues on frontends. We divide by the number of frontends; + // 2 in this case in order to apply the global limit in aggregate. + // This is basically base * shard_factor * query_split_factor / num_frontends where + max_outstanding_per_tenant: std.floor(200 * $._config.queryFrontend.shard_factor * $._config.queryFrontend.query_split_factor / $._config.queryFrontend.replicas), + } + else { max_outstanding_per_tenant: 200, }, frontend_worker: { frontend_address: 'query-frontend.%s.svc.cluster.local:9095' % $._config.namespace, // Limit to N/2 worker threads per frontend, as we have two frontends. - parallelism: $._config.querierConcurrency / 2, + parallelism: std.floor($._config.querier.concurrency / $._config.queryFrontend.replicas), grpc_client_config: { max_send_msg_size: $._config.grpc_server_max_msg_size, }, @@ -131,11 +162,19 @@ max_idle_conns: 16, }, }, - }, + } + + if $._config.queryFrontend.sharded_queries_enabled then { + parallelise_shardable_queries: 'true', + } + else {}, + }, + querier: { + query_ingesters_within: '2h', // twice the max-chunk age (1h default) for safety buffer }, limits_config: { enforce_metric_name: false, - max_query_parallelism: 32, + // align middleware parallelism with shard factor to optimize one-legged sharded queries. + max_query_parallelism: $._config.queryFrontend.shard_factor, reject_old_samples: true, reject_old_samples_max_age: '168h', max_query_length: '12000h', // 500 days diff --git a/production/ksonnet/loki/querier.libsonnet b/production/ksonnet/loki/querier.libsonnet index a63f063f6a84b..37db70ad71de6 100644 --- a/production/ksonnet/loki/querier.libsonnet +++ b/production/ksonnet/loki/querier.libsonnet @@ -13,12 +13,13 @@ container.mixin.readinessProbe.httpGet.withPath('/ready') + container.mixin.readinessProbe.httpGet.withPort($._config.http_listen_port) + container.mixin.readinessProbe.withInitialDelaySeconds(15) + - container.mixin.readinessProbe.withTimeoutSeconds(1), + container.mixin.readinessProbe.withTimeoutSeconds(1) + + $.util.resourcesRequests('4', '2Gi'), local deployment = $.apps.v1.deployment, querier_deployment: - deployment.new('querier', 3, [$.querier_container]) + + deployment.new('querier', $._config.querier.replicas, [$.querier_container]) + $.config_hash_mixin + $.util.configVolumeMount('loki', '/etc/loki/config') + $.util.configVolumeMount('overrides', '/etc/loki/overrides') + diff --git a/production/ksonnet/loki/query-frontend.libsonnet b/production/ksonnet/loki/query-frontend.libsonnet index ce983c7a81278..a74e1dabd6e2a 100644 --- a/production/ksonnet/loki/query-frontend.libsonnet +++ b/production/ksonnet/loki/query-frontend.libsonnet @@ -11,14 +11,20 @@ container.new('query-frontend', $._images.query_frontend) + container.withPorts($.util.defaultPorts) + container.withArgsMixin($.util.mapToFlags($.query_frontend_args)) + - $.util.resourcesRequests('2', '600Mi') + - $.util.resourcesLimits(null, '1200Mi') + - $.jaeger_mixin, + $.jaeger_mixin + + if $._config.queryFrontend.sharded_queries_enabled then + $.util.resourcesRequests('2', '2Gi') + + $.util.resourcesLimits(null, '6Gi') + + container.withEnvMap({ + JAEGER_REPORTER_MAX_QUEUE_SIZE: '5000', + }) + else $.util.resourcesRequests('2', '600Mi') + + $.util.resourcesLimits(null, '1200Mi'), local deployment = $.apps.v1.deployment, query_frontend_deployment: - deployment.new('query-frontend', 2, [$.query_frontend_container]) + + deployment.new('query-frontend', $._config.queryFrontend.replicas, [$.query_frontend_container]) + $.config_hash_mixin + $.util.configVolumeMount('loki', '/etc/loki/config') + $.util.configVolumeMount('overrides', '/etc/loki/overrides') + From 430af9609291acf82a2ef04e90feddc8bd50e27b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 20 Apr 2020 15:48:02 -0400 Subject: [PATCH 43/67] removes querier replicas --- production/ksonnet/loki/config.libsonnet | 6 ------ production/ksonnet/loki/querier.libsonnet | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/production/ksonnet/loki/config.libsonnet b/production/ksonnet/loki/config.libsonnet index 9050e3b37fcdc..b8c9274868c19 100644 --- a/production/ksonnet/loki/config.libsonnet +++ b/production/ksonnet/loki/config.libsonnet @@ -10,13 +10,7 @@ grpc_server_max_msg_size: 100 << 20, // 100MB - // The expectation is that if sharding is enabled, we can force more (smaller) - // queries on the queriers. However this can't be extended too far because most queries - // concern recent (ingester) data, which isn't sharded. Therefore, we must strike a balance - // which allows us to process more sharded queries in parallel when requested, but not overload - // queriers during normal queries. querier: { - replicas: if $._config.queryFrontend.sharded_queries_enabled then 6 else 3, concurrency: 16, }, diff --git a/production/ksonnet/loki/querier.libsonnet b/production/ksonnet/loki/querier.libsonnet index 37db70ad71de6..6641945a34f16 100644 --- a/production/ksonnet/loki/querier.libsonnet +++ b/production/ksonnet/loki/querier.libsonnet @@ -19,7 +19,7 @@ local deployment = $.apps.v1.deployment, querier_deployment: - deployment.new('querier', $._config.querier.replicas, [$.querier_container]) + + deployment.new('querier', 3, [$.querier_container]) + $.config_hash_mixin + $.util.configVolumeMount('loki', '/etc/loki/config') + $.util.configVolumeMount('overrides', '/etc/loki/overrides') + From 539e4323cddb4393d39a81c74384b375244ea672 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 21 Apr 2020 11:09:12 -0400 Subject: [PATCH 44/67] default 32 concurrency for workers --- production/ksonnet/loki/config.libsonnet | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/ksonnet/loki/config.libsonnet b/production/ksonnet/loki/config.libsonnet index b8c9274868c19..fa50fcc75b64c 100644 --- a/production/ksonnet/loki/config.libsonnet +++ b/production/ksonnet/loki/config.libsonnet @@ -11,7 +11,7 @@ querier: { - concurrency: 16, + concurrency: 32, }, queryFrontend: { From 54334d8034652fc77b3e0d61b8e543cc85f1558f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 21 Apr 2020 11:16:16 -0400 Subject: [PATCH 45/67] jsonnet correct level override --- production/ksonnet/loki/config.libsonnet | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/production/ksonnet/loki/config.libsonnet b/production/ksonnet/loki/config.libsonnet index fa50fcc75b64c..2ced53ca78096 100644 --- a/production/ksonnet/loki/config.libsonnet +++ b/production/ksonnet/loki/config.libsonnet @@ -156,12 +156,10 @@ max_idle_conns: 16, }, }, - } + - if $._config.queryFrontend.sharded_queries_enabled then { - parallelise_shardable_queries: 'true', - } - else {}, - }, + }, + } + if $._config.queryFrontend.sharded_queries_enabled then { + parallelise_shardable_queries: 'true', + } else {}, querier: { query_ingesters_within: '2h', // twice the max-chunk age (1h default) for safety buffer }, From e07c2d86d831fe6c67bd9b68c2c54259c76c3046 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 21 Apr 2020 11:19:34 -0400 Subject: [PATCH 46/67] unquote true in yaml --- production/ksonnet/loki/config.libsonnet | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/production/ksonnet/loki/config.libsonnet b/production/ksonnet/loki/config.libsonnet index 2ced53ca78096..ba4964947f371 100644 --- a/production/ksonnet/loki/config.libsonnet +++ b/production/ksonnet/loki/config.libsonnet @@ -158,7 +158,7 @@ }, }, } + if $._config.queryFrontend.sharded_queries_enabled then { - parallelise_shardable_queries: 'true', + parallelise_shardable_queries: true, } else {}, querier: { query_ingesters_within: '2h', // twice the max-chunk age (1h default) for safety buffer From 5ae035e7373711b5bb5fc442a943f3153a444a6c Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 23 Apr 2020 09:44:16 -0400 Subject: [PATCH 47/67] lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator --- pkg/logql/sharding.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 128dacd5e5996..4bd33cbccb122 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -160,11 +160,8 @@ func (ev *DownstreamEvaluator) StepEvaluator( return ConcatEvaluator(xs) - case *vectorAggregationExpr, *binOpExpr: - return ev.defaultEvaluator.StepEvaluator(ctx, nextEv, e, params) - default: - return nil, EvaluatorUnsupportedType(expr, ev) + return ev.defaultEvaluator.StepEvaluator(ctx, nextEv, e, params) } } @@ -267,7 +264,7 @@ func ResultStepEvaluator(res Result, params Params) (StepEvaluator, error) { func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { streams, ok := res.Data.(Streams) if !ok { - return nil, fmt.Errorf("Unexpected type (%s) for ResultIterator; expected %s", res.Data.Type(), ValueTypeStreams) + return nil, fmt.Errorf("unexpected type (%s) for ResultIterator; expected %s", res.Data.Type(), ValueTypeStreams) } return iter.NewStreamsIterator(context.Background(), streams, params.Direction()), nil From 1d6e513e77b2da01431cf3a5cce4348c30456604 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 23 Apr 2020 09:45:48 -0400 Subject: [PATCH 48/67] makes shardRecorder private --- pkg/logql/shardmapper.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 36e757bf60e52..9c8948069ccbe 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -48,31 +48,31 @@ func NewShardingMetrics(registerer prometheus.Registerer) *ShardingMetrics { } } -// ShardRecorder constructs a recorder using the underlying metrics. -func (m *ShardingMetrics) ShardRecorder() *ShardRecorder { - return &ShardRecorder{ +// shardRecorder constructs a recorder using the underlying metrics. +func (m *ShardingMetrics) ShardRecorder() *shardRecorder { + return &shardRecorder{ ShardingMetrics: m, } } -// ShardRecorder wraps a vector & histogram, providing an easy way to increment sharding counts. +// shardRecorder wraps a vector & histogram, providing an easy way to increment sharding counts. // and unify them into histogram entries. // NOT SAFE FOR CONCURRENT USE! We avoid introducing mutex locking here // because AST mapping is single threaded. -type ShardRecorder struct { +type shardRecorder struct { done bool total int *ShardingMetrics } // Add increments both the shard count and tracks it for the eventual histogram entry. -func (r *ShardRecorder) Add(x int, key string) { +func (r *shardRecorder) Add(x int, key string) { r.total += x r.shards.WithLabelValues(key).Add(float64(x)) } // Finish idemptotently records a histogram entry with the total shard factor. -func (r *ShardRecorder) Finish() { +func (r *shardRecorder) Finish() { if !r.done { r.done = true r.shardFactor.Observe(float64(r.total)) @@ -130,7 +130,7 @@ func (m ShardMapper) Parse(query string) (Expr, error) { return mapped, err } -func (m ShardMapper) Map(expr Expr, r *ShardRecorder) (Expr, error) { +func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) { switch e := expr.(type) { case *literalExpr: return e, nil @@ -165,7 +165,7 @@ func (m ShardMapper) Map(expr Expr, r *ShardRecorder) (Expr, error) { } } -func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *ShardRecorder) LogSelectorExpr { +func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *shardRecorder) LogSelectorExpr { var head *ConcatLogSelectorExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatLogSelectorExpr{ @@ -184,7 +184,7 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *ShardRecorder) return head } -func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *ShardRecorder) SampleExpr { +func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *shardRecorder) SampleExpr { var head *ConcatSampleExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatSampleExpr{ @@ -205,7 +205,7 @@ func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *ShardRecorder) SampleExpr // technically, std{dev,var} are also parallelizable if there is no cross-shard merging // in descendent nodes in the AST. This optimization is currently avoided for simplicity. -func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr, r *ShardRecorder) (SampleExpr, error) { +func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr, r *shardRecorder) (SampleExpr, error) { // if this AST contains unshardable operations, don't shard this at this level, // but attempt to shard a child node. @@ -282,7 +282,7 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *vectorAggregationExpr, r *Sh } } -func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr, r *ShardRecorder) SampleExpr { +func (m ShardMapper) mapRangeAggregationExpr(expr *rangeAggregationExpr, r *shardRecorder) SampleExpr { switch expr.operation { case OpTypeCountOverTime, OpTypeRate: // count_over_time(x) -> count_over_time(x, shard=1) ++ count_over_time(x, shard=2)... From 42438e6b75d72bac2e0c755860db95a4e5291e79 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 23 Apr 2020 09:46:14 -0400 Subject: [PATCH 49/67] logs query on failed parse --- pkg/logql/shardmapper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 9c8948069ccbe..bedf35ade1711 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -110,7 +110,7 @@ func (m ShardMapper) Parse(query string) (Expr, error) { mapped, err := m.Map(parsed, recorder) if err != nil { - level.Warn(m.logger).Log("msg", "failed mapping AST", "err", err.Error()) + level.Warn(m.logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query) m.metrics.parsed.WithLabelValues(FailureKey).Inc() return nil, err } From 2fa2900c99f75d79005086e54b452b0817d515de Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 23 Apr 2020 11:26:08 -0400 Subject: [PATCH 50/67] refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface --- pkg/logcli/query/query.go | 5 +-- pkg/logql/engine.go | 46 ++++++++---------------- pkg/logql/engine_test.go | 12 +++---- pkg/logql/sharding.go | 47 +++++++++++++------------ pkg/logql/sharding_test.go | 8 ++--- pkg/logql/shardmapper.go | 23 +++++------- pkg/logql/shardmapper_test.go | 13 ++++--- pkg/logql/test_utils.go | 4 +-- pkg/querier/http.go | 6 ++-- pkg/querier/querier.go | 2 +- pkg/querier/queryrange/querysharding.go | 25 +++++-------- 11 files changed, 81 insertions(+), 110 deletions(-) diff --git a/pkg/logcli/query/query.go b/pkg/logcli/query/query.go index be0484cd76a5f..50d6ccbcc3430 100644 --- a/pkg/logcli/query/query.go +++ b/pkg/logcli/query/query.go @@ -117,8 +117,9 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string eng := logql.NewEngine(conf.Querier.Engine, querier) var query logql.Query + if q.isInstant() { - query = eng.NewInstantQuery(logql.NewLiteralParams( + query = eng.Query(logql.NewLiteralParams( q.QueryString, q.Start, q.Start, @@ -128,7 +129,7 @@ func (q *Query) DoLocalQuery(out output.LogOutput, statistics bool, orgID string nil, )) } else { - query = eng.NewRangeQuery(logql.NewLiteralParams( + query = eng.Query(logql.NewLiteralParams( q.QueryString, q.Start, q.End, diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index b470efd98f8d8..751158117ea68 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -73,27 +73,31 @@ func (opts *EngineOpts) applyDefault() { } } -// Engine interface used to construct queries -type Engine interface { - NewRangeQuery(Params) Query - NewInstantQuery(Params) Query -} - -// engine is the LogQL engine. -type engine struct { +// Engine is the LogQL engine. +type Engine struct { timeout time.Duration evaluator Evaluator } -// NewEngine creates a new LogQL engine. -func NewEngine(opts EngineOpts, q Querier) Engine { +// NewEngine creates a new LogQL Engine. +func NewEngine(opts EngineOpts, q Querier) *Engine { opts.applyDefault() - return &engine{ + return &Engine{ timeout: opts.Timeout, evaluator: NewDefaultEvaluator(q, opts.MaxLookBackPeriod), } } +// Query creates a new LogQL query. Instant/Range type is derived from the parameters. +func (ng *Engine) Query(params Params) Query { + return &query{ + timeout: ng.timeout, + params: params, + evaluator: ng.evaluator, + parse: ParseExpr, + } +} + // Query is a LogQL query to be executed. type Query interface { // Exec processes the query. @@ -168,26 +172,6 @@ func (q *query) Eval(ctx context.Context) (promql.Value, error) { } } -// NewRangeQuery creates a new LogQL range query. -func (ng *engine) NewRangeQuery(params Params) Query { - return &query{ - timeout: ng.timeout, - params: params, - evaluator: ng.evaluator, - parse: ParseExpr, - } -} - -// NewInstantQuery creates a new LogQL instant query. -func (ng *engine) NewInstantQuery(params Params) Query { - return &query{ - timeout: ng.timeout, - params: params, - evaluator: ng.evaluator, - parse: ParseExpr, - } -} - // evalSample evaluate a sampleExpr func (q *query) evalSample(ctx context.Context, expr SampleExpr) (promql.Value, error) { if lit, ok := expr.(*literalExpr); ok { diff --git a/pkg/logql/engine_test.go b/pkg/logql/engine_test.go index f35da48cdc73c..ce54d700aaaff 100644 --- a/pkg/logql/engine_test.go +++ b/pkg/logql/engine_test.go @@ -20,7 +20,7 @@ import ( var testSize = int64(300) -func TestEngine_NewInstantQuery(t *testing.T) { +func TestEngine_InstantQuery(t *testing.T) { t.Parallel() for _, test := range []struct { qs string @@ -325,7 +325,7 @@ func TestEngine_NewInstantQuery(t *testing.T) { t.Parallel() eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) - q := eng.NewInstantQuery(LiteralParams{ + q := eng.Query(LiteralParams{ qs: test.qs, start: test.ts, end: test.ts, @@ -341,7 +341,7 @@ func TestEngine_NewInstantQuery(t *testing.T) { } } -func TestEngine_NewRangeQuery(t *testing.T) { +func TestEngine_RangeQuery(t *testing.T) { t.Parallel() for _, test := range []struct { qs string @@ -1082,7 +1082,7 @@ func TestEngine_NewRangeQuery(t *testing.T) { eng := NewEngine(EngineOpts{}, newQuerierRecorder(test.streams, test.params)) - q := eng.NewRangeQuery(LiteralParams{ + q := eng.Query(LiteralParams{ qs: test.qs, start: test.start, end: test.end, @@ -1106,7 +1106,7 @@ func TestEngine_Stats(t *testing.T) { return iter.NoopIterator, nil })) - q := eng.NewInstantQuery(LiteralParams{ + q := eng.Query(LiteralParams{ qs: `{foo="bar"}`, start: time.Now(), end: time.Now(), @@ -1168,7 +1168,7 @@ func benchmarkRangeQuery(testsize int64, b *testing.B) { {`bottomk(2,rate(({app=~"foo|bar"} |~".+bar")[1m]))`, logproto.FORWARD}, {`bottomk(3,rate(({app=~"foo|bar"} |~".+bar")[1m])) without (app)`, logproto.FORWARD}, } { - q := eng.NewRangeQuery(LiteralParams{ + q := eng.Query(LiteralParams{ qs: test.qs, start: start, end: end, diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 4bd33cbccb122..a07f30acc11a2 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -8,6 +8,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/go-kit/kit/log" + "github.com/go-kit/kit/log/level" "github.com/prometheus/prometheus/promql" "github.com/grafana/loki/pkg/iter" @@ -270,44 +271,44 @@ func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { } -type shardedEngine struct { +type ShardedEngine struct { timeout time.Duration - mapper ShardMapper evaluator Evaluator metrics *ShardingMetrics - logger log.Logger } -func NewShardedEngine(opts EngineOpts, shards int, downstreamer Downstreamer, metrics *ShardingMetrics, log log.Logger) (Engine, error) { +func NewShardedEngine(opts EngineOpts, downstreamer Downstreamer, metrics *ShardingMetrics) *ShardedEngine { opts.applyDefault() - mapper, err := NewShardMapper(shards, metrics, log) - if err != nil { - return nil, err - } - - return &shardedEngine{ + return &ShardedEngine{ timeout: opts.Timeout, - mapper: mapper, evaluator: NewDownstreamEvaluator(downstreamer), metrics: metrics, - logger: log, - }, nil + } } -func (ng *shardedEngine) query(p Params) Query { +func (ng *ShardedEngine) Query(p Params, shards int, logger log.Logger) Query { + if logger == nil { + logger = log.NewNopLogger() + } + return &query{ timeout: ng.timeout, params: p, evaluator: ng.evaluator, - parse: ng.mapper.Parse, - } -} - -func (ng *shardedEngine) NewRangeQuery(p Params) Query { - return ng.query(p) -} + parse: func(query string) (Expr, error) { + mapper, err := NewShardMapper(shards, ng.metrics) + if err != nil { + return nil, err + } + noop, parsed, err := mapper.Parse(query) + if err != nil { + level.Warn(logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query) + return nil, err + } -func (ng *shardedEngine) NewInstantQuery(p Params) Query { - return ng.query(p) + level.Debug(logger).Log("no-op", noop, "mapped", parsed.String()) + return parsed, nil + }, + } } diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 6828b60bb60fb..c6e30c644dbd9 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -6,7 +6,6 @@ import ( "testing" "time" - "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/require" @@ -57,8 +56,7 @@ func TestMappingEquivalence(t *testing.T) { opts := EngineOpts{} regular := NewEngine(opts, q) - sharded, err := NewShardedEngine(opts, shards, MockDownstreamer{regular}, nilMetrics, log.NewNopLogger()) - require.Nil(t, err) + sharded := NewShardedEngine(opts, MockDownstreamer{regular}, nilMetrics) t.Run(tc.query, func(t *testing.T) { params := NewLiteralParams( @@ -70,8 +68,8 @@ func TestMappingEquivalence(t *testing.T) { uint32(limit), nil, ) - qry := regular.NewRangeQuery(params) - shardedQry := sharded.NewRangeQuery(params) + qry := regular.Query(params) + shardedQry := sharded.Query(params, shards, nil) res, err := qry.Exec(context.Background()) require.Nil(t, err) diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index bedf35ade1711..4692f9f229881 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -5,7 +5,6 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/util" - "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -49,7 +48,7 @@ func NewShardingMetrics(registerer prometheus.Registerer) *ShardingMetrics { } // shardRecorder constructs a recorder using the underlying metrics. -func (m *ShardingMetrics) ShardRecorder() *shardRecorder { +func (m *ShardingMetrics) shardRecorder() *shardRecorder { return &shardRecorder{ ShardingMetrics: m, } @@ -83,51 +82,47 @@ func badASTMapping(expected string, got Expr) error { return fmt.Errorf("Bad AST mapping: expected one type (%s), but got (%T)", expected, got) } -func NewShardMapper(shards int, metrics *ShardingMetrics, log log.Logger) (ShardMapper, error) { +func NewShardMapper(shards int, metrics *ShardingMetrics) (ShardMapper, error) { if shards < 2 { return ShardMapper{}, fmt.Errorf("Cannot create ShardMapper with <2 shards. Received %d", shards) } return ShardMapper{ shards: shards, metrics: metrics, - logger: log, }, nil } type ShardMapper struct { shards int metrics *ShardingMetrics - logger log.Logger } -func (m ShardMapper) Parse(query string) (Expr, error) { +func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) { parsed, err := ParseExpr(query) if err != nil { - return nil, err + return false, nil, err } - recorder := m.metrics.ShardRecorder() + recorder := m.metrics.shardRecorder() mapped, err := m.Map(parsed, recorder) if err != nil { - level.Warn(m.logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query) m.metrics.parsed.WithLabelValues(FailureKey).Inc() - return nil, err + return false, nil, err } mappedStr := mapped.String() originalStr := parsed.String() - noop := originalStr == mappedStr + noop = originalStr == mappedStr if noop { m.metrics.parsed.WithLabelValues(NoopKey).Inc() } else { m.metrics.parsed.WithLabelValues(SuccessKey).Inc() } - recorder.Finish() - level.Debug(m.logger).Log("mapped", !noop, "query", mappedStr, "original", originalStr) + recorder.Finish() // only record metrics for successful mappings - return mapped, err + return noop, mapped, err } func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) { diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 5da086544980d..f8ea937a670b3 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -5,7 +5,6 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" - "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" ) @@ -53,7 +52,7 @@ func TestStringer(t *testing.T) { } func TestMapSampleExpr(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { @@ -114,14 +113,14 @@ func TestMapSampleExpr(t *testing.T) { }, } { t.Run(tc.in.String(), func(t *testing.T) { - require.Equal(t, tc.out, m.mapSampleExpr(tc.in, nilMetrics.ShardRecorder())) + require.Equal(t, tc.out, m.mapSampleExpr(tc.in, nilMetrics.shardRecorder())) }) } } func TestMappingStrings(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { in string @@ -160,7 +159,7 @@ func TestMappingStrings(t *testing.T) { ast, err := ParseExpr(tc.in) require.Nil(t, err) - mapped, err := m.Map(ast, nilMetrics.ShardRecorder()) + mapped, err := m.Map(ast, nilMetrics.shardRecorder()) require.Nil(t, err) require.Equal(t, tc.out, mapped.String()) @@ -170,7 +169,7 @@ func TestMappingStrings(t *testing.T) { } func TestMapping(t *testing.T) { - m, err := NewShardMapper(2, nilMetrics, log.NewNopLogger()) + m, err := NewShardMapper(2, nilMetrics) require.Nil(t, err) for _, tc := range []struct { @@ -939,7 +938,7 @@ func TestMapping(t *testing.T) { ast, err := ParseExpr(tc.in) require.Equal(t, tc.err, err) - mapped, err := m.Map(ast, nilMetrics.ShardRecorder()) + mapped, err := m.Map(ast, nilMetrics.shardRecorder()) require.Equal(t, tc.err, err) require.Equal(t, tc.expr.String(), mapped.String()) diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index c8d3e23ae0ba1..8f9e27e60c48b 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -98,7 +98,7 @@ outer: } type MockDownstreamer struct { - Engine + *Engine } func (d MockDownstreamer) Downstream(expr Expr, p Params, shards Shards) (Query, error) { @@ -111,7 +111,7 @@ func (d MockDownstreamer) Downstream(expr Expr, p Params, shards Shards) (Query, p.Limit(), shards.Encode(), ) - return d.NewRangeQuery(params), nil + return d.Query(params), nil } // create nStreams of nEntries with labelNames each where each label value diff --git a/pkg/querier/http.go b/pkg/querier/http.go index 7fac8049fe736..15c351e993111 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -59,7 +59,7 @@ func (q *Querier) RangeQueryHandler(w http.ResponseWriter, r *http.Request) { request.Limit, request.Shards, ) - query := q.engine.NewRangeQuery(params) + query := q.engine.Query(params) result, err := query.Exec(ctx) if err != nil { writeError(err, w) @@ -98,7 +98,7 @@ func (q *Querier) InstantQueryHandler(w http.ResponseWriter, r *http.Request) { request.Limit, nil, ) - query := q.engine.NewInstantQuery(params) + query := q.engine.Query(params) result, err := query.Exec(ctx) if err != nil { writeError(err, w) @@ -154,7 +154,7 @@ func (q *Querier) LogQueryHandler(w http.ResponseWriter, r *http.Request) { request.Limit, request.Shards, ) - query := q.engine.NewRangeQuery(params) + query := q.engine.Query(params) result, err := query.Exec(ctx) if err != nil { diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 9c7e4d4c124f7..c5d4cd8949ca2 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -62,7 +62,7 @@ type Querier struct { ring ring.ReadRing pool *ring_client.Pool store storage.Store - engine logql.Engine + engine *logql.Engine limits *validation.Overrides } diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index d8ed92585df0b..6c1904575194c 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -64,18 +64,18 @@ func newASTMapperware( ) *astMapperware { return &astMapperware{ - confs: confs, - logger: log.With(logger, "middleware", "QueryShard.astMapperware"), - next: next, - metrics: metrics, + confs: confs, + logger: log.With(logger, "middleware", "QueryShard.astMapperware"), + next: next, + ng: logql.NewShardedEngine(logql.EngineOpts{}, DownstreamHandler{next}, metrics), } } type astMapperware struct { - confs queryrange.ShardingConfigs - logger log.Logger - next queryrange.Handler - metrics *logql.ShardingMetrics + confs queryrange.ShardingConfigs + logger log.Logger + next queryrange.Handler + ng *logql.ShardedEngine } func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryrange.Response, error) { @@ -89,19 +89,12 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra shardedLog, ctx := spanlogger.New(ctx, "shardedEngine") defer shardedLog.Finish() - ng, err := logql.NewShardedEngine(logql.EngineOpts{}, int(conf.RowShards), DownstreamHandler{ast.next}, ast.metrics, shardedLog) - - if err != nil { - level.Warn(ast.logger).Log("err", err.Error(), "msg", "failed to create sharded engine") - return ast.next.Do(ctx, r) - } - req, ok := r.(*LokiRequest) if !ok { return nil, fmt.Errorf("expected *LokiRequest, got (%T)", r) } params := paramsFromRequest(req) - query := ng.NewRangeQuery(params) + query := ast.ng.Query(params, int(conf.RowShards), shardedLog) res, err := query.Exec(ctx) if err != nil { From 9b82ebc7290ed0f1b2f424a065766e31c21a42fd Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 24 Apr 2020 11:21:32 -0400 Subject: [PATCH 51/67] basic tests for querysharding mware --- pkg/logproto/extensions.go | 7 + pkg/querier/queryrange/querysharding_test.go | 152 +++++++++++++++++++ 2 files changed, 159 insertions(+) create mode 100644 pkg/querier/queryrange/querysharding_test.go diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index ecd2b4dbe36a9..fe328c60a6ff5 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -2,6 +2,7 @@ package logproto import "github.com/prometheus/prometheus/pkg/labels" +// Note, this is not very efficient and use should be minimized as it requires label construction on each comparison type SeriesIdentifiers []SeriesIdentifier func (ids SeriesIdentifiers) Len() int { return len(ids) } @@ -10,3 +11,9 @@ func (ids SeriesIdentifiers) Less(i, j int) bool { a, b := labels.FromMap(ids[i].Labels), labels.FromMap(ids[j].Labels) return labels.Compare(a, b) <= 0 } + +type Streams []Stream + +func (xs Streams) Len() int { return len(xs) } +func (xs Streams) Swap(i, j int) { xs[i], xs[j] = xs[j], xs[i] } +func (xs Streams) Less(i, j int) bool { return xs[i].Labels <= xs[j].Labels } diff --git a/pkg/querier/queryrange/querysharding_test.go b/pkg/querier/queryrange/querysharding_test.go new file mode 100644 index 0000000000000..3f8f53df62772 --- /dev/null +++ b/pkg/querier/queryrange/querysharding_test.go @@ -0,0 +1,152 @@ +package queryrange + +import ( + "context" + "sort" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/go-kit/kit/log" + "github.com/grafana/loki/pkg/loghttp" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" + "github.com/stretchr/testify/require" +) + +var ( + nilShardingMetrics = logql.NewShardingMetrics(nil) + defaultReq = func() *LokiRequest { + return &LokiRequest{ + Limit: 100, + StartTs: start, + EndTs: end, + Direction: logproto.BACKWARD, + Path: "/loki/api/v1/query_range", + } + } + lokiResps = []queryrange.Response{ + &LokiResponse{ + Status: loghttp.QueryStatusSuccess, + Direction: logproto.BACKWARD, + Limit: defaultReq().Limit, + Version: 1, + Data: LokiData{ + ResultType: loghttp.ResultTypeStream, + Result: []logproto.Stream{ + { + Labels: `{foo="bar", level="debug"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 6), Line: "6"}, + {Timestamp: time.Unix(0, 5), Line: "5"}, + }, + }, + }, + }, + }, + &LokiResponse{ + Status: loghttp.QueryStatusSuccess, + Direction: logproto.BACKWARD, + Limit: 100, + Version: 1, + Data: LokiData{ + ResultType: loghttp.ResultTypeStream, + Result: []logproto.Stream{ + { + Labels: `{foo="bar", level="error"}`, + Entries: []logproto.Entry{ + {Timestamp: time.Unix(0, 2), Line: "2"}, + {Timestamp: time.Unix(0, 1), Line: "1"}, + }, + }, + }, + }, + }, + } +) + +func Test_PartitionRequest(t *testing.T) { + midpt := time.Unix(0, 0).Add(500 * time.Millisecond) + cutoff := TimeToMillis(midpt) + + // test split + req := defaultReq().WithStartEnd(0, cutoff*2) + before, after := partitionRequest(req, midpt) + require.Equal(t, req.WithStartEnd(0, cutoff), before) + require.Equal(t, req.WithStartEnd(cutoff, 2*cutoff), after) + + // test all before cutoff + before, after = partitionRequest(req, midpt.Add(1000*time.Millisecond)) + require.Equal(t, req, before) + require.Nil(t, after) + + // test after cutoff + before, after = partitionRequest(req, time.Unix(0, 0)) + require.Nil(t, before) + require.Equal(t, req, after) + +} + +func Test_shardSplitter(t *testing.T) { + splitter := &shardSplitter{ + shardingware: mockHandler(lokiResps[0], nil), + next: mockHandler(lokiResps[1], nil), + now: time.Now, + MinShardingLookback: 0, + } + + req := defaultReq().WithStartEnd( + TimeToMillis(time.Now().Add(-time.Hour)), + TimeToMillis(time.Now().Add(time.Hour)), + ) + + resp, err := splitter.Do(context.Background(), req) + require.Nil(t, err) + expected, err := lokiCodec.MergeResponse(lokiResps...) + require.Nil(t, err) + require.Equal(t, expected, resp) +} + +func Test_astMapper(t *testing.T) { + called := 0 + + handler := queryrange.HandlerFunc(func(ctx context.Context, req queryrange.Request) (queryrange.Response, error) { + resp := lokiResps[called] + called++ + return resp, nil + }) + + mware := newASTMapperware( + queryrange.ShardingConfigs{ + chunk.PeriodConfig{ + RowShards: 2, + }, + }, + handler, + log.NewNopLogger(), + nilShardingMetrics, + ) + + resp, err := mware.Do(context.Background(), defaultReq().WithQuery(`{food="bar"}`)) + require.Nil(t, err) + + expected, err := lokiCodec.MergeResponse(lokiResps...) + sort.Sort(logproto.Streams(expected.(*LokiResponse).Data.Result)) + require.Nil(t, err) + require.Equal(t, called, 2) + require.Equal(t, expected.(*LokiResponse).Data, resp.(*LokiResponse).Data) + +} + +// astmapper successful stream & prom conversion + +func mockHandler(resp queryrange.Response, err error) queryrange.Handler { + return queryrange.HandlerFunc(func(ctx context.Context, req queryrange.Request) (queryrange.Response, error) { + if expired := ctx.Err(); expired != nil { + return nil, expired + } + + return resp, err + }) +} From e8ff7c33e67081b36847079565bf8d46dda7c52f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 2 May 2020 09:32:28 -0400 Subject: [PATCH 52/67] [wip] concurrent evaluator --- pkg/logql/sharding.go | 103 ++++++++++++++++++++++++++++++++++-------- 1 file changed, 83 insertions(+), 20 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index a07f30acc11a2..9226db0b40e6c 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -124,6 +124,7 @@ func (ev *DownstreamEvaluator) StepEvaluator( params Params, ) (StepEvaluator, error) { switch e := expr.(type) { + case DownstreamSampleExpr: // downstream to a querier var shards []astmapper.ShardAnnotation @@ -142,21 +143,50 @@ func (ev *DownstreamEvaluator) StepEvaluator( return ResultStepEvaluator(res, params) case *ConcatSampleExpr: - // ensure they all impl the same (SampleExpr, LogSelectorExpr) & concat - var xs []StepEvaluator + type result struct { + stepper StepEvaluator + err error + } + ctx, cancel := context.WithCancel(ctx) cur := e + ch := make(chan result) + done := make(chan struct{}) + count := 0 for cur != nil { - eval, err := ev.StepEvaluator(ctx, nextEv, cur.SampleExpr, params) - if err != nil { - // Close previously opened StepEvaluators - for _, x := range xs { - x.Close() + go func(expr SampleExpr) { + eval, err := ev.StepEvaluator(ctx, nextEv, expr, params) + select { + case <-done: + case ch <- result{eval, err}: } - return nil, err - } - xs = append(xs, eval) + }(cur.SampleExpr) cur = cur.next + count++ + } + + xs := make([]StepEvaluator, 0, count) + cleanup := func() { + cancel() // cancel ctx + done <- struct{}{} // send done signal to awaiting goroutines + // Close previously opened StepEvaluators + for _, x := range xs { + x.Close() // close unused StepEvaluators + } + } + + for i := 0; i < count; i++ { + select { + case <-ctx.Done(): + defer cleanup() + return nil, ctx.Err() + case res := <-ch: + if res.err != nil { + defer cleanup() + return nil, res.err + } + xs = append(xs, res.stepper) + } } return ConcatEvaluator(xs) @@ -191,21 +221,54 @@ func (ev *DownstreamEvaluator) Iterator( return ResultIterator(res, params) case *ConcatLogSelectorExpr: - var iters []iter.EntryIterator + type result struct { + iterator iter.EntryIterator + err error + } + ctx, cancel := context.WithCancel(ctx) cur := e + ch := make(chan result) + done := make(chan struct{}) + count := 0 + for cur != nil { - iterator, err := ev.Iterator(ctx, cur.LogSelectorExpr, params) - if err != nil { - // Close previously opened StepEvaluators - for _, x := range iters { - x.Close() + go func(expr LogSelectorExpr) { + iterator, err := ev.Iterator(ctx, expr, params) + select { + case <-done: + case ch <- result{iterator, err}: } - return nil, err - } - iters = append(iters, iterator) + }(cur.LogSelectorExpr) cur = cur.next + count++ } - return iter.NewHeapIterator(ctx, iters, params.Direction()), nil + + xs := make([]iter.EntryIterator, 0, count) + + cleanup := func() { + cancel() // cancel ctx + done <- struct{}{} // send done signal to awaiting goroutines + // Close previously opened Iterators + for _, x := range xs { + x.Close() // close unused Iterators + } + } + + for i := 0; i < count; i++ { + select { + case <-ctx.Done(): + defer cleanup() + return nil, ctx.Err() + case res := <-ch: + if res.err != nil { + defer cleanup() + return nil, res.err + } + xs = append(xs, res.iterator) + } + } + + return iter.NewHeapIterator(ctx, xs, params.Direction()), nil default: return nil, EvaluatorUnsupportedType(expr, ev) From 2dc37eeb64ec000def3b45e29df0ee16f9c119d5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 May 2020 12:33:23 -0400 Subject: [PATCH 53/67] integrates stat propagation into sharding evaluator --- pkg/logql/sharding.go | 36 +++++++++++++++------- pkg/logql/stats/context.go | 54 +++++++++++++++++++++++++++++++-- pkg/logql/stats/context_test.go | 47 ++++++++++++++++++++++++++++ 3 files changed, 124 insertions(+), 13 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 9226db0b40e6c..d16bb32f745cf 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -7,11 +7,13 @@ import ( "time" "github.com/cortexproject/cortex/pkg/querier/astmapper" + "github.com/cortexproject/cortex/pkg/util" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/prometheus/prometheus/promql" "github.com/grafana/loki/pkg/iter" + "github.com/grafana/loki/pkg/logql/stats" ) // DownstreamSampleExpr is a SampleExpr which signals downstream computation @@ -103,6 +105,27 @@ type DownstreamEvaluator struct { defaultEvaluator *DefaultEvaluator } +// Exec runs a query and collects stats from the embedded Downstreamer +func (ev DownstreamEvaluator) Exec(ctx context.Context, expr Expr, p Params, shards Shards) (Result, error) { + qry, err := ev.Downstream(expr, p, shards) + if err != nil { + return Result{}, err + } + + res, err := qry.Exec(ctx) + if err != nil { + return Result{}, err + } + + err = stats.JoinResults(ctx, res.Statistics) + if err != nil { + level.Warn(util.Logger).Log("msg", "unable to merge downstream results", "err", err) + } + + return res, nil + +} + func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator { return &DownstreamEvaluator{ Downstreamer: downstreamer, @@ -131,12 +154,7 @@ func (ev *DownstreamEvaluator) StepEvaluator( if e.shard != nil { shards = append(shards, *e.shard) } - qry, err := ev.Downstream(e.SampleExpr, params, shards) - if err != nil { - return nil, err - } - - res, err := qry.Exec(ctx) + res, err := ev.Exec(ctx, e.SampleExpr, params, shards) if err != nil { return nil, err } @@ -209,15 +227,11 @@ func (ev *DownstreamEvaluator) Iterator( if e.shard != nil { shards = append(shards, *e.shard) } - qry, err := ev.Downstream(e.LogSelectorExpr, params, shards) + res, err := ev.Exec(ctx, e.LogSelectorExpr, params, shards) if err != nil { return nil, err } - res, err := qry.Exec(ctx) - if err != nil { - return nil, err - } return ResultIterator(res, params) case *ConcatLogSelectorExpr: diff --git a/pkg/logql/stats/context.go b/pkg/logql/stats/context.go index a481b68671e0d..300e8ef50995c 100644 --- a/pkg/logql/stats/context.go +++ b/pkg/logql/stats/context.go @@ -22,6 +22,9 @@ package stats import ( "context" + "errors" + fmt "fmt" + "sync" "time" "github.com/dustin/go-humanize" @@ -35,6 +38,8 @@ const ( chunksKey ctxKeyType = "chunks" ingesterKey ctxKeyType = "ingester" storeKey ctxKeyType = "store" + resultKey ctxKeyType = "result" // key for pre-computed results to be merged in `Snapshot` + lockKey ctxKeyType = "lock" // key for locking a context when stats is used concurrently ) // Log logs a query statistics result. @@ -82,6 +87,8 @@ func NewContext(ctx context.Context) context.Context { ctx = context.WithValue(ctx, storeKey, &StoreData{}) ctx = context.WithValue(ctx, chunksKey, &ChunkData{}) ctx = context.WithValue(ctx, ingesterKey, &IngesterData{}) + ctx = context.WithValue(ctx, resultKey, &Result{}) + ctx = context.WithValue(ctx, lockKey, &sync.Mutex{}) return ctx } @@ -157,8 +164,17 @@ func Snapshot(ctx context.Context, execTime time.Duration) Result { res.Store.CompressedBytes = c.CompressedBytes res.Store.TotalDuplicates = c.TotalDuplicates } - res.ComputeSummary(execTime) - return res + + // see if there is a pre-computed Result embedded in the context which needs merging + _ = JoinResults(ctx, res) + merged, err := GetResult(ctx) + if err != nil { + merged = &res + } + + merged.ComputeSummary(execTime) + return *merged + } // ComputeSummary calculates the summary based on store and ingester data. @@ -204,3 +220,37 @@ func (r *Result) Merge(m Result) { r.ComputeSummary(time.Duration(int64((r.Summary.ExecTime + m.Summary.ExecTime) * float64(time.Second)))) } + +// JoinResults merges a Result with the embedded Result in a context in a concurrency-safe manner. +func JoinResults(ctx context.Context, res Result) error { + mtx, err := GetMutex(ctx) + if err != nil { + return err + } + mtx.Lock() + defer mtx.Unlock() + + v, err := GetResult(ctx) + if err != nil { + return err + } + v.Merge(res) + return nil +} + +func GetResult(ctx context.Context) (*Result, error) { + v, ok := ctx.Value(resultKey).(*Result) + if !ok { + return nil, errors.New("unpopulated Results key") + } + return v, nil +} + +// GetChunkData returns the chunks statistics data from the current context. +func GetMutex(ctx context.Context) (*sync.Mutex, error) { + res, ok := ctx.Value(lockKey).(*sync.Mutex) + if !ok { + return nil, fmt.Errorf("no mutex available under %s", string(lockKey)) + } + return res, nil +} diff --git a/pkg/logql/stats/context_test.go b/pkg/logql/stats/context_test.go index 8c0187d663dce..8a688494ff572 100644 --- a/pkg/logql/stats/context_test.go +++ b/pkg/logql/stats/context_test.go @@ -64,6 +64,53 @@ func TestSnapshot(t *testing.T) { require.Equal(t, expected, res) } +func TestSnapshot_MergesResults(t *testing.T) { + ctx := NewContext(context.Background()) + expected := Result{ + Ingester: Ingester{ + TotalChunksMatched: 200, + TotalBatches: 50, + TotalLinesSent: 60, + HeadChunkBytes: 10, + HeadChunkLines: 20, + DecompressedBytes: 24, + DecompressedLines: 40, + CompressedBytes: 60, + TotalDuplicates: 2, + TotalReached: 2, + }, + Store: Store{ + TotalChunksRef: 50, + TotalChunksDownloaded: 60, + ChunksDownloadTime: time.Second.Seconds(), + HeadChunkBytes: 10, + HeadChunkLines: 20, + DecompressedBytes: 40, + DecompressedLines: 20, + CompressedBytes: 30, + TotalDuplicates: 10, + }, + Summary: Summary{ + ExecTime: 2 * time.Second.Seconds(), + BytesProcessedPerSecond: int64(42), + LinesProcessedPerSecond: int64(50), + TotalBytesProcessed: int64(84), + TotalLinesProcessed: int64(100), + }, + } + + err := JoinResults(ctx, expected) + require.Nil(t, err) + res := Snapshot(ctx, 2*time.Second) + require.Equal(t, expected, res) +} + +func TestGetResult_ErrsNonexistant(t *testing.T) { + out, err := GetResult(context.Background()) + require.NotNil(t, err) + require.Nil(t, out) +} + func fakeIngesterQuery(ctx context.Context) { d, _ := ctx.Value(trailersKey).(*trailerCollector) meta := d.addTrailer() From 5dc90a2ca6cda90376ee350cac98ac2978912e9d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 May 2020 13:50:03 -0400 Subject: [PATCH 54/67] splitby histogram --- pkg/querier/queryrange/roundtrip.go | 11 +++-- pkg/querier/queryrange/split_by_interval.go | 48 +++++++++++++------ .../queryrange/split_by_interval_test.go | 5 ++ 3 files changed, 46 insertions(+), 18 deletions(-) diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 1b21a2dcf88fa..ddd2da3561d16 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -53,12 +53,13 @@ func NewTripperware( instrumentMetrics := queryrange.NewInstrumentMiddlewareMetrics(registerer) retryMetrics := queryrange.NewRetryMiddlewareMetrics(registerer) shardingMetrics := logql.NewShardingMetrics(registerer) + splitByMetrics := NewSplitByMetrics(registerer) - metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics, shardingMetrics) + metricsTripperware, cache, err := NewMetricTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, prometheusResponseExtractor, instrumentMetrics, retryMetrics, shardingMetrics, splitByMetrics) if err != nil { return nil, nil, err } - logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, shardingMetrics) + logFilterTripperware, err := NewLogFilterTripperware(cfg, log, limits, schema, minShardingLookback, lokiCodec, instrumentMetrics, retryMetrics, shardingMetrics, splitByMetrics) if err != nil { return nil, nil, err } @@ -164,10 +165,11 @@ func NewLogFilterTripperware( instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, shardingMetrics *logql.ShardingMetrics, + splitByMetrics *SplitByMetrics, ) (frontend.Tripperware, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.SplitQueriesByInterval != 0 { - queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec)) + queryRangeMiddleware = append(queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), SplitByIntervalMiddleware(limits, codec, splitByMetrics)) } if cfg.ShardedQueries { @@ -209,6 +211,7 @@ func NewMetricTripperware( instrumentMetrics *queryrange.InstrumentMiddlewareMetrics, retryMiddlewareMetrics *queryrange.RetryMiddlewareMetrics, shardingMetrics *logql.ShardingMetrics, + splitByMetrics *SplitByMetrics, ) (frontend.Tripperware, Stopper, error) { queryRangeMiddleware := []queryrange.Middleware{StatsCollectorMiddleware(), queryrange.LimitsMiddleware(limits)} if cfg.AlignQueriesWithStep { @@ -227,7 +230,7 @@ func NewMetricTripperware( queryRangeMiddleware = append( queryRangeMiddleware, queryrange.InstrumentMiddleware("split_by_interval", instrumentMetrics), - SplitByIntervalMiddleware(limits, codec), + SplitByIntervalMiddleware(limits, codec, splitByMetrics), ) var c cache.Cache diff --git a/pkg/querier/queryrange/split_by_interval.go b/pkg/querier/queryrange/split_by_interval.go index 3e57970f759dd..af6c58c957c80 100644 --- a/pkg/querier/queryrange/split_by_interval.go +++ b/pkg/querier/queryrange/split_by_interval.go @@ -8,23 +8,14 @@ import ( "github.com/cortexproject/cortex/pkg/querier/queryrange" "github.com/opentracing/opentracing-go" otlog "github.com/opentracing/opentracing-go/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/user" "github.com/grafana/loki/pkg/logproto" ) -// SplitByIntervalMiddleware creates a new Middleware that splits log requests by a given interval. -func SplitByIntervalMiddleware(limits Limits, merger queryrange.Merger) queryrange.Middleware { - return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { - return &splitByInterval{ - next: next, - limits: limits, - merger: merger, - } - }) -} - type lokiResult struct { req queryrange.Request ch chan *packedResp @@ -35,10 +26,38 @@ type packedResp struct { err error } +type SplitByMetrics struct { + splits prometheus.Histogram +} + +func NewSplitByMetrics(r prometheus.Registerer) *SplitByMetrics { + return &SplitByMetrics{ + splits: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: "loki", + Name: "query_frontend_partitions", + Help: "Number of time-based partitions (sub-requests) per request", + Buckets: prometheus.ExponentialBuckets(1, 4, 5), // 1 -> 1024 + }), + } +} + type splitByInterval struct { - next queryrange.Handler - limits Limits - merger queryrange.Merger + next queryrange.Handler + limits Limits + merger queryrange.Merger + metrics *SplitByMetrics +} + +// SplitByIntervalMiddleware creates a new Middleware that splits log requests by a given interval. +func SplitByIntervalMiddleware(limits Limits, merger queryrange.Merger, metrics *SplitByMetrics) queryrange.Middleware { + return queryrange.MiddlewareFunc(func(next queryrange.Handler) queryrange.Handler { + return &splitByInterval{ + next: next, + limits: limits, + merger: merger, + metrics: metrics, + } + }) } func (h *splitByInterval) Feed(ctx context.Context, input []*lokiResult) chan *lokiResult { @@ -148,6 +167,7 @@ func (h *splitByInterval) Do(ctx context.Context, r queryrange.Request) (queryra } intervals := splitByTime(lokiRequest, interval) + h.metrics.splits.Observe(float64(len(intervals))) // no interval should not be processed by the frontend. if len(intervals) == 0 { diff --git a/pkg/querier/queryrange/split_by_interval_test.go b/pkg/querier/queryrange/split_by_interval_test.go index 4a2b7736ed167..0a469d71d307a 100644 --- a/pkg/querier/queryrange/split_by_interval_test.go +++ b/pkg/querier/queryrange/split_by_interval_test.go @@ -16,6 +16,8 @@ import ( "github.com/grafana/loki/pkg/logproto" ) +var nilMetrics = NewSplitByMetrics(nil) + func Test_splitQuery(t *testing.T) { tests := []struct { @@ -105,6 +107,7 @@ func Test_splitByInterval_Do(t *testing.T) { split := SplitByIntervalMiddleware( l, lokiCodec, + nilMetrics, ).Wrap(next) tests := []struct { @@ -288,6 +291,7 @@ func Test_ExitEarly(t *testing.T) { split := SplitByIntervalMiddleware( l, lokiCodec, + nilMetrics, ).Wrap(next) req := &LokiRequest{ @@ -366,6 +370,7 @@ func Test_DoesntDeadlock(t *testing.T) { split := SplitByIntervalMiddleware( l, lokiCodec, + nilMetrics, ).Wrap(next) // split into n requests w/ n/2 limit, ensuring unused responses are cleaned up properly From 76423b6db85817b4966e10452f9fb2ca03f1c4f8 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 May 2020 16:25:24 -0400 Subject: [PATCH 55/67] extends le bounds for bytes processed --- pkg/logql/metrics.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 63b3ec0ecfb22..34348ae7ee001 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -28,8 +28,8 @@ var ( Namespace: "loki", Name: "logql_querystats_bytes_processed_per_seconds", Help: "Distribution of bytes processed per second for LogQL queries.", - // 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB - Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9}, + // 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB 30GB, 40GB + Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9, 30 * 1e9, 40 * 1e9}, }, []string{"status_code", "type", "range", "latency_type"}) execLatency = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "loki", From 34b045a5d13593970f9bdd0b2f7f31d2b7a63ca5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 May 2020 15:17:22 -0400 Subject: [PATCH 56/67] byte throughput histogram buckets to 40gb --- pkg/logql/metrics.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 34348ae7ee001..ba8b351b206ab 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -28,8 +28,8 @@ var ( Namespace: "loki", Name: "logql_querystats_bytes_processed_per_seconds", Help: "Distribution of bytes processed per second for LogQL queries.", - // 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB 30GB, 40GB - Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9, 30 * 1e9, 40 * 1e9}, + // 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB 30GB, 40GB 50GB 60GB + Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9, 30 * 1e9, 40 * 1e9, 50 * 1e9, 60 * 1e9}, }, []string{"status_code", "type", "range", "latency_type"}) execLatency = promauto.NewHistogramVec(prometheus.HistogramOpts{ Namespace: "loki", From e9b6f695e2654c3a630a606ffc33d8f59604b936 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 May 2020 15:40:45 -0400 Subject: [PATCH 57/67] chunk duration mixin --- production/loki-mixin/dashboards.libsonnet | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/production/loki-mixin/dashboards.libsonnet b/production/loki-mixin/dashboards.libsonnet index 2ca8376f70315..e91bfebe2c524 100644 --- a/production/loki-mixin/dashboards.libsonnet +++ b/production/loki-mixin/dashboards.libsonnet @@ -142,6 +142,24 @@ local utils = import 'mixin-utils/utils.libsonnet'; g.panel('Flush Rate') + g.qpsPanel('loki_ingester_chunk_age_seconds_count{cluster="$cluster", job="$namespace/ingester"}'), ), + ) + .addRow( + g.row('Duration') + .addPanel( + g.panel('Chunk Duration hours (end-start)') + + g.queryPanel( + [ + 'histogram_quantile(0.5, sum(rate(loki_ingester_chunk_bounds_hours_bucket{cluster="$cluster", job="$namespace/ingester"}[5m])) by (le))', + 'histogram_quantile(0.99, sum(rate(loki_ingester_chunk_bounds_hours_bucket{cluster="$cluster", job="$namespace/ingester"}[5m])) by (le))', + 'sum(rate(loki_ingester_chunk_bounds_hours_sum{cluster="$cluster", job="$namespace/ingester"}[5m])) / sum(rate(loki_ingester_chunk_bounds_hours_count{cluster="$cluster", job="$namespace/ingester"}[5m]))', + ], + [ + 'p50', + 'p99', + 'avg', + ], + ), + ) ), }, } From 404487d96530df2d8463e040606fe45c627b89b3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 May 2020 17:37:24 -0400 Subject: [PATCH 58/67] fixes merge w/ field rename --- pkg/loki/modules.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 36a895dccb6d0..9b1a118130a08 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -303,7 +303,7 @@ func (t *Loki) initQueryFrontend() (_ services.Service, err error) { util.Logger, t.overrides, t.cfg.SchemaConfig, - t.cfg.Querier.IngesterMaxQueryLookback, + t.cfg.Querier.QueryIngestersWithin, prometheus.DefaultRegisterer, ) if err != nil { From 15a74611082e9b90c8c323d55033d6bf72aafb11 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 09:21:14 -0400 Subject: [PATCH 59/67] derives logger in sharded engine via ctx & logs some downstream evaluators --- pkg/logql/engine.go | 10 ++++++---- pkg/logql/sharding.go | 17 ++++++++++------- pkg/logql/sharding_test.go | 2 +- pkg/querier/queryrange/querysharding.go | 2 +- 4 files changed, 18 insertions(+), 13 deletions(-) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index e80aea411ac7d..51f9249f908d4 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -95,7 +95,9 @@ func (ng *Engine) Query(params Params) Query { timeout: ng.timeout, params: params, evaluator: ng.evaluator, - parse: ParseExpr, + parse: func(_ context.Context, query string) (Expr, error) { + return ParseExpr(query) + }, } } @@ -108,13 +110,13 @@ type Query interface { type query struct { timeout time.Duration params Params - parse func(string) (Expr, error) + parse func(context.Context, string) (Expr, error) evaluator Evaluator } // Exec Implements `Query`. It handles instrumentation & defers to Eval. func (q *query) Exec(ctx context.Context) (Result, error) { - log, ctx := spanlogger.New(ctx, "Engine.Exec") + log, ctx := spanlogger.New(ctx, "query.Exec") defer log.Finish() rangeType := GetRangeType(q.params) @@ -150,7 +152,7 @@ func (q *query) Eval(ctx context.Context) (promql.Value, error) { ctx, cancel := context.WithTimeout(ctx, q.timeout) defer cancel() - expr, err := q.parse(q.params.Query()) + expr, err := q.parse(ctx, q.params.Query()) if err != nil { return nil, err } diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index d16bb32f745cf..18c1d36cb5036 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -8,7 +8,7 @@ import ( "github.com/cortexproject/cortex/pkg/querier/astmapper" "github.com/cortexproject/cortex/pkg/util" - "github.com/go-kit/kit/log" + "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/go-kit/kit/log/level" "github.com/prometheus/prometheus/promql" @@ -174,6 +174,9 @@ func (ev *DownstreamEvaluator) StepEvaluator( for cur != nil { go func(expr SampleExpr) { eval, err := ev.StepEvaluator(ctx, nextEv, expr, params) + if err != nil { + level.Warn(util.Logger).Log("msg", "could not extract StepEvaluator", "err", err, "expr", expr.String()) + } select { case <-done: case ch <- result{eval, err}: @@ -248,6 +251,9 @@ func (ev *DownstreamEvaluator) Iterator( for cur != nil { go func(expr LogSelectorExpr) { iterator, err := ev.Iterator(ctx, expr, params) + if err != nil { + level.Warn(util.Logger).Log("msg", "could not extract Iterator", "err", err, "expr", expr.String()) + } select { case <-done: case ch <- result{iterator, err}: @@ -364,16 +370,13 @@ func NewShardedEngine(opts EngineOpts, downstreamer Downstreamer, metrics *Shard } -func (ng *ShardedEngine) Query(p Params, shards int, logger log.Logger) Query { - if logger == nil { - logger = log.NewNopLogger() - } - +func (ng *ShardedEngine) Query(p Params, shards int) Query { return &query{ timeout: ng.timeout, params: p, evaluator: ng.evaluator, - parse: func(query string) (Expr, error) { + parse: func(ctx context.Context, query string) (Expr, error) { + logger := spanlogger.FromContext(ctx) mapper, err := NewShardMapper(shards, ng.metrics) if err != nil { return nil, err diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index b4c430ec4b1f3..bf4a777dddc76 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -71,7 +71,7 @@ func TestMappingEquivalence(t *testing.T) { nil, ) qry := regular.Query(params) - shardedQry := sharded.Query(params, shards, nil) + shardedQry := sharded.Query(params, shards) res, err := qry.Exec(context.Background()) require.Nil(t, err) diff --git a/pkg/querier/queryrange/querysharding.go b/pkg/querier/queryrange/querysharding.go index 6c1904575194c..9aae4a0542d68 100644 --- a/pkg/querier/queryrange/querysharding.go +++ b/pkg/querier/queryrange/querysharding.go @@ -94,7 +94,7 @@ func (ast *astMapperware) Do(ctx context.Context, r queryrange.Request) (queryra return nil, fmt.Errorf("expected *LokiRequest, got (%T)", r) } params := paramsFromRequest(req) - query := ast.ng.Query(params, int(conf.RowShards), shardedLog) + query := ast.ng.Query(params, int(conf.RowShards)) res, err := query.Exec(ctx) if err != nil { From fd33919cd1cfe22d1dff79814b743561d5820aa1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 09:36:42 -0400 Subject: [PATCH 60/67] moves sharded engine to top, adds comments --- pkg/logql/sharding.go | 86 +++++++++++++++++++++++-------------------- 1 file changed, 47 insertions(+), 39 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 18c1d36cb5036..51a4a4d00e53c 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -16,6 +16,53 @@ import ( "github.com/grafana/loki/pkg/logql/stats" ) +/* +This includes a bunch of tooling for parallelization improvements based on backend shard factors. In schemas 10+ a shard factor (default 16) is introduced in the index store, calculated by hashing the label set of a log stream. This allows us to perform certain optimizations that fall under the umbrella of query remapping and querying shards individually. For instance, `{app="foo"} |= "bar"` can be executed on each shard independently, then reaggregated. There are also a class of optimizations that can be performed by altering a query into a functionally equivalent, but more parallelizable form. For instance, an average can be remapped into a sum/count, which can then take advantage of our sharded execution model. +*/ + +// ShardedEngine is an Engine implementation that can split queries into more parallelizable forms via +// querying the underlying backend shards individually and reaggregating them. +type ShardedEngine struct { + timeout time.Duration + evaluator Evaluator + metrics *ShardingMetrics +} + +// NewShardedEngine constructs a *ShardedEngine +func NewShardedEngine(opts EngineOpts, downstreamer Downstreamer, metrics *ShardingMetrics) *ShardedEngine { + opts.applyDefault() + return &ShardedEngine{ + timeout: opts.Timeout, + evaluator: NewDownstreamEvaluator(downstreamer), + metrics: metrics, + } + +} + +// Query constructs a Query +func (ng *ShardedEngine) Query(p Params, shards int) Query { + return &query{ + timeout: ng.timeout, + params: p, + evaluator: ng.evaluator, + parse: func(ctx context.Context, query string) (Expr, error) { + logger := spanlogger.FromContext(ctx) + mapper, err := NewShardMapper(shards, ng.metrics) + if err != nil { + return nil, err + } + noop, parsed, err := mapper.Parse(query) + if err != nil { + level.Warn(logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query) + return nil, err + } + + level.Debug(logger).Log("no-op", noop, "mapped", parsed.String()) + return parsed, nil + }, + } +} + // DownstreamSampleExpr is a SampleExpr which signals downstream computation type DownstreamSampleExpr struct { shard *astmapper.ShardAnnotation @@ -353,42 +400,3 @@ func ResultIterator(res Result, params Params) (iter.EntryIterator, error) { return iter.NewStreamsIterator(context.Background(), streams, params.Direction()), nil } - -type ShardedEngine struct { - timeout time.Duration - evaluator Evaluator - metrics *ShardingMetrics -} - -func NewShardedEngine(opts EngineOpts, downstreamer Downstreamer, metrics *ShardingMetrics) *ShardedEngine { - opts.applyDefault() - return &ShardedEngine{ - timeout: opts.Timeout, - evaluator: NewDownstreamEvaluator(downstreamer), - metrics: metrics, - } - -} - -func (ng *ShardedEngine) Query(p Params, shards int) Query { - return &query{ - timeout: ng.timeout, - params: p, - evaluator: ng.evaluator, - parse: func(ctx context.Context, query string) (Expr, error) { - logger := spanlogger.FromContext(ctx) - mapper, err := NewShardMapper(shards, ng.metrics) - if err != nil { - return nil, err - } - noop, parsed, err := mapper.Parse(query) - if err != nil { - level.Warn(logger).Log("msg", "failed mapping AST", "err", err.Error(), "query", query) - return nil, err - } - - level.Debug(logger).Log("no-op", noop, "mapped", parsed.String()) - return parsed, nil - }, - } -} From e8c9e66fc9776dcaac087aba8eac2b0705e46f52 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 09:56:57 -0400 Subject: [PATCH 61/67] logs failed merge results in stats ctx --- pkg/chunkenc/memchunk_test.go | 5 +++-- pkg/logql/engine.go | 2 +- pkg/logql/stats/context.go | 9 ++++++--- pkg/logql/stats/context_test.go | 4 ++-- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/chunkenc/memchunk_test.go b/pkg/chunkenc/memchunk_test.go index 72b1961d41e63..e5582e6d4f023 100644 --- a/pkg/chunkenc/memchunk_test.go +++ b/pkg/chunkenc/memchunk_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/cortexproject/cortex/pkg/util" "github.com/stretchr/testify/assert" "github.com/dustin/go-humanize" @@ -436,7 +437,7 @@ func TestChunkStats(t *testing.T) { t.Fatal(err) } // test on a chunk filling up - s := stats.Snapshot(ctx, time.Since(first)) + s := stats.Snapshot(ctx, time.Since(first), util.Logger) require.Equal(t, int64(expectedSize), s.Summary.TotalBytesProcessed) require.Equal(t, int64(inserted), s.Summary.TotalLinesProcessed) @@ -464,7 +465,7 @@ func TestChunkStats(t *testing.T) { if err := it.Close(); err != nil { t.Fatal(err) } - s = stats.Snapshot(ctx, time.Since(first)) + s = stats.Snapshot(ctx, time.Since(first), util.Logger) require.Equal(t, int64(expectedSize), s.Summary.TotalBytesProcessed) require.Equal(t, int64(inserted), s.Summary.TotalLinesProcessed) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 51f9249f908d4..3550d5d2819bd 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -130,7 +130,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) { data, err := q.Eval(ctx) - statResult = stats.Snapshot(ctx, time.Since(start)) + statResult = stats.Snapshot(ctx, time.Since(start), log) statResult.Log(level.Debug(log)) status := "200" diff --git a/pkg/logql/stats/context.go b/pkg/logql/stats/context.go index 300e8ef50995c..0bfafa8dc2c2a 100644 --- a/pkg/logql/stats/context.go +++ b/pkg/logql/stats/context.go @@ -13,7 +13,7 @@ Then you can update statistics by mutating data by using: Finally to get a snapshot of the current query statistic use - stats.Snapshot(ctx,time.Since(start)) + stats.Snapshot(ctx, time.Since(start), logger) Ingester statistics are sent across the GRPC stream using Trailers see https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md @@ -29,6 +29,7 @@ import ( "github.com/dustin/go-humanize" "github.com/go-kit/kit/log" + "github.com/go-kit/kit/log/level" ) type ctxKeyType string @@ -144,7 +145,7 @@ func GetStoreData(ctx context.Context) *StoreData { } // Snapshot compute query statistics from a context using the total exec time. -func Snapshot(ctx context.Context, execTime time.Duration) Result { +func Snapshot(ctx context.Context, execTime time.Duration, log log.Logger) Result { // ingester data is decoded from grpc trailers. res := decodeTrailers(ctx) // collect data from store. @@ -166,7 +167,9 @@ func Snapshot(ctx context.Context, execTime time.Duration) Result { } // see if there is a pre-computed Result embedded in the context which needs merging - _ = JoinResults(ctx, res) + if err := JoinResults(ctx, res); err != nil { + level.Warn(log).Log("msg", "could not merge Stats embedded in ctx", "err", err) + } merged, err := GetResult(ctx) if err != nil { merged = &res diff --git a/pkg/logql/stats/context_test.go b/pkg/logql/stats/context_test.go index 8a688494ff572..a7ecace04ac1e 100644 --- a/pkg/logql/stats/context_test.go +++ b/pkg/logql/stats/context_test.go @@ -27,7 +27,7 @@ func TestSnapshot(t *testing.T) { fakeIngesterQuery(ctx) fakeIngesterQuery(ctx) - res := Snapshot(ctx, 2*time.Second) + res := Snapshot(ctx, 2*time.Second, util.Logger) res.Log(util.Logger) expected := Result{ Ingester: Ingester{ @@ -101,7 +101,7 @@ func TestSnapshot_MergesResults(t *testing.T) { err := JoinResults(ctx, expected) require.Nil(t, err) - res := Snapshot(ctx, 2*time.Second) + res := Snapshot(ctx, 2*time.Second, util.Logger) require.Equal(t, expected, res) } From ab3e7aa9e96c7fc8d0864812a1690469272695fb Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 11:04:37 -0400 Subject: [PATCH 62/67] snapshotting stats merge logic is done more effectively --- pkg/logql/stats/context.go | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/pkg/logql/stats/context.go b/pkg/logql/stats/context.go index 0bfafa8dc2c2a..9a97b82351c41 100644 --- a/pkg/logql/stats/context.go +++ b/pkg/logql/stats/context.go @@ -29,7 +29,6 @@ import ( "github.com/dustin/go-humanize" "github.com/go-kit/kit/log" - "github.com/go-kit/kit/log/level" ) type ctxKeyType string @@ -166,17 +165,15 @@ func Snapshot(ctx context.Context, execTime time.Duration, log log.Logger) Resul res.Store.TotalDuplicates = c.TotalDuplicates } - // see if there is a pre-computed Result embedded in the context which needs merging - if err := JoinResults(ctx, res); err != nil { - level.Warn(log).Log("msg", "could not merge Stats embedded in ctx", "err", err) - } - merged, err := GetResult(ctx) + existing, err := GetResult(ctx) if err != nil { - merged = &res + res.ComputeSummary(execTime) + return res } - merged.ComputeSummary(execTime) - return *merged + existing.Merge(res) + existing.ComputeSummary(execTime) + return *existing } From 9547ec22fe4d136c761b75db4f74c175b5fa20fc Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 16:10:29 -0400 Subject: [PATCH 63/67] per query concurrency controlled via downstreamer --- pkg/logql/sharding.go | 204 +++++++++++-------------- pkg/logql/sharding_test.go | 1 + pkg/logql/shardmapper.go | 4 +- pkg/logql/shardmapper_test.go | 68 ++++----- pkg/logql/test_utils.go | 36 +++-- pkg/querier/queryrange/downstreamer.go | 96 ++++++++++-- 6 files changed, 237 insertions(+), 172 deletions(-) diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index 51a4a4d00e53c..cc6948eeccbbd 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -23,18 +23,18 @@ This includes a bunch of tooling for parallelization improvements based on backe // ShardedEngine is an Engine implementation that can split queries into more parallelizable forms via // querying the underlying backend shards individually and reaggregating them. type ShardedEngine struct { - timeout time.Duration - evaluator Evaluator - metrics *ShardingMetrics + timeout time.Duration + downstreamable Downstreamable + metrics *ShardingMetrics } // NewShardedEngine constructs a *ShardedEngine -func NewShardedEngine(opts EngineOpts, downstreamer Downstreamer, metrics *ShardingMetrics) *ShardedEngine { +func NewShardedEngine(opts EngineOpts, downstreamable Downstreamable, metrics *ShardingMetrics) *ShardedEngine { opts.applyDefault() return &ShardedEngine{ - timeout: opts.Timeout, - evaluator: NewDownstreamEvaluator(downstreamer), - metrics: metrics, + timeout: opts.Timeout, + downstreamable: downstreamable, + metrics: metrics, } } @@ -44,7 +44,7 @@ func (ng *ShardedEngine) Query(p Params, shards int) Query { return &query{ timeout: ng.timeout, params: p, - evaluator: ng.evaluator, + evaluator: NewDownstreamEvaluator(ng.downstreamable.Downstreamer()), parse: func(ctx context.Context, query string) (Expr, error) { logger := spanlogger.FromContext(ctx) mapper, err := NewShardMapper(shards, ng.metrics) @@ -87,30 +87,30 @@ func (d DownstreamLogSelectorExpr) String() string { // Contract: The embedded SampleExprs within a linked list of ConcatSampleExprs must be of the // same structure. This makes special implementations of SampleExpr.Associative() unnecessary. type ConcatSampleExpr struct { - SampleExpr + DownstreamSampleExpr next *ConcatSampleExpr } func (c ConcatSampleExpr) String() string { if c.next == nil { - return c.SampleExpr.String() + return c.DownstreamSampleExpr.String() } - return fmt.Sprintf("%s ++ %s", c.SampleExpr.String(), c.next.String()) + return fmt.Sprintf("%s ++ %s", c.DownstreamSampleExpr.String(), c.next.String()) } // ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr type ConcatLogSelectorExpr struct { - LogSelectorExpr + DownstreamLogSelectorExpr next *ConcatLogSelectorExpr } func (c ConcatLogSelectorExpr) String() string { if c.next == nil { - return c.LogSelectorExpr.String() + return c.DownstreamLogSelectorExpr.String() } - return fmt.Sprintf("%s ++ %s", c.LogSelectorExpr.String(), c.next.String()) + return fmt.Sprintf("%s ++ %s", c.DownstreamLogSelectorExpr.String(), c.next.String()) } type Shards []astmapper.ShardAnnotation @@ -140,10 +140,20 @@ func ParseShards(strs []string) (Shards, error) { return shards, nil } +type Downstreamable interface { + Downstreamer() Downstreamer +} + // Downstreamer is an interface for deferring responsibility for query execution. // It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. +type DownstreamQuery struct { + Expr Expr + Params Params + Shards Shards +} + type Downstreamer interface { - Downstream(Expr, Params, Shards) (Query, error) + Downstream(context.Context, []DownstreamQuery) ([]Result, error) } // DownstreamEvaluator is an evaluator which handles shard aware AST nodes @@ -152,24 +162,20 @@ type DownstreamEvaluator struct { defaultEvaluator *DefaultEvaluator } -// Exec runs a query and collects stats from the embedded Downstreamer -func (ev DownstreamEvaluator) Exec(ctx context.Context, expr Expr, p Params, shards Shards) (Result, error) { - qry, err := ev.Downstream(expr, p, shards) +// Downstream runs queries and collects stats from the embedded Downstreamer +func (ev DownstreamEvaluator) Downstream(ctx context.Context, queries []DownstreamQuery) ([]Result, error) { + results, err := ev.Downstreamer.Downstream(ctx, queries) if err != nil { - return Result{}, err + return nil, err } - res, err := qry.Exec(ctx) - if err != nil { - return Result{}, err - } - - err = stats.JoinResults(ctx, res.Statistics) - if err != nil { - level.Warn(util.Logger).Log("msg", "unable to merge downstream results", "err", err) + for _, res := range results { + if err := stats.JoinResults(ctx, res.Statistics); err != nil { + level.Warn(util.Logger).Log("msg", "unable to merge downstream results", "err", err) + } } - return res, nil + return results, nil } @@ -201,60 +207,48 @@ func (ev *DownstreamEvaluator) StepEvaluator( if e.shard != nil { shards = append(shards, *e.shard) } - res, err := ev.Exec(ctx, e.SampleExpr, params, shards) + results, err := ev.Downstream(ctx, []DownstreamQuery{{ + Expr: e.SampleExpr, + Params: params, + Shards: shards, + }}) if err != nil { return nil, err } - return ResultStepEvaluator(res, params) + return ResultStepEvaluator(results[0], params) case *ConcatSampleExpr: - type result struct { - stepper StepEvaluator - err error - } - ctx, cancel := context.WithCancel(ctx) cur := e - ch := make(chan result) - done := make(chan struct{}) - count := 0 - + var queries []DownstreamQuery for cur != nil { - go func(expr SampleExpr) { - eval, err := ev.StepEvaluator(ctx, nextEv, expr, params) - if err != nil { - level.Warn(util.Logger).Log("msg", "could not extract StepEvaluator", "err", err, "expr", expr.String()) - } - select { - case <-done: - case ch <- result{eval, err}: - } - }(cur.SampleExpr) + qry := DownstreamQuery{ + Expr: cur.DownstreamSampleExpr.SampleExpr, + Params: params, + } + if shard := cur.DownstreamSampleExpr.shard; shard != nil { + qry.Shards = Shards{*shard} + } + queries = append(queries, qry) cur = cur.next - count++ } - xs := make([]StepEvaluator, 0, count) - cleanup := func() { - cancel() // cancel ctx - done <- struct{}{} // send done signal to awaiting goroutines - // Close previously opened StepEvaluators - for _, x := range xs { - x.Close() // close unused StepEvaluators - } + results, err := ev.Downstream(ctx, queries) + if err != nil { + return nil, err } - for i := 0; i < count; i++ { - select { - case <-ctx.Done(): - defer cleanup() - return nil, ctx.Err() - case res := <-ch: - if res.err != nil { - defer cleanup() - return nil, res.err - } - xs = append(xs, res.stepper) + xs := make([]StepEvaluator, 0, len(queries)) + for i, res := range results { + stepper, err := ResultStepEvaluator(res, params) + if err != nil { + level.Warn(util.Logger).Log( + "msg", "could not extract StepEvaluator", + "err", err, + "expr", queries[i].Expr.String(), + ) + return nil, err } + xs = append(xs, stepper) } return ConcatEvaluator(xs) @@ -277,62 +271,48 @@ func (ev *DownstreamEvaluator) Iterator( if e.shard != nil { shards = append(shards, *e.shard) } - res, err := ev.Exec(ctx, e.LogSelectorExpr, params, shards) + results, err := ev.Downstream(ctx, []DownstreamQuery{{ + Expr: e.LogSelectorExpr, + Params: params, + Shards: shards, + }}) + if err != nil { return nil, err } - - return ResultIterator(res, params) + return ResultIterator(results[0], params) case *ConcatLogSelectorExpr: - type result struct { - iterator iter.EntryIterator - err error - } - ctx, cancel := context.WithCancel(ctx) cur := e - ch := make(chan result) - done := make(chan struct{}) - count := 0 - + var queries []DownstreamQuery for cur != nil { - go func(expr LogSelectorExpr) { - iterator, err := ev.Iterator(ctx, expr, params) - if err != nil { - level.Warn(util.Logger).Log("msg", "could not extract Iterator", "err", err, "expr", expr.String()) - } - select { - case <-done: - case ch <- result{iterator, err}: - } - }(cur.LogSelectorExpr) + qry := DownstreamQuery{ + Expr: cur.DownstreamLogSelectorExpr.LogSelectorExpr, + Params: params, + } + if shard := cur.DownstreamLogSelectorExpr.shard; shard != nil { + qry.Shards = Shards{*shard} + } + queries = append(queries, qry) cur = cur.next - count++ } - xs := make([]iter.EntryIterator, 0, count) - - cleanup := func() { - cancel() // cancel ctx - done <- struct{}{} // send done signal to awaiting goroutines - // Close previously opened Iterators - for _, x := range xs { - x.Close() // close unused Iterators - } + results, err := ev.Downstream(ctx, queries) + if err != nil { + return nil, err } - for i := 0; i < count; i++ { - select { - case <-ctx.Done(): - defer cleanup() - return nil, ctx.Err() - case res := <-ch: - if res.err != nil { - defer cleanup() - return nil, res.err - } - xs = append(xs, res.iterator) + xs := make([]iter.EntryIterator, 0, len(queries)) + for i, res := range results { + iter, err := ResultIterator(res, params) + if err != nil { + level.Warn(util.Logger).Log( + "msg", "could not extract Iterator", + "err", err, + "expr", queries[i].Expr.String(), + ) } + xs = append(xs, iter) } return iter.NewHeapIterator(ctx, xs, params.Direction()), nil diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index bf4a777dddc76..adfd487f5bb2c 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -75,6 +75,7 @@ func TestMappingEquivalence(t *testing.T) { res, err := qry.Exec(context.Background()) require.Nil(t, err) + shardedRes, err := shardedQry.Exec(context.Background()) require.Nil(t, err) diff --git a/pkg/logql/shardmapper.go b/pkg/logql/shardmapper.go index 4692f9f229881..2424c38b8fe10 100644 --- a/pkg/logql/shardmapper.go +++ b/pkg/logql/shardmapper.go @@ -164,7 +164,7 @@ func (m ShardMapper) mapLogSelectorExpr(expr LogSelectorExpr, r *shardRecorder) var head *ConcatLogSelectorExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: i, Of: m.shards, @@ -183,7 +183,7 @@ func (m ShardMapper) mapSampleExpr(expr SampleExpr, r *shardRecorder) SampleExpr var head *ConcatSampleExpr for i := m.shards - 1; i >= 0; i-- { head = &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: i, Of: m.shards, diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index f8ea937a670b3..96a5c7b033e6e 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -16,7 +16,7 @@ func TestStringer(t *testing.T) { }{ { in: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -28,7 +28,7 @@ func TestStringer(t *testing.T) { }, }, next: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -72,7 +72,7 @@ func TestMapSampleExpr(t *testing.T) { }, }, out: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -90,7 +90,7 @@ func TestMapSampleExpr(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -180,7 +180,7 @@ func TestMapping(t *testing.T) { { in: `{foo="bar"}`, expr: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -192,7 +192,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -210,7 +210,7 @@ func TestMapping(t *testing.T) { { in: `{foo="bar"} |= "error"`, expr: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -226,7 +226,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatLogSelectorExpr{ - LogSelectorExpr: DownstreamLogSelectorExpr{ + DownstreamLogSelectorExpr: DownstreamLogSelectorExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -248,7 +248,7 @@ func TestMapping(t *testing.T) { { in: `rate({foo="bar"}[5m])`, expr: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -266,7 +266,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -290,7 +290,7 @@ func TestMapping(t *testing.T) { { in: `count_over_time({foo="bar"}[5m])`, expr: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -308,7 +308,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -335,7 +335,7 @@ func TestMapping(t *testing.T) { grouping: &grouping{}, operation: OpTypeSum, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -357,7 +357,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -390,7 +390,7 @@ func TestMapping(t *testing.T) { params: 3, operation: OpTypeTopK, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -408,7 +408,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -439,7 +439,7 @@ func TestMapping(t *testing.T) { }, operation: OpTypeMax, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -457,7 +457,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -485,7 +485,7 @@ func TestMapping(t *testing.T) { operation: OpTypeSum, grouping: &grouping{}, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -507,7 +507,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -541,7 +541,7 @@ func TestMapping(t *testing.T) { grouping: &grouping{}, operation: OpTypeSum, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -563,7 +563,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -592,7 +592,7 @@ func TestMapping(t *testing.T) { operation: OpTypeSum, grouping: &grouping{}, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -614,7 +614,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -652,7 +652,7 @@ func TestMapping(t *testing.T) { }, operation: OpTypeSum, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -676,7 +676,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -715,7 +715,7 @@ func TestMapping(t *testing.T) { grouping: &grouping{}, operation: OpTypeMax, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -733,7 +733,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -766,7 +766,7 @@ func TestMapping(t *testing.T) { operation: OpTypeSum, grouping: &grouping{}, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -788,7 +788,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -828,7 +828,7 @@ func TestMapping(t *testing.T) { }, operation: OpTypeSum, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -852,7 +852,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, @@ -884,7 +884,7 @@ func TestMapping(t *testing.T) { operation: OpTypeSum, grouping: &grouping{}, left: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 0, Of: 2, @@ -906,7 +906,7 @@ func TestMapping(t *testing.T) { }, }, next: &ConcatSampleExpr{ - SampleExpr: DownstreamSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ shard: &astmapper.ShardAnnotation{ Shard: 1, Of: 2, diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 59850e3446862..f4e49e1c4c2c1 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -101,18 +101,30 @@ type MockDownstreamer struct { *Engine } -func (d MockDownstreamer) Downstream(expr Expr, p Params, shards Shards) (Query, error) { - params := NewLiteralParams( - expr.String(), - p.Start(), - p.End(), - p.Step(), - p.Interval(), - p.Direction(), - p.Limit(), - shards.Encode(), - ) - return d.Query(params), nil +func (m MockDownstreamer) Downstreamer() Downstreamer { return m } + +func (d MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery) ([]Result, error) { + results := make([]Result, 0, len(queries)) + for _, query := range queries { + params := NewLiteralParams( + query.Expr.String(), + query.Params.Start(), + query.Params.End(), + query.Params.Step(), + query.Params.Interval(), + query.Params.Direction(), + query.Params.Limit(), + query.Shards.Encode(), + ) + res, err := d.Query(params).Exec(ctx) + if err != nil { + return nil, err + } + + results = append(results, res) + } + return results, nil + } // create nStreams of nEntries with labelNames each where each label value diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index be7525f0d4931..e2a8e18e0cda1 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -14,16 +14,14 @@ import ( "github.com/grafana/loki/pkg/logql" ) +const ( + DefaultDownstreamConcurrency = 32 +) + type DownstreamHandler struct { next queryrange.Handler } -type QuerierFunc func(context.Context) (logql.Result, error) - -func (fn QuerierFunc) Exec(ctx context.Context) (logql.Result, error) { - return fn(ctx) -} - func ParamsToLokiRequest(params logql.Params) *LokiRequest { return &LokiRequest{ Query: params.Query(), @@ -36,21 +34,95 @@ func ParamsToLokiRequest(params logql.Params) *LokiRequest { } } -func (h DownstreamHandler) Downstream(expr logql.Expr, params logql.Params, shards logql.Shards) (logql.Query, error) { - req := ParamsToLokiRequest(params).WithShards(shards).WithQuery(expr.String()).(*LokiRequest) +func (h DownstreamHandler) Downstreamer() logql.Downstreamer { + p := DefaultDownstreamConcurrency + locks := make(chan struct{}, p) + for i := 0; i < p; i++ { + locks <- struct{}{} + } + return &instance{ + locks: locks, + handler: h.next, + } +} - return QuerierFunc(func(ctx context.Context) (logql.Result, error) { +// instance is an intermediate struct for controlling concurrency across a single query +type instance struct { + locks chan struct{} + handler queryrange.Handler +} - logger, ctx := spanlogger.New(ctx, "DownstreamHandler") +func (i instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery) ([]logql.Result, error) { + return i.For(queries, func(qry logql.DownstreamQuery) (logql.Result, error) { + req := ParamsToLokiRequest(qry.Params).WithShards(qry.Shards).WithQuery(qry.Expr.String()).(*LokiRequest) + logger, ctx := spanlogger.New(ctx, "DownstreamHandler.instance") defer logger.Finish() level.Debug(logger).Log("shards", req.Shards, "query", req.Query) - res, err := h.next.Do(ctx, req) + res, err := i.handler.Do(ctx, req) if err != nil { return logql.Result{}, err } return ResponseToResult(res) - }), nil + }) +} + +// For runs a function against a list of queries, collecting the results or returning an error. The indices are preserved such that input[i] maps to output[i]. +func (in instance) For( + queries []logql.DownstreamQuery, + fn func(logql.DownstreamQuery) (logql.Result, error), +) ([]logql.Result, error) { + type resp struct { + i int + res logql.Result + err error + } + + done := make(chan struct{}) + defer close(done) + + ch := make(chan resp) + + // Make one goroutine to dispatch the other goroutines, bounded by instance parallelism + go func() { + for i := 0; i < len(queries); i++ { + select { + case <-done: + break + case <-in.locks: + go func(i int) { + // release lock back into pool + defer func() { + in.locks <- struct{}{} + }() + + res, err := fn(queries[i]) + response := resp{ + i: i, + res: res, + err: err, + } + + // Feed the result into the channel unless the work has completed. + select { + case <-done: + case ch <- response: + } + }(i) + } + } + }() + + results := make([]logql.Result, len(queries)) + for i := 0; i < len(queries); i++ { + resp := <-ch + if resp.err != nil { + return nil, resp.err + } + results[resp.i] = resp.res + } + return results, nil + } // convert to matrix From b13abf9745f2a9de2620637aa60019d232b067ae Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 16:15:05 -0400 Subject: [PATCH 64/67] unexports decodereq --- pkg/storage/store.go | 8 ++++---- pkg/storage/store_test.go | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index bef4d8a197ec1..66e400439fd40 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -71,9 +71,9 @@ func NewTableClient(name string, cfg Config) (chunk.TableClient, error) { return storage.NewTableClient(name, cfg.Config) } -// DecodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher, +// decodeReq sanitizes an incoming request, rounds bounds, appends the __name__ matcher, // and adds the "__cortex_shard__" label if this is a sharded query. -func DecodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) { +func decodeReq(req logql.SelectParams) ([]*labels.Matcher, logql.LineFilter, model.Time, model.Time, error) { expr, err := req.LogSelector() if err != nil { return nil, nil, 0, 0, err @@ -149,7 +149,7 @@ func (s *store) lazyChunks(ctx context.Context, matchers []*labels.Matcher, from } func (s *store) GetSeries(ctx context.Context, req logql.SelectParams) ([]logproto.SeriesIdentifier, error) { - matchers, _, from, through, err := DecodeReq(req) + matchers, _, from, through, err := decodeReq(req) if err != nil { return nil, err } @@ -216,7 +216,7 @@ func (s *store) GetSeries(ctx context.Context, req logql.SelectParams) ([]logpro // LazyQuery returns an iterator that will query the store for more chunks while iterating instead of fetching all chunks upfront // for that request. func (s *store) LazyQuery(ctx context.Context, req logql.SelectParams) (iter.EntryIterator, error) { - matchers, filter, from, through, err := DecodeReq(req) + matchers, filter, from, through, err := decodeReq(req) if err != nil { return nil, err } diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 9619551b15c7f..c85d0561de2ad 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -464,7 +464,7 @@ func Test_store_decodeReq_Matchers(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - ms, _, _, _, err := DecodeReq(logql.SelectParams{QueryRequest: tt.req}) + ms, _, _, _, err := decodeReq(logql.SelectParams{QueryRequest: tt.req}) if err != nil { t.Errorf("store.GetSeries() error = %v", err) return From 8f3471b021d805f27a03732ed26b07d3bbc9d0d0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 17:03:21 -0400 Subject: [PATCH 65/67] queryrange testware --- pkg/querier/queryrange/downstreamer_test.go | 179 +++++++++++++++++++ pkg/querier/queryrange/querysharding_test.go | 37 ++++ 2 files changed, 216 insertions(+) create mode 100644 pkg/querier/queryrange/downstreamer_test.go diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go new file mode 100644 index 0000000000000..fa4000e3cfe8b --- /dev/null +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -0,0 +1,179 @@ +package queryrange + +import ( + "testing" + + "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/cortexproject/cortex/pkg/querier/queryrange" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/stats" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/promql" + "github.com/stretchr/testify/require" +) + +func testSampleStreams() []queryrange.SampleStream { + return []queryrange.SampleStream{ + { + Labels: []client.LabelAdapter{{Name: "foo", Value: "bar"}}, + Samples: []client.Sample{ + { + Value: 0, + TimestampMs: 0, + }, + { + Value: 1, + TimestampMs: 1, + }, + { + Value: 2, + TimestampMs: 2, + }, + }, + }, + { + Labels: []client.LabelAdapter{{Name: "bazz", Value: "buzz"}}, + Samples: []client.Sample{ + { + Value: 4, + TimestampMs: 4, + }, + { + Value: 5, + TimestampMs: 5, + }, + { + Value: 6, + TimestampMs: 6, + }, + }, + }, + } +} + +func TestSampleStreamToMatrix(t *testing.T) { + input := testSampleStreams() + expected := promql.Matrix{ + { + Metric: labels.FromMap(map[string]string{ + "foo": "bar", + }), + Points: []promql.Point{ + { + V: 0, + T: 0, + }, + { + V: 1, + T: 1, + }, + { + V: 2, + T: 2, + }, + }, + }, + { + Metric: labels.FromMap(map[string]string{ + "bazz": "buzz", + }), + Points: []promql.Point{ + { + V: 4, + T: 4, + }, + { + V: 5, + T: 5, + }, + { + V: 6, + T: 6, + }, + }, + }, + } + require.Equal(t, expected, sampleStreamToMatrix(input)) +} + +func TestResponseToResult(t *testing.T) { + for _, tc := range []struct { + desc string + input queryrange.Response + err bool + expected logql.Result + }{ + { + desc: "LokiResponse", + input: &LokiResponse{ + Data: LokiData{ + Result: []logproto.Stream{{ + Labels: `{foo="bar"}`, + }}, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ExecTime: 1}, + }, + }, + expected: logql.Result{ + Statistics: stats.Result{ + Summary: stats.Summary{ExecTime: 1}, + }, + Data: logql.Streams{{ + Labels: `{foo="bar"}`, + }}, + }, + }, + { + desc: "LokiResponseError", + input: &LokiResponse{ + Error: "foo", + ErrorType: "bar", + }, + err: true, + }, + { + desc: "LokiPromResponse", + input: &LokiPromResponse{ + Statistics: stats.Result{ + Summary: stats.Summary{ExecTime: 1}, + }, + Response: &queryrange.PrometheusResponse{ + Data: queryrange.PrometheusData{ + Result: testSampleStreams(), + }, + }, + }, + expected: logql.Result{ + Statistics: stats.Result{ + Summary: stats.Summary{ExecTime: 1}, + }, + Data: sampleStreamToMatrix(testSampleStreams()), + }, + }, + { + desc: "LokiPromResponseError", + input: &LokiPromResponse{ + Response: &queryrange.PrometheusResponse{ + Error: "foo", + ErrorType: "bar", + }, + }, + err: true, + }, + { + desc: "UnexpectedTypeError", + input: nil, + err: true, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + out, err := ResponseToResult(tc.input) + if tc.err { + require.NotNil(t, err) + } + require.Equal(t, tc.expected, out) + }) + } +} diff --git a/pkg/querier/queryrange/querysharding_test.go b/pkg/querier/queryrange/querysharding_test.go index 3f8f53df62772..670169f3948a1 100644 --- a/pkg/querier/queryrange/querysharding_test.go +++ b/pkg/querier/queryrange/querysharding_test.go @@ -2,6 +2,7 @@ package queryrange import ( "context" + "fmt" "sort" "testing" "time" @@ -139,6 +140,42 @@ func Test_astMapper(t *testing.T) { } +func Test_hasShards(t *testing.T) { + for i, tc := range []struct { + input queryrange.ShardingConfigs + expected bool + }{ + { + input: queryrange.ShardingConfigs{ + {}, + }, + expected: false, + }, + { + input: queryrange.ShardingConfigs{ + {RowShards: 16}, + }, + expected: true, + }, + { + input: queryrange.ShardingConfigs{ + {}, + {RowShards: 16}, + {}, + }, + expected: true, + }, + { + input: nil, + expected: false, + }, + } { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + require.Equal(t, tc.expected, hasShards(tc.input)) + }) + } +} + // astmapper successful stream & prom conversion func mockHandler(resp queryrange.Response, err error) queryrange.Handler { From df85eebd9015d495ff9b311e0301e72cc6881770 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 May 2020 18:23:59 -0400 Subject: [PATCH 66/67] downstreamer tests --- pkg/querier/queryrange/downstreamer.go | 10 +- pkg/querier/queryrange/downstreamer_test.go | 165 ++++++++++++++++++++ 2 files changed, 171 insertions(+), 4 deletions(-) diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index e2a8e18e0cda1..2fba180e33be9 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -41,15 +41,17 @@ func (h DownstreamHandler) Downstreamer() logql.Downstreamer { locks <- struct{}{} } return &instance{ - locks: locks, - handler: h.next, + parallelism: p, + locks: locks, + handler: h.next, } } // instance is an intermediate struct for controlling concurrency across a single query type instance struct { - locks chan struct{} - handler queryrange.Handler + parallelism int + locks chan struct{} + handler queryrange.Handler } func (i instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery) ([]logql.Result, error) { diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index fa4000e3cfe8b..dbdbe47a1e820 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -1,7 +1,11 @@ package queryrange import ( + "context" + "errors" + "sync" "testing" + "time" "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/querier/queryrange" @@ -177,3 +181,164 @@ func TestResponseToResult(t *testing.T) { }) } } + +func TestDownstreamHandler(t *testing.T) { + + // Pretty poor test, but this is just a passthrough struct, so ensure we create locks + // and can consume them + h := DownstreamHandler{nil} + in := h.Downstreamer().(*instance) + require.Equal(t, DefaultDownstreamConcurrency, in.parallelism) + require.NotNil(t, in.locks) + ensureParallelism(t, in, in.parallelism) +} + +// Consumes the locks in an instance, making sure they're all available. Does not replace them and thus instance is unusuable after. This is a cleanup test to ensure internal state +func ensureParallelism(t *testing.T, in *instance, n int) { + for i := 0; i < n; i++ { + select { + case <-in.locks: + case <-time.After(time.Millisecond): + require.FailNow(t, "lock couldn't be acquired") + } + } + // ensure no more locks available + select { + case <-in.locks: + require.FailNow(t, "unexpected lock acquisition") + default: + } +} + +func TestInstanceFor(t *testing.T) { + mkIn := func() *instance { return DownstreamHandler{nil}.Downstreamer().(*instance) } + in := mkIn() + + queries := make([]logql.DownstreamQuery, in.parallelism+1) + var mtx sync.Mutex + var ct int + + // ensure we can execute queries that number more than the parallelism parameter + _, err := in.For(queries, func(_ logql.DownstreamQuery) (logql.Result, error) { + mtx.Lock() + defer mtx.Unlock() + ct++ + return logql.Result{}, nil + }) + require.Nil(t, err) + require.Equal(t, len(queries), ct) + ensureParallelism(t, in, in.parallelism) + + // ensure an early error abandons the other queues queries + in = mkIn() + ct = 0 + _, err = in.For(queries, func(_ logql.DownstreamQuery) (logql.Result, error) { + mtx.Lock() + defer mtx.Unlock() + ct++ + return logql.Result{}, errors.New("testerr") + }) + require.NotNil(t, err) + // Ensure no more than the initial batch was parallelized. + require.LessOrEqual(t, ct, in.parallelism) + ensureParallelism(t, in, in.parallelism) + + in = mkIn() + results, err := in.For( + []logql.DownstreamQuery{ + { + Shards: logql.Shards{ + {Shard: 0, Of: 2}, + }, + }, + { + Shards: logql.Shards{ + {Shard: 1, Of: 2}, + }, + }, + }, + func(qry logql.DownstreamQuery) (logql.Result, error) { + + return logql.Result{ + Data: logql.Streams{{ + Labels: qry.Shards[0].String(), + }}, + }, nil + }, + ) + require.Nil(t, err) + require.Equal( + t, + []logql.Result{ + logql.Result{ + Data: logql.Streams{{Labels: "0_of_2"}}, + }, + logql.Result{ + Data: logql.Streams{{Labels: "1_of_2"}}, + }, + }, + results, + ) + ensureParallelism(t, in, in.parallelism) + +} + +func TestInstanceDownstream(t *testing.T) { + params := logql.NewLiteralParams( + "", + time.Now(), + time.Now(), + 0, + 0, + logproto.BACKWARD, + 1000, + nil, + ) + expr, err := logql.ParseExpr(`{foo="bar"}`) + require.Nil(t, err) + + expectedResp := func() *LokiResponse { + return &LokiResponse{ + Data: LokiData{ + Result: []logproto.Stream{{ + Labels: `{foo="bar"}`, + }}, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ExecTime: 1}, + }, + } + } + + queries := []logql.DownstreamQuery{ + { + Expr: expr, + Params: params, + Shards: logql.Shards{{Shard: 0, Of: 2}}, + }, + } + + var got queryrange.Request + var want queryrange.Request + handler := queryrange.HandlerFunc( + func(_ context.Context, req queryrange.Request) (queryrange.Response, error) { + // for some reason these seemingly can't be checked in their own goroutines, + // so we assign them to scoped variables for later comparison. + got = req + want = ParamsToLokiRequest(params).WithShards(logql.Shards{{Shard: 0, Of: 2}}).WithQuery(expr.String()) + + return expectedResp(), nil + }, + ) + + expected, err := ResponseToResult(expectedResp()) + require.Nil(t, err) + + results, err := DownstreamHandler{handler}.Downstreamer().Downstream(context.Background(), queries) + + require.Equal(t, want, got) + + require.Nil(t, err) + require.Equal(t, []logql.Result{expected}, results) + +} From 388c547d5e81e87eedd14b6081f9cac20d070955 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 26 May 2020 17:24:46 -0400 Subject: [PATCH 67/67] pr requests --- pkg/chunkenc/memchunk_test.go | 5 ++--- pkg/logql/engine.go | 2 +- pkg/logql/sharding.go | 13 ++++++++++--- pkg/logql/stats/context.go | 4 ++-- pkg/logql/stats/context_test.go | 4 ++-- 5 files changed, 17 insertions(+), 11 deletions(-) diff --git a/pkg/chunkenc/memchunk_test.go b/pkg/chunkenc/memchunk_test.go index e5582e6d4f023..72b1961d41e63 100644 --- a/pkg/chunkenc/memchunk_test.go +++ b/pkg/chunkenc/memchunk_test.go @@ -10,7 +10,6 @@ import ( "testing" "time" - "github.com/cortexproject/cortex/pkg/util" "github.com/stretchr/testify/assert" "github.com/dustin/go-humanize" @@ -437,7 +436,7 @@ func TestChunkStats(t *testing.T) { t.Fatal(err) } // test on a chunk filling up - s := stats.Snapshot(ctx, time.Since(first), util.Logger) + s := stats.Snapshot(ctx, time.Since(first)) require.Equal(t, int64(expectedSize), s.Summary.TotalBytesProcessed) require.Equal(t, int64(inserted), s.Summary.TotalLinesProcessed) @@ -465,7 +464,7 @@ func TestChunkStats(t *testing.T) { if err := it.Close(); err != nil { t.Fatal(err) } - s = stats.Snapshot(ctx, time.Since(first), util.Logger) + s = stats.Snapshot(ctx, time.Since(first)) require.Equal(t, int64(expectedSize), s.Summary.TotalBytesProcessed) require.Equal(t, int64(inserted), s.Summary.TotalLinesProcessed) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index 3550d5d2819bd..51f9249f908d4 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -130,7 +130,7 @@ func (q *query) Exec(ctx context.Context) (Result, error) { data, err := q.Eval(ctx) - statResult = stats.Snapshot(ctx, time.Since(start), log) + statResult = stats.Snapshot(ctx, time.Since(start)) statResult.Log(level.Debug(log)) status := "200" diff --git a/pkg/logql/sharding.go b/pkg/logql/sharding.go index cc6948eeccbbd..135b7bcc04253 100644 --- a/pkg/logql/sharding.go +++ b/pkg/logql/sharding.go @@ -17,7 +17,14 @@ import ( ) /* -This includes a bunch of tooling for parallelization improvements based on backend shard factors. In schemas 10+ a shard factor (default 16) is introduced in the index store, calculated by hashing the label set of a log stream. This allows us to perform certain optimizations that fall under the umbrella of query remapping and querying shards individually. For instance, `{app="foo"} |= "bar"` can be executed on each shard independently, then reaggregated. There are also a class of optimizations that can be performed by altering a query into a functionally equivalent, but more parallelizable form. For instance, an average can be remapped into a sum/count, which can then take advantage of our sharded execution model. +This includes a bunch of tooling for parallelization improvements based on backend shard factors. +In schemas 10+ a shard factor (default 16) is introduced in the index store, +calculated by hashing the label set of a log stream. This allows us to perform certain optimizations +that fall under the umbrella of query remapping and querying shards individually. +For instance, `{app="foo"} |= "bar"` can be executed on each shard independently, then reaggregated. +There are also a class of optimizations that can be performed by altering a query into a functionally equivalent, +but more parallelizable form. For instance, an average can be remapped into a sum/count, +which can then take advantage of our sharded execution model. */ // ShardedEngine is an Engine implementation that can split queries into more parallelizable forms via @@ -144,14 +151,14 @@ type Downstreamable interface { Downstreamer() Downstreamer } -// Downstreamer is an interface for deferring responsibility for query execution. -// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type DownstreamQuery struct { Expr Expr Params Params Shards Shards } +// Downstreamer is an interface for deferring responsibility for query execution. +// It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type Downstreamer interface { Downstream(context.Context, []DownstreamQuery) ([]Result, error) } diff --git a/pkg/logql/stats/context.go b/pkg/logql/stats/context.go index 9a97b82351c41..cf45356a1bdcb 100644 --- a/pkg/logql/stats/context.go +++ b/pkg/logql/stats/context.go @@ -13,7 +13,7 @@ Then you can update statistics by mutating data by using: Finally to get a snapshot of the current query statistic use - stats.Snapshot(ctx, time.Since(start), logger) + stats.Snapshot(ctx, time.Since(start)) Ingester statistics are sent across the GRPC stream using Trailers see https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md @@ -144,7 +144,7 @@ func GetStoreData(ctx context.Context) *StoreData { } // Snapshot compute query statistics from a context using the total exec time. -func Snapshot(ctx context.Context, execTime time.Duration, log log.Logger) Result { +func Snapshot(ctx context.Context, execTime time.Duration) Result { // ingester data is decoded from grpc trailers. res := decodeTrailers(ctx) // collect data from store. diff --git a/pkg/logql/stats/context_test.go b/pkg/logql/stats/context_test.go index a7ecace04ac1e..8a688494ff572 100644 --- a/pkg/logql/stats/context_test.go +++ b/pkg/logql/stats/context_test.go @@ -27,7 +27,7 @@ func TestSnapshot(t *testing.T) { fakeIngesterQuery(ctx) fakeIngesterQuery(ctx) - res := Snapshot(ctx, 2*time.Second, util.Logger) + res := Snapshot(ctx, 2*time.Second) res.Log(util.Logger) expected := Result{ Ingester: Ingester{ @@ -101,7 +101,7 @@ func TestSnapshot_MergesResults(t *testing.T) { err := JoinResults(ctx, expected) require.Nil(t, err) - res := Snapshot(ctx, 2*time.Second, util.Logger) + res := Snapshot(ctx, 2*time.Second) require.Equal(t, expected, res) }