mirror of
https://github.com/prometheus/prometheus.git
synced 2025-03-05 20:59:13 -08:00
working commit
This commit is contained in:
parent
106542a1fe
commit
07f178a4be
|
@ -1259,7 +1259,7 @@ func labelsSetPromQL(query, labelMatchType, name, value string) error {
|
|||
return fmt.Errorf("invalid label match type: %s", labelMatchType)
|
||||
}
|
||||
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
if n, ok := node.(*parser.VectorSelector); ok {
|
||||
var found bool
|
||||
for i, l := range n.LabelMatchers {
|
||||
|
@ -1278,7 +1278,7 @@ func labelsSetPromQL(query, labelMatchType, name, value string) error {
|
|||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
|
||||
fmt.Println(expr.Pretty(0))
|
||||
return nil
|
||||
|
@ -1290,7 +1290,7 @@ func labelsDeletePromQL(query, name string) error {
|
|||
return err
|
||||
}
|
||||
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
if n, ok := node.(*parser.VectorSelector); ok {
|
||||
for i, l := range n.LabelMatchers {
|
||||
if l.Name == name {
|
||||
|
@ -1299,7 +1299,7 @@ func labelsDeletePromQL(query, name string) error {
|
|||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
|
||||
fmt.Println(expr.Pretty(0))
|
||||
return nil
|
||||
|
|
|
@ -344,6 +344,7 @@ type Engine struct {
|
|||
enableNegativeOffset bool
|
||||
enablePerStepStats bool
|
||||
enableDelayedNameRemoval bool
|
||||
NodeReplacer parser.NodeReplacer
|
||||
}
|
||||
|
||||
// NewEngine returns a new engine.
|
||||
|
@ -485,10 +486,11 @@ func (ng *Engine) NewInstantQuery(ctx context.Context, q storage.Queryable, opts
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := ng.validateOpts(expr); err != nil {
|
||||
|
||||
if err := ng.validateOpts(pExpr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
*pExpr = PreprocessExpr(expr, ts, ts)
|
||||
pExpr.Expr = PreprocessExpr(expr, ts, ts)
|
||||
|
||||
return qry, nil
|
||||
}
|
||||
|
@ -506,18 +508,24 @@ func (ng *Engine) NewRangeQuery(ctx context.Context, q storage.Queryable, opts Q
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := ng.validateOpts(expr); err != nil {
|
||||
|
||||
lookbackDelta := opts.LookbackDelta()
|
||||
if lookbackDelta <= 0 {
|
||||
lookbackDelta = ng.lookbackDelta
|
||||
}
|
||||
|
||||
if err := ng.validateOpts(pExpr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if expr.Type() != parser.ValueTypeVector && expr.Type() != parser.ValueTypeScalar {
|
||||
return nil, fmt.Errorf("invalid expression type %q for range query, must be Scalar or instant Vector", parser.DocumentedType(expr.Type()))
|
||||
}
|
||||
*pExpr = PreprocessExpr(expr, start, end)
|
||||
pExpr.Expr = PreprocessExpr(expr, start, end)
|
||||
|
||||
return qry, nil
|
||||
}
|
||||
|
||||
func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start, end time.Time, interval time.Duration) (*parser.Expr, *query) {
|
||||
func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start, end time.Time, interval time.Duration) (*parser.EvalStmt, *query) {
|
||||
if opts == nil {
|
||||
opts = NewPrometheusQueryOpts(false, 0)
|
||||
}
|
||||
|
@ -541,7 +549,7 @@ func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start
|
|||
sampleStats: stats.NewQuerySamples(ng.enablePerStepStats && opts.EnablePerStepStats()),
|
||||
queryable: q,
|
||||
}
|
||||
return &es.Expr, qry
|
||||
return es, qry
|
||||
}
|
||||
|
||||
var (
|
||||
|
@ -549,7 +557,7 @@ var (
|
|||
ErrValidationNegativeOffsetDisabled = errors.New("negative offset is disabled")
|
||||
)
|
||||
|
||||
func (ng *Engine) validateOpts(expr parser.Expr) error {
|
||||
func (ng *Engine) validateOpts(expr *parser.EvalStmt) error {
|
||||
if ng.enableAtModifier && ng.enableNegativeOffset {
|
||||
return nil
|
||||
}
|
||||
|
@ -557,7 +565,7 @@ func (ng *Engine) validateOpts(expr parser.Expr) error {
|
|||
var atModifierUsed, negativeOffsetUsed bool
|
||||
|
||||
var validationErr error
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), expr, func(node parser.Node, path []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
if n.Timestamp != nil || n.StartOrEnd == parser.START || n.StartOrEnd == parser.END {
|
||||
|
@ -595,7 +603,7 @@ func (ng *Engine) validateOpts(expr parser.Expr) error {
|
|||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
|
||||
return validationErr
|
||||
}
|
||||
|
@ -869,7 +877,7 @@ func FindMinMaxTime(s *parser.EvalStmt) (int64, int64) {
|
|||
// The evaluation of the VectorSelector inside then evaluates the given range and unsets
|
||||
// the variable.
|
||||
var evalRange time.Duration
|
||||
parser.Inspect(s.Expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), s, func(node parser.Node, path []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
start, end := getTimeRangesForSelector(s, n, path, evalRange)
|
||||
|
@ -884,7 +892,7 @@ func FindMinMaxTime(s *parser.EvalStmt) (int64, int64) {
|
|||
evalRange = n.Range
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
|
||||
if maxTimestamp == math.MinInt64 {
|
||||
// This happens when there was no selector. Hence no time range to select.
|
||||
|
@ -954,8 +962,11 @@ func (ng *Engine) populateSeries(ctx context.Context, querier storage.Querier, s
|
|||
// The evaluation of the VectorSelector inside then evaluates the given range and unsets
|
||||
// the variable.
|
||||
var evalRange time.Duration
|
||||
l := sync.Mutex{}
|
||||
|
||||
parser.Inspect(s.Expr, func(node parser.Node, path []parser.Node) error {
|
||||
n, err := parser.Inspect(context.TODO(), s, func(node parser.Node, path []parser.Node) error {
|
||||
l.Lock()
|
||||
defer l.Unlock()
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
start, end := getTimeRangesForSelector(s, n, path, evalRange)
|
||||
|
@ -978,7 +989,15 @@ func (ng *Engine) populateSeries(ctx context.Context, querier storage.Querier, s
|
|||
evalRange = n.Range
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, ng.NodeReplacer)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if nTyped, ok := n.(parser.Expr); ok {
|
||||
s.Expr = nTyped
|
||||
}
|
||||
}
|
||||
|
||||
// extractFuncFromPath walks up the path and searches for the first instance of
|
||||
|
@ -3600,7 +3619,7 @@ func setOffsetForAtModifier(evalTime int64, expr parser.Expr) {
|
|||
return originalOffset + offsetDiff
|
||||
}
|
||||
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
n.Offset = getOffset(n.Timestamp, n.OriginalOffset, path)
|
||||
|
@ -3613,7 +3632,7 @@ func setOffsetForAtModifier(evalTime int64, expr parser.Expr) {
|
|||
n.Offset = getOffset(n.Timestamp, n.OriginalOffset, path)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
}
|
||||
|
||||
// detectHistogramStatsDecoding modifies the expression by setting the
|
||||
|
@ -3622,7 +3641,7 @@ func setOffsetForAtModifier(evalTime int64, expr parser.Expr) {
|
|||
// and buckets. The function can be treated as an optimization and is not
|
||||
// required for correctness.
|
||||
func detectHistogramStatsDecoding(expr parser.Expr) {
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
if n, ok := node.(*parser.BinaryExpr); ok {
|
||||
detectHistogramStatsDecoding(n.LHS)
|
||||
detectHistogramStatsDecoding(n.RHS)
|
||||
|
@ -3649,7 +3668,7 @@ func detectHistogramStatsDecoding(expr parser.Expr) {
|
|||
}
|
||||
}
|
||||
return fmt.Errorf("stop")
|
||||
})
|
||||
}, nil)
|
||||
}
|
||||
|
||||
func makeInt64Pointer(val int64) *int64 {
|
||||
|
|
|
@ -83,7 +83,7 @@ loop:
|
|||
func (ev *evaluator) infoSelectHints(expr parser.Expr) storage.SelectHints {
|
||||
var nodeTimestamp *int64
|
||||
var offset int64
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
if n.Timestamp != nil {
|
||||
|
@ -94,7 +94,7 @@ func (ev *evaluator) infoSelectHints(expr parser.Expr) storage.SelectHints {
|
|||
default:
|
||||
return nil
|
||||
}
|
||||
})
|
||||
}, nil)
|
||||
|
||||
start := ev.startTimestamp
|
||||
end := ev.endTimestamp
|
||||
|
|
|
@ -313,32 +313,53 @@ type Visitor interface {
|
|||
// invoked recursively with visitor w for each of the non-nil children of node,
|
||||
// followed by a call of w.Visit(nil), returning an error
|
||||
// As the tree is descended the path of previous nodes is provided.
|
||||
func Walk(v Visitor, node Node, path []Node) error {
|
||||
func Walk(ctx context.Context, v Visitor, s *EvalStmt, node Node, path []Node, nr NodeReplacer) (Node, error) {
|
||||
// Check if the context is closed already
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
default:
|
||||
}
|
||||
|
||||
if nr != nil {
|
||||
replacement, err := nr(ctx, s, node)
|
||||
if replacement != nil {
|
||||
node = replacement
|
||||
}
|
||||
if err != nil {
|
||||
return node, err
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
var err error
|
||||
if v, err = v.Visit(node, path); v == nil || err != nil {
|
||||
return err
|
||||
return node, err
|
||||
}
|
||||
path = append(path, node)
|
||||
|
||||
for _, e := range Children(node) {
|
||||
if err := Walk(v, e, path); err != nil {
|
||||
return err
|
||||
// TODO: parallel execution of children
|
||||
for i, e := range Children(node) {
|
||||
if childNode, err := Walk(ctx, v, s, e, path, nr); err != nil {
|
||||
return node, err
|
||||
} else {
|
||||
SetChild(node, i, childNode)
|
||||
}
|
||||
}
|
||||
|
||||
_, err = v.Visit(nil, nil)
|
||||
return err
|
||||
return node, err
|
||||
}
|
||||
|
||||
func ExtractSelectors(expr Expr) [][]*labels.Matcher {
|
||||
var selectors [][]*labels.Matcher
|
||||
Inspect(expr, func(node Node, _ []Node) error {
|
||||
Inspect(context.TODO(), &EvalStmt{Expr: expr}, func(node Node, _ []Node) error {
|
||||
vs, ok := node.(*VectorSelector)
|
||||
if ok {
|
||||
selectors = append(selectors, vs.LabelMatchers)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
return selectors
|
||||
}
|
||||
|
||||
|
@ -355,8 +376,52 @@ func (f inspector) Visit(node Node, path []Node) (Visitor, error) {
|
|||
// Inspect traverses an AST in depth-first order: It starts by calling
|
||||
// f(node, path); node must not be nil. If f returns a nil error, Inspect invokes f
|
||||
// for all the non-nil children of node, recursively.
|
||||
func Inspect(node Node, f inspector) {
|
||||
Walk(f, node, nil) //nolint:errcheck
|
||||
func Inspect(ctx context.Context, s *EvalStmt, f inspector, nr NodeReplacer) (Node, error) {
|
||||
//nolint: errcheck
|
||||
return Walk(ctx, inspector(f), s, s.Expr, nil, nr)
|
||||
}
|
||||
|
||||
func SetChild(node Node, i int, child Node) {
|
||||
// For some reasons these switches have significantly better performance than interfaces
|
||||
switch n := node.(type) {
|
||||
case *EvalStmt:
|
||||
n.Expr = child.(Expr)
|
||||
case Expressions:
|
||||
n[i] = child.(Expr)
|
||||
case *AggregateExpr:
|
||||
// While this does not look nice, it should avoid unnecessary allocations
|
||||
// caused by slice resizing
|
||||
if n.Expr == nil && n.Param == nil {
|
||||
} else if n.Expr == nil {
|
||||
n.Param = child.(Expr)
|
||||
} else if n.Param == nil {
|
||||
n.Expr = child.(Expr)
|
||||
} else {
|
||||
switch i {
|
||||
case 0:
|
||||
n.Expr = child.(Expr)
|
||||
case 1:
|
||||
n.Param = child.(Expr)
|
||||
}
|
||||
}
|
||||
case *BinaryExpr:
|
||||
switch i {
|
||||
case 0:
|
||||
n.LHS = child.(Expr)
|
||||
case 1:
|
||||
n.RHS = child.(Expr)
|
||||
}
|
||||
case *Call:
|
||||
n.Args[i] = child.(Expr)
|
||||
case *SubqueryExpr:
|
||||
n.Expr = child.(Expr)
|
||||
case *ParenExpr:
|
||||
n.Expr = child.(Expr)
|
||||
case *UnaryExpr:
|
||||
n.Expr = child.(Expr)
|
||||
case *MatrixSelector:
|
||||
case *NumberLiteral, *StringLiteral, *VectorSelector:
|
||||
}
|
||||
}
|
||||
|
||||
// Children returns a list of all child nodes of a syntax tree node.
|
||||
|
@ -494,3 +559,5 @@ func (e *UnaryExpr) PositionRange() posrange.PositionRange {
|
|||
func (e *VectorSelector) PositionRange() posrange.PositionRange {
|
||||
return e.PosRange
|
||||
}
|
||||
|
||||
type NodeReplacer func(context.Context, *EvalStmt, Node) (Node, error)
|
||||
|
|
|
@ -996,7 +996,7 @@ func atModifierTestCases(exprStr string, evalTime time.Time) ([]atModifierTestCa
|
|||
// Setting the @ timestamp for all selectors to be evalTime.
|
||||
// If there is a subquery, then the selectors inside it don't get the @ timestamp.
|
||||
// If any selector already has the @ timestamp set, then it is untouched.
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: expr}, func(node parser.Node, path []parser.Node) error {
|
||||
if hasAtModifier(path) {
|
||||
// There is a subquery with timestamp in the path,
|
||||
// hence don't change any timestamps further.
|
||||
|
@ -1023,7 +1023,7 @@ func atModifierTestCases(exprStr string, evalTime time.Time) ([]atModifierTestCa
|
|||
containsNonStepInvariant = containsNonStepInvariant || ok
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
|
||||
if containsNonStepInvariant {
|
||||
// Expression contains a function whose result can vary with evaluation
|
||||
|
|
|
@ -1064,7 +1064,7 @@ func buildDependencyMap(rules []Rule) dependencyMap {
|
|||
name := rule.Name()
|
||||
outputs[name] = append(outputs[name], rule)
|
||||
|
||||
parser.Inspect(rule.Query(), func(node parser.Node, path []parser.Node) error {
|
||||
parser.Inspect(context.TODO(), &parser.EvalStmt{Expr: rule.Query()}, func(node parser.Node, path []parser.Node) error {
|
||||
if n, ok := node.(*parser.VectorSelector); ok {
|
||||
// A wildcard metric expression means we cannot reliably determine if this rule depends on any other,
|
||||
// which means we cannot safely run any rules concurrently.
|
||||
|
@ -1083,7 +1083,7 @@ func buildDependencyMap(rules []Rule) dependencyMap {
|
|||
inputs[n.Name] = append(inputs[n.Name], rule)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}, nil)
|
||||
}
|
||||
|
||||
if indeterminate {
|
||||
|
|
Loading…
Reference in a new issue