Logql/parallel binop (#5317)

* adds justification for keeping Downstreamer parallelism

* loads binop legs in parallel

* increases downstreamer default concurrency

* astmapper spanlogger

* always clone expr during mapping to prevent mutability bugs

* Revert "astmapper spanlogger"

This reverts commit 23f6b55c14.

* cleanup + use errgroup
pull/5289/head^2
Owen Diehl 3 years ago committed by GitHub
parent c477e29c3f
commit 2af3ca077e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      pkg/logql/ast.go
  2. 36
      pkg/logql/evaluator.go
  3. 8
      pkg/logql/shardmapper.go
  4. 8
      pkg/querier/queryrange/downstreamer.go

@ -26,6 +26,10 @@ type Expr interface {
fmt.Stringer
}
func Clone(e Expr) (Expr, error) {
return ParseExpr(e.String())
}
type QueryParams interface {
LogSelector() (LogSelectorExpr, error)
GetStart() time.Time

@ -11,6 +11,7 @@ import (
"github.com/pkg/errors"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
"golang.org/x/sync/errgroup"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
@ -547,16 +548,31 @@ func binOpStepEvaluator(
)
}
// we have two non literal legs
lse, err := ev.StepEvaluator(ctx, ev, expr.SampleExpr, q)
if err != nil {
return nil, err
}
rse, err := ev.StepEvaluator(ctx, ev, expr.RHS, q)
if err != nil {
var lse, rse StepEvaluator
g, ctx := errgroup.WithContext(ctx)
// We have two non literal legs,
// load them in parallel
g.Go(func() error {
var err error
lse, err = ev.StepEvaluator(ctx, ev, expr.SampleExpr, q)
return err
})
g.Go(func() error {
var err error
rse, err = ev.StepEvaluator(ctx, ev, expr.RHS, q)
return err
})
// ensure both sides are loaded before returning the combined evaluator
if err := g.Wait(); err != nil {
return nil, err
}
// keep a scoped reference to err as it's referenced in the Error()
// implementation of this StepEvaluator
var scopedErr error
return newStepEvaluator(func() (bool, int64, promql.Vector) {
var (
ts int64
@ -593,7 +609,7 @@ func binOpStepEvaluator(
case OpTypeUnless:
results = vectorUnless(lhs, rhs, lsigs, rsigs)
default:
results, err = vectorBinop(expr.Op, expr.Opts, lhs, rhs, lsigs, rsigs)
results, scopedErr = vectorBinop(expr.Op, expr.Opts, lhs, rhs, lsigs, rsigs)
}
return true, ts, results
}, func() (lastError error) {
@ -605,8 +621,8 @@ func binOpStepEvaluator(
return lastError
}, func() error {
var errs []error
if err != nil {
errs = append(errs, err)
if scopedErr != nil {
errs = append(errs, scopedErr)
}
for _, ev := range []StepEvaluator{lse, rse} {
if err := ev.Error(); err != nil {

@ -111,8 +111,8 @@ func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) {
return false, nil, err
}
mappedStr := mapped.String()
originalStr := parsed.String()
mappedStr := mapped.String()
noop = originalStr == mappedStr
if noop {
m.metrics.parsed.WithLabelValues(NoopKey).Inc()
@ -126,6 +126,12 @@ func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) {
}
func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) {
// immediately clone the passed expr to avoid mutating the original
expr, err := Clone(expr)
if err != nil {
return nil, err
}
switch e := expr.(type) {
case *LiteralExpr:
return e, nil

@ -18,7 +18,7 @@ import (
)
const (
DefaultDownstreamConcurrency = 32
DefaultDownstreamConcurrency = 128
)
type DownstreamHandler struct {
@ -48,6 +48,12 @@ func ParamsToLokiRequest(params logql.Params, shards logql.Shards) queryrangebas
}
}
// Note: After the introduction of the LimitedRoundTripper,
// bounding concurrency in the downstreamer is mostly redundant
// The reason we don't remove it is to prevent malicious queries
// from creating an unreasonably large number of goroutines, such as
// the case of a query like `a / a / a / a / a ..etc`, which could try
// to shard each leg, quickly dispatching an unreasonable number of goroutines.
func (h DownstreamHandler) Downstreamer() logql.Downstreamer {
p := DefaultDownstreamConcurrency
locks := make(chan struct{}, p)

Loading…
Cancel
Save