mirror of
https://github.com/prometheus/prometheus
synced 2025-03-19 01:44:47 +00:00
feat(promql): offset from static time expression
Related to #12318 . Allow using a PromQL expression for setting the offset, provided that: the expression evaluates to a scalar the expression does not contain vector selector (no TSDB access) the expression does not call the info() function (no TSDB access) the time() function is only called on the top level of the expression, not inside a subquery We'll call such expression a time expression. During parsing we no longer set OriginalOffset field in sub-queries and vector selectors, instead we set a new OriginalOffsetExpr field to the time expression, even if the expression is a literal number. Before evaluating the overall PromQL expression, the engine shall evaluate the time expressions and set the OriginalOffset field in vector selectors and subqueries to the calculated value. This makes the change fairly non intrusive. Later we can change the code to start using the expression more directly. Note: this prohibits using time() function inside a sub-query as the evaluation time is not constant inside a sub-query. Signed-off-by: György Krajcsovits <gyorgy.krajcsovits@grafana.com>
This commit is contained in:
parent
8be416a67c
commit
2f144dd3ad
210
promql/engine.go
210
promql/engine.go
@ -100,6 +100,9 @@ type (
|
||||
// ErrStorage is returned if an error was encountered in the storage layer
|
||||
// during query handling.
|
||||
ErrStorage struct{ Err error }
|
||||
// ErrTimeExprNotAFloat is returned if a time expression evaluates to something
|
||||
// other than a single float value.
|
||||
ErrTimeExprNotAFloat string
|
||||
)
|
||||
|
||||
func (e ErrQueryTimeout) Error() string {
|
||||
@ -118,6 +121,20 @@ func (e ErrStorage) Error() string {
|
||||
return e.Err.Error()
|
||||
}
|
||||
|
||||
func (e ErrTimeExprNotAFloat) Error() string {
|
||||
return fmt.Sprintf("time expression evaluates to %s not one float", string(e))
|
||||
}
|
||||
|
||||
var (
|
||||
// ErrTimeExprUsesStorage is returned if a time expression
|
||||
// (timestamp/duration) calculation includes vector/range selector.
|
||||
ErrTimeExprUsesStorage = errors.New("time expression must not use storage")
|
||||
// ErrTimeExprDependsOnEvalTime is returned if a time expression
|
||||
// depends on the evaluation time, which is only allowed for the top level
|
||||
// time expressions.
|
||||
ErrTimeExprDependsOnEvalTime = errors.New("inner time expression must not depend on the evaluation time")
|
||||
)
|
||||
|
||||
// QueryEngine defines the interface for the *promql.Engine, so it can be replaced, wrapped or mocked.
|
||||
type QueryEngine interface {
|
||||
NewInstantQuery(ctx context.Context, q storage.Queryable, opts QueryOpts, qs string, ts time.Time) (Query, error)
|
||||
@ -486,6 +503,11 @@ func (ng *Engine) NewInstantQuery(ctx context.Context, q storage.Queryable, opts
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := ng.PreprocessTimeExpr(expr, ts, ts, 0, opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := ng.validateOpts(expr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -507,6 +529,11 @@ func (ng *Engine) NewRangeQuery(ctx context.Context, q storage.Queryable, opts Q
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := ng.PreprocessTimeExpr(expr, start, end, interval, opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := ng.validateOpts(expr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -518,6 +545,117 @@ func (ng *Engine) NewRangeQuery(ctx context.Context, q storage.Queryable, opts Q
|
||||
return qry, nil
|
||||
}
|
||||
|
||||
type timeExprVisitor struct {
|
||||
err *error
|
||||
ev *evaluator
|
||||
inSubquery bool
|
||||
}
|
||||
|
||||
func (tev timeExprVisitor) Visit(node parser.Node, path []parser.Node) (parser.Visitor, error) {
|
||||
checkTimeExpr := func(expr parser.Expr) (float64, error) {
|
||||
if expr == nil {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
hasStorageDependency, hasEvalTimeDependency := inspectTimeExpr(expr)
|
||||
if hasStorageDependency {
|
||||
return 0, ErrTimeExprUsesStorage
|
||||
}
|
||||
if tev.inSubquery && hasEvalTimeDependency {
|
||||
return 0, ErrTimeExprDependsOnEvalTime
|
||||
}
|
||||
if nl, ok := expr.(*parser.NumberLiteral); ok {
|
||||
return nl.Val, nil
|
||||
}
|
||||
val, err := tev.ev.timeValueOf(expr)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
// Return the precalculated value.
|
||||
return val, nil
|
||||
}
|
||||
|
||||
switch e := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
v, err := checkTimeExpr(e.OriginalOffsetExpr)
|
||||
if err != nil {
|
||||
*tev.err = err
|
||||
return tev, err
|
||||
}
|
||||
e.OriginalOffset = secondsValueToDuration(v)
|
||||
return tev, nil
|
||||
case *parser.SubqueryExpr:
|
||||
v, err := checkTimeExpr(e.OriginalOffsetExpr)
|
||||
if err != nil {
|
||||
*tev.err = err
|
||||
return tev, err
|
||||
}
|
||||
e.OriginalOffset = secondsValueToDuration(v)
|
||||
return timeExprVisitor{err: tev.err, inSubquery: true}, nil
|
||||
default:
|
||||
return tev, nil
|
||||
}
|
||||
}
|
||||
|
||||
func inspectTimeExpr(e parser.Expr) (bool, bool) {
|
||||
var hasStorageDependency, hasEvalTimeDependency bool
|
||||
parser.Inspect(e, func(node parser.Node, _ []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.Call:
|
||||
if n.Func.EvalTimeDependent {
|
||||
hasEvalTimeDependency = true
|
||||
}
|
||||
if n.Func.StorageDependent {
|
||||
hasStorageDependency = true
|
||||
}
|
||||
case *parser.VectorSelector:
|
||||
hasStorageDependency = true
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return hasStorageDependency, hasEvalTimeDependency
|
||||
}
|
||||
|
||||
// PreprocessTimeExpr checks the time expressions don't try to load data from
|
||||
// storage. If the time expression is eval time independent, it will calculate
|
||||
// its value and replace with a number literal to avoid re-calculations later.
|
||||
func (ng *Engine) PreprocessTimeExpr(e parser.Expr, start, end time.Time, interval time.Duration, opts QueryOpts) error {
|
||||
if opts == nil {
|
||||
opts = NewPrometheusQueryOpts(false, 0)
|
||||
}
|
||||
|
||||
lookbackDelta := opts.LookbackDelta()
|
||||
if lookbackDelta <= 0 {
|
||||
lookbackDelta = ng.lookbackDelta
|
||||
}
|
||||
|
||||
// Make an evaulator on the heap for the time expressions.
|
||||
// We only allow time() function on the top level, so using
|
||||
// "end" timestamp as evaluation time.
|
||||
ev := evaluator{
|
||||
startTimestamp: timeMilliseconds(end),
|
||||
endTimestamp: timeMilliseconds(end),
|
||||
interval: 1,
|
||||
maxSamples: ng.maxSamplesPerQuery,
|
||||
logger: ng.logger,
|
||||
lookbackDelta: lookbackDelta,
|
||||
samplesStats: nil,
|
||||
noStepSubqueryIntervalFn: ng.noStepSubqueryIntervalFn,
|
||||
enableDelayedNameRemoval: ng.enableDelayedNameRemoval,
|
||||
querier: nil, // Fail if we do try to use querier.
|
||||
}
|
||||
|
||||
var err error
|
||||
tev := timeExprVisitor{
|
||||
err: &err,
|
||||
ev: &ev,
|
||||
}
|
||||
|
||||
parser.Walk(tev, e, nil) //nolint:errcheck
|
||||
|
||||
return *tev.err
|
||||
}
|
||||
|
||||
func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start, end time.Time, interval time.Duration) (*parser.Expr, *query) {
|
||||
if opts == nil {
|
||||
opts = NewPrometheusQueryOpts(false, 0)
|
||||
@ -3844,3 +3982,75 @@ func (ev *evaluator) gatherVector(ts int64, input Matrix, output Vector, bufHelp
|
||||
|
||||
return output, bufHelpers
|
||||
}
|
||||
|
||||
// timeValueOf calculates the scalar time value of an expression in float64.
|
||||
func (ev *evaluator) timeValueOf(e parser.Expr) (float64, error) {
|
||||
if e == nil {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
if ev.querier != nil {
|
||||
panic("evaulator does not disallow querier for time expressions")
|
||||
}
|
||||
|
||||
// Shortcut for number literals.
|
||||
if nl, ok := e.(*parser.NumberLiteral); ok {
|
||||
return nl.Val, nil
|
||||
}
|
||||
|
||||
// We can only use scalars as timestamp or duration.
|
||||
if e.Type() != parser.ValueTypeScalar {
|
||||
panic("parser should have discarded non scalar time expression")
|
||||
}
|
||||
|
||||
// TODO(krajorama): would we ever get annotations here?
|
||||
res, _, err := ev.Eval(context.Background(), e)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
var value float64
|
||||
|
||||
switch v := res.(type) {
|
||||
case Scalar:
|
||||
value = v.V
|
||||
case String:
|
||||
return 0, ErrTimeExprNotAFloat("string")
|
||||
case Vector:
|
||||
if len(v) > 0 {
|
||||
return 0, ErrTimeExprNotAFloat("vector with multiple samples")
|
||||
}
|
||||
if len(v) == 0 {
|
||||
return 0, ErrTimeExprNotAFloat("vector with no samples")
|
||||
}
|
||||
if v[0].H != nil {
|
||||
return 0, ErrTimeExprNotAFloat("vector with a histogram element")
|
||||
}
|
||||
return v[0].F, nil
|
||||
case Matrix:
|
||||
if len(v) > 1 {
|
||||
return 0, ErrTimeExprNotAFloat("matrix with more than one series")
|
||||
}
|
||||
if len(v) == 0 {
|
||||
return 0, ErrTimeExprNotAFloat("matrix with no series")
|
||||
}
|
||||
s := v[0]
|
||||
if len(s.Histograms) > 0 {
|
||||
return 0, ErrTimeExprNotAFloat("matrix with histograms")
|
||||
}
|
||||
if len(s.Floats) > 1 {
|
||||
return 0, ErrTimeExprNotAFloat("matrix with more than one element")
|
||||
}
|
||||
if len(s.Floats) == 0 {
|
||||
return 0, ErrTimeExprNotAFloat("matrix with no element")
|
||||
}
|
||||
return s.Floats[0].F, nil
|
||||
default:
|
||||
return 0, ErrTimeExprNotAFloat("unknown type")
|
||||
}
|
||||
return value, nil
|
||||
}
|
||||
|
||||
func secondsValueToDuration(v float64) time.Duration {
|
||||
return time.Duration(v*1000.0) * time.Millisecond
|
||||
}
|
||||
|
@ -2739,8 +2739,8 @@ func TestPreprocessAndWrapWithStepInvariantExpr(t *testing.T) {
|
||||
Start: 0,
|
||||
End: 27,
|
||||
},
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffsetExpr: &parser.NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
},
|
||||
},
|
||||
Range: 10 * time.Minute,
|
||||
@ -2762,11 +2762,11 @@ func TestPreprocessAndWrapWithStepInvariantExpr(t *testing.T) {
|
||||
End: 11,
|
||||
},
|
||||
},
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
EndPos: 35,
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffsetExpr: &parser.NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
EndPos: 35,
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -133,8 +133,10 @@ type MatrixSelector struct {
|
||||
type SubqueryExpr struct {
|
||||
Expr Expr
|
||||
Range time.Duration
|
||||
// OriginalOffsetExpr is the parsed expression for OriginalOffset
|
||||
OriginalOffsetExpr Expr
|
||||
// OriginalOffset is the actual offset that was set in the query.
|
||||
// This never changes.
|
||||
// Precalculated from the OriginalOffsetExpr and then never changes.
|
||||
OriginalOffset time.Duration
|
||||
// Offset is the offset used during the query execution
|
||||
// which is calculated using the original offset, at modifier time,
|
||||
@ -192,8 +194,10 @@ func (e *StepInvariantExpr) PositionRange() posrange.PositionRange {
|
||||
// VectorSelector represents a Vector selection.
|
||||
type VectorSelector struct {
|
||||
Name string
|
||||
// OriginalOffsetExpr is the parsed expression for OriginalOffset
|
||||
OriginalOffsetExpr Expr
|
||||
// OriginalOffset is the actual offset that was set in the query.
|
||||
// This never changes.
|
||||
// Precalculated from the OriginalOffsetExpr and then never changes.
|
||||
OriginalOffset time.Duration
|
||||
// Offset is the offset used during the query execution
|
||||
// which is calculated using the original offset, at modifier time,
|
||||
|
@ -21,6 +21,14 @@ type Function struct {
|
||||
Variadic int
|
||||
ReturnType ValueType
|
||||
Experimental bool
|
||||
|
||||
// EvalTimeDependent means that the output of the function depends on what
|
||||
// the current evaluation timestamp is. For example time().
|
||||
EvalTimeDependent bool
|
||||
|
||||
// StorageDependent means that the out of the function depends on the
|
||||
// content of TSDB at the time. For example info().
|
||||
StorageDependent bool
|
||||
}
|
||||
|
||||
// EnableExperimentalFunctions controls whether experimentalFunctions are enabled.
|
||||
@ -225,11 +233,12 @@ var Functions = map[string]*Function{
|
||||
ReturnType: ValueTypeVector,
|
||||
},
|
||||
"info": {
|
||||
Name: "info",
|
||||
ArgTypes: []ValueType{ValueTypeVector, ValueTypeVector},
|
||||
ReturnType: ValueTypeVector,
|
||||
Experimental: true,
|
||||
Variadic: 1,
|
||||
Name: "info",
|
||||
ArgTypes: []ValueType{ValueTypeVector, ValueTypeVector},
|
||||
ReturnType: ValueTypeVector,
|
||||
Experimental: true,
|
||||
Variadic: 1,
|
||||
StorageDependent: true,
|
||||
},
|
||||
"irate": {
|
||||
Name: "irate",
|
||||
@ -411,9 +420,10 @@ var Functions = map[string]*Function{
|
||||
ReturnType: ValueTypeVector,
|
||||
},
|
||||
"time": {
|
||||
Name: "time",
|
||||
ArgTypes: []ValueType{},
|
||||
ReturnType: ValueTypeScalar,
|
||||
Name: "time",
|
||||
ArgTypes: []ValueType{},
|
||||
ReturnType: ValueTypeScalar,
|
||||
EvalTimeDependent: true,
|
||||
},
|
||||
"timestamp": {
|
||||
Name: "timestamp",
|
||||
|
@ -436,19 +436,27 @@ paren_expr : LEFT_PAREN expr RIGHT_PAREN
|
||||
offset_expr: expr OFFSET number_duration_literal
|
||||
{
|
||||
numLit, _ := $3.(*NumberLiteral)
|
||||
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
|
||||
dur := &NumberLiteral{Val: numLit.Val}
|
||||
yylex.(*parser).addOffset($1, dur)
|
||||
$$ = $1
|
||||
}
|
||||
| expr OFFSET SUB number_duration_literal
|
||||
{
|
||||
numLit, _ := $4.(*NumberLiteral)
|
||||
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
|
||||
yylex.(*parser).addOffset($1, -dur)
|
||||
dur := &NumberLiteral{Val: numLit.Val * -1.0}
|
||||
yylex.(*parser).addOffset($1, dur)
|
||||
$$ = $1
|
||||
}
|
||||
| expr OFFSET paren_expr
|
||||
{
|
||||
dur, _ := $3.(*ParenExpr)
|
||||
if dur.Type() != ValueTypeScalar {
|
||||
yylex.(*parser).unexpected("offset", "time expression does not evaluate to a scalar")
|
||||
}
|
||||
yylex.(*parser).addOffset($1, dur)
|
||||
}
|
||||
| expr OFFSET error
|
||||
{ yylex.(*parser).unexpected("offset", "number or duration"); $$ = $1 }
|
||||
{ yylex.(*parser).unexpected("offset", "number or duration or time expression in parentheses"); $$ = $1 }
|
||||
;
|
||||
/*
|
||||
* @ modifiers.
|
||||
@ -480,7 +488,7 @@ matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET
|
||||
vs, ok := $1.(*VectorSelector)
|
||||
if !ok{
|
||||
errMsg = "ranges only allowed for vector selectors"
|
||||
} else if vs.OriginalOffset != 0{
|
||||
} else if vs.OriginalOffsetExpr != nil {
|
||||
errMsg = "no offset modifiers allowed before range"
|
||||
} else if vs.Timestamp != nil {
|
||||
errMsg = "no @ modifiers allowed before range"
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -937,13 +937,13 @@ func (p *parser) newMetricNameMatcher(value Item) *labels.Matcher {
|
||||
}
|
||||
|
||||
// addOffset is used to set the offset in the generated parser.
|
||||
func (p *parser) addOffset(e Node, offset time.Duration) {
|
||||
var orgoffsetp *time.Duration
|
||||
func (p *parser) addOffset(e Node, offset Expr) {
|
||||
var orgoffsetp *Expr
|
||||
var endPosp *posrange.Pos
|
||||
|
||||
switch s := e.(type) {
|
||||
case *VectorSelector:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetp = &s.OriginalOffsetExpr
|
||||
endPosp = &s.PosRange.End
|
||||
case *MatrixSelector:
|
||||
vs, ok := s.VectorSelector.(*VectorSelector)
|
||||
@ -951,10 +951,10 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
|
||||
p.addParseErrf(e.PositionRange(), "ranges only allowed for vector selectors")
|
||||
return
|
||||
}
|
||||
orgoffsetp = &vs.OriginalOffset
|
||||
orgoffsetp = &vs.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
case *SubqueryExpr:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetp = &s.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
default:
|
||||
p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant vector selector or range vector selector or a subquery")
|
||||
@ -963,7 +963,7 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
|
||||
|
||||
// it is already ensured by parseDuration func that there never will be a zero offset modifier
|
||||
switch {
|
||||
case *orgoffsetp != 0:
|
||||
case *orgoffsetp != nil:
|
||||
p.addParseErrf(e.PositionRange(), "offset may not be set multiple times")
|
||||
case orgoffsetp != nil:
|
||||
*orgoffsetp = offset
|
||||
|
@ -1472,8 +1472,8 @@ var testExpr = []struct {
|
||||
{
|
||||
input: "foo offset 5m",
|
||||
expected: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffset: 5 * time.Minute,
|
||||
Name: "foo",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(5 * time.Minute / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
@ -1486,8 +1486,8 @@ var testExpr = []struct {
|
||||
{
|
||||
input: "foo offset -7m",
|
||||
expected: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffset: -7 * time.Minute,
|
||||
Name: "foo",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(-7 * time.Minute / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
@ -1500,8 +1500,8 @@ var testExpr = []struct {
|
||||
{
|
||||
input: `foo OFFSET 1h30m`,
|
||||
expected: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffset: 90 * time.Minute,
|
||||
Name: "foo",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(90 * time.Minute / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
@ -1514,8 +1514,8 @@ var testExpr = []struct {
|
||||
{
|
||||
input: `foo OFFSET 1m30ms`,
|
||||
expected: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffset: time.Minute + 30*time.Millisecond,
|
||||
Name: "foo",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: 60.03},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
@ -1525,6 +1525,60 @@ var testExpr = []struct {
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
// 0 1 2
|
||||
// 123456789012345678901234567890
|
||||
input: `foo offset (scalar(vector(1)))`,
|
||||
expected: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffsetExpr: &ParenExpr{
|
||||
Expr: &Call{
|
||||
Func: &Function{
|
||||
Name: "scalar",
|
||||
ArgTypes: []ValueType{ValueTypeVector},
|
||||
ReturnType: ValueTypeScalar,
|
||||
},
|
||||
Args: Expressions{
|
||||
&Call{
|
||||
Func: &Function{
|
||||
Name: "vector",
|
||||
ArgTypes: []ValueType{ValueTypeScalar},
|
||||
ReturnType: ValueTypeVector,
|
||||
},
|
||||
Args: Expressions{
|
||||
&NumberLiteral{
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 26,
|
||||
End: 27,
|
||||
},
|
||||
Val: 1.0,
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 19,
|
||||
End: 28,
|
||||
},
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 12,
|
||||
End: 29,
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 11,
|
||||
End: 30,
|
||||
},
|
||||
},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 30,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo @ 1603774568`,
|
||||
expected: &VectorSelector{
|
||||
@ -2064,8 +2118,8 @@ var testExpr = []struct {
|
||||
input: "test[5h] OFFSET 5m",
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "test",
|
||||
OriginalOffset: 5 * time.Minute,
|
||||
Name: "test",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(5 * time.Minute / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"),
|
||||
},
|
||||
@ -2082,8 +2136,8 @@ var testExpr = []struct {
|
||||
input: "test[5d] OFFSET 10s",
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "test",
|
||||
OriginalOffset: 10 * time.Second,
|
||||
Name: "test",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: 10},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"),
|
||||
},
|
||||
@ -2100,8 +2154,8 @@ var testExpr = []struct {
|
||||
input: "test[5w] offset 2w",
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "test",
|
||||
OriginalOffset: 14 * 24 * time.Hour,
|
||||
Name: "test",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(14 * 24 * time.Hour / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"),
|
||||
},
|
||||
@ -2118,8 +2172,8 @@ var testExpr = []struct {
|
||||
input: `test{a="b"}[5y] OFFSET 3d`,
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "test",
|
||||
OriginalOffset: 3 * 24 * time.Hour,
|
||||
Name: "test",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(3 * 24 * time.Hour / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, "a", "b"),
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"),
|
||||
@ -2137,8 +2191,8 @@ var testExpr = []struct {
|
||||
input: `test{a="b"}[5m] OFFSET 3600`,
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "test",
|
||||
OriginalOffset: 1 * time.Hour,
|
||||
Name: "test",
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(1 * time.Hour / time.Second)},
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, "a", "b"),
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"),
|
||||
@ -2355,6 +2409,11 @@ var testExpr = []struct {
|
||||
fail: true,
|
||||
errMsg: "unexpected end of input in offset, expected number or duration",
|
||||
},
|
||||
{
|
||||
input: `some_metric[5m] OFFSET (vector(1))`,
|
||||
fail: true,
|
||||
errMsg: "time expression does not evaluate to a scalar",
|
||||
},
|
||||
{
|
||||
input: `some_metric OFFSET 1m[5m]`,
|
||||
fail: true,
|
||||
@ -3267,9 +3326,9 @@ var testExpr = []struct {
|
||||
End: 38,
|
||||
},
|
||||
},
|
||||
Range: 5 * time.Minute,
|
||||
OriginalOffset: 4 * time.Minute,
|
||||
EndPos: 53,
|
||||
Range: 5 * time.Minute,
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(4 * time.Minute / time.Second)},
|
||||
EndPos: 53,
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
@ -3413,7 +3472,7 @@ var testExpr = []struct {
|
||||
Start: 0,
|
||||
End: 21,
|
||||
},
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
},
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
@ -3451,8 +3510,8 @@ var testExpr = []struct {
|
||||
Start: 0,
|
||||
End: 27,
|
||||
},
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
},
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
@ -3471,8 +3530,8 @@ var testExpr = []struct {
|
||||
Start: 0,
|
||||
End: 27,
|
||||
},
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
},
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
@ -3492,11 +3551,11 @@ var testExpr = []struct {
|
||||
End: 11,
|
||||
},
|
||||
},
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffset: 1 * time.Minute,
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
EndPos: 35,
|
||||
Timestamp: makeInt64Pointer(123000),
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(1 * time.Minute / time.Second)},
|
||||
Range: 10 * time.Minute,
|
||||
Step: 5 * time.Second,
|
||||
EndPos: 35,
|
||||
},
|
||||
},
|
||||
{
|
||||
@ -3575,9 +3634,9 @@ var testExpr = []struct {
|
||||
End: 21,
|
||||
},
|
||||
},
|
||||
Range: 5 * time.Minute,
|
||||
OriginalOffset: 10 * time.Minute,
|
||||
EndPos: 37,
|
||||
Range: 5 * time.Minute,
|
||||
OriginalOffsetExpr: &NumberLiteral{Val: float64(10 * time.Minute / time.Second)},
|
||||
EndPos: 37,
|
||||
},
|
||||
},
|
||||
{
|
||||
@ -3886,8 +3945,7 @@ var testExpr = []struct {
|
||||
Args: Expressions{
|
||||
&MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "http_request_counter_total",
|
||||
OriginalOffset: 0,
|
||||
Name: "http_request_counter_total",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "http_request_counter_total"),
|
||||
},
|
||||
@ -4006,6 +4064,7 @@ func TestParseExpressions(t *testing.T) {
|
||||
|
||||
require.Equal(t, expected, expr, "error on input '%s'", test.input)
|
||||
} else {
|
||||
require.Error(t, err, "expected an error on input '%s', got nil", test.input)
|
||||
require.ErrorContains(t, err, test.errMsg, "unexpected error on input '%s', expected '%s', got '%s'", test.input, test.errMsg, err.Error())
|
||||
|
||||
var errorList ParseErrors
|
||||
|
@ -158,11 +158,8 @@ func (node *MatrixSelector) atOffset() (string, string) {
|
||||
// Copy the Vector selector before changing the offset
|
||||
vecSelector := node.VectorSelector.(*VectorSelector)
|
||||
offset := ""
|
||||
switch {
|
||||
case vecSelector.OriginalOffset > time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset %s", model.Duration(vecSelector.OriginalOffset))
|
||||
case vecSelector.OriginalOffset < time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset -%s", model.Duration(-vecSelector.OriginalOffset))
|
||||
if vecSelector.OriginalOffsetExpr != nil {
|
||||
offset = fmt.Sprintf(" offset %s", timeExprString(vecSelector.OriginalOffsetExpr))
|
||||
}
|
||||
at := ""
|
||||
switch {
|
||||
@ -181,14 +178,14 @@ func (node *MatrixSelector) String() string {
|
||||
// Copy the Vector selector before changing the offset
|
||||
vecSelector := *node.VectorSelector.(*VectorSelector)
|
||||
// Do not print the @ and offset twice.
|
||||
offsetVal, atVal, preproc := vecSelector.OriginalOffset, vecSelector.Timestamp, vecSelector.StartOrEnd
|
||||
vecSelector.OriginalOffset = 0
|
||||
offsetExpr, atVal, preproc := vecSelector.OriginalOffsetExpr, vecSelector.Timestamp, vecSelector.StartOrEnd
|
||||
vecSelector.OriginalOffsetExpr = nil
|
||||
vecSelector.Timestamp = nil
|
||||
vecSelector.StartOrEnd = 0
|
||||
|
||||
str := fmt.Sprintf("%s[%s]%s%s", vecSelector.String(), model.Duration(node.Range), at, offset)
|
||||
|
||||
vecSelector.OriginalOffset, vecSelector.Timestamp, vecSelector.StartOrEnd = offsetVal, atVal, preproc
|
||||
vecSelector.OriginalOffsetExpr, vecSelector.Timestamp, vecSelector.StartOrEnd = offsetExpr, atVal, preproc
|
||||
|
||||
return str
|
||||
}
|
||||
@ -213,11 +210,8 @@ func (node *SubqueryExpr) getSubqueryTimeSuffix() string {
|
||||
step = model.Duration(node.Step).String()
|
||||
}
|
||||
offset := ""
|
||||
switch {
|
||||
case node.OriginalOffset > time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset))
|
||||
case node.OriginalOffset < time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset -%s", model.Duration(-node.OriginalOffset))
|
||||
if node.OriginalOffsetExpr != nil {
|
||||
offset = fmt.Sprintf(" offset %s", timeExprString(node.OriginalOffsetExpr))
|
||||
}
|
||||
at := ""
|
||||
switch {
|
||||
@ -264,11 +258,8 @@ func (node *VectorSelector) String() string {
|
||||
labelStrings = append(labelStrings, matcher.String())
|
||||
}
|
||||
offset := ""
|
||||
switch {
|
||||
case node.OriginalOffset > time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset))
|
||||
case node.OriginalOffset < time.Duration(0):
|
||||
offset = fmt.Sprintf(" offset -%s", model.Duration(-node.OriginalOffset))
|
||||
if node.OriginalOffsetExpr != nil {
|
||||
offset = fmt.Sprintf(" offset %s", timeExprString(node.OriginalOffsetExpr))
|
||||
}
|
||||
at := ""
|
||||
switch {
|
||||
@ -286,3 +277,15 @@ func (node *VectorSelector) String() string {
|
||||
sort.Strings(labelStrings)
|
||||
return fmt.Sprintf("%s{%s}%s%s", node.Name, strings.Join(labelStrings, ","), at, offset)
|
||||
}
|
||||
|
||||
func timeExprString(e Expr) string {
|
||||
if nl, ok := e.(*NumberLiteral); ok {
|
||||
v, sign := nl.Val, ""
|
||||
if nl.Val < 0 {
|
||||
v = -1 * v
|
||||
sign = "-"
|
||||
}
|
||||
return fmt.Sprintf("%s%s", sign, model.Duration(time.Duration(v)*time.Second))
|
||||
}
|
||||
return e.String()
|
||||
}
|
||||
|
@ -112,6 +112,11 @@ func TestExprString(t *testing.T) {
|
||||
{
|
||||
in: `a[1h:5m] offset 1m`,
|
||||
},
|
||||
{
|
||||
in: `a[12m] offset (1m + 1m)`,
|
||||
// TODO(krajorama): maybe handle time expressions special for numbers?
|
||||
out: `a[12m] offset (60 + 60)`,
|
||||
},
|
||||
{
|
||||
in: `{__name__="a"}`,
|
||||
},
|
||||
|
36
promql/promqltest/testdata/duration_arithmetic.test
vendored
Normal file
36
promql/promqltest/testdata/duration_arithmetic.test
vendored
Normal file
@ -0,0 +1,36 @@
|
||||
# This file test requirements from
|
||||
# https://github.com/prometheus/prometheus/issues/12318
|
||||
# We only supply float samples in the inputs and assume that complex type behave
|
||||
# the same.
|
||||
|
||||
load 1m
|
||||
metric 0+1x60
|
||||
|
||||
# offset modifier.
|
||||
#
|
||||
# Regression test offset modifier has higher precedence.
|
||||
# Value at 10m-5m and then plus 1.
|
||||
eval instant at 10m metric offset 5m + 1
|
||||
{} 6
|
||||
|
||||
# Regression test offset modifier has higher precedence.
|
||||
# Value at 10m-5m and then plus 60(s).
|
||||
eval instant at 10m metric offset 5m + 1m
|
||||
{} 65
|
||||
|
||||
# Value at 10m-(5m+1s) == 4m59s.
|
||||
eval instant at 10m metric offset (5m + 1)
|
||||
metric{} 4
|
||||
|
||||
# Value at 10m-(5m-120s) == 7m.
|
||||
eval instant at 10m metric offset (5m - 120)
|
||||
metric{} 7
|
||||
|
||||
# Value at 10m-(5m+1m) == 4m.
|
||||
eval instant at 10m metric offset (5m + 1m)
|
||||
metric{} 4
|
||||
|
||||
# The time() function is allowed on top level.
|
||||
# Value at 10m-(1) == 9.59
|
||||
eval instant at 10m metric offset (time() != bool 0 + time() == bool 0)
|
||||
metric{} 9
|
Loading…
Reference in New Issue
Block a user