diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 83aa48df46..651730f2dd 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.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)) diff --git a/pkg/logql/engine.go b/pkg/logql/engine.go index fdd5e7f51b..d593480733 100644 --- a/pkg/logql/engine.go +++ b/pkg/logql/engine.go @@ -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 { diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index aa22cf534c..d26b2fcb11 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -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 } diff --git a/pkg/logql/evaluator_test.go b/pkg/logql/evaluator_test.go index 987783a6ad..9f5279c0dd 100644 --- a/pkg/logql/evaluator_test.go +++ b/pkg/logql/evaluator_test.go @@ -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) } }) diff --git a/pkg/logql/log/label_filter.go b/pkg/logql/log/label_filter.go index b0a86baeb2..f769aae728 100644 --- a/pkg/logql/log/label_filter.go +++ b/pkg/logql/log/label_filter.go @@ -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) diff --git a/pkg/logql/syntax/ast.go b/pkg/logql/syntax/ast.go index f9dc906b20..151f92266a 100644 --- a/pkg/logql/syntax/ast.go +++ b/pkg/logql/syntax/ast.go @@ -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 } diff --git a/pkg/logql/syntax/ast_test.go b/pkg/logql/syntax/ast_test.go index bbefc5d922..b7c3f016f8 100644 --- a/pkg/logql/syntax/ast_test.go +++ b/pkg/logql/syntax/ast_test.go @@ -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) diff --git a/pkg/logql/syntax/extractor.go b/pkg/logql/syntax/extractor.go index 34674ccfaf..3b6473a346 100644 --- a/pkg/logql/syntax/extractor.go +++ b/pkg/logql/syntax/extractor.go @@ -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 } diff --git a/pkg/logql/syntax/parser.go b/pkg/logql/syntax/parser.go index 844b28673b..35021452b9 100644 --- a/pkg/logql/syntax/parser.go +++ b/pkg/logql/syntax/parser.go @@ -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()) } } diff --git a/pkg/querier/multi_tenant_querier.go b/pkg/querier/multi_tenant_querier.go index d525541370..687accc823 100644 --- a/pkg/querier/multi_tenant_querier.go +++ b/pkg/querier/multi_tenant_querier.go @@ -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 } diff --git a/pkg/querier/queryrange/shard_resolver.go b/pkg/querier/queryrange/shard_resolver.go index a02bf6e26d..245d153732 100644 --- a/pkg/querier/queryrange/shard_resolver.go +++ b/pkg/querier/queryrange/shard_resolver.go @@ -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 {