LogQL: [optimization] syntax: Replace "panic" in "/pkg/logql/syntax" with "error" (#7208)

pull/8425/head
李国忠 2 years ago committed by GitHub
parent 2139dbf55f
commit 322783e3d8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 7
      pkg/ingester/instance.go
  2. 8
      pkg/logql/engine.go
  3. 52
      pkg/logql/evaluator.go
  4. 37
      pkg/logql/evaluator_test.go
  5. 1
      pkg/logql/log/label_filter.go
  6. 176
      pkg/logql/syntax/ast.go
  7. 6
      pkg/logql/syntax/ast_test.go
  8. 3
      pkg/logql/syntax/extractor.go
  9. 24
      pkg/logql/syntax/parser.go
  10. 6
      pkg/querier/multi_tenant_querier.go
  11. 5
      pkg/querier/queryrange/shard_resolver.go

@ -425,11 +425,14 @@ func (i *instance) QuerySample(ctx context.Context, req logql.SelectSampleParams
if len(shards) == 1 {
shard = &shards[0]
}
selector, err := expr.Selector()
if err != nil {
return nil, err
}
err = i.forMatchingStreams(
ctx,
req.Start,
expr.Selector().Matchers(),
selector.Matchers(),
shard,
func(stream *stream) error {
iter, err := stream.SampleIterator(ctx, stats, req.Start, req.End, extractor.ForStream(stream.labels))

@ -99,7 +99,7 @@ func (s SelectSampleParams) LogSelector() (syntax.LogSelectorExpr, error) {
if err != nil {
return nil, err
}
return expr.Selector(), nil
return expr.Selector()
}
// Querier allows a LogQL expression to fetch an EntryIterator for a
@ -403,9 +403,13 @@ func (q *query) evalSample(ctx context.Context, expr syntax.SampleExpr) (promql_
}
func (q *query) evalLiteral(_ context.Context, expr *syntax.LiteralExpr) (promql_parser.Value, error) {
value, err := expr.Value()
if err != nil {
return nil, err
}
s := promql.Scalar{
T: q.params.Start().UnixNano() / int64(time.Millisecond),
V: expr.Value(),
V: value,
}
if GetRangeType(q.params) == InstantType {

@ -472,9 +472,13 @@ func rangeAggEvaluator(
return nil, err
}
if expr.Operation == syntax.OpRangeTypeAbsent {
absentLabels, err := absentLabels(expr)
if err != nil {
return nil, err
}
return &absentRangeVectorEvaluator{
iter: iter,
lbs: absentLabels(expr),
lbs: absentLabels,
}, nil
}
return &rangeVectorEvaluator{
@ -771,8 +775,11 @@ func vectorBinop(op string, opts *syntax.BinOpOptions, lhs, rhs promql.Vector, l
ls, rs = rs, ls
}
}
if merged := syntax.MergeBinOp(op, ls, rs, filter, syntax.IsComparisonOperator(op)); merged != nil {
merged, err := syntax.MergeBinOp(op, ls, rs, filter, syntax.IsComparisonOperator(op))
if err != nil {
return nil, err
}
if merged != nil {
// replace with labels specified by expr
merged.Metric = metric
results = append(results, *merged)
@ -886,29 +893,39 @@ func literalStepEvaluator(
inverted bool,
returnBool bool,
) (StepEvaluator, error) {
val, err := lit.Value()
if err != nil {
return nil, err
}
var mergeErr error
return newStepEvaluator(
func() (bool, int64, promql.Vector) {
ok, ts, vec := eval.Next()
results := make(promql.Vector, 0, len(vec))
for _, sample := range vec {
literalPoint := promql.Sample{
Metric: sample.Metric,
Point: promql.Point{T: ts, V: lit.Value()},
Point: promql.Point{T: ts, V: val},
}
left, right := &literalPoint, &sample
if inverted {
left, right = right, left
}
if merged := syntax.MergeBinOp(
merged, err := syntax.MergeBinOp(
op,
left,
right,
!returnBool,
syntax.IsComparisonOperator(op),
); merged != nil {
)
if err != nil {
mergeErr = err
return false, 0, nil
}
if merged != nil {
results = append(results, *merged)
}
}
@ -916,7 +933,12 @@ func literalStepEvaluator(
return ok, ts, results
},
eval.Close,
eval.Error,
func() error {
if mergeErr != nil {
return mergeErr
}
return eval.Error()
},
)
}
@ -1010,12 +1032,16 @@ func labelReplaceEvaluator(
}
// This is to replace missing timeseries during absent_over_time aggregation.
func absentLabels(expr syntax.SampleExpr) labels.Labels {
func absentLabels(expr syntax.SampleExpr) (labels.Labels, error) {
m := labels.Labels{}
lm := expr.Selector().Matchers()
selector, err := expr.Selector()
if err != nil {
return nil, err
}
lm := selector.Matchers()
if len(lm) == 0 {
return m
return m, nil
}
empty := []string{}
@ -1033,5 +1059,5 @@ func absentLabels(expr syntax.SampleExpr) labels.Labels {
for _, v := range empty {
m = labels.NewBuilder(m).Del(v).Labels(nil)
}
return m
return m, nil
}

@ -11,7 +11,7 @@ import (
)
func TestDefaultEvaluator_DivideByZero(t *testing.T) {
require.Equal(t, true, math.IsNaN(syntax.MergeBinOp(syntax.OpTypeDiv,
op, err := syntax.MergeBinOp(syntax.OpTypeDiv,
&promql.Sample{
Point: promql.Point{T: 1, V: 1},
},
@ -20,9 +20,11 @@ func TestDefaultEvaluator_DivideByZero(t *testing.T) {
},
false,
false,
).Point.V))
)
require.NoError(t, err)
require.Equal(t, true, math.IsNaN(syntax.MergeBinOp(syntax.OpTypeMod,
require.Equal(t, true, math.IsNaN(op.Point.V))
binOp, err := syntax.MergeBinOp(syntax.OpTypeMod,
&promql.Sample{
Point: promql.Point{T: 1, V: 1},
},
@ -31,7 +33,9 @@ func TestDefaultEvaluator_DivideByZero(t *testing.T) {
},
false,
false,
).Point.V))
)
require.NoError(t, err)
require.Equal(t, true, math.IsNaN(binOp.Point.V))
}
func TestDefaultEvaluator_Sortable(t *testing.T) {
logqlSort := `sort(rate(({app=~"foo|bar"} |~".+bar")[1m])) `
@ -216,20 +220,33 @@ 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, 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))
op, err := syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, false, false)
require.NoError(t, err)
require.Equal(t, tc.expected, op)
op2, err := syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, false, true)
require.NoError(t, err)
require.Equal(t, tc.expected, op2)
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, nil, false, true))
op3, err := syntax.MergeBinOp(tc.op, tc.lhs, nil, false, true)
require.NoError(t, err)
require.Nil(t, op3)
// test filtered variants
if tc.expected.V == 0 {
// ensure zeroed predicates are filtered out
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))
op, err := syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, true, false)
require.NoError(t, err)
require.Nil(t, op)
op2, err := syntax.MergeBinOp(tc.op, tc.lhs, tc.rhs, true, true)
require.NoError(t, err)
require.Nil(t, op2)
// for vector-vector comparisons, ensure that nil right hand sides
// translate into nil results
require.Nil(t, syntax.MergeBinOp(tc.op, tc.lhs, nil, true, true))
op3, err := syntax.MergeBinOp(tc.op, tc.lhs, nil, true, true)
require.NoError(t, err)
require.Nil(t, op3)
}
})

@ -269,6 +269,7 @@ type NumericLabelFilter struct {
Name string
Value float64
Type LabelFilterType
err error
}
// NewNumericLabelFilter creates a new label filterer which parses float64 string representation (5.2)

@ -791,9 +791,9 @@ func IsLogicalBinOp(op string) bool {
// SampleExpr is a LogQL expression filtering logs and returning metric samples.
type SampleExpr interface {
// Selector is the LogQL selector to apply when retrieving logs.
Selector() LogSelectorExpr
Selector() (LogSelectorExpr, error)
Extractor() (SampleExtractor, error)
MatcherGroups() []MatcherRange
MatcherGroups() ([]MatcherRange, error)
Expr
}
@ -803,6 +803,7 @@ type RangeAggregationExpr struct {
Params *float64
Grouping *Grouping
err error
implicit
}
@ -810,18 +811,18 @@ func newRangeAggregationExpr(left *LogRange, operation string, gr *Grouping, str
var params *float64
if stringParams != nil {
if operation != OpRangeTypeQuantile {
panic(logqlmodel.NewParseError(fmt.Sprintf("parameter %s not supported for operation %s", *stringParams, operation), 0, 0))
return &RangeAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter %s not supported for operation %s", *stringParams, operation), 0, 0)}
}
var err error
params = new(float64)
*params, err = strconv.ParseFloat(*stringParams, 64)
if err != nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("invalid parameter for operation %s: %s", operation, err), 0, 0))
return &RangeAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("invalid parameter for operation %s: %s", operation, err), 0, 0)}
}
} else {
if operation == OpRangeTypeQuantile {
panic(logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0))
return &RangeAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0)}
}
}
e := &RangeAggregationExpr{
@ -831,16 +832,22 @@ func newRangeAggregationExpr(left *LogRange, operation string, gr *Grouping, str
Params: params,
}
if err := e.validate(); err != nil {
panic(logqlmodel.NewParseError(err.Error(), 0, 0))
return &RangeAggregationExpr{err: logqlmodel.NewParseError(err.Error(), 0, 0)}
}
return e
}
func (e *RangeAggregationExpr) Selector() LogSelectorExpr {
return e.Left.Left
func (e *RangeAggregationExpr) Selector() (LogSelectorExpr, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.Left, nil
}
func (e *RangeAggregationExpr) MatcherGroups() []MatcherRange {
func (e *RangeAggregationExpr) MatcherGroups() ([]MatcherRange, error) {
if e.err != nil {
return nil, e.err
}
xs := e.Left.Left.Matchers()
if len(xs) > 0 {
return []MatcherRange{
@ -849,9 +856,9 @@ func (e *RangeAggregationExpr) MatcherGroups() []MatcherRange {
Interval: e.Left.Interval,
Offset: e.Left.Offset,
},
}
}, nil
}
return nil
return nil, nil
}
func (e RangeAggregationExpr) validate() error {
@ -943,6 +950,7 @@ type VectorAggregationExpr struct {
Grouping *Grouping
Params int
Operation string
err error
implicit
}
@ -952,19 +960,19 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin
switch operation {
case OpTypeBottomK, OpTypeTopK:
if params == nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0))
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("parameter required for operation %s", operation), 0, 0)}
}
p, err = strconv.Atoi(*params)
if err != nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("invalid parameter %s(%s,", operation, *params), 0, 0))
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("invalid parameter %s(%s,", operation, *params), 0, 0)}
}
if p <= 0 {
panic(logqlmodel.NewParseError(fmt.Sprintf("invalid parameter (must be greater than 0) %s(%s", operation, *params), 0, 0))
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("invalid parameter (must be greater than 0) %s(%s", operation, *params), 0, 0)}
}
default:
if params != nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("unsupported parameter for operation %s(%s,", operation, *params), 0, 0))
return &VectorAggregationExpr{err: logqlmodel.NewParseError(fmt.Sprintf("unsupported parameter for operation %s(%s,", operation, *params), 0, 0)}
}
}
if gr == nil {
@ -978,15 +986,24 @@ func mustNewVectorAggregationExpr(left SampleExpr, operation string, gr *Groupin
}
}
func (e *VectorAggregationExpr) MatcherGroups() []MatcherRange {
func (e *VectorAggregationExpr) MatcherGroups() ([]MatcherRange, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.MatcherGroups()
}
func (e *VectorAggregationExpr) Selector() LogSelectorExpr {
func (e *VectorAggregationExpr) Selector() (LogSelectorExpr, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.Selector()
}
func (e *VectorAggregationExpr) Extractor() (log.SampleExtractor, error) {
if e.err != nil {
return nil, e.err
}
// inject in the range vector extractor the outer groups to improve performance.
// This is only possible if the operation is a sum. Anything else needs all labels.
if r, ok := e.Left.(*RangeAggregationExpr); ok && canInjectVectorGrouping(e.Operation, r.Operation) {
@ -1107,10 +1124,22 @@ type BinOpExpr struct {
RHS SampleExpr
Op string
Opts *BinOpOptions
err error
}
func (e *BinOpExpr) MatcherGroups() []MatcherRange {
return append(e.SampleExpr.MatcherGroups(), e.RHS.MatcherGroups()...)
func (e *BinOpExpr) MatcherGroups() ([]MatcherRange, error) {
if e.err != nil {
return nil, e.err
}
groups, err := e.SampleExpr.MatcherGroups()
if err != nil {
return nil, err
}
RHSGroups, err := e.RHS.MatcherGroups()
if err != nil {
return nil, err
}
return append(groups, RHSGroups...), nil
}
func (e *BinOpExpr) String() string {
@ -1158,46 +1187,62 @@ func (e *BinOpExpr) Walk(f WalkFn) {
func mustNewBinOpExpr(op string, opts *BinOpOptions, lhs, rhs Expr) SampleExpr {
left, ok := lhs.(SampleExpr)
if !ok {
panic(logqlmodel.NewParseError(fmt.Sprintf(
return &BinOpExpr{err: logqlmodel.NewParseError(fmt.Sprintf(
"unexpected type for left leg of binary operation (%s): %T",
op,
lhs,
), 0, 0))
), 0, 0)}
}
right, ok := rhs.(SampleExpr)
if !ok {
panic(logqlmodel.NewParseError(fmt.Sprintf(
return &BinOpExpr{err: logqlmodel.NewParseError(fmt.Sprintf(
"unexpected type for right leg of binary operation (%s): %T",
op,
rhs,
), 0, 0))
), 0, 0)}
}
leftLit, lOk := left.(*LiteralExpr)
rightLit, rOk := right.(*LiteralExpr)
var leftVal float64
var rightVal float64
if lOk {
leftV, err := leftLit.Value()
if err != nil {
return &BinOpExpr{err: err}
}
leftVal = leftV
}
if rOk {
rightV, err := rightLit.Value()
if err != nil {
return &BinOpExpr{err: err}
}
rightVal = rightV
}
if IsLogicalBinOp(op) {
if lOk {
panic(logqlmodel.NewParseError(fmt.Sprintf(
return &BinOpExpr{err: logqlmodel.NewParseError(fmt.Sprintf(
"unexpected literal for left leg of logical/set binary operation (%s): %f",
op,
leftLit.Val,
), 0, 0))
leftVal,
), 0, 0)}
}
if rOk {
panic(logqlmodel.NewParseError(fmt.Sprintf(
return &BinOpExpr{err: logqlmodel.NewParseError(fmt.Sprintf(
"unexpected literal for right leg of logical/set binary operation (%s): %f",
op,
rightLit.Val,
), 0, 0))
rightVal,
), 0, 0)}
}
}
// map expr like (1+1) -> 2
if lOk && rOk {
return reduceBinOp(op, leftLit, rightLit)
return reduceBinOp(op, leftVal, rightVal)
}
return &BinOpExpr{
@ -1205,24 +1250,28 @@ func mustNewBinOpExpr(op string, opts *BinOpOptions, lhs, rhs Expr) SampleExpr {
RHS: right,
Op: op,
Opts: opts,
err: nil,
}
}
// Reduces a binary operation expression. A binop is reducible if both of its legs are literal expressions.
// 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(
func reduceBinOp(op string, left, right float64) *LiteralExpr {
merged, err := MergeBinOp(
op,
&promql.Sample{Point: promql.Point{V: left.Val}},
&promql.Sample{Point: promql.Point{V: right.Val}},
&promql.Sample{Point: promql.Point{V: left}},
&promql.Sample{Point: promql.Point{V: right}},
false,
false,
)
if err != nil {
return &LiteralExpr{err: err}
}
return &LiteralExpr{Val: merged.V}
}
func MergeBinOp(op string, left, right *promql.Sample, filter, isVectorComparison bool) *promql.Sample {
func MergeBinOp(op string, left, right *promql.Sample, filter, isVectorComparison bool) (*promql.Sample, error) {
var merger func(left, right *promql.Sample) *promql.Sample
switch op {
@ -1443,12 +1492,12 @@ func MergeBinOp(op string, left, right *promql.Sample, filter, isVectorCompariso
}
default:
panic(errors.Errorf("should never happen: unexpected operation: (%s)", op))
return nil, errors.Errorf("should never happen: unexpected operation: (%s)", op)
}
res := merger(left, right)
if !isVectorComparison {
return res
return res, nil
}
if filter {
@ -1456,21 +1505,22 @@ func MergeBinOp(op string, left, right *promql.Sample, filter, isVectorCompariso
// 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 left, nil
}
}
return res
return res, nil
}
type LiteralExpr struct {
Val float64
err error
implicit
}
func mustNewLiteralExpr(s string, invert bool) *LiteralExpr {
n, err := strconv.ParseFloat(s, 64)
if err != nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("unable to parse literal as a float: %s", err.Error()), 0, 0))
err = logqlmodel.NewParseError(fmt.Sprintf("unable to parse literal as a float: %s", err.Error()), 0, 0)
}
if invert {
@ -1479,6 +1529,7 @@ func mustNewLiteralExpr(s string, invert bool) *LiteralExpr {
return &LiteralExpr{
Val: n,
err: err,
}
}
@ -1489,15 +1540,20 @@ func (e *LiteralExpr) String() string {
// literlExpr impls SampleExpr & LogSelectorExpr mainly to reduce the need for more complicated typings
// to facilitate sum types. We'll be type switching when evaluating them anyways
// and they will only be present in binary operation legs.
func (e *LiteralExpr) Selector() LogSelectorExpr { return e }
func (e *LiteralExpr) Selector() (LogSelectorExpr, error) { return e, e.err }
func (e *LiteralExpr) HasFilter() bool { return false }
func (e *LiteralExpr) Shardable() bool { return true }
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) MatcherGroups() []MatcherRange { return nil }
func (e *LiteralExpr) Extractor() (log.SampleExtractor, error) { return nil, nil }
func (e *LiteralExpr) Value() float64 { return e.Val }
func (e *LiteralExpr) MatcherGroups() ([]MatcherRange, error) { return nil, e.err }
func (e *LiteralExpr) Extractor() (log.SampleExtractor, error) { return nil, e.err }
func (e *LiteralExpr) Value() (float64, error) {
if e.err != nil {
return 0, e.err
}
return e.Val, nil
}
// helper used to impl Stringer for vector and range aggregations
// nolint:interfacer
@ -1527,6 +1583,7 @@ type LabelReplaceExpr struct {
Src string
Regex string
Re *regexp.Regexp
err error
implicit
}
@ -1534,7 +1591,9 @@ type LabelReplaceExpr struct {
func mustNewLabelReplaceExpr(left SampleExpr, dst, replacement, src, regex string) *LabelReplaceExpr {
re, err := regexp.Compile("^(?:" + regex + ")$")
if err != nil {
panic(logqlmodel.NewParseError(fmt.Sprintf("invalid regex in label_replace: %s", err.Error()), 0, 0))
return &LabelReplaceExpr{
err: logqlmodel.NewParseError(fmt.Sprintf("invalid regex in label_replace: %s", err.Error()), 0, 0),
}
}
return &LabelReplaceExpr{
Left: left,
@ -1546,15 +1605,24 @@ func mustNewLabelReplaceExpr(left SampleExpr, dst, replacement, src, regex strin
}
}
func (e *LabelReplaceExpr) Selector() LogSelectorExpr {
func (e *LabelReplaceExpr) Selector() (LogSelectorExpr, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.Selector()
}
func (e *LabelReplaceExpr) MatcherGroups() []MatcherRange {
func (e *LabelReplaceExpr) MatcherGroups() ([]MatcherRange, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.MatcherGroups()
}
func (e *LabelReplaceExpr) Extractor() (SampleExtractor, error) {
if e.err != nil {
return nil, e.err
}
return e.Left.Extractor()
}
@ -1630,7 +1698,7 @@ type MatcherRange struct {
Interval, Offset time.Duration
}
func MatcherGroups(expr Expr) []MatcherRange {
func MatcherGroups(expr Expr) ([]MatcherRange, error) {
switch e := expr.(type) {
case SampleExpr:
return e.MatcherGroups()
@ -1640,11 +1708,11 @@ func MatcherGroups(expr Expr) []MatcherRange {
{
Matchers: xs,
},
}
}, nil
}
return nil
return nil, nil
default:
return nil
return nil, nil
}
}
@ -1685,11 +1753,11 @@ func (e *VectorExpr) Value() (float64, error) {
return e.Val, nil
}
func (e *VectorExpr) Selector() LogSelectorExpr { return e }
func (e *VectorExpr) Selector() (LogSelectorExpr, error) { return e, e.err }
func (e *VectorExpr) HasFilter() bool { return false }
func (e *VectorExpr) Shardable() bool { return true }
func (e *VectorExpr) Walk(f WalkFn) { f(e) }
func (e *VectorExpr) Pipeline() (log.Pipeline, error) { return log.NewNoopPipeline(), nil }
func (e *VectorExpr) Matchers() []*labels.Matcher { return nil }
func (e *VectorExpr) MatcherGroups() []MatcherRange { return nil }
func (e *VectorExpr) MatcherGroups() ([]MatcherRange, error) { return nil, e.err }
func (e *VectorExpr) Extractor() (log.SampleExtractor, error) { return nil, nil }

@ -249,7 +249,8 @@ func TestMatcherGroups(t *testing.T) {
t.Run(fmt.Sprint(i), func(t *testing.T) {
expr, err := ParseExpr(tc.query)
require.Nil(t, err)
out := MatcherGroups(expr)
out, err := MatcherGroups(expr)
require.Nil(t, err)
require.Equal(t, tc.exp, out)
})
}
@ -595,7 +596,7 @@ func Test_canInjectVectorGrouping(t *testing.T) {
}
func Test_MergeBinOpVectors_Filter(t *testing.T) {
res := MergeBinOp(
res, err := MergeBinOp(
OpTypeGT,
&promql.Sample{
Point: promql.Point{V: 2},
@ -606,6 +607,7 @@ func Test_MergeBinOpVectors_Filter(t *testing.T) {
true,
true,
)
require.NoError(t, err)
// ensure we return the left hand side's value (2) instead of the
// comparison operator's result (1: the truthy answer)

@ -15,6 +15,9 @@ func (r RangeAggregationExpr) Extractor() (log.SampleExtractor, error) {
// extractor creates a SampleExtractor but allows for the grouping to be overridden.
func (r RangeAggregationExpr) extractor(override *Grouping) (log.SampleExtractor, error) {
if r.err != nil {
return nil, r.err
}
if err := r.validate(); err != nil {
return nil, err
}

@ -162,14 +162,28 @@ func ParseSampleExpr(input string) (SampleExpr, error) {
func validateSampleExpr(expr SampleExpr) error {
switch e := expr.(type) {
case *BinOpExpr:
if e.err != nil {
return e.err
}
if err := validateSampleExpr(e.SampleExpr); err != nil {
return err
}
return validateSampleExpr(e.RHS)
case *LiteralExpr, *VectorExpr:
case *LiteralExpr:
if e.err != nil {
return e.err
}
return nil
case *VectorExpr:
if e.err != nil {
return e.err
}
return nil
case *VectorAggregationExpr:
if e.err != nil {
return e.err
}
if e.Operation == OpTypeSort || e.Operation == OpTypeSortDesc {
if err := validateSortGrouping(e.Grouping); err != nil {
return err
@ -177,7 +191,11 @@ func validateSampleExpr(expr SampleExpr) error {
}
return validateSampleExpr(e.Left)
default:
return validateMatchers(expr.Selector().Matchers())
selector, err := e.Selector()
if err != nil {
return err
}
return validateMatchers(selector.Matchers())
}
}

@ -192,7 +192,11 @@ func removeTenantSelector(params logql.SelectSampleParams, tenantIDs []string) (
if err != nil {
return nil, nil, err
}
matchedTenants, filteredMatchers := filterValuesByMatchers(defaultTenantLabel, tenantIDs, expr.Selector().Matchers()...)
selector, err := expr.Selector()
if err != nil {
return nil, nil, err
}
matchedTenants, filteredMatchers := filterValuesByMatchers(defaultTenantLabel, tenantIDs, selector.Matchers()...)
updatedExpr := replaceMatchers(expr, filteredMatchers)
return matchedTenants, updatedExpr, nil
}

@ -64,7 +64,10 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, error) {
// We try to shard subtrees in the AST independently if possible, although
// nested binary expressions can make this difficult. In this case,
// we query the index stats for all matcher groups then sum the results.
grps := syntax.MatcherGroups(e)
grps, err := syntax.MatcherGroups(e)
if err != nil {
return 0, err
}
// If there are zero matchers groups, we'll inject one to query everything
if len(grps) == 0 {

Loading…
Cancel
Save