Refactor LogQL syntax into its own package (#5539)

* Refactor LogQL syntax into its own package

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Merge upstream

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Update pkg/loghttp/series.go

Co-authored-by: Susana Ferreira <ssncferreira@gmail.com>

* lint

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* lint

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

Co-authored-by: Susana Ferreira <ssncferreira@gmail.com>
pull/5502/head^2
Cyril Tovena 4 years ago committed by GitHub
parent 86d196ef21
commit 79f0e349f5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      cmd/migrate/main.go
  2. 8
      pkg/chunkenc/memchunk_test.go
  3. 4
      pkg/distributor/distributor.go
  4. 4
      pkg/distributor/validator_test.go
  5. 3
      pkg/ingester/ingester.go
  6. 3
      pkg/ingester/instance.go
  7. 3
      pkg/ingester/instance_test.go
  8. 8
      pkg/ingester/tailer.go
  9. 4
      pkg/loghttp/push/push.go
  10. 14
      pkg/loghttp/series.go
  11. 19
      pkg/logql/downstream.go
  12. 74
      pkg/logql/engine.go
  13. 5
      pkg/logql/engine_test.go
  14. 353
      pkg/logql/evaluator.go
  15. 64
      pkg/logql/evaluator_test.go
  16. 357
      pkg/logql/functions.go
  17. 4
      pkg/logql/log/parser_hints_test.go
  18. 25
      pkg/logql/matchers.go
  19. 66
      pkg/logql/matchers_test.go
  20. 7
      pkg/logql/metrics.go
  21. 26
      pkg/logql/optimize.go
  22. 4
      pkg/logql/optimize_test.go
  23. 273
      pkg/logql/range_vector.go
  24. 5
      pkg/logql/range_vector_test.go
  25. 119
      pkg/logql/shardmapper.go
  26. 721
      pkg/logql/shardmapper_test.go
  27. 374
      pkg/logql/syntax/ast.go
  28. 25
      pkg/logql/syntax/ast_test.go
  29. 2
      pkg/logql/syntax/expr.y
  30. 4
      pkg/logql/syntax/expr.y.go
  31. 86
      pkg/logql/syntax/extractor.go
  32. 2
      pkg/logql/syntax/extractor_test.go
  33. 0
      pkg/logql/syntax/fuzz.go
  34. 0
      pkg/logql/syntax/fuzz_test.go
  35. 2
      pkg/logql/syntax/lex.go
  36. 54
      pkg/logql/syntax/lex_test.go
  37. 39
      pkg/logql/syntax/parser.go
  38. 187
      pkg/logql/syntax/parser_test.go
  39. 2
      pkg/logql/syntax/walk.go
  40. 2
      pkg/logql/syntax/walk_test.go
  41. 65
      pkg/logql/vector/heap.go
  42. 15
      pkg/querier/http.go
  43. 9
      pkg/querier/multi_tenant_querier.go
  44. 5
      pkg/querier/multi_tenant_querier_test.go
  45. 1
      pkg/querier/querier.go
  46. 7
      pkg/querier/queryrange/codec.go
  47. 4
      pkg/querier/queryrange/downstreamer_test.go
  48. 18
      pkg/querier/queryrange/roundtrip.go
  49. 6
      pkg/querier/queryrange/split_by_interval.go
  50. 7
      pkg/ruler/compat.go
  51. 10
      pkg/storage/batch.go
  52. 4
      pkg/storage/hack/main.go
  53. 4
      pkg/storage/stores/shipper/compactor/deletion/delete_request_test.go
  54. 4
      pkg/storage/stores/shipper/compactor/deletion/delete_requests_manager_test.go
  55. 4
      pkg/storage/stores/shipper/compactor/retention/util_test.go
  56. 4
      pkg/storage/tsdb/querier_test.go
  57. 6
      pkg/storage/util_test.go
  58. 6
      pkg/validation/limits.go

@ -15,12 +15,12 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/tenant"
"github.com/prometheus/prometheus/model/labels"
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/loki"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk"
@ -126,7 +126,7 @@ func main() {
matchers := []*labels.Matcher{nameLabelMatcher}
if *match != "" {
m, err := logql.ParseMatchers(*match)
m, err := syntax.ParseMatchers(*match)
if err != nil {
log.Println("Failed to parse log matcher:", err)
os.Exit(1)

@ -23,8 +23,8 @@ import (
"github.com/grafana/loki/pkg/chunkenc/testdata"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
@ -1037,7 +1037,7 @@ func BenchmarkBufferedIteratorLabels(b *testing.B) {
} {
b.Run(test, func(b *testing.B) {
b.ReportAllocs()
expr, err := logql.ParseLogSelector(test, true)
expr, err := syntax.ParseLogSelector(test, true)
if err != nil {
b.Fatal(err)
}
@ -1076,7 +1076,7 @@ func BenchmarkBufferedIteratorLabels(b *testing.B) {
} {
b.Run(test, func(b *testing.B) {
b.ReportAllocs()
expr, err := logql.ParseSampleExpr(test)
expr, err := syntax.ParseSampleExpr(test)
if err != nil {
b.Fatal(err)
}
@ -1119,7 +1119,7 @@ func Test_HeadIteratorReverse(t *testing.T) {
}
assertOrder := func(t *testing.T, total int64) {
expr, err := logql.ParseLogSelector(`{app="foo"} | logfmt`, true)
expr, err := syntax.ParseLogSelector(`{app="foo"} | logfmt`, true)
require.NoError(t, err)
p, err := expr.Pipeline()
require.NoError(t, err)

@ -24,7 +24,7 @@ import (
"github.com/grafana/loki/pkg/distributor/clientpool"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/runtime"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention"
"github.com/grafana/loki/pkg/tenant"
@ -423,7 +423,7 @@ func (d *Distributor) parseStreamLabels(vContext validationContext, key string,
if ok {
return labelVal.(string), nil
}
ls, err := logql.ParseLabels(key)
ls, err := syntax.ParseLabels(key)
if err != nil {
return "", httpgrpc.Errorf(http.StatusBadRequest, validation.InvalidLabelsErrorMsg, key, err)
}

@ -12,7 +12,7 @@ import (
"github.com/weaveworks/common/httpgrpc"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/validation"
)
@ -203,7 +203,7 @@ func TestValidator_ValidateLabels(t *testing.T) {
}
func mustParseLabels(s string) labels.Labels {
ls, err := logql.ParseLabels(s)
ls, err := syntax.ParseLabels(s)
if err != nil {
panic(err)
}

@ -26,6 +26,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/runtime"
"github.com/grafana/loki/pkg/storage"
@ -677,7 +678,7 @@ func (i *Ingester) GetChunkIDs(ctx context.Context, req *logproto.GetChunkIDsReq
// parse the request
start, end := errUtil.RoundToMilliseconds(reqStart, req.End)
matchers, err := logql.ParseMatchers(req.Matchers)
matchers, err := syntax.ParseMatchers(req.Matchers)
if err != nil {
return nil, err
}

@ -22,6 +22,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/runtime"
@ -220,7 +221,7 @@ func (i *instance) createStream(pushReqStream logproto.Stream, record *WALRecord
return nil, httpgrpc.Errorf(http.StatusTooManyRequests, validation.StreamLimitErrorMsg)
}
labels, err := logql.ParseLabels(pushReqStream.Labels)
labels, err := syntax.ParseLabels(pushReqStream.Labels)
if err != nil {
if i.configs.LogStreamCreation(i.instanceID) {
level.Debug(util_log.Logger).Log(

@ -10,6 +10,7 @@ import (
"testing"
"time"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/pkg/errors"
@ -592,7 +593,7 @@ func Test_ChunkFilter(t *testing.T) {
for it.Next() {
require.NoError(t, it.Error())
lbs, err := logql.ParseLabels(it.Labels())
lbs, err := syntax.ParseLabels(it.Labels())
require.NoError(t, err)
require.NotEqual(t, "dispatcher", lbs.Get("log_stream"))
}

@ -11,8 +11,8 @@ import (
"golang.org/x/net/context"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/util"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -28,8 +28,8 @@ type tailer struct {
id uint32
orgID string
matchers []*labels.Matcher
pipeline logql.Pipeline
expr logql.Expr
pipeline syntax.Pipeline
expr syntax.Expr
pipelineMtx sync.Mutex
sendChan chan *logproto.Stream
@ -48,7 +48,7 @@ type tailer struct {
}
func newTailer(orgID, query string, conn TailServer, maxDroppedStreams int) (*tailer, error) {
expr, err := logql.ParseLogSelector(query, true)
expr, err := syntax.ParseLogSelector(query, true)
if err != nil {
return nil, err
}

@ -19,7 +19,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
loki_util "github.com/grafana/loki/pkg/util"
@ -124,7 +124,7 @@ func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRete
streamLabelsSize += int64(len(s.Labels))
var retentionHours string
if tenantsRetention != nil {
lbs, err := logql.ParseLabels(s.Labels)
lbs, err := syntax.ParseLabels(s.Labels)
if err != nil {
return nil, err
}

@ -6,6 +6,7 @@ import (
"strings"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
)
type SeriesResponse struct {
@ -61,3 +62,16 @@ func union(cols ...[]string) []string {
return res
}
func ParseAndValidateSeriesQuery(r *http.Request) (*logproto.SeriesRequest, error) {
req, err := ParseSeriesQuery(r)
if err != nil {
return nil, err
}
// ensure matchers are valid before fanning out to ingesters/store as well as returning valuable parsing errors
// instead of 500s
if _, err = logql.Match(req.Groups); err != nil {
return nil, err
}
return req, nil
}

@ -11,6 +11,7 @@ import (
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
@ -61,13 +62,13 @@ func NewDownstreamEngine(opts EngineOpts, downstreamable Downstreamable, metrics
}
// Query constructs a Query
func (ng *DownstreamEngine) Query(p Params, mapped Expr) Query {
func (ng *DownstreamEngine) Query(p Params, mapped syntax.Expr) Query {
return &query{
logger: ng.logger,
timeout: ng.timeout,
params: p,
evaluator: NewDownstreamEvaluator(ng.downstreamable.Downstreamer()),
parse: func(_ context.Context, _ string) (Expr, error) {
parse: func(_ context.Context, _ string) (syntax.Expr, error) {
return mapped, nil
},
limits: ng.limits,
@ -77,7 +78,7 @@ func (ng *DownstreamEngine) Query(p Params, mapped Expr) Query {
// DownstreamSampleExpr is a SampleExpr which signals downstream computation
type DownstreamSampleExpr struct {
shard *astmapper.ShardAnnotation
SampleExpr
syntax.SampleExpr
}
func (d DownstreamSampleExpr) String() string {
@ -87,14 +88,14 @@ func (d DownstreamSampleExpr) String() string {
// DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation
type DownstreamLogSelectorExpr struct {
shard *astmapper.ShardAnnotation
LogSelectorExpr
syntax.LogSelectorExpr
}
func (d DownstreamLogSelectorExpr) String() string {
return fmt.Sprintf("downstream<%s, shard=%s>", d.LogSelectorExpr.String(), d.shard)
}
func (d DownstreamSampleExpr) Walk(f WalkFn) { f(d) }
func (d DownstreamSampleExpr) Walk(f syntax.WalkFn) { f(d) }
// ConcatSampleExpr is an expr for concatenating multiple SampleExpr
// Contract: The embedded SampleExprs within a linked list of ConcatSampleExprs must be of the
@ -112,7 +113,7 @@ func (c ConcatSampleExpr) String() string {
return fmt.Sprintf("%s ++ %s", c.DownstreamSampleExpr.String(), c.next.String())
}
func (c ConcatSampleExpr) Walk(f WalkFn) {
func (c ConcatSampleExpr) Walk(f syntax.WalkFn) {
f(c)
f(c.next)
}
@ -163,7 +164,7 @@ type Downstreamable interface {
}
type DownstreamQuery struct {
Expr Expr
Expr syntax.Expr
Params Params
Shards Shards
}
@ -215,7 +216,7 @@ func NewDownstreamEvaluator(downstreamer Downstreamer) *DownstreamEvaluator {
func (ev *DownstreamEvaluator) StepEvaluator(
ctx context.Context,
nextEv SampleEvaluator,
expr SampleExpr,
expr syntax.SampleExpr,
params Params,
) (StepEvaluator, error) {
switch e := expr.(type) {
@ -280,7 +281,7 @@ func (ev *DownstreamEvaluator) StepEvaluator(
// Iterator returns the iter.EntryIterator for a given LogSelectorExpr
func (ev *DownstreamEvaluator) Iterator(
ctx context.Context,
expr LogSelectorExpr,
expr syntax.LogSelectorExpr,
params Params,
) (iter.EntryIterator, error) {
switch e := expr.(type) {

@ -4,8 +4,10 @@ import (
"context"
"errors"
"flag"
"fmt"
"math"
"sort"
"strings"
"time"
"github.com/go-kit/log"
@ -18,6 +20,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/tenant"
@ -36,6 +39,59 @@ var (
lastEntryMinTime = time.Unix(-100, 0)
)
type QueryParams interface {
LogSelector() (syntax.LogSelectorExpr, error)
GetStart() time.Time
GetEnd() time.Time
GetShards() []string
}
// SelectParams specifies parameters passed to data selections.
type SelectLogParams struct {
*logproto.QueryRequest
}
func (s SelectLogParams) String() string {
if s.QueryRequest != nil {
return fmt.Sprintf("selector=%s, direction=%s, start=%s, end=%s, limit=%d, shards=%s",
s.Selector, logproto.Direction_name[int32(s.Direction)], s.Start, s.End, s.Limit, strings.Join(s.Shards, ","))
}
return ""
}
// LogSelector returns the LogSelectorExpr from the SelectParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectLogParams) LogSelector() (syntax.LogSelectorExpr, error) {
return syntax.ParseLogSelector(s.Selector, true)
}
type SelectSampleParams struct {
*logproto.SampleQueryRequest
}
// Expr returns the SampleExpr from the SelectSampleParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectSampleParams) Expr() (syntax.SampleExpr, error) {
return syntax.ParseSampleExpr(s.Selector)
}
// LogSelector returns the LogSelectorExpr from the SelectParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectSampleParams) LogSelector() (syntax.LogSelectorExpr, error) {
expr, err := syntax.ParseSampleExpr(s.Selector)
if err != nil {
return nil, err
}
return expr.Selector(), nil
}
// Querier allows a LogQL expression to fetch an EntryIterator for a
// set of matchers and filters
type Querier interface {
SelectLogs(context.Context, SelectLogParams) (iter.EntryIterator, error)
SelectSamples(context.Context, SelectSampleParams) (iter.SampleIterator, error)
}
// EngineOpts is the list of options to use with the LogQL query engine.
type EngineOpts struct {
// Timeout for queries execution
@ -88,8 +144,8 @@ func (ng *Engine) Query(params Params) Query {
timeout: ng.timeout,
params: params,
evaluator: ng.evaluator,
parse: func(_ context.Context, query string) (Expr, error) {
return ParseExpr(query)
parse: func(_ context.Context, query string) (syntax.Expr, error) {
return syntax.ParseExpr(query)
},
record: true,
limits: ng.limits,
@ -106,7 +162,7 @@ type query struct {
logger log.Logger
timeout time.Duration
params Params
parse func(context.Context, string) (Expr, error)
parse func(context.Context, string) (syntax.Expr, error)
limits Limits
evaluator Evaluator
record bool
@ -163,11 +219,11 @@ func (q *query) Eval(ctx context.Context) (promql_parser.Value, error) {
}
switch e := expr.(type) {
case SampleExpr:
case syntax.SampleExpr:
value, err := q.evalSample(ctx, e)
return value, err
case LogSelectorExpr:
case syntax.LogSelectorExpr:
iter, err := q.evaluator.Iterator(ctx, e, q.params)
if err != nil {
return nil, err
@ -182,8 +238,8 @@ func (q *query) Eval(ctx context.Context) (promql_parser.Value, error) {
}
// evalSample evaluate a sampleExpr
func (q *query) evalSample(ctx context.Context, expr SampleExpr) (promql_parser.Value, error) {
if lit, ok := expr.(*LiteralExpr); ok {
func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_parser.Value, error) {
if lit, ok := expr.(*syntax.LiteralExpr); ok {
return q.evalLiteral(ctx, lit)
}
@ -267,10 +323,10 @@ func (q *query) evalSample(ctx context.Context, expr SampleExpr) (promql_parser.
return result, stepEvaluator.Error()
}
func (q *query) evalLiteral(_ context.Context, expr *LiteralExpr) (promql_parser.Value, error) {
func (q *query) evalLiteral(_ context.Context, expr *syntax.LiteralExpr) (promql_parser.Value, error) {
s := promql.Scalar{
T: q.params.Start().UnixNano() / int64(time.Millisecond),
V: expr.value,
V: expr.Value(),
}
if GetRangeType(q.params) == InstantType {

@ -20,6 +20,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/util"
@ -2366,7 +2367,7 @@ type logData struct {
type generator func(i int64) logData
func newStream(n int64, f generator, lbsString string) logproto.Stream {
labels, err := ParseLabels(lbsString)
labels, err := syntax.ParseLabels(lbsString)
if err != nil {
panic(err)
}
@ -2381,7 +2382,7 @@ func newStream(n int64, f generator, lbsString string) logproto.Stream {
}
func newSeries(n int64, f generator, lbsString string) logproto.Series {
labels, err := ParseLabels(lbsString)
labels, err := syntax.ParseLabels(lbsString)
if err != nil {
panic(err)
}

@ -15,6 +15,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/util"
)
@ -111,22 +112,22 @@ type Evaluator interface {
type SampleEvaluator interface {
// StepEvaluator returns a StepEvaluator for a given SampleExpr. It's explicitly passed another StepEvaluator// in order to enable arbitrary computation of embedded expressions. This allows more modular & extensible
// StepEvaluator implementations which can be composed.
StepEvaluator(ctx context.Context, nextEvaluator SampleEvaluator, expr SampleExpr, p Params) (StepEvaluator, error)
StepEvaluator(ctx context.Context, nextEvaluator SampleEvaluator, expr syntax.SampleExpr, p Params) (StepEvaluator, error)
}
type SampleEvaluatorFunc func(ctx context.Context, nextEvaluator SampleEvaluator, expr SampleExpr, p Params) (StepEvaluator, error)
type SampleEvaluatorFunc func(ctx context.Context, nextEvaluator SampleEvaluator, expr syntax.SampleExpr, p Params) (StepEvaluator, error)
func (s SampleEvaluatorFunc) StepEvaluator(ctx context.Context, nextEvaluator SampleEvaluator, expr SampleExpr, p Params) (StepEvaluator, error) {
func (s SampleEvaluatorFunc) StepEvaluator(ctx context.Context, nextEvaluator SampleEvaluator, expr syntax.SampleExpr, p Params) (StepEvaluator, error) {
return s(ctx, nextEvaluator, expr, p)
}
type EntryEvaluator interface {
// Iterator returns the iter.EntryIterator for a given LogSelectorExpr
Iterator(context.Context, LogSelectorExpr, Params) (iter.EntryIterator, error)
Iterator(context.Context, syntax.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 {
func EvaluatorUnsupportedType(expr syntax.Expr, ev Evaluator) error {
return errors.Errorf("unexpected expr type (%T) for Evaluator type (%T) ", expr, ev)
}
@ -143,7 +144,7 @@ func NewDefaultEvaluator(querier Querier, maxLookBackPeriod time.Duration) *Defa
}
}
func (ev *DefaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr, q Params) (iter.EntryIterator, error) {
func (ev *DefaultEvaluator) Iterator(ctx context.Context, expr syntax.LogSelectorExpr, q Params) (iter.EntryIterator, error) {
params := SelectLogParams{
QueryRequest: &logproto.QueryRequest{
Start: q.Start(),
@ -165,15 +166,15 @@ func (ev *DefaultEvaluator) Iterator(ctx context.Context, expr LogSelectorExpr,
func (ev *DefaultEvaluator) StepEvaluator(
ctx context.Context,
nextEv SampleEvaluator,
expr SampleExpr,
expr syntax.SampleExpr,
q Params,
) (StepEvaluator, error) {
switch e := expr.(type) {
case *VectorAggregationExpr:
if rangExpr, ok := e.Left.(*RangeAggregationExpr); ok && e.Operation == OpTypeSum {
case *syntax.VectorAggregationExpr:
if rangExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum {
// if range expression is wrapped with a vector expression
// we should send the vector expression for allowing reducing labels at the source.
nextEv = SampleEvaluatorFunc(func(ctx context.Context, nextEvaluator SampleEvaluator, expr SampleExpr, p Params) (StepEvaluator, error) {
nextEv = SampleEvaluatorFunc(func(ctx context.Context, nextEvaluator SampleEvaluator, expr syntax.SampleExpr, p Params) (StepEvaluator, error) {
it, err := ev.querier.SelectSamples(ctx, SelectSampleParams{
&logproto.SampleQueryRequest{
Start: q.Start().Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset),
@ -189,7 +190,7 @@ func (ev *DefaultEvaluator) StepEvaluator(
})
}
return vectorAggEvaluator(ctx, nextEv, e, q)
case *RangeAggregationExpr:
case *syntax.RangeAggregationExpr:
it, err := ev.querier.SelectSamples(ctx, SelectSampleParams{
&logproto.SampleQueryRequest{
Start: q.Start().Add(-e.Left.Interval).Add(-e.Left.Offset),
@ -202,9 +203,9 @@ func (ev *DefaultEvaluator) StepEvaluator(
return nil, err
}
return rangeAggEvaluator(iter.NewPeekingSampleIterator(it), e, q, e.Left.Offset)
case *BinOpExpr:
case *syntax.BinOpExpr:
return binOpStepEvaluator(ctx, nextEv, e, q)
case *LabelReplaceExpr:
case *syntax.LabelReplaceExpr:
return labelReplaceEvaluator(ctx, nextEv, e, q)
default:
return nil, EvaluatorUnsupportedType(e, ev)
@ -214,7 +215,7 @@ func (ev *DefaultEvaluator) StepEvaluator(
func vectorAggEvaluator(
ctx context.Context,
ev SampleEvaluator,
expr *VectorAggregationExpr,
expr *syntax.VectorAggregationExpr,
q Params,
) (StepEvaluator, error) {
if expr.Grouping == nil {
@ -234,7 +235,7 @@ func vectorAggEvaluator(
return false, 0, promql.Vector{}
}
result := map[uint64]*groupedAggregation{}
if expr.Operation == OpTypeTopK || expr.Operation == OpTypeBottomK {
if expr.Operation == syntax.OpTypeTopK || expr.Operation == syntax.OpTypeBottomK {
if expr.Params < 1 {
return next, ts, promql.Vector{}
}
@ -282,15 +283,15 @@ func vectorAggEvaluator(
if expr.Params > inputVecLen {
resultSize = inputVecLen
}
if expr.Operation == OpTypeStdvar || expr.Operation == OpTypeStddev {
if expr.Operation == syntax.OpTypeStdvar || expr.Operation == syntax.OpTypeStddev {
result[groupingKey].value = 0.0
} else if expr.Operation == OpTypeTopK {
} else if expr.Operation == syntax.OpTypeTopK {
result[groupingKey].heap = make(vectorByValueHeap, 0, resultSize)
heap.Push(&result[groupingKey].heap, &promql.Sample{
Point: promql.Point{V: s.V},
Metric: s.Metric,
})
} else if expr.Operation == OpTypeBottomK {
} else if expr.Operation == syntax.OpTypeBottomK {
result[groupingKey].reverseHeap = make(vectorByReverseValueHeap, 0, resultSize)
heap.Push(&result[groupingKey].reverseHeap, &promql.Sample{
Point: promql.Point{V: s.V},
@ -300,33 +301,33 @@ func vectorAggEvaluator(
continue
}
switch expr.Operation {
case OpTypeSum:
case syntax.OpTypeSum:
group.value += s.V
case OpTypeAvg:
case syntax.OpTypeAvg:
group.groupCount++
group.mean += (s.V - group.mean) / float64(group.groupCount)
case OpTypeMax:
case syntax.OpTypeMax:
if group.value < s.V || math.IsNaN(group.value) {
group.value = s.V
}
case OpTypeMin:
case syntax.OpTypeMin:
if group.value > s.V || math.IsNaN(group.value) {
group.value = s.V
}
case OpTypeCount:
case syntax.OpTypeCount:
group.groupCount++
case OpTypeStddev, OpTypeStdvar:
case syntax.OpTypeStddev, syntax.OpTypeStdvar:
group.groupCount++
delta := s.V - group.mean
group.mean += delta / float64(group.groupCount)
group.value += delta * (s.V - group.mean)
case OpTypeTopK:
case syntax.OpTypeTopK:
if len(group.heap) < expr.Params || group.heap[0].V < s.V || math.IsNaN(group.heap[0].V) {
if len(group.heap) == expr.Params {
heap.Pop(&group.heap)
@ -337,7 +338,7 @@ func vectorAggEvaluator(
})
}
case OpTypeBottomK:
case syntax.OpTypeBottomK:
if len(group.reverseHeap) < expr.Params || group.reverseHeap[0].V > s.V || math.IsNaN(group.reverseHeap[0].V) {
if len(group.reverseHeap) == expr.Params {
heap.Pop(&group.reverseHeap)
@ -354,19 +355,19 @@ func vectorAggEvaluator(
vec = vec[:0]
for _, aggr := range result {
switch expr.Operation {
case OpTypeAvg:
case syntax.OpTypeAvg:
aggr.value = aggr.mean
case OpTypeCount:
case syntax.OpTypeCount:
aggr.value = float64(aggr.groupCount)
case OpTypeStddev:
case syntax.OpTypeStddev:
aggr.value = math.Sqrt(aggr.value / float64(aggr.groupCount))
case OpTypeStdvar:
case syntax.OpTypeStdvar:
aggr.value = aggr.value / float64(aggr.groupCount)
case OpTypeTopK:
case syntax.OpTypeTopK:
// The heap keeps the lowest value on top, so reverse it.
sort.Sort(sort.Reverse(aggr.heap))
for _, v := range aggr.heap {
@ -380,7 +381,7 @@ func vectorAggEvaluator(
}
continue // Bypass default append.
case OpTypeBottomK:
case syntax.OpTypeBottomK:
// The heap keeps the lowest value on top, so reverse it.
sort.Sort(sort.Reverse(aggr.reverseHeap))
for _, v := range aggr.reverseHeap {
@ -409,11 +410,11 @@ func vectorAggEvaluator(
func rangeAggEvaluator(
it iter.PeekingSampleIterator,
expr *RangeAggregationExpr,
expr *syntax.RangeAggregationExpr,
q Params,
o time.Duration,
) (StepEvaluator, error) {
agg, err := expr.aggregator()
agg, err := aggregator(expr)
if err != nil {
return nil, err
}
@ -423,7 +424,7 @@ func rangeAggEvaluator(
q.Step().Nanoseconds(),
q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(),
)
if expr.Operation == OpRangeTypeAbsent {
if expr.Operation == syntax.OpRangeTypeAbsent {
return &absentRangeVectorEvaluator{
iter: iter,
lbs: absentLabels(expr),
@ -516,12 +517,12 @@ func (r absentRangeVectorEvaluator) Error() error {
func binOpStepEvaluator(
ctx context.Context,
ev SampleEvaluator,
expr *BinOpExpr,
expr *syntax.BinOpExpr,
q Params,
) (StepEvaluator, error) {
// first check if either side is a literal
leftLit, lOk := expr.SampleExpr.(*LiteralExpr)
rightLit, rOk := expr.RHS.(*LiteralExpr)
leftLit, lOk := expr.SampleExpr.(*syntax.LiteralExpr)
rightLit, rOk := expr.RHS.(*syntax.LiteralExpr)
// match a literal expr with all labels in the other leg
if lOk {
@ -613,11 +614,11 @@ func binOpStepEvaluator(
var results promql.Vector
switch expr.Op {
case OpTypeAnd:
case syntax.OpTypeAnd:
results = vectorAnd(lhs, rhs, lsigs, rsigs)
case OpTypeOr:
case syntax.OpTypeOr:
results = vectorOr(lhs, rhs, lsigs, rsigs)
case OpTypeUnless:
case syntax.OpTypeUnless:
results = vectorUnless(lhs, rhs, lsigs, rsigs)
default:
results, scopedErr = vectorBinop(expr.Op, expr.Opts, lhs, rhs, lsigs, rsigs)
@ -651,7 +652,7 @@ func binOpStepEvaluator(
})
}
func matchingSignature(sample promql.Sample, opts *BinOpOptions) uint64 {
func matchingSignature(sample promql.Sample, opts *syntax.BinOpOptions) uint64 {
if opts == nil || opts.VectorMatching == nil {
return sample.Metric.Hash()
} else if opts.VectorMatching.On {
@ -661,10 +662,10 @@ func matchingSignature(sample promql.Sample, opts *BinOpOptions) uint64 {
}
}
func vectorBinop(op string, opts *BinOpOptions, lhs, rhs promql.Vector, lsigs, rsigs []uint64) (promql.Vector, error) {
func vectorBinop(op string, opts *syntax.BinOpOptions, lhs, rhs promql.Vector, lsigs, rsigs []uint64) (promql.Vector, error) {
// handle one-to-one or many-to-one matching
// for one-to-many, swap
if opts != nil && opts.VectorMatching.Card == CardOneToMany {
if opts != nil && opts.VectorMatching.Card == syntax.CardOneToMany {
lhs, rhs = rhs, lhs
lsigs, rsigs = rsigs, lsigs
}
@ -677,7 +678,7 @@ func vectorBinop(op string, opts *BinOpOptions, lhs, rhs promql.Vector, lsigs, r
sig := rsigs[i]
if rightSigs[sig] != nil {
side := "right"
if opts.VectorMatching.Card == CardOneToMany {
if opts.VectorMatching.Card == syntax.CardOneToMany {
side = "left"
}
return nil, fmt.Errorf("found duplicate series on the %s hand-side"+
@ -701,7 +702,7 @@ func vectorBinop(op string, opts *BinOpOptions, lhs, rhs promql.Vector, lsigs, r
insertedSigs, exists := matchedSigs[sig]
filter := true
if opts != nil {
if opts.VectorMatching.Card == CardOneToOne {
if opts.VectorMatching.Card == syntax.CardOneToOne {
if exists {
return nil, errors.New("multiple matches for labels: many-to-one matching must be explicit (group_left/group_right)")
}
@ -721,12 +722,12 @@ func vectorBinop(op string, opts *BinOpOptions, lhs, rhs promql.Vector, lsigs, r
filter = false
}
// swap back before apply binary operator
if opts.VectorMatching.Card == CardOneToMany {
if opts.VectorMatching.Card == syntax.CardOneToMany {
ls, rs = rs, ls
}
}
if merged := mergeBinOp(op, ls, rs, filter, IsComparisonOperator(op)); merged != nil {
if merged := syntax.MergeBinOp(op, ls, rs, filter, syntax.IsComparisonOperator(op)); merged != nil {
// replace with labels specified by expr
merged.Metric = metric
results = append(results, *merged)
@ -798,12 +799,12 @@ func vectorUnless(lhs, rhs promql.Vector, lsigs, rsigs []uint64) promql.Vector {
// resultMetric returns the metric for the given sample(s) based on the Vector
// binary operation and the matching options.
func resultMetric(lhs, rhs labels.Labels, opts *BinOpOptions) labels.Labels {
func resultMetric(lhs, rhs labels.Labels, opts *syntax.BinOpOptions) labels.Labels {
lb := labels.NewBuilder(lhs)
if opts != nil {
matching := opts.VectorMatching
if matching.Card == CardOneToOne {
if matching.Card == syntax.CardOneToOne {
if matching.On {
Outer:
for _, l := range lhs {
@ -831,251 +832,11 @@ func resultMetric(lhs, rhs labels.Labels, opts *BinOpOptions) labels.Labels {
return lb.Labels()
}
func mergeBinOp(op string, left, right *promql.Sample, filter, isVectorComparison bool) *promql.Sample {
var merger func(left, right *promql.Sample) *promql.Sample
switch op {
case OpTypeAdd:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V += right.Point.V
return &res
}
case OpTypeSub:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V -= right.Point.V
return &res
}
case OpTypeMul:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V *= right.Point.V
return &res
}
case OpTypeDiv:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
// guard against divide by zero
if right.Point.V == 0 {
res.Point.V = math.NaN()
} else {
res.Point.V /= right.Point.V
}
return &res
}
case OpTypeMod:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
// guard against divide by zero
if right.Point.V == 0 {
res.Point.V = math.NaN()
} else {
res.Point.V = math.Mod(res.Point.V, right.Point.V)
}
return &res
}
case OpTypePow:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V = math.Pow(left.Point.V, right.Point.V)
return &res
}
case OpTypeCmpEQ:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V == right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeNEQ:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V != right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeGT:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V > right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeGTE:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V >= right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeLT:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V < right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeLTE:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V <= right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
default:
panic(errors.Errorf("should never happen: unexpected operation: (%s)", op))
}
res := merger(left, right)
if !isVectorComparison {
return res
}
if filter {
// if a filter-enabled vector-wise comparison has returned non-nil,
// ensure we return the left-hand side's value (2) instead of the
// comparison operator's result (1: the truthy answer)
if res != nil {
return left
}
}
return res
}
// 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 literalStepEvaluator(
op string,
lit *LiteralExpr,
lit *syntax.LiteralExpr,
eval StepEvaluator,
inverted bool,
returnBool bool,
@ -1088,7 +849,7 @@ func literalStepEvaluator(
for _, sample := range vec {
literalPoint := promql.Sample{
Metric: sample.Metric,
Point: promql.Point{T: ts, V: lit.value},
Point: promql.Point{T: ts, V: lit.Value()},
}
left, right := &literalPoint, &sample
@ -1096,12 +857,12 @@ func literalStepEvaluator(
left, right = right, left
}
if merged := mergeBinOp(
if merged := syntax.MergeBinOp(
op,
left,
right,
!returnBool,
IsComparisonOperator(op),
syntax.IsComparisonOperator(op),
); merged != nil {
results = append(results, *merged)
}
@ -1117,7 +878,7 @@ func literalStepEvaluator(
func labelReplaceEvaluator(
ctx context.Context,
ev SampleEvaluator,
expr *LabelReplaceExpr,
expr *syntax.LabelReplaceExpr,
q Params,
) (StepEvaluator, error) {
nextEvaluator, err := ev.StepEvaluator(ctx, ev, expr.Left, q)
@ -1163,7 +924,7 @@ func labelReplaceEvaluator(
}
// This is to replace missing timeseries during absent_over_time aggregation.
func absentLabels(expr SampleExpr) labels.Labels {
func absentLabels(expr syntax.SampleExpr) labels.Labels {
m := labels.Labels{}
lm := expr.Selector().Matchers()

@ -6,11 +6,12 @@ import (
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql/syntax"
)
func TestDefaultEvaluator_DivideByZero(t *testing.T) {
require.Equal(t, true, math.IsNaN(mergeBinOp(OpTypeDiv,
require.Equal(t, true, math.IsNaN(syntax.MergeBinOp(syntax.OpTypeDiv,
&promql.Sample{
Point: promql.Point{T: 1, V: 1},
},
@ -21,7 +22,7 @@ func TestDefaultEvaluator_DivideByZero(t *testing.T) {
false,
).Point.V))
require.Equal(t, true, math.IsNaN(mergeBinOp(OpTypeMod,
require.Equal(t, true, math.IsNaN(syntax.MergeBinOp(syntax.OpTypeMod,
&promql.Sample{
Point: promql.Point{T: 1, V: 1},
},
@ -42,7 +43,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
}{
{
`eq_0`,
OpTypeCmpEQ,
syntax.OpTypeCmpEQ,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -55,7 +56,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`eq_1`,
OpTypeCmpEQ,
syntax.OpTypeCmpEQ,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -68,7 +69,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`neq_0`,
OpTypeNEQ,
syntax.OpTypeNEQ,
&promql.Sample{
Point: promql.Point{V: 0},
},
@ -81,7 +82,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`neq_1`,
OpTypeNEQ,
syntax.OpTypeNEQ,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -94,7 +95,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`gt_0`,
OpTypeGT,
syntax.OpTypeGT,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -107,7 +108,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`gt_1`,
OpTypeGT,
syntax.OpTypeGT,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -120,7 +121,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`lt_0`,
OpTypeLT,
syntax.OpTypeLT,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -133,7 +134,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`lt_1`,
OpTypeLT,
syntax.OpTypeLT,
&promql.Sample{
Point: promql.Point{V: 0},
},
@ -146,7 +147,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`gte_0`,
OpTypeGTE,
syntax.OpTypeGTE,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -159,7 +160,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`gt_1`,
OpTypeGTE,
syntax.OpTypeGTE,
&promql.Sample{
Point: promql.Point{V: 0},
},
@ -172,7 +173,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`lte_0`,
OpTypeLTE,
syntax.OpTypeLTE,
&promql.Sample{
Point: promql.Point{V: 0},
},
@ -185,7 +186,7 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
},
{
`lte_1`,
OpTypeLTE,
syntax.OpTypeLTE,
&promql.Sample{
Point: promql.Point{V: 1},
},
@ -200,43 +201,22 @@ func TestEvaluator_mergeBinOpComparisons(t *testing.T) {
t.Run(tc.desc, func(t *testing.T) {
// comparing a binop should yield the unfiltered (non-nil variant) regardless
// of whether this is a vector-vector comparison or not.
require.Equal(t, tc.expected, mergeBinOp(tc.op, tc.lhs, tc.rhs, false, false))
require.Equal(t, tc.expected, mergeBinOp(tc.op, tc.lhs, tc.rhs, false, true))
require.Equal(t, tc.expected, syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, false, false))
require.Equal(t, tc.expected, syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, false, true))
require.Nil(t, mergeBinOp(tc.op, tc.lhs, nil, false, true))
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, nil, false, true))
// test filtered variants
if tc.expected.V == 0 {
// ensure zeroed predicates are filtered out
require.Nil(t, mergeBinOp(tc.op, tc.lhs, tc.rhs, true, false))
require.Nil(t, mergeBinOp(tc.op, tc.lhs, tc.rhs, true, true))
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, true, false))
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, true, true))
// for vector-vector comparisons, ensure that nil right hand sides
// translate into nil results
require.Nil(t, mergeBinOp(tc.op, tc.lhs, nil, true, true))
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, nil, true, true))
}
})
}
}
func Test_MergeBinOpVectors_Filter(t *testing.T) {
res := mergeBinOp(
OpTypeGT,
&promql.Sample{
Point: promql.Point{V: 2},
},
&promql.Sample{
Point: promql.Point{V: 0},
},
true,
true,
)
// ensure we return the left hand side's value (2) instead of the
// comparison operator's result (1: the truthy answer)
require.Equal(t, &promql.Sample{
Point: promql.Point{V: 2},
}, res)
}

@ -1,357 +0,0 @@
package logql
import (
"fmt"
"math"
"sort"
"time"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/logql/log"
)
const unsupportedErr = "unsupported range vector aggregation operation: %s"
func (r RangeAggregationExpr) Extractor() (log.SampleExtractor, error) {
return r.extractor(nil)
}
// extractor creates a SampleExtractor but allows for the grouping to be overridden.
func (r RangeAggregationExpr) extractor(override *Grouping) (log.SampleExtractor, error) {
if err := r.validate(); err != nil {
return nil, err
}
var groups []string
var without bool
var noLabels bool
if r.Grouping != nil {
groups = r.Grouping.Groups
without = r.Grouping.Without
if len(groups) == 0 {
noLabels = true
}
}
// uses override if it exists
if override != nil {
groups = override.Groups
without = override.Without
if len(groups) == 0 {
noLabels = true
}
}
// absent_over_time cannot be grouped (yet?), so set noLabels=true
// to make extraction more efficient and less likely to strip per query series limits.
if r.Operation == OpRangeTypeAbsent {
noLabels = true
}
sort.Strings(groups)
var stages []log.Stage
if p, ok := r.Left.Left.(*PipelineExpr); ok {
// if the expression is a pipeline then take all stages into account first.
st, err := p.MultiStages.stages()
if err != nil {
return nil, err
}
stages = st
}
// unwrap...means we want to extract metrics from labels.
if r.Left.Unwrap != nil {
var convOp string
switch r.Left.Unwrap.Operation {
case OpConvBytes:
convOp = log.ConvertBytes
case OpConvDuration, OpConvDurationSeconds:
convOp = log.ConvertDuration
default:
convOp = log.ConvertFloat
}
return log.LabelExtractorWithStages(
r.Left.Unwrap.Identifier,
convOp, groups, without, noLabels, stages,
log.ReduceAndLabelFilter(r.Left.Unwrap.PostFilters),
)
}
// otherwise we extract metrics from the log line.
switch r.Operation {
case OpRangeTypeRate, OpRangeTypeCount, OpRangeTypeAbsent:
return log.NewLineSampleExtractor(log.CountExtractor, stages, groups, without, noLabels)
case OpRangeTypeBytes, OpRangeTypeBytesRate:
return log.NewLineSampleExtractor(log.BytesExtractor, stages, groups, without, noLabels)
default:
return nil, fmt.Errorf(unsupportedErr, r.Operation)
}
}
func (r RangeAggregationExpr) aggregator() (RangeVectorAggregator, error) {
switch r.Operation {
case OpRangeTypeRate:
return rateLogs(r.Left.Interval, r.Left.Unwrap != nil), nil
case OpRangeTypeCount:
return countOverTime, nil
case OpRangeTypeBytesRate:
return rateLogBytes(r.Left.Interval), nil
case OpRangeTypeBytes, OpRangeTypeSum:
return sumOverTime, nil
case OpRangeTypeAvg:
return avgOverTime, nil
case OpRangeTypeMax:
return maxOverTime, nil
case OpRangeTypeMin:
return minOverTime, nil
case OpRangeTypeStddev:
return stddevOverTime, nil
case OpRangeTypeStdvar:
return stdvarOverTime, nil
case OpRangeTypeQuantile:
return quantileOverTime(*r.Params), nil
case OpRangeTypeFirst:
return first, nil
case OpRangeTypeLast:
return last, nil
case OpRangeTypeAbsent:
return one, nil
default:
return nil, fmt.Errorf(unsupportedErr, r.Operation)
}
}
// rateLogs calculates the per-second rate of log lines.
func rateLogs(selRange time.Duration, computeValues bool) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
if !computeValues {
return float64(len(samples)) / selRange.Seconds()
}
return extrapolatedRate(samples, selRange, true, true)
}
}
// extrapolatedRate function is taken from prometheus code promql/functions.go:59
// extrapolatedRate is a utility function for rate/increase/delta.
// It calculates the rate (allowing for counter resets if isCounter is true),
// extrapolates if the first/last sample is close to the boundary, and returns
// the result as either per-second (if isRate is true) or overall.
func extrapolatedRate(samples []promql.Point, selRange time.Duration, isCounter, isRate bool) float64 {
// No sense in trying to compute a rate without at least two points. Drop
// this Vector element.
if len(samples) < 2 {
return 0
}
var (
rangeStart = samples[0].T - durationMilliseconds(selRange)
rangeEnd = samples[len(samples)-1].T
)
resultValue := samples[len(samples)-1].V - samples[0].V
if isCounter {
var lastValue float64
for _, sample := range samples {
if sample.V < lastValue {
resultValue += lastValue
}
lastValue = sample.V
}
}
// Duration between first/last samples and boundary of range.
durationToStart := float64(samples[0].T-rangeStart) / 1000
durationToEnd := float64(rangeEnd-samples[len(samples)-1].T) / 1000
sampledInterval := float64(samples[len(samples)-1].T-samples[0].T) / 1000
averageDurationBetweenSamples := sampledInterval / float64(len(samples)-1)
if isCounter && resultValue > 0 && samples[0].V >= 0 {
// Counters cannot be negative. If we have any slope at
// all (i.e. resultValue went up), we can extrapolate
// the zero point of the counter. If the duration to the
// zero point is shorter than the durationToStart, we
// take the zero point as the start of the series,
// thereby avoiding extrapolation to negative counter
// values.
durationToZero := sampledInterval * (samples[0].V / resultValue)
if durationToZero < durationToStart {
durationToStart = durationToZero
}
}
// If the first/last samples are close to the boundaries of the range,
// extrapolate the result. This is as we expect that another sample
// will exist given the spacing between samples we've seen thus far,
// with an allowance for noise.
extrapolationThreshold := averageDurationBetweenSamples * 1.1
extrapolateToInterval := sampledInterval
if durationToStart < extrapolationThreshold {
extrapolateToInterval += durationToStart
} else {
extrapolateToInterval += averageDurationBetweenSamples / 2
}
if durationToEnd < extrapolationThreshold {
extrapolateToInterval += durationToEnd
} else {
extrapolateToInterval += averageDurationBetweenSamples / 2
}
resultValue = resultValue * (extrapolateToInterval / sampledInterval)
if isRate {
seconds := selRange.Seconds()
resultValue = resultValue / seconds
}
return resultValue
}
func durationMilliseconds(d time.Duration) int64 {
return int64(d / (time.Millisecond / time.Nanosecond))
}
// rateLogBytes calculates the per-second rate of log bytes.
func rateLogBytes(selRange time.Duration) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
return sumOverTime(samples) / selRange.Seconds()
}
}
// countOverTime counts the amount of log lines.
func countOverTime(samples []promql.Point) float64 {
return float64(len(samples))
}
func sumOverTime(samples []promql.Point) float64 {
var sum float64
for _, v := range samples {
sum += v.V
}
return sum
}
func avgOverTime(samples []promql.Point) float64 {
var mean, count float64
for _, v := range samples {
count++
if math.IsInf(mean, 0) {
if math.IsInf(v.V, 0) && (mean > 0) == (v.V > 0) {
// The `mean` and `v.V` values are `Inf` of the same sign. They
// can't be subtracted, but the value of `mean` is correct
// already.
continue
}
if !math.IsInf(v.V, 0) && !math.IsNaN(v.V) {
// At this stage, the mean is an infinite. If the added
// value is neither an Inf or a Nan, we can keep that mean
// value.
// This is required because our calculation below removes
// the mean value, which would look like Inf += x - Inf and
// end up as a NaN.
continue
}
}
mean += v.V/count - mean/count
}
return mean
}
func maxOverTime(samples []promql.Point) float64 {
max := samples[0].V
for _, v := range samples {
if v.V > max || math.IsNaN(max) {
max = v.V
}
}
return max
}
func minOverTime(samples []promql.Point) float64 {
min := samples[0].V
for _, v := range samples {
if v.V < min || math.IsNaN(min) {
min = v.V
}
}
return min
}
func stdvarOverTime(samples []promql.Point) float64 {
var aux, count, mean float64
for _, v := range samples {
count++
delta := v.V - mean
mean += delta / count
aux += delta * (v.V - mean)
}
return aux / count
}
func stddevOverTime(samples []promql.Point) float64 {
var aux, count, mean float64
for _, v := range samples {
count++
delta := v.V - mean
mean += delta / count
aux += delta * (v.V - mean)
}
return math.Sqrt(aux / count)
}
func quantileOverTime(q float64) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
values := make(vectorByValueHeap, 0, len(samples))
for _, v := range samples {
values = append(values, promql.Sample{Point: promql.Point{V: v.V}})
}
return quantile(q, values)
}
}
// quantile calculates the given quantile of a vector of samples.
//
// The Vector will be sorted.
// If 'values' has zero elements, NaN is returned.
// If q<0, -Inf is returned.
// If q>1, +Inf is returned.
func quantile(q float64, values vectorByValueHeap) float64 {
if len(values) == 0 {
return math.NaN()
}
if q < 0 {
return math.Inf(-1)
}
if q > 1 {
return math.Inf(+1)
}
sort.Sort(values)
n := float64(len(values))
// When the quantile lies between two samples,
// we use a weighted average of the two samples.
rank := q * (n - 1)
lowerIndex := math.Max(0, math.Floor(rank))
upperIndex := math.Min(n-1, lowerIndex+1)
weight := rank - math.Floor(rank)
return values[int(lowerIndex)].V*(1-weight) + values[int(upperIndex)].V*weight
}
func first(samples []promql.Point) float64 {
if len(samples) == 0 {
return math.NaN()
}
return samples[0].V
}
func last(samples []promql.Point) float64 {
if len(samples) == 0 {
return math.NaN()
}
return samples[len(samples)-1].V
}
func one(samples []promql.Point) float64 {
return 1.0
}

@ -7,7 +7,7 @@ import (
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
)
var (
@ -216,7 +216,7 @@ func Test_ParserHints(t *testing.T) {
tt := tt
t.Run(tt.expr, func(t *testing.T) {
t.Parallel()
expr, err := logql.ParseSampleExpr(tt.expr)
expr, err := syntax.ParseSampleExpr(tt.expr)
require.NoError(t, err)
ex, err := expr.Extractor()

@ -0,0 +1,25 @@
package logql
import (
"github.com/pkg/errors"
"github.com/prometheus/prometheus/model/labels"
"github.com/grafana/loki/pkg/logql/syntax"
)
// Match extracts and parses multiple matcher groups from a slice of strings
func Match(xs []string) ([][]*labels.Matcher, error) {
groups := make([][]*labels.Matcher, 0, len(xs))
for _, x := range xs {
ms, err := syntax.ParseMatchers(x)
if err != nil {
return nil, err
}
if len(ms) == 0 {
return nil, errors.Errorf("0 matchers in group: %s", x)
}
groups = append(groups, ms)
}
return groups, nil
}

@ -0,0 +1,66 @@
package logql
import (
"testing"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
)
func Test_match(t *testing.T) {
tests := []struct {
name string
input []string
want [][]*labels.Matcher
wantErr bool
}{
{"malformed", []string{`{a="1`}, nil, true},
{"empty on nil input", nil, [][]*labels.Matcher{}, false},
{"empty on empty input", []string{}, [][]*labels.Matcher{}, false},
{
"single",
[]string{`{a="1"}`},
[][]*labels.Matcher{
{mustMatcher(labels.MatchEqual, "a", "1")},
},
false,
},
{
"multiple groups",
[]string{`{a="1"}`, `{b="2", c=~"3", d!="4"}`},
[][]*labels.Matcher{
{mustMatcher(labels.MatchEqual, "a", "1")},
{
mustMatcher(labels.MatchEqual, "b", "2"),
mustMatcher(labels.MatchRegexp, "c", "3"),
mustMatcher(labels.MatchNotEqual, "d", "4"),
},
},
false,
},
{
"errors on empty group",
[]string{`{}`},
nil,
true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := Match(tt.input)
if tt.wantErr {
require.Error(t, err)
} else {
require.Equal(t, tt.want, got)
}
})
}
}
func mustMatcher(t labels.MatchType, n string, v string) *labels.Matcher {
m, err := labels.NewMatcher(t, n, v)
if err != nil {
panic(err)
}
return m
}

@ -11,6 +11,7 @@ import (
"github.com/prometheus/client_golang/prometheus/promauto"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
logql_stats "github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/usagestats"
@ -148,14 +149,14 @@ func recordUsageStats(queryType string, stats logql_stats.Result) {
}
func QueryType(query string) (string, error) {
expr, err := ParseExpr(query)
expr, err := syntax.ParseExpr(query)
if err != nil {
return "", err
}
switch e := expr.(type) {
case SampleExpr:
case syntax.SampleExpr:
return QueryTypeMetric, nil
case LogSelectorExpr:
case syntax.LogSelectorExpr:
if e.HasFilter() {
return QueryTypeFilter, nil
}

@ -1,7 +1,9 @@
package logql
import "github.com/grafana/loki/pkg/logql/syntax"
// optimizeSampleExpr Attempt to optimize the SampleExpr to another that will run faster but will produce the same result.
func optimizeSampleExpr(expr SampleExpr) (SampleExpr, error) {
func optimizeSampleExpr(expr syntax.SampleExpr) (syntax.SampleExpr, error) {
var skip bool
// we skip sharding AST for now, it's not easy to clone them since they are not part of the language.
expr.Walk(func(e interface{}) {
@ -16,7 +18,7 @@ func optimizeSampleExpr(expr SampleExpr) (SampleExpr, error) {
}
// clone the expr.
q := expr.String()
expr, err := ParseSampleExpr(q)
expr, err := syntax.ParseSampleExpr(q)
if err != nil {
return nil, err
}
@ -25,24 +27,24 @@ func optimizeSampleExpr(expr SampleExpr) (SampleExpr, error) {
}
// removeLineformat removes unnecessary line_format within a SampleExpr.
func removeLineformat(expr SampleExpr) {
func removeLineformat(expr syntax.SampleExpr) {
expr.Walk(func(e interface{}) {
rangeExpr, ok := e.(*RangeAggregationExpr)
rangeExpr, ok := e.(*syntax.RangeAggregationExpr)
if !ok {
return
}
// bytes operation count bytes of the log line so line_format changes the result.
if rangeExpr.Operation == OpRangeTypeBytes ||
rangeExpr.Operation == OpRangeTypeBytesRate {
if rangeExpr.Operation == syntax.OpRangeTypeBytes ||
rangeExpr.Operation == syntax.OpRangeTypeBytesRate {
return
}
pipelineExpr, ok := rangeExpr.Left.Left.(*PipelineExpr)
pipelineExpr, ok := rangeExpr.Left.Left.(*syntax.PipelineExpr)
if !ok {
return
}
temp := pipelineExpr.MultiStages[:0]
for i, s := range pipelineExpr.MultiStages {
_, ok := s.(*LineFmtExpr)
_, ok := s.(*syntax.LineFmtExpr)
if !ok {
temp = append(temp, s)
continue
@ -51,11 +53,11 @@ func removeLineformat(expr SampleExpr) {
// in which case it could be useful for further processing.
var found bool
for j := i; j < len(pipelineExpr.MultiStages); j++ {
if _, ok := pipelineExpr.MultiStages[j].(*LabelParserExpr); ok {
if _, ok := pipelineExpr.MultiStages[j].(*syntax.LabelParserExpr); ok {
found = true
break
}
if _, ok := pipelineExpr.MultiStages[j].(*LineFilterExpr); ok {
if _, ok := pipelineExpr.MultiStages[j].(*syntax.LineFilterExpr); ok {
found = true
break
}
@ -68,9 +70,7 @@ func removeLineformat(expr SampleExpr) {
pipelineExpr.MultiStages = temp
// transform into a matcherExpr if there's no more pipeline.
if len(pipelineExpr.MultiStages) == 0 {
rangeExpr.Left.Left = &MatchersExpr{
matchers: rangeExpr.Left.Left.Matchers(),
}
rangeExpr.Left.Left = &syntax.MatchersExpr{Mts: rangeExpr.Left.Left.Matchers()}
}
})
}

@ -4,6 +4,8 @@ import (
"testing"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql/syntax"
)
func Test_optimizeSampleExpr(t *testing.T) {
@ -27,7 +29,7 @@ func Test_optimizeSampleExpr(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.in, func(t *testing.T) {
e, err := ParseSampleExpr(tt.in)
e, err := syntax.ParseSampleExpr(tt.in)
require.NoError(t, err)
got, err := optimizeSampleExpr(e)
require.NoError(t, err)

@ -1,13 +1,19 @@
package logql
import (
"fmt"
"math"
"sort"
"sync"
"time"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logql/vector"
)
// RangeVectorAggregator aggregates samples for a given range of samples.
@ -178,3 +184,270 @@ func getSeries() *promql.Series {
func putSeries(s *promql.Series) {
seriesPool.Put(s)
}
func aggregator(r *syntax.RangeAggregationExpr) (RangeVectorAggregator, error) {
switch r.Operation {
case syntax.OpRangeTypeRate:
return rateLogs(r.Left.Interval, r.Left.Unwrap != nil), nil
case syntax.OpRangeTypeCount:
return countOverTime, nil
case syntax.OpRangeTypeBytesRate:
return rateLogBytes(r.Left.Interval), nil
case syntax.OpRangeTypeBytes, syntax.OpRangeTypeSum:
return sumOverTime, nil
case syntax.OpRangeTypeAvg:
return avgOverTime, nil
case syntax.OpRangeTypeMax:
return maxOverTime, nil
case syntax.OpRangeTypeMin:
return minOverTime, nil
case syntax.OpRangeTypeStddev:
return stddevOverTime, nil
case syntax.OpRangeTypeStdvar:
return stdvarOverTime, nil
case syntax.OpRangeTypeQuantile:
return quantileOverTime(*r.Params), nil
case syntax.OpRangeTypeFirst:
return first, nil
case syntax.OpRangeTypeLast:
return last, nil
case syntax.OpRangeTypeAbsent:
return one, nil
default:
return nil, fmt.Errorf(syntax.UnsupportedErr, r.Operation)
}
}
// rateLogs calculates the per-second rate of log lines.
func rateLogs(selRange time.Duration, computeValues bool) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
if !computeValues {
return float64(len(samples)) / selRange.Seconds()
}
return extrapolatedRate(samples, selRange, true, true)
}
}
// extrapolatedRate function is taken from prometheus code promql/functions.go:59
// extrapolatedRate is a utility function for rate/increase/delta.
// It calculates the rate (allowing for counter resets if isCounter is true),
// extrapolates if the first/last sample is close to the boundary, and returns
// the result as either per-second (if isRate is true) or overall.
func extrapolatedRate(samples []promql.Point, selRange time.Duration, isCounter, isRate bool) float64 {
// No sense in trying to compute a rate without at least two points. Drop
// this Vector element.
if len(samples) < 2 {
return 0
}
var (
rangeStart = samples[0].T - durationMilliseconds(selRange)
rangeEnd = samples[len(samples)-1].T
)
resultValue := samples[len(samples)-1].V - samples[0].V
if isCounter {
var lastValue float64
for _, sample := range samples {
if sample.V < lastValue {
resultValue += lastValue
}
lastValue = sample.V
}
}
// Duration between first/last samples and boundary of range.
durationToStart := float64(samples[0].T-rangeStart) / 1000
durationToEnd := float64(rangeEnd-samples[len(samples)-1].T) / 1000
sampledInterval := float64(samples[len(samples)-1].T-samples[0].T) / 1000
averageDurationBetweenSamples := sampledInterval / float64(len(samples)-1)
if isCounter && resultValue > 0 && samples[0].V >= 0 {
// Counters cannot be negative. If we have any slope at
// all (i.e. resultValue went up), we can extrapolate
// the zero point of the counter. If the duration to the
// zero point is shorter than the durationToStart, we
// take the zero point as the start of the series,
// thereby avoiding extrapolation to negative counter
// values.
durationToZero := sampledInterval * (samples[0].V / resultValue)
if durationToZero < durationToStart {
durationToStart = durationToZero
}
}
// If the first/last samples are close to the boundaries of the range,
// extrapolate the result. This is as we expect that another sample
// will exist given the spacing between samples we've seen thus far,
// with an allowance for noise.
extrapolationThreshold := averageDurationBetweenSamples * 1.1
extrapolateToInterval := sampledInterval
if durationToStart < extrapolationThreshold {
extrapolateToInterval += durationToStart
} else {
extrapolateToInterval += averageDurationBetweenSamples / 2
}
if durationToEnd < extrapolationThreshold {
extrapolateToInterval += durationToEnd
} else {
extrapolateToInterval += averageDurationBetweenSamples / 2
}
resultValue = resultValue * (extrapolateToInterval / sampledInterval)
if isRate {
seconds := selRange.Seconds()
resultValue = resultValue / seconds
}
return resultValue
}
func durationMilliseconds(d time.Duration) int64 {
return int64(d / (time.Millisecond / time.Nanosecond))
}
// rateLogBytes calculates the per-second rate of log bytes.
func rateLogBytes(selRange time.Duration) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
return sumOverTime(samples) / selRange.Seconds()
}
}
// countOverTime counts the amount of log lines.
func countOverTime(samples []promql.Point) float64 {
return float64(len(samples))
}
func sumOverTime(samples []promql.Point) float64 {
var sum float64
for _, v := range samples {
sum += v.V
}
return sum
}
func avgOverTime(samples []promql.Point) float64 {
var mean, count float64
for _, v := range samples {
count++
if math.IsInf(mean, 0) {
if math.IsInf(v.V, 0) && (mean > 0) == (v.V > 0) {
// The `mean` and `v.V` values are `Inf` of the same sign. They
// can't be subtracted, but the value of `mean` is correct
// already.
continue
}
if !math.IsInf(v.V, 0) && !math.IsNaN(v.V) {
// At this stage, the mean is an infinite. If the added
// value is neither an Inf or a Nan, we can keep that mean
// value.
// This is required because our calculation below removes
// the mean value, which would look like Inf += x - Inf and
// end up as a NaN.
continue
}
}
mean += v.V/count - mean/count
}
return mean
}
func maxOverTime(samples []promql.Point) float64 {
max := samples[0].V
for _, v := range samples {
if v.V > max || math.IsNaN(max) {
max = v.V
}
}
return max
}
func minOverTime(samples []promql.Point) float64 {
min := samples[0].V
for _, v := range samples {
if v.V < min || math.IsNaN(min) {
min = v.V
}
}
return min
}
func stdvarOverTime(samples []promql.Point) float64 {
var aux, count, mean float64
for _, v := range samples {
count++
delta := v.V - mean
mean += delta / count
aux += delta * (v.V - mean)
}
return aux / count
}
func stddevOverTime(samples []promql.Point) float64 {
var aux, count, mean float64
for _, v := range samples {
count++
delta := v.V - mean
mean += delta / count
aux += delta * (v.V - mean)
}
return math.Sqrt(aux / count)
}
func quantileOverTime(q float64) func(samples []promql.Point) float64 {
return func(samples []promql.Point) float64 {
values := make(vector.HeapByMaxValue, 0, len(samples))
for _, v := range samples {
values = append(values, promql.Sample{Point: promql.Point{V: v.V}})
}
return quantile(q, values)
}
}
// quantile calculates the given quantile of a vector of samples.
//
// The Vector will be sorted.
// If 'values' has zero elements, NaN is returned.
// If q<0, -Inf is returned.
// If q>1, +Inf is returned.
func quantile(q float64, values vector.HeapByMaxValue) float64 {
if len(values) == 0 {
return math.NaN()
}
if q < 0 {
return math.Inf(-1)
}
if q > 1 {
return math.Inf(+1)
}
sort.Sort(values)
n := float64(len(values))
// When the quantile lies between two samples,
// we use a weighted average of the two samples.
rank := q * (n - 1)
lowerIndex := math.Max(0, math.Floor(rank))
upperIndex := math.Min(n-1, lowerIndex+1)
weight := rank - math.Floor(rank)
return values[int(lowerIndex)].V*(1-weight) + values[int(upperIndex)].V*weight
}
func first(samples []promql.Point) float64 {
if len(samples) == 0 {
return math.NaN()
}
return samples[0].V
}
func last(samples []promql.Point) float64 {
if len(samples) == 0 {
return math.NaN()
}
return samples[len(samples)-1].V
}
func one(samples []promql.Point) float64 {
return 1.0
}

@ -11,6 +11,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
)
var samples = []logproto.Sample{
@ -28,8 +29,8 @@ var samples = []logproto.Sample{
}
var (
labelFoo, _ = ParseLabels("{app=\"foo\"}")
labelBar, _ = ParseLabels("{app=\"bar\"}")
labelFoo, _ = syntax.ParseLabels("{app=\"foo\"}")
labelBar, _ = syntax.ParseLabels("{app=\"bar\"}")
)
func newSampleIterator() iter.SampleIterator {

@ -8,6 +8,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/querier/astmapper"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -78,6 +79,10 @@ func (r *shardRecorder) Finish() {
}
}
func badASTMapping(got syntax.Expr) error {
return fmt.Errorf("bad AST mapping: expected SampleExpr, but got (%T)", 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)
@ -93,8 +98,8 @@ type ShardMapper struct {
metrics *ShardingMetrics
}
func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) {
parsed, err := ParseExpr(query)
func (m ShardMapper) Parse(query string) (noop bool, expr syntax.Expr, err error) {
parsed, err := syntax.ParseExpr(query)
if err != nil {
return false, nil, err
}
@ -121,25 +126,25 @@ func (m ShardMapper) Parse(query string) (noop bool, expr Expr, err error) {
return noop, mapped, err
}
func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) {
func (m ShardMapper) Map(expr syntax.Expr, r *shardRecorder) (syntax.Expr, error) {
// immediately clone the passed expr to avoid mutating the original
expr, err := Clone(expr)
expr, err := syntax.Clone(expr)
if err != nil {
return nil, err
}
switch e := expr.(type) {
case *LiteralExpr:
case *syntax.LiteralExpr:
return e, nil
case *MatchersExpr, *PipelineExpr:
return m.mapLogSelectorExpr(e.(LogSelectorExpr), r), nil
case *VectorAggregationExpr:
case *syntax.MatchersExpr, *syntax.PipelineExpr:
return m.mapLogSelectorExpr(e.(syntax.LogSelectorExpr), r), nil
case *syntax.VectorAggregationExpr:
return m.mapVectorAggregationExpr(e, r)
case *LabelReplaceExpr:
case *syntax.LabelReplaceExpr:
return m.mapLabelReplaceExpr(e, r)
case *RangeAggregationExpr:
case *syntax.RangeAggregationExpr:
return m.mapRangeAggregationExpr(e, r), nil
case *BinOpExpr:
case *syntax.BinOpExpr:
lhsMapped, err := m.Map(e.SampleExpr, r)
if err != nil {
return nil, err
@ -148,11 +153,11 @@ func (m ShardMapper) Map(expr Expr, r *shardRecorder) (Expr, error) {
if err != nil {
return nil, err
}
lhsSampleExpr, ok := lhsMapped.(SampleExpr)
lhsSampleExpr, ok := lhsMapped.(syntax.SampleExpr)
if !ok {
return nil, badASTMapping(lhsMapped)
}
rhsSampleExpr, ok := rhsMapped.(SampleExpr)
rhsSampleExpr, ok := rhsMapped.(syntax.SampleExpr)
if !ok {
return nil, badASTMapping(rhsMapped)
}
@ -164,7 +169,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 syntax.LogSelectorExpr, r *shardRecorder) syntax.LogSelectorExpr {
var head *ConcatLogSelectorExpr
for i := m.shards - 1; i >= 0; i-- {
head = &ConcatLogSelectorExpr{
@ -183,7 +188,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 syntax.SampleExpr, r *shardRecorder) syntax.SampleExpr {
var head *ConcatSampleExpr
for i := m.shards - 1; i >= 0; i-- {
head = &ConcatSampleExpr{
@ -204,7 +209,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 *syntax.VectorAggregationExpr, r *shardRecorder) (syntax.SampleExpr, error) {
// if this AST contains unshardable operations, don't shard this at this level,
// but attempt to shard a child node.
if !expr.Shardable() {
@ -212,12 +217,12 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *VectorAggregationExpr, r *sh
if err != nil {
return nil, err
}
sampleExpr, ok := subMapped.(SampleExpr)
sampleExpr, ok := subMapped.(syntax.SampleExpr)
if !ok {
return nil, badASTMapping(subMapped)
}
return &VectorAggregationExpr{
return &syntax.VectorAggregationExpr{
Left: sampleExpr,
Grouping: expr.Grouping,
Params: expr.Params,
@ -227,47 +232,47 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *VectorAggregationExpr, r *sh
}
switch expr.Operation {
case OpTypeSum:
case syntax.OpTypeSum:
// sum(x) -> sum(sum(x, shard=1) ++ sum(x, shard=2)...)
return &VectorAggregationExpr{
return &syntax.VectorAggregationExpr{
Left: m.mapSampleExpr(expr, r),
Grouping: expr.Grouping,
Params: expr.Params,
Operation: expr.Operation,
}, nil
case OpTypeAvg:
case syntax.OpTypeAvg:
// avg(x) -> sum(x)/count(x)
lhs, err := m.mapVectorAggregationExpr(&VectorAggregationExpr{
lhs, err := m.mapVectorAggregationExpr(&syntax.VectorAggregationExpr{
Left: expr.Left,
Grouping: expr.Grouping,
Operation: OpTypeSum,
Operation: syntax.OpTypeSum,
}, r)
if err != nil {
return nil, err
}
rhs, err := m.mapVectorAggregationExpr(&VectorAggregationExpr{
rhs, err := m.mapVectorAggregationExpr(&syntax.VectorAggregationExpr{
Left: expr.Left,
Grouping: expr.Grouping,
Operation: OpTypeCount,
Operation: syntax.OpTypeCount,
}, r)
if err != nil {
return nil, err
}
return &BinOpExpr{
return &syntax.BinOpExpr{
SampleExpr: lhs,
RHS: rhs,
Op: OpTypeDiv,
Op: syntax.OpTypeDiv,
}, nil
case OpTypeCount:
case syntax.OpTypeCount:
// count(x) -> sum(count(x, shard=1) ++ count(x, shard=2)...)
sharded := m.mapSampleExpr(expr, r)
return &VectorAggregationExpr{
return &syntax.VectorAggregationExpr{
Left: sharded,
Grouping: expr.Grouping,
Operation: OpTypeSum,
Operation: syntax.OpTypeSum,
}, nil
default:
// this should not be reachable. If an operation is shardable it should
@ -280,17 +285,17 @@ func (m ShardMapper) mapVectorAggregationExpr(expr *VectorAggregationExpr, r *sh
}
}
func (m ShardMapper) mapLabelReplaceExpr(expr *LabelReplaceExpr, r *shardRecorder) (SampleExpr, error) {
func (m ShardMapper) mapLabelReplaceExpr(expr *syntax.LabelReplaceExpr, r *shardRecorder) (syntax.SampleExpr, error) {
subMapped, err := m.Map(expr.Left, r)
if err != nil {
return nil, err
}
cpy := *expr
cpy.Left = subMapped.(SampleExpr)
cpy.Left = subMapped.(syntax.SampleExpr)
return &cpy, nil
}
func (m ShardMapper) mapRangeAggregationExpr(expr *RangeAggregationExpr, r *shardRecorder) SampleExpr {
func (m ShardMapper) mapRangeAggregationExpr(expr *syntax.RangeAggregationExpr, r *shardRecorder) syntax.SampleExpr {
if hasLabelModifier(expr) {
// if an expr can modify labels this means multiple shards can returns the same labelset.
// When this happens the merge strategy needs to be different than a simple concatenation.
@ -299,7 +304,7 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *RangeAggregationExpr, r *shar
return expr
}
switch expr.Operation {
case OpRangeTypeCount, OpRangeTypeRate, OpRangeTypeBytesRate, OpRangeTypeBytes:
case syntax.OpRangeTypeCount, syntax.OpRangeTypeRate, syntax.OpRangeTypeBytesRate, syntax.OpRangeTypeBytes:
// 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)...
// same goes for bytes_rate and bytes_over_time
@ -311,54 +316,16 @@ func (m ShardMapper) mapRangeAggregationExpr(expr *RangeAggregationExpr, r *shar
// hasLabelModifier tells if an expression contains pipelines that can modify stream labels
// parsers introduce new labels but does not alter original one for instance.
func hasLabelModifier(expr *RangeAggregationExpr) bool {
func hasLabelModifier(expr *syntax.RangeAggregationExpr) bool {
switch ex := expr.Left.Left.(type) {
case *MatchersExpr:
case *syntax.MatchersExpr:
return false
case *PipelineExpr:
case *syntax.PipelineExpr:
for _, p := range ex.MultiStages {
if _, ok := p.(*LabelFmtExpr); ok {
if _, ok := p.(*syntax.LabelFmtExpr); ok {
return true
}
}
}
return false
}
// 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
OpRangeTypeCount: true,
OpRangeTypeRate: true,
OpRangeTypeBytes: true,
OpRangeTypeBytesRate: true,
OpRangeTypeSum: true,
OpRangeTypeMax: true,
OpRangeTypeMin: true,
// binops - arith
OpTypeAdd: true,
OpTypeMul: true,
}

File diff suppressed because it is too large Load Diff

@ -1,19 +1,18 @@
package logql
package syntax
import (
"context"
"fmt"
"math"
"regexp"
"strconv"
"strings"
"time"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/logqlmodel"
)
@ -30,64 +29,11 @@ func Clone(e Expr) (Expr, error) {
return ParseExpr(e.String())
}
type QueryParams interface {
LogSelector() (LogSelectorExpr, error)
GetStart() time.Time
GetEnd() time.Time
GetShards() []string
}
// implicit holds default implementations
type implicit struct{}
func (implicit) logQLExpr() {}
// SelectParams specifies parameters passed to data selections.
type SelectLogParams struct {
*logproto.QueryRequest
}
func (s SelectLogParams) String() string {
if s.QueryRequest != nil {
return fmt.Sprintf("selector=%s, direction=%s, start=%s, end=%s, limit=%d, shards=%s",
s.Selector, logproto.Direction_name[int32(s.Direction)], s.Start, s.End, s.Limit, strings.Join(s.Shards, ","))
}
return ""
}
// LogSelector returns the LogSelectorExpr from the SelectParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectLogParams) LogSelector() (LogSelectorExpr, error) {
return ParseLogSelector(s.Selector, true)
}
type SelectSampleParams struct {
*logproto.SampleQueryRequest
}
// Expr returns the SampleExpr from the SelectSampleParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectSampleParams) Expr() (SampleExpr, error) {
return ParseSampleExpr(s.Selector)
}
// LogSelector returns the LogSelectorExpr from the SelectParams.
// The `LogSelectorExpr` can then returns all matchers and filters to use for that request.
func (s SelectSampleParams) LogSelector() (LogSelectorExpr, error) {
expr, err := ParseSampleExpr(s.Selector)
if err != nil {
return nil, err
}
return expr.Selector(), nil
}
// Querier allows a LogQL expression to fetch an EntryIterator for a
// set of matchers and filters
type Querier interface {
SelectLogs(context.Context, SelectLogParams) (iter.EntryIterator, error)
SelectSamples(context.Context, SelectSampleParams) (iter.SampleIterator, error)
}
// LogSelectorExpr is a LogQL expression filtering and returning logs.
type LogSelectorExpr interface {
Matchers() []*labels.Matcher
@ -154,20 +100,20 @@ func (m MultiStageExpr) String() string {
func (MultiStageExpr) logQLExpr() {} // nolint:unused
type MatchersExpr struct {
matchers []*labels.Matcher
Mts []*labels.Matcher
implicit
}
func newMatcherExpr(matchers []*labels.Matcher) *MatchersExpr {
return &MatchersExpr{matchers: matchers}
return &MatchersExpr{Mts: matchers}
}
func (e *MatchersExpr) Matchers() []*labels.Matcher {
return e.matchers
return e.Mts
}
func (e *MatchersExpr) AppendMatchers(m []*labels.Matcher) {
e.matchers = append(e.matchers, m...)
e.Mts = append(e.Mts, m...)
}
func (e *MatchersExpr) Shardable() bool { return true }
@ -177,9 +123,9 @@ func (e *MatchersExpr) Walk(f WalkFn) { f(e) }
func (e *MatchersExpr) String() string {
var sb strings.Builder
sb.WriteString("{")
for i, m := range e.matchers {
for i, m := range e.Mts {
sb.WriteString(m.String())
if i+1 != len(e.matchers) {
if i+1 != len(e.Mts) {
sb.WriteString(", ")
}
}
@ -764,10 +710,6 @@ type SampleExpr interface {
Expr
}
func badASTMapping(got Expr) error {
return fmt.Errorf("bad AST mapping: expected SampleExpr, but got (%T)", got)
}
type RangeAggregationExpr struct {
Left *LogRange
Operation string
@ -1108,7 +1050,7 @@ func mustNewBinOpExpr(op string, opts *BinOpOptions, lhs, rhs Expr) SampleExpr {
panic(logqlmodel.NewParseError(fmt.Sprintf(
"unexpected literal for left leg of logical/set binary operation (%s): %f",
op,
leftLit.value,
leftLit.Val,
), 0, 0))
}
@ -1116,7 +1058,7 @@ func mustNewBinOpExpr(op string, opts *BinOpOptions, lhs, rhs Expr) SampleExpr {
panic(logqlmodel.NewParseError(fmt.Sprintf(
"unexpected literal for right leg of logical/set binary operation (%s): %f",
op,
rightLit.value,
rightLit.Val,
), 0, 0))
}
}
@ -1138,18 +1080,258 @@ func mustNewBinOpExpr(op string, opts *BinOpOptions, 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 := mergeBinOp(
merged := MergeBinOp(
op,
&promql.Sample{Point: promql.Point{V: left.value}},
&promql.Sample{Point: promql.Point{V: right.value}},
&promql.Sample{Point: promql.Point{V: left.Val}},
&promql.Sample{Point: promql.Point{V: right.Val}},
false,
false,
)
return &LiteralExpr{value: merged.V}
return &LiteralExpr{Val: merged.V}
}
func MergeBinOp(op string, left, right *promql.Sample, filter, isVectorComparison bool) *promql.Sample {
var merger func(left, right *promql.Sample) *promql.Sample
switch op {
case OpTypeAdd:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V += right.Point.V
return &res
}
case OpTypeSub:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V -= right.Point.V
return &res
}
case OpTypeMul:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V *= right.Point.V
return &res
}
case OpTypeDiv:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
// guard against divide by zero
if right.Point.V == 0 {
res.Point.V = math.NaN()
} else {
res.Point.V /= right.Point.V
}
return &res
}
case OpTypeMod:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
// guard against divide by zero
if right.Point.V == 0 {
res.Point.V = math.NaN()
} else {
res.Point.V = math.Mod(res.Point.V, right.Point.V)
}
return &res
}
case OpTypePow:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
res.Point.V = math.Pow(left.Point.V, right.Point.V)
return &res
}
case OpTypeCmpEQ:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V == right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeNEQ:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V != right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeGT:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V > right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeGTE:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V >= right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeLT:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V < right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
case OpTypeLTE:
merger = func(left, right *promql.Sample) *promql.Sample {
if left == nil || right == nil {
return nil
}
res := &promql.Sample{
Metric: left.Metric,
Point: left.Point,
}
val := 0.
if left.Point.V <= right.Point.V {
val = 1.
} else if filter {
return nil
}
res.Point.V = val
return res
}
default:
panic(errors.Errorf("should never happen: unexpected operation: (%s)", op))
}
res := merger(left, right)
if !isVectorComparison {
return res
}
if filter {
// if a filter-enabled vector-wise comparison has returned non-nil,
// ensure we return the left hand side's value (2) instead of the
// comparison operator's result (1: the truthy answer)
if res != nil {
return left
}
}
return res
}
type LiteralExpr struct {
value float64
Val float64
implicit
}
@ -1164,12 +1346,12 @@ func mustNewLiteralExpr(s string, invert bool) *LiteralExpr {
}
return &LiteralExpr{
value: n,
Val: n,
}
}
func (e *LiteralExpr) String() string {
return fmt.Sprint(e.value)
return fmt.Sprint(e.Val)
}
// literlExpr impls SampleExpr & LogSelectorExpr mainly to reduce the need for more complicated typings
@ -1182,7 +1364,7 @@ func (e *LiteralExpr) Walk(f WalkFn) { f(e) }
func (e *LiteralExpr) Pipeline() (log.Pipeline, error) { return log.NewNoopPipeline(), nil }
func (e *LiteralExpr) Matchers() []*labels.Matcher { return nil }
func (e *LiteralExpr) Extractor() (log.SampleExtractor, error) { return nil, nil }
func (e *LiteralExpr) Value() float64 { return e.value }
func (e *LiteralExpr) Value() float64 { return e.Val }
// helper used to impl Stringer for vector and range aggregations
// nolint:interfacer
@ -1267,3 +1449,41 @@ func (e *LabelReplaceExpr) String() string {
sb.WriteString(")")
return sb.String()
}
// 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
OpRangeTypeCount: true,
OpRangeTypeRate: true,
OpRangeTypeBytes: true,
OpRangeTypeBytesRate: true,
OpRangeTypeSum: true,
OpRangeTypeMax: true,
OpRangeTypeMin: true,
// binops - arith
OpTypeAdd: true,
OpTypeMul: true,
}

@ -1,15 +1,18 @@
package logql
package syntax
import (
"testing"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql/log"
)
var labelBar, _ = ParseLabels("{app=\"bar\"}")
func Test_logSelectorExpr_String(t *testing.T) {
t.Parallel()
tests := []struct {
@ -483,3 +486,23 @@ func Test_canInjectVectorGrouping(t *testing.T) {
})
}
}
func Test_MergeBinOpVectors_Filter(t *testing.T) {
res := MergeBinOp(
OpTypeGT,
&promql.Sample{
Point: promql.Point{V: 2},
},
&promql.Sample{
Point: promql.Point{V: 0},
},
true,
true,
)
// ensure we return the left hand side's value (2) instead of the
// comparison operator's result (1: the truthy answer)
require.Equal(t, &promql.Sample{
Point: promql.Point{V: 2},
}, res)
}

@ -1,5 +1,5 @@
%{
package logql
package syntax
import (
"time"

@ -1,6 +1,6 @@
// Code generated by goyacc -p expr -o pkg/logql/expr.y.go pkg/logql/expr.y. DO NOT EDIT.
// Code generated by goyacc -p expr -o pkg/logql/syntax/expr.y.go pkg/logql/syntax/expr.y. DO NOT EDIT.
package logql
package syntax
import __yyfmt__ "fmt"

@ -0,0 +1,86 @@
package syntax
import (
"fmt"
"sort"
"github.com/grafana/loki/pkg/logql/log"
)
const UnsupportedErr = "unsupported range vector aggregation operation: %s"
func (r RangeAggregationExpr) Extractor() (log.SampleExtractor, error) {
return r.extractor(nil)
}
// extractor creates a SampleExtractor but allows for the grouping to be overridden.
func (r RangeAggregationExpr) extractor(override *Grouping) (log.SampleExtractor, error) {
if err := r.validate(); err != nil {
return nil, err
}
var groups []string
var without bool
var noLabels bool
if r.Grouping != nil {
groups = r.Grouping.Groups
without = r.Grouping.Without
if len(groups) == 0 {
noLabels = true
}
}
// uses override if it exists
if override != nil {
groups = override.Groups
without = override.Without
if len(groups) == 0 {
noLabels = true
}
}
// absent_over_time cannot be grouped (yet?), so set noLabels=true
// to make extraction more efficient and less likely to strip per query series limits.
if r.Operation == OpRangeTypeAbsent {
noLabels = true
}
sort.Strings(groups)
var stages []log.Stage
if p, ok := r.Left.Left.(*PipelineExpr); ok {
// if the expression is a pipeline then take all stages into account first.
st, err := p.MultiStages.stages()
if err != nil {
return nil, err
}
stages = st
}
// unwrap...means we want to extract metrics from labels.
if r.Left.Unwrap != nil {
var convOp string
switch r.Left.Unwrap.Operation {
case OpConvBytes:
convOp = log.ConvertBytes
case OpConvDuration, OpConvDurationSeconds:
convOp = log.ConvertDuration
default:
convOp = log.ConvertFloat
}
return log.LabelExtractorWithStages(
r.Left.Unwrap.Identifier,
convOp, groups, without, noLabels, stages,
log.ReduceAndLabelFilter(r.Left.Unwrap.PostFilters),
)
}
// otherwise we extract metrics from the log line.
switch r.Operation {
case OpRangeTypeRate, OpRangeTypeCount, OpRangeTypeAbsent:
return log.NewLineSampleExtractor(log.CountExtractor, stages, groups, without, noLabels)
case OpRangeTypeBytes, OpRangeTypeBytesRate:
return log.NewLineSampleExtractor(log.BytesExtractor, stages, groups, without, noLabels)
default:
return nil, fmt.Errorf(UnsupportedErr, r.Operation)
}
}

@ -1,4 +1,4 @@
package logql
package syntax
import (
"testing"

@ -1,4 +1,4 @@
package logql
package syntax
import (
"strings"

@ -1,4 +1,4 @@
package logql
package syntax
import (
"strings"
@ -21,27 +21,47 @@ func TestLex(t *testing.T) {
{`{foo="bar"} |~ "\\w+" | foo = 0ms`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, EQ, DURATION}},
{`{foo="bar"} |~ "\\w+" | latency > 1h15m30.918273645s`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, GT, DURATION}},
{`{foo="bar"} |~ "\\w+" | latency > 1h0.0m0s`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, GT, DURATION}},
{`{foo="bar"} |~ "\\w+" | latency > 1h0.0m0s or foo == 4.00 and bar ="foo"`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING}},
{`{foo="bar"} |~ "\\w+" | duration > 1h0.0m0s or avg == 4.00 and bar ="foo"`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING}},
{`{foo="bar"} |~ "\\w+" | latency > 1h0.0m0s or foo == 4.00 and bar ="foo" | unwrap foo`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING, PIPE, UNWRAP, IDENTIFIER}},
{
`{foo="bar"} |~ "\\w+" | latency > 1h0.0m0s or foo == 4.00 and bar ="foo"`,
[]int{
OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING,
},
},
{
`{foo="bar"} |~ "\\w+" | duration > 1h0.0m0s or avg == 4.00 and bar ="foo"`,
[]int{
OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING,
},
},
{
`{foo="bar"} |~ "\\w+" | latency > 1h0.0m0s or foo == 4.00 and bar ="foo" | unwrap foo`,
[]int{
OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING,
PIPE, IDENTIFIER, GT, DURATION, OR, IDENTIFIER, CMP_EQ, NUMBER, AND, IDENTIFIER, EQ, STRING, PIPE, UNWRAP, IDENTIFIER,
},
},
{`{foo="bar"} |~ "\\w+" | size > 250kB`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, GT, BYTES}},
{`{foo="bar"} |~ "\\w+" | size > 250kB and latency <= 1h15m30s or bar=1`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE,
IDENTIFIER, GT, BYTES, AND, IDENTIFIER, LTE, DURATION, OR, IDENTIFIER, EQ, NUMBER}},
{`{foo="bar"} |~ "\\w+" | size > 200MiB or foo == 4.00`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, GT, BYTES, OR, IDENTIFIER, CMP_EQ, NUMBER}},
{
`{foo="bar"} |~ "\\w+" | size > 250kB and latency <= 1h15m30s or bar=1`,
[]int{
OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE,
IDENTIFIER, GT, BYTES, AND, IDENTIFIER, LTE, DURATION, OR, IDENTIFIER, EQ, NUMBER,
},
},
{
`{foo="bar"} |~ "\\w+" | size > 200MiB or foo == 4.00`,
[]int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, PIPE_MATCH, STRING, PIPE, IDENTIFIER, GT, BYTES, OR, IDENTIFIER, CMP_EQ, NUMBER},
},
{`{ foo = "bar" }`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE}},
{`{ foo != "bar" }`, []int{OPEN_BRACE, IDENTIFIER, NEQ, STRING, CLOSE_BRACE}},
{`{ foo =~ "bar" }`, []int{OPEN_BRACE, IDENTIFIER, RE, STRING, CLOSE_BRACE}},
{`{ foo !~ "bar" }`, []int{OPEN_BRACE, IDENTIFIER, NRE, STRING, CLOSE_BRACE}},
{`{ foo = "bar", bar != "baz" }`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING,
COMMA, IDENTIFIER, NEQ, STRING, CLOSE_BRACE}},
{`{ foo = "bar", bar != "baz" }`, []int{
OPEN_BRACE, IDENTIFIER, EQ, STRING,
COMMA, IDENTIFIER, NEQ, STRING, CLOSE_BRACE,
}},
{`{ foo = "ba\"r" }`, []int{OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE}},
{`rate({foo="bar"}[10s])`, []int{RATE, OPEN_PARENTHESIS, OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, RANGE, CLOSE_PARENTHESIS}},
{`count_over_time({foo="bar"}[5m])`, []int{COUNT_OVER_TIME, OPEN_PARENTHESIS, OPEN_BRACE, IDENTIFIER, EQ, STRING, CLOSE_BRACE, RANGE, CLOSE_PARENTHESIS}},

@ -1,20 +1,16 @@
package logql
package syntax
import (
"errors"
"fmt"
"net/http"
"sort"
"strings"
"sync"
"text/scanner"
errors2 "github.com/pkg/errors"
"github.com/prometheus/prometheus/model/labels"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/util"
)
@ -142,7 +138,7 @@ func ParseMatchers(input string) ([]*labels.Matcher, error) {
if !ok {
return nil, errors.New("only label matchers is supported")
}
return matcherExpr.matchers, nil
return matcherExpr.Mts, nil
}
// ParseSampleExpr parses a string and returns the sampleExpr
@ -201,34 +197,3 @@ func ParseLabels(lbs string) (labels.Labels, error) {
sort.Sort(ls)
return ls, nil
}
// Match extracts and parses multiple matcher groups from a slice of strings
func Match(xs []string) ([][]*labels.Matcher, error) {
groups := make([][]*labels.Matcher, 0, len(xs))
for _, x := range xs {
ms, err := ParseMatchers(x)
if err != nil {
return nil, err
}
if len(ms) == 0 {
return nil, errors2.Errorf("0 matchers in group: %s", x)
}
groups = append(groups, ms)
}
return groups, nil
}
func ParseAndValidateSeriesQuery(r *http.Request) (*logproto.SeriesRequest, error) {
req, err := loghttp.ParseSeriesQuery(r)
if err != nil {
return nil, err
}
// ensure matchers are valid before fanning out to ingesters/store as well as returning valuable parsing errors
// instead of 500s
_, err = Match(req.Groups)
if err != nil {
return nil, err
}
return req, nil
}

@ -1,4 +1,4 @@
package logql
package syntax
import (
"errors"
@ -34,7 +34,7 @@ func TestParse(t *testing.T) {
newLineFilterExpr(labels.MatchRegexp, "", "error\\"),
},
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchRegexp, "foo", "bar\\w+"),
},
},
@ -75,26 +75,26 @@ func TestParse(t *testing.T) {
},
{
in: `{foo="bar"}`,
exp: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
exp: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
},
{
in: `{ foo = "bar" }`,
exp: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
exp: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
},
{
in: `{ namespace="buzz", foo != "bar" }`,
exp: &MatchersExpr{matchers: []*labels.Matcher{
exp: &MatchersExpr{Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "namespace", "buzz"),
mustNewMatcher(labels.MatchNotEqual, "foo", "bar"),
}},
},
{
in: `{ foo =~ "bar" }`,
exp: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchRegexp, "foo", "bar")}},
exp: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchRegexp, "foo", "bar")}},
},
{
in: `{ namespace="buzz", foo !~ "bar" }`,
exp: &MatchersExpr{matchers: []*labels.Matcher{
exp: &MatchersExpr{Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "namespace", "buzz"),
mustNewMatcher(labels.MatchNotRegexp, "foo", "bar"),
}},
@ -103,7 +103,7 @@ func TestParse(t *testing.T) {
in: `count_over_time({ foo = "bar" }[12m])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
@ -113,7 +113,7 @@ func TestParse(t *testing.T) {
in: `bytes_over_time({ foo = "bar" }[12m])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: OpRangeTypeBytes,
@ -123,7 +123,7 @@ func TestParse(t *testing.T) {
in: `bytes_rate({ foo = "bar" }[12m])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: OpRangeTypeBytesRate,
@ -133,7 +133,7 @@ func TestParse(t *testing.T) {
in: `rate({ foo = "bar" }[5h])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "rate",
@ -153,7 +153,7 @@ func TestParse(t *testing.T) {
in: `rate({ foo = "bar" }[5d])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * 24 * time.Hour,
},
Operation: "rate",
@ -163,7 +163,7 @@ func TestParse(t *testing.T) {
in: `count_over_time({ foo = "bar" }[1w])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 7 * 24 * time.Hour,
},
Operation: "count_over_time",
@ -173,7 +173,7 @@ func TestParse(t *testing.T) {
in: `absent_over_time({ foo = "bar" }[1w])`,
exp: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 7 * 24 * time.Hour,
},
Operation: OpRangeTypeAbsent,
@ -183,7 +183,7 @@ func TestParse(t *testing.T) {
in: `sum(rate({ foo = "bar" }[5h]))`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "rate",
@ -193,7 +193,7 @@ func TestParse(t *testing.T) {
in: `sum(rate({ foo ="bar" }[1y]))`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 365 * 24 * time.Hour,
},
Operation: "rate",
@ -203,7 +203,7 @@ func TestParse(t *testing.T) {
in: `avg(count_over_time({ foo = "bar" }[5h])) by (bar,foo)`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -226,7 +226,7 @@ func TestParse(t *testing.T) {
mustNewLabelReplaceExpr(
&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -242,7 +242,7 @@ func TestParse(t *testing.T) {
in: `avg(count_over_time({ foo = "bar" }[5h])) by ()`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -255,7 +255,7 @@ func TestParse(t *testing.T) {
in: `max without (bar) (count_over_time({ foo = "bar" }[5h]))`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -268,7 +268,7 @@ func TestParse(t *testing.T) {
in: `max without () (count_over_time({ foo = "bar" }[5h]))`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -281,7 +281,7 @@ func TestParse(t *testing.T) {
in: `topk(10,count_over_time({ foo = "bar" }[5h])) without (bar)`,
exp: mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -294,7 +294,7 @@ func TestParse(t *testing.T) {
in: `bottomk(30 ,sum(rate({ foo = "bar" }[5h])) by (foo))`,
exp: mustNewVectorAggregationExpr(mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "rate",
@ -308,7 +308,7 @@ func TestParse(t *testing.T) {
in: `max( sum(count_over_time({ foo = "bar" }[5h])) without (foo,bar) ) by (foo)`,
exp: mustNewVectorAggregationExpr(mustNewVectorAggregationExpr(&RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 5 * time.Hour,
},
Operation: "count_over_time",
@ -616,7 +616,7 @@ func TestParse(t *testing.T) {
},
{
in: `{ foo = "bar", bar != "baz" }`,
exp: &MatchersExpr{matchers: []*labels.Matcher{
exp: &MatchersExpr{Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
mustNewMatcher(labels.MatchNotEqual, "bar", "baz"),
}},
@ -1019,7 +1019,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1035,7 +1035,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1052,7 +1052,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1086,7 +1086,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1102,7 +1102,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1119,7 +1119,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1149,7 +1149,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1170,7 +1170,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1186,7 +1186,7 @@ func TestParse(t *testing.T) {
mustNewVectorAggregationExpr(newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1227,7 +1227,7 @@ func TestParse(t *testing.T) {
newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "namespace", "tns"),
},
},
@ -1261,7 +1261,7 @@ func TestParse(t *testing.T) {
newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "namespace", "tns"),
},
},
@ -1282,7 +1282,7 @@ func TestParse(t *testing.T) {
newRangeAggregationExpr(
&LogRange{
Left: &MatchersExpr{
matchers: []*labels.Matcher{
Mts: []*labels.Matcher{
mustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
@ -1295,7 +1295,7 @@ func TestParse(t *testing.T) {
},
nil,
),
&LiteralExpr{value: 0.5},
&LiteralExpr{Val: 0.5},
),
},
{
@ -1306,10 +1306,10 @@ func TestParse(t *testing.T) {
&BinOpOptions{
VectorMatching: &VectorMatching{Card: CardOneToOne},
},
&LiteralExpr{value: 1},
&LiteralExpr{Val: 1},
mustNewBinOpExpr(OpTypeDiv, &BinOpOptions{
VectorMatching: &VectorMatching{Card: CardOneToOne},
}, &LiteralExpr{value: -2}, &LiteralExpr{value: 1}),
}, &LiteralExpr{Val: -2}, &LiteralExpr{Val: 1}),
),
},
{
@ -1322,8 +1322,8 @@ func TestParse(t *testing.T) {
},
mustNewBinOpExpr(OpTypeAdd, &BinOpOptions{
VectorMatching: &VectorMatching{Card: CardOneToOne},
}, &LiteralExpr{value: 1}, &LiteralExpr{value: 1}),
&LiteralExpr{value: -1},
}, &LiteralExpr{Val: 1}, &LiteralExpr{Val: 1}),
&LiteralExpr{Val: -1},
),
},
{
@ -2208,7 +2208,8 @@ func TestParse(t *testing.T) {
) by (foo,bar)
`,
exp: mustNewBinOpExpr(OpTypeAdd, &BinOpOptions{
VectorMatching: &VectorMatching{Card: CardOneToOne}, ReturnBool: false},
VectorMatching: &VectorMatching{Card: CardOneToOne}, ReturnBool: false,
},
mustNewVectorAggregationExpr(
newRangeAggregationExpr(
newLogRange(&PipelineExpr{
@ -2672,54 +2673,54 @@ func TestParse(t *testing.T) {
{
// ensure binary ops with two literals are reduced recursively
in: `1 + 1 + 1`,
exp: &LiteralExpr{value: 3},
exp: &LiteralExpr{Val: 3},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 == 1`,
exp: &LiteralExpr{value: 1},
exp: &LiteralExpr{Val: 1},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 != 1`,
exp: &LiteralExpr{value: 0},
exp: &LiteralExpr{Val: 0},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 > 1`,
exp: &LiteralExpr{value: 0},
exp: &LiteralExpr{Val: 0},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 >= 1`,
exp: &LiteralExpr{value: 1},
exp: &LiteralExpr{Val: 1},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 < 1`,
exp: &LiteralExpr{value: 0},
exp: &LiteralExpr{Val: 0},
},
{
// ensure binary ops with two literals are reduced when comparisons are used
in: `1 <= 1`,
exp: &LiteralExpr{value: 1},
exp: &LiteralExpr{Val: 1},
},
{
// ensure binary ops with two literals are reduced recursively when comparisons are used
in: `1 >= 1 > 1`,
exp: &LiteralExpr{value: 0},
exp: &LiteralExpr{Val: 0},
},
{
in: `{foo="bar"} + {foo="bar"}`,
err: logqlmodel.NewParseError(`unexpected type for left leg of binary operation (+): *logql.MatchersExpr`, 0, 0),
err: logqlmodel.NewParseError(`unexpected type for left leg of binary operation (+): *syntax.MatchersExpr`, 0, 0),
},
{
in: `sum(count_over_time({foo="bar"}[5m])) by (foo) - {foo="bar"}`,
err: logqlmodel.NewParseError(`unexpected type for right leg of binary operation (-): *logql.MatchersExpr`, 0, 0),
err: logqlmodel.NewParseError(`unexpected type for right leg of binary operation (-): *syntax.MatchersExpr`, 0, 0),
},
{
in: `{foo="bar"} / sum(count_over_time({foo="bar"}[5m])) by (foo)`,
err: logqlmodel.NewParseError(`unexpected type for left leg of binary operation (/): *logql.MatchersExpr`, 0, 0),
err: logqlmodel.NewParseError(`unexpected type for left leg of binary operation (/): *syntax.MatchersExpr`, 0, 0),
},
{
in: `sum(count_over_time({foo="bar"}[5m])) by (foo) or 1`,
@ -2743,14 +2744,14 @@ func TestParse(t *testing.T) {
},
SampleExpr: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
},
RHS: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
@ -2767,18 +2768,18 @@ func TestParse(t *testing.T) {
},
SampleExpr: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
},
RHS: &LiteralExpr{value: 1},
RHS: &LiteralExpr{Val: 1},
},
},
{
// cannot compare metric & log queries
in: `count_over_time({ foo = "bar" }[12m]) > { foo = "bar" }`,
err: logqlmodel.NewParseError("unexpected type for right leg of binary operation (>): *logql.MatchersExpr", 0, 0),
err: logqlmodel.NewParseError("unexpected type for right leg of binary operation (>): *syntax.MatchersExpr", 0, 0),
},
{
in: `count_over_time({ foo = "bar" }[12m]) or count_over_time({ foo = "bar" }[12m]) > 1`,
@ -2790,7 +2791,7 @@ func TestParse(t *testing.T) {
},
SampleExpr: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
@ -2803,24 +2804,24 @@ func TestParse(t *testing.T) {
},
SampleExpr: &RangeAggregationExpr{
Left: &LogRange{
Left: &MatchersExpr{matchers: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Left: &MatchersExpr{Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}},
Interval: 12 * time.Minute,
},
Operation: "count_over_time",
},
RHS: &LiteralExpr{value: 1},
RHS: &LiteralExpr{Val: 1},
},
},
},
{
// test associativity
in: `1 > 1 < 1`,
exp: &LiteralExpr{value: 1},
exp: &LiteralExpr{Val: 1},
},
{
// bool modifiers are reduced-away between two literal legs
in: `1 > 1 > bool 1`,
exp: &LiteralExpr{value: 0},
exp: &LiteralExpr{Val: 0},
},
{
// cannot lead with bool modifier
@ -3170,61 +3171,3 @@ func TestParseLogSelectorExpr_equalityMatcher(t *testing.T) {
})
}
}
func Test_match(t *testing.T) {
tests := []struct {
name string
input []string
want [][]*labels.Matcher
wantErr bool
}{
{"malformed", []string{`{a="1`}, nil, true},
{"empty on nil input", nil, [][]*labels.Matcher{}, false},
{"empty on empty input", []string{}, [][]*labels.Matcher{}, false},
{
"single",
[]string{`{a="1"}`},
[][]*labels.Matcher{
{mustMatcher(labels.MatchEqual, "a", "1")},
},
false,
},
{
"multiple groups",
[]string{`{a="1"}`, `{b="2", c=~"3", d!="4"}`},
[][]*labels.Matcher{
{mustMatcher(labels.MatchEqual, "a", "1")},
{
mustMatcher(labels.MatchEqual, "b", "2"),
mustMatcher(labels.MatchRegexp, "c", "3"),
mustMatcher(labels.MatchNotEqual, "d", "4"),
},
},
false,
},
{
"errors on empty group",
[]string{`{}`},
nil,
true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := Match(tt.input)
if tt.wantErr {
require.Error(t, err)
} else {
require.Equal(t, tt.want, got)
}
})
}
}
func mustMatcher(t labels.MatchType, n string, v string) *labels.Matcher {
m, err := labels.NewMatcher(t, n, v)
if err != nil {
panic(err)
}
return m
}

@ -1,4 +1,4 @@
package logql
package syntax
type WalkFn = func(e interface{})

@ -1,4 +1,4 @@
package logql
package syntax
import (
"testing"

@ -0,0 +1,65 @@
package vector
import (
"math"
"github.com/prometheus/prometheus/promql"
)
type HeapByMaxValue promql.Vector
func (s HeapByMaxValue) Len() int {
return len(s)
}
func (s HeapByMaxValue) Less(i, j int) bool {
if math.IsNaN(s[i].V) {
return true
}
return s[i].V < s[j].V
}
func (s HeapByMaxValue) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}
func (s *HeapByMaxValue) Push(x interface{}) {
*s = append(*s, *(x.(*promql.Sample)))
}
func (s *HeapByMaxValue) Pop() interface{} {
old := *s
n := len(old)
el := old[n-1]
*s = old[0 : n-1]
return el
}
type HeapByMinValue promql.Vector
func (s HeapByMinValue) Len() int {
return len(s)
}
func (s HeapByMinValue) Less(i, j int) bool {
if math.IsNaN(s[i].V) {
return true
}
return s[i].V > s[j].V
}
func (s HeapByMinValue) Swap(i, j int) {
s[i], s[j] = s[j], s[i]
}
func (s *HeapByMinValue) Push(x interface{}) {
*s = append(*s, *(x.(*promql.Sample)))
}
func (s *HeapByMinValue) Pop() interface{} {
old := *s
n := len(old)
el := old[n-1]
*s = old[0 : n-1]
return el
}

@ -15,6 +15,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
loghttp_legacy "github.com/grafana/loki/pkg/loghttp/legacy"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/tenant"
util_log "github.com/grafana/loki/pkg/util/log"
@ -148,14 +149,14 @@ func (q *QuerierAPI) LogQueryHandler(w http.ResponseWriter, r *http.Request) {
return
}
expr, err := logql.ParseExpr(request.Query)
expr, err := syntax.ParseExpr(request.Query)
if err != nil {
serverutil.WriteError(err, w)
return
}
// short circuit metric queries
if _, ok := expr.(logql.SampleExpr); ok {
if _, ok := expr.(syntax.SampleExpr); ok {
serverutil.WriteError(httpgrpc.Errorf(http.StatusBadRequest, "legacy endpoints only support %s result type", logqlmodel.ValueTypeStreams), w)
return
}
@ -340,7 +341,7 @@ func (q *QuerierAPI) TailHandler(w http.ResponseWriter, r *http.Request) {
// SeriesHandler returns the list of time series that match a certain label set.
// See https://prometheus.io/docs/prometheus/latest/querying/api/#finding-series-by-label-matchers
func (q *QuerierAPI) SeriesHandler(w http.ResponseWriter, r *http.Request) {
req, err := logql.ParseAndValidateSeriesQuery(r)
req, err := loghttp.ParseAndValidateSeriesQuery(r)
if err != nil {
serverutil.WriteError(httpgrpc.Errorf(http.StatusBadRequest, err.Error()), w)
return
@ -365,11 +366,11 @@ func parseRegexQuery(httpRequest *http.Request) (string, error) {
query := httpRequest.Form.Get("query")
regexp := httpRequest.Form.Get("regexp")
if regexp != "" {
expr, err := logql.ParseLogSelector(query, true)
expr, err := syntax.ParseLogSelector(query, true)
if err != nil {
return "", err
}
newExpr, err := logql.AddFilterExpr(expr, labels.MatchRegexp, "", regexp)
newExpr, err := syntax.AddFilterExpr(expr, labels.MatchRegexp, "", regexp)
if err != nil {
return "", err
}
@ -384,13 +385,13 @@ func (q *QuerierAPI) validateEntriesLimits(ctx context.Context, query string, li
return httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
expr, err := logql.ParseExpr(query)
expr, err := syntax.ParseExpr(query)
if err != nil {
return err
}
// entry limit does not apply to metric queries.
if _, ok := expr.(logql.SampleExpr); ok {
if _, ok := expr.(syntax.SampleExpr); ok {
return nil
}

@ -9,6 +9,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/tenant"
)
@ -32,7 +33,6 @@ func NewMultiTenantQuerier(querier Querier, logger log.Logger) *MultiTenantQueri
}
func (q *MultiTenantQuerier) SelectLogs(ctx context.Context, params logql.SelectLogParams) (iter.EntryIterator, error) {
tenantIDs, err := q.resolver.TenantIDs(ctx)
if err != nil {
return nil, err
@ -47,7 +47,6 @@ func (q *MultiTenantQuerier) SelectLogs(ctx context.Context, params logql.Select
for i, id := range tenantIDs {
singleContext := user.InjectUserID(ctx, id)
iter, err := q.Querier.SelectLogs(singleContext, params)
if err != nil {
return nil, err
}
@ -58,7 +57,6 @@ func (q *MultiTenantQuerier) SelectLogs(ctx context.Context, params logql.Select
}
func (q *MultiTenantQuerier) SelectSamples(ctx context.Context, params logql.SelectSampleParams) (iter.SampleIterator, error) {
tenantIDs, err := q.resolver.TenantIDs(ctx)
if err != nil {
return nil, err
@ -73,7 +71,6 @@ func (q *MultiTenantQuerier) SelectSamples(ctx context.Context, params logql.Sel
for i, id := range tenantIDs {
singleContext := user.InjectUserID(ctx, id)
iter, err := q.Querier.SelectSamples(singleContext, params)
if err != nil {
return nil, err
}
@ -95,7 +92,7 @@ func NewTenantEntryIterator(iter iter.EntryIterator, id string) *TenantEntryIter
func (i *TenantEntryIterator) Labels() string {
// TODO: cache manipulated labels and add a benchmark.
lbls, _ := logql.ParseLabels(i.EntryIterator.Labels())
lbls, _ := syntax.ParseLabels(i.EntryIterator.Labels())
builder := labels.NewBuilder(lbls.WithoutLabels(defaultTenantLabel))
// Prefix label if it conflicts with the tenant label.
@ -119,7 +116,7 @@ func NewTenantSampleIterator(iter iter.SampleIterator, id string) *TenantSampleI
func (i *TenantSampleIterator) Labels() string {
// TODO: cache manipulated labels
lbls, _ := logql.ParseLabels(i.SampleIterator.Labels())
lbls, _ := syntax.ParseLabels(i.SampleIterator.Labels())
builder := labels.NewBuilder(lbls.WithoutLabels(defaultTenantLabel))
// Prefix label if it conflicts with the tenant label.

@ -13,6 +13,7 @@ import (
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
)
func TestMultiTenantQuerier_SelectLogs(t *testing.T) {
@ -130,8 +131,8 @@ var samples = []logproto.Sample{
}
var (
labelFoo, _ = logql.ParseLabels("{app=\"foo\"}")
labelBar, _ = logql.ParseLabels("{app=\"bar\"}")
labelFoo, _ = syntax.ParseLabels("{app=\"foo\"}")
labelBar, _ = syntax.ParseLabels("{app=\"bar\"}")
)
func newSampleIterator() iter.SampleIterator {

@ -461,7 +461,6 @@ func (q *SingleTenantQuerier) seriesForMatchers(
groups []string,
shards []string,
) ([]logproto.SeriesIdentifier, error) {
var results []logproto.SeriesIdentifier
// If no matchers were specified for the series query,
// we send a query with an empty matcher which will match every series.

@ -22,6 +22,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
@ -229,7 +230,7 @@ func (Codec) DecodeRequest(_ context.Context, r *http.Request, forwardHeaders []
Shards: req.Shards,
}, nil
case SeriesOp:
req, err := logql.ParseAndValidateSeriesQuery(r)
req, err := loghttp.ParseAndValidateSeriesQuery(r)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
@ -887,11 +888,11 @@ func NewEmptyResponse(r queryrangebase.Request) (queryrangebase.Response, error)
}, nil
case *LokiRequest:
// range query can either be metrics or logs
expr, err := logql.ParseExpr(req.Query)
expr, err := syntax.ParseExpr(req.Query)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
if _, ok := expr.(logql.SampleExpr); ok {
if _, ok := expr.(syntax.SampleExpr); ok {
return &LokiPromResponse{
Response: queryrangebase.NewEmptyPrometheusResponse(),
}, nil

@ -14,6 +14,7 @@ import (
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
@ -298,7 +299,7 @@ func TestInstanceDownstream(t *testing.T) {
1000,
nil,
)
expr, err := logql.ParseExpr(`{foo="bar"}`)
expr, err := syntax.ParseExpr(`{foo="bar"}`)
require.Nil(t, err)
expectedResp := func() *LokiResponse {
@ -374,5 +375,4 @@ func TestCancelWhileWaitingResponse(t *testing.T) {
}, 5*time.Second, 10*time.Millisecond,
"The parent context calling the Downstreamer For method was canceled "+
"but the For method did not return as expected.")
}

@ -13,6 +13,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
@ -42,7 +43,6 @@ func NewTripperware(
schema chunk.SchemaConfig,
registerer prometheus.Registerer,
) (queryrangebase.Tripperware, Stopper, error) {
instrumentMetrics := queryrangebase.NewInstrumentMiddlewareMetrics(registerer)
retryMetrics := queryrangebase.NewRetryMiddlewareMetrics(registerer)
shardingMetrics := logql.NewShardingMetrics(registerer)
@ -116,14 +116,14 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
expr, err := logql.ParseExpr(rangeQuery.Query)
expr, err := syntax.ParseExpr(rangeQuery.Query)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
switch e := expr.(type) {
case logql.SampleExpr:
case syntax.SampleExpr:
return r.metric.RoundTrip(req)
case logql.LogSelectorExpr:
case syntax.LogSelectorExpr:
expr, err := transformRegexQuery(req, e)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
@ -141,7 +141,7 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
return r.next.RoundTrip(req)
}
case SeriesOp:
_, err := logql.ParseAndValidateSeriesQuery(req)
_, err := loghttp.ParseAndValidateSeriesQuery(req)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
@ -157,12 +157,12 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
expr, err := logql.ParseExpr(instantQuery.Query)
expr, err := syntax.ParseExpr(instantQuery.Query)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
switch expr.(type) {
case logql.SampleExpr:
case syntax.SampleExpr:
return r.instantMetric.RoundTrip(req)
default:
return r.next.RoundTrip(req)
@ -173,10 +173,10 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
}
// transformRegexQuery backport the old regexp params into the v1 query format
func transformRegexQuery(req *http.Request, expr logql.LogSelectorExpr) (logql.LogSelectorExpr, error) {
func transformRegexQuery(req *http.Request, expr syntax.LogSelectorExpr) (syntax.LogSelectorExpr, error) {
regexp := req.Form.Get("regexp")
if regexp != "" {
filterExpr, err := logql.AddFilterExpr(expr, labels.MatchRegexp, "", regexp)
filterExpr, err := syntax.AddFilterExpr(expr, labels.MatchRegexp, "", regexp)
if err != nil {
return nil, err
}

@ -12,7 +12,7 @@ import (
"github.com/weaveworks/common/httpgrpc"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/tenant"
"github.com/grafana/loki/pkg/util"
@ -293,13 +293,13 @@ func forInterval(interval time.Duration, start, end time.Time, endTimeInclusive
// maxRangeVectorDuration returns the maximum range vector duration within a LogQL query.
func maxRangeVectorDuration(q string) (time.Duration, error) {
expr, err := logql.ParseSampleExpr(q)
expr, err := syntax.ParseSampleExpr(q)
if err != nil {
return 0, err
}
var max time.Duration
expr.Walk(func(e interface{}) {
if r, ok := e.(*logql.LogRange); ok && r.Interval > max {
if r, ok := e.(*syntax.LogRange); ok && r.Interval > max {
max = r.Interval
}
})

@ -25,6 +25,7 @@ import (
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
ruler "github.com/grafana/loki/pkg/ruler/base"
"github.com/grafana/loki/pkg/ruler/rulespb"
"github.com/grafana/loki/pkg/ruler/util"
@ -170,7 +171,7 @@ func MultiTenantRuleManager(cfg Config, engine *logql.Engine, overrides RulesLim
type GroupLoader struct{}
func (GroupLoader) Parse(query string) (parser.Expr, error) {
expr, err := logql.ParseExpr(query)
expr, err := syntax.ParseExpr(query)
if err != nil {
return nil, err
}
@ -248,7 +249,7 @@ func validateRuleNode(r *rulefmt.RuleNode, groupName string) error {
if r.Expr.Value == "" {
return errors.Errorf("field 'expr' must be set in rule")
} else if _, err := logql.ParseExpr(r.Expr.Value); err != nil {
} else if _, err := syntax.ParseExpr(r.Expr.Value); err != nil {
return errors.Wrapf(err, fmt.Sprintf("could not parse expression for record '%s' in group '%s'", r.Record.Value, groupName))
}
@ -337,7 +338,7 @@ func testTemplateParsing(rl *rulefmt.RuleNode) (errs []error) {
// Allows logql expressions to be treated as promql expressions by the prometheus rules pkg.
type exprAdapter struct {
logql.Expr
syntax.Expr
}
func (exprAdapter) PositionRange() parser.PositionRange { return parser.PositionRange{} }

@ -16,8 +16,8 @@ import (
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/storage/chunk"
@ -310,7 +310,7 @@ type logBatchIterator struct {
ctx context.Context
cancel context.CancelFunc
pipeline logql.Pipeline
pipeline syntax.Pipeline
}
func newLogBatchIterator(
@ -320,7 +320,7 @@ func newLogBatchIterator(
chunks []*LazyChunk,
batchSize int,
matchers []*labels.Matcher,
pipeline logql.Pipeline,
pipeline syntax.Pipeline,
direction logproto.Direction,
start, end time.Time,
chunkFilterer ChunkFilterer,
@ -456,7 +456,7 @@ type sampleBatchIterator struct {
ctx context.Context
cancel context.CancelFunc
extractor logql.SampleExtractor
extractor syntax.SampleExtractor
}
func newSampleBatchIterator(
@ -466,7 +466,7 @@ func newSampleBatchIterator(
chunks []*LazyChunk,
batchSize int,
matchers []*labels.Matcher,
extractor logql.SampleExtractor,
extractor syntax.SampleExtractor,
start, end time.Time,
chunkFilterer ChunkFilterer,
) (iter.SampleIterator, error) {

@ -17,7 +17,7 @@ import (
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
lstore "github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/local"
@ -101,7 +101,7 @@ func fillStore(cm storage.ClientMetrics) error {
wgPush.Add(1)
go func(j int) {
defer wgPush.Done()
lbs, err := logql.ParseLabels(fmt.Sprintf("{foo=\"bar\",level=\"%d\"}", j))
lbs, err := syntax.ParseLabels(fmt.Sprintf("{foo=\"bar\",level=\"%d\"}", j))
if err != nil {
panic(err)
}

@ -8,7 +8,7 @@ import (
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention"
)
@ -172,7 +172,7 @@ func TestDeleteRequest_IsDeleted(t *testing.T) {
}
func mustParseLabel(input string) labels.Labels {
lbls, err := logql.ParseLabels(input)
lbls, err := syntax.ParseLabels(input)
if err != nil {
panic(err)
}

@ -8,7 +8,7 @@ import (
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention"
)
@ -61,7 +61,7 @@ func TestDeleteRequestsManager_Expired(t *testing.T) {
}
now := model.Now()
lblFoo, err := logql.ParseLabels(`{foo="bar"}`)
lblFoo, err := syntax.ParseLabels(`{foo="bar"}`)
require.NoError(t, err)
chunkEntry := retention.ChunkEntry{

@ -16,7 +16,7 @@ import (
"github.com/weaveworks/common/user"
"go.etcd.io/bbolt"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/local"
@ -106,7 +106,7 @@ var (
)
func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry {
lbs, err := logql.ParseLabels(labels)
lbs, err := syntax.ParseLabels(labels)
if err != nil {
panic(err)
}

@ -7,12 +7,12 @@ import (
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage/tsdb/index"
)
func mustParseLabels(s string) labels.Labels {
ls, err := logql.ParseLabels(s)
ls, err := syntax.ParseLabels(s)
if err != nil {
panic(err)
}

@ -14,7 +14,7 @@ import (
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
@ -89,7 +89,7 @@ func newLazyInvalidChunk(stream logproto.Stream) *LazyChunk {
}
func newChunk(stream logproto.Stream) chunk.Chunk {
lbs, err := logql.ParseLabels(stream.Labels)
lbs, err := syntax.ParseLabels(stream.Labels)
if err != nil {
panic(err)
}
@ -113,7 +113,7 @@ func newChunk(stream logproto.Stream) chunk.Chunk {
}
func newMatchers(matchers string) []*labels.Matcher {
res, err := logql.ParseMatchers(matchers)
res, err := syntax.ParseMatchers(matchers)
if err != nil {
panic(err)
}

@ -13,7 +13,7 @@ import (
"golang.org/x/time/rate"
"gopkg.in/yaml.v2"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/ruler/util"
"github.com/grafana/loki/pkg/util/flagext"
)
@ -213,10 +213,9 @@ func (l *Limits) UnmarshalYAML(unmarshal func(interface{}) error) error {
// Validate validates that this limits config is valid.
func (l *Limits) Validate() error {
if l.StreamRetention != nil {
for i, rule := range l.StreamRetention {
matchers, err := logql.ParseMatchers(rule.Selector)
matchers, err := syntax.ParseMatchers(rule.Selector)
if err != nil {
return fmt.Errorf("invalid labels matchers: %w", err)
}
@ -576,7 +575,6 @@ func (sm *OverwriteMarshalingStringMap) UnmarshalJSON(val []byte) error {
sm.m = def
return nil
}
// MarshalYAML explicitly uses the the type receiver and not pointer receiver

Loading…
Cancel
Save