From 2c4f65c05752fb00f9740b8a17e1c3ead8cf3bab Mon Sep 17 00:00:00 2001 From: Gediminas Guoba Date: Thu, 11 Jun 2020 15:31:44 +0300 Subject: [PATCH] [m3comparator] adaption of Prometheus testdata to validate m3query (#2361) --- scripts/comparator/run.sh | 3 + src/cmd/services/m3comparator/main/querier.go | 2 +- src/query/test/m3comparator_client.go | 74 ++ src/query/test/m3query_client.go | 67 ++ src/query/test/promql_test.go | 59 ++ src/query/test/test.go | 578 ++++++++++++++++ src/query/test/testdata/LICENSE.txt | 34 + src/query/test/testdata/aggregators.test | 304 +++++++++ src/query/test/testdata/functions.test | 644 ++++++++++++++++++ src/query/test/testdata/histograms.test | 181 +++++ src/query/test/testdata/legacy.test | 391 +++++++++++ src/query/test/testdata/literals.test | 61 ++ src/query/test/testdata/operators.test | 440 ++++++++++++ src/query/test/testdata/selectors.test | 59 ++ src/query/test/testdata/staleness.test | 51 ++ src/query/test/testdata/subquery.test | 117 ++++ 16 files changed, 3064 insertions(+), 1 deletion(-) create mode 100644 src/query/test/m3comparator_client.go create mode 100644 src/query/test/m3query_client.go create mode 100644 src/query/test/promql_test.go create mode 100644 src/query/test/test.go create mode 100644 src/query/test/testdata/LICENSE.txt create mode 100644 src/query/test/testdata/aggregators.test create mode 100644 src/query/test/testdata/functions.test create mode 100644 src/query/test/testdata/histograms.test create mode 100644 src/query/test/testdata/legacy.test create mode 100644 src/query/test/testdata/literals.test create mode 100644 src/query/test/testdata/operators.test create mode 100644 src/query/test/testdata/selectors.test create mode 100644 src/query/test/testdata/staleness.test create mode 100644 src/query/test/testdata/subquery.test diff --git a/scripts/comparator/run.sh b/scripts/comparator/run.sh index 8c7903cf54..d1bb833188 100755 --- a/scripts/comparator/run.sh +++ b/scripts/comparator/run.sh @@ -73,3 +73,6 @@ $comparator -input=$QUERY_FILE \ -e=$END \ -comparator=$COMPARATOR_WRITE \ -regressionDir=$REGRESSION_DIR + +# Run PromQL testdata tests +go test -v -timeout 300s -tags=compatibility -count=1 github.com/m3db/m3/src/query/test/ diff --git a/src/cmd/services/m3comparator/main/querier.go b/src/cmd/services/m3comparator/main/querier.go index 0a40aacae5..fcabcc7cc1 100644 --- a/src/cmd/services/m3comparator/main/querier.go +++ b/src/cmd/services/m3comparator/main/querier.go @@ -174,7 +174,7 @@ func (q *querier) FetchCompressed( return m3.SeriesFetchResult{}, noop, err } } - + if iters == nil || iters.Len() == 0 { randomSeries, ignoreFilter, err = q.generateRandomSeries(query) if err != nil { diff --git a/src/query/test/m3comparator_client.go b/src/query/test/m3comparator_client.go new file mode 100644 index 0000000000..a3582909fb --- /dev/null +++ b/src/query/test/m3comparator_client.go @@ -0,0 +1,74 @@ +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package test + +import ( + "bytes" + "fmt" + "io/ioutil" + "net/http" +) + +type m3comparatorClient struct { + host string + port int +} + +func newM3ComparatorClient(host string, port int) *m3comparatorClient { + return &m3comparatorClient{ + host: host, + port: port, + } +} + +func (c *m3comparatorClient) clear() error { + comparatorURL := fmt.Sprintf("http://%s:%d", c.host, c.port) + req, err := http.NewRequest(http.MethodDelete, comparatorURL, nil) + if err != nil { + return err + } + + _, err = http.DefaultClient.Do(req) + if err != nil { + return err + } + return nil +} + +func (c *m3comparatorClient) load(data []byte) error { + comparatorURL := fmt.Sprintf("http://%s:%d", c.host, c.port) + resp, err := http.Post(comparatorURL, "application/json", bytes.NewReader(data)) + if err != nil { + return fmt.Errorf("got error loading data to comparator %v", err) + } + defer resp.Body.Close() + + if resp.StatusCode/200 == 1 { + return nil + } + + bodyBytes, err := ioutil.ReadAll(resp.Body) + if err != nil { + return fmt.Errorf("load status code %d. Error: %v", resp.StatusCode, err) + } + + return fmt.Errorf("load status code %d. Response: %s", resp.StatusCode, string(bodyBytes)) +} diff --git a/src/query/test/m3query_client.go b/src/query/test/m3query_client.go new file mode 100644 index 0000000000..8b0c57bad1 --- /dev/null +++ b/src/query/test/m3query_client.go @@ -0,0 +1,67 @@ +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package test + +import ( + "fmt" + "io/ioutil" + "net/http" + "time" + + "github.com/pkg/errors" +) + +type m3queryClient struct { + host string + port int +} + +func newM3QueryClient(host string, port int) *m3queryClient { + return &m3queryClient{ + host: host, + port: port, + } +} + +func (c *m3queryClient) query(expr string, t time.Time) ([]byte, error) { + url := fmt.Sprintf("http://%s:%d/m3query/api/v1/query", c.host, c.port) + req, err := http.NewRequest("GET", url, nil) + if err != nil { + return nil, err + } + + q := req.URL.Query() + q.Add("query", expr) + q.Add("time", fmt.Sprint(t.Unix())) + req.URL.RawQuery = q.Encode() + resp, err := http.DefaultClient.Do(req) + + if err != nil { + return nil, errors.Wrapf(err, "error evaluating query %s", expr) + } + + defer resp.Body.Close() + if resp.StatusCode/200 != 1 { + return nil, fmt.Errorf("invalid status %+v received sending query: %+v", resp.StatusCode, req) + } + + return ioutil.ReadAll(resp.Body) +} diff --git a/src/query/test/promql_test.go b/src/query/test/promql_test.go new file mode 100644 index 0000000000..7dab41d803 --- /dev/null +++ b/src/query/test/promql_test.go @@ -0,0 +1,59 @@ +// +build compatibility + +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2015 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// This code was taken from prometheus repo: https://github.com/prometheus/prometheus/blob/master/promql/promql_test.go + +package test + +import ( + "path/filepath" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestEvaluations(t *testing.T) { + files, err := filepath.Glob("testdata/*.test") + require.NoError(t, err) + + for _, fn := range files { + test, err := newTestFromFile(t, fn) + require.NoError(t, err) + + require.NoError(t, test.Run()) + + test.Close() + } +} diff --git a/src/query/test/test.go b/src/query/test/test.go new file mode 100644 index 0000000000..97dd3b52a3 --- /dev/null +++ b/src/query/test/test.go @@ -0,0 +1,578 @@ +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2015 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Parts of code were taken from prometheus repo: https://github.com/prometheus/prometheus/blob/master/promql/test.go + +package test + +import ( + "context" + "encoding/json" + "fmt" + "io/ioutil" + "math" + "regexp" + "sort" + "strconv" + "strings" + "time" + + cparser "github.com/m3db/m3/src/cmd/services/m3comparator/main/parser" + + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/util/testutil" +) + +var ( + minNormal = math.Float64frombits(0x0010000000000000) // The smallest positive normal value of type float64. + + patSpace = regexp.MustCompile("[\t ]+") + patLoad = regexp.MustCompile(`^load\s+(.+?)$`) + patEvalInstant = regexp.MustCompile(`^eval(?:_(fail|ordered))?\s+instant\s+(?:at\s+(.+?))?\s+(.+)$`) +) + +const ( + epsilon = 0.000001 // Relative error allowed for sample values. + startingTime = 1587393285000000000 // 2020-04-20 17:34:45 +) + +var testStartTime = time.Unix(0, 0).UTC() + +// Test is a sequence of read and write commands that are run +// against a test storage. +type Test struct { + testutil.T + + cmds []testCommand + + context context.Context + + m3comparator *m3comparatorClient +} + +// NewTest returns an initialized empty Test. +func NewTest(t testutil.T, input string) (*Test, error) { + test := &Test{ + T: t, + cmds: []testCommand{}, + m3comparator: newM3ComparatorClient("localhost", 9001), + } + err := test.parse(input) + if err != nil { + return test, err + } + err = test.clear() + return test, err +} + +func newTestFromFile(t testutil.T, filename string) (*Test, error) { + content, err := ioutil.ReadFile(filename) + if err != nil { + return nil, err + } + return NewTest(t, string(content)) +} + +func raise(line int, format string, v ...interface{}) error { + return &parser.ParseErr{ + LineOffset: line, + Err: errors.Errorf(format, v...), + } +} + +func (t *Test) parseLoad(lines []string, i int) (int, *loadCmd, error) { + if !patLoad.MatchString(lines[i]) { + return i, nil, raise(i, "invalid load command. (load )") + } + parts := patLoad.FindStringSubmatch(lines[i]) + + gap, err := model.ParseDuration(parts[1]) + if err != nil { + return i, nil, raise(i, "invalid step definition %q: %s", parts[1], err) + } + cmd := newLoadCmd(t.m3comparator, time.Duration(gap)) + for i+1 < len(lines) { + i++ + defLine := lines[i] + if len(defLine) == 0 { + i-- + break + } + metric, vals, err := parser.ParseSeriesDesc(defLine) + if err != nil { + if perr, ok := err.(*parser.ParseErr); ok { + perr.LineOffset = i + } + return i, nil, err + } + cmd.set(metric, vals...) + } + return i, cmd, nil +} + +func (t *Test) parseEval(lines []string, i int) (int, *evalCmd, error) { + if !patEvalInstant.MatchString(lines[i]) { + return i, nil, raise(i, "invalid evaluation command. (eval[_fail|_ordered] instant [at ] ") + } + parts := patEvalInstant.FindStringSubmatch(lines[i]) + var ( + mod = parts[1] + at = parts[2] + expr = parts[3] + ) + _, err := parser.ParseExpr(expr) + if err != nil { + if perr, ok := err.(*parser.ParseErr); ok { + perr.LineOffset = i + posOffset := parser.Pos(strings.Index(lines[i], expr)) + perr.PositionRange.Start += posOffset + perr.PositionRange.End += posOffset + perr.Query = lines[i] + } + return i, nil, err + } + + offset, err := model.ParseDuration(at) + if err != nil { + return i, nil, raise(i, "invalid step definition %q: %s", parts[1], err) + } + ts := testStartTime.Add(time.Duration(offset)) + + cmd := newEvalCmd(expr, ts, i+1) + switch mod { + case "ordered": + cmd.ordered = true + case "fail": + cmd.fail = true + } + + for j := 1; i+1 < len(lines); j++ { + i++ + defLine := lines[i] + if len(defLine) == 0 { + i-- + break + } + if f, err := parseNumber(defLine); err == nil { + cmd.expect(0, nil, parser.SequenceValue{Value: f}) + break + } + metric, vals, err := parser.ParseSeriesDesc(defLine) + if err != nil { + if perr, ok := err.(*parser.ParseErr); ok { + perr.LineOffset = i + } + return i, nil, err + } + + // Currently, we are not expecting any matrices. + if len(vals) > 1 { + return i, nil, raise(i, "expecting multiple values in instant evaluation not allowed") + } + cmd.expect(j, metric, vals...) + } + return i, cmd, nil +} + +// getLines returns trimmed lines after removing the comments. +func getLines(input string) []string { + lines := strings.Split(input, "\n") + for i, l := range lines { + l = strings.TrimSpace(l) + if strings.HasPrefix(l, "#") { + l = "" + } + lines[i] = l + } + return lines +} + +// parse the given command sequence and appends it to the test. +func (t *Test) parse(input string) error { + lines := getLines(input) + var err error + // Scan for steps line by line. + for i := 0; i < len(lines); i++ { + l := lines[i] + if len(l) == 0 { + continue + } + var cmd testCommand + + switch c := strings.ToLower(patSpace.Split(l, 2)[0]); { + case c == "clear": + cmd = &clearCmd{} + case c == "load": + i, cmd, err = t.parseLoad(lines, i) + case strings.HasPrefix(c, "eval"): + i, cmd, err = t.parseEval(lines, i) + default: + return raise(i, "invalid command %q", l) + } + if err != nil { + return err + } + t.cmds = append(t.cmds, cmd) + } + return nil +} + +// testCommand is an interface that ensures that only the package internal +// types can be a valid command for a test. +type testCommand interface { + testCmd() +} + +func (*clearCmd) testCmd() {} +func (*loadCmd) testCmd() {} +func (*evalCmd) testCmd() {} + +// loadCmd is a command that loads sequences of sample values for specific +// metrics into the storage. +type loadCmd struct { + gap time.Duration + metrics map[uint64]labels.Labels + defs map[uint64][]promql.Point + m3compClient *m3comparatorClient +} + +func newLoadCmd(m3compClient *m3comparatorClient, gap time.Duration) *loadCmd { + return &loadCmd{ + gap: gap, + metrics: map[uint64]labels.Labels{}, + defs: map[uint64][]promql.Point{}, + m3compClient: m3compClient, + } +} + +func (cmd loadCmd) String() string { + return "load" +} + +// set a sequence of sample values for the given metric. +func (cmd *loadCmd) set(m labels.Labels, vals ...parser.SequenceValue) { + h := m.Hash() + + samples := make([]promql.Point, 0, len(vals)) + ts := testStartTime + for _, v := range vals { + if !v.Omitted { + samples = append(samples, promql.Point{ + T: ts.UnixNano() / int64(time.Millisecond/time.Nanosecond), + V: v.Value, + }) + } + ts = ts.Add(cmd.gap) + } + cmd.defs[h] = samples + cmd.metrics[h] = m +} + +// append the defined time series to the storage. +func (cmd *loadCmd) append() error { + series := make([]cparser.Series, 0, len(cmd.defs)) + + for h, smpls := range cmd.defs { + m := cmd.metrics[h] + start := time.Unix(0, startingTime) + + ser := cparser.Series{ + Tags: make(cparser.Tags, 0, len(m)), + Start: start, + Datapoints: make(cparser.Datapoints, 0, len(smpls)), + } + for _, l := range m { + ser.Tags = append(ser.Tags, cparser.NewTag(l.Name, l.Value)) + } + + for _, s := range smpls { + ts := start.Add(time.Duration(s.T) * time.Millisecond) + ser.Datapoints = append(ser.Datapoints, cparser.Datapoint{ + Timestamp: ts, + Value: cparser.Value(s.V), + }) + + ser.End = ts.Add(cmd.gap * time.Millisecond) + } + series = append(series, ser) + } + + j, err := json.Marshal(series) + if err != nil { + return err + } + + return cmd.m3compClient.load(j) +} + +// evalCmd is a command that evaluates an expression for the given time (range) +// and expects a specific result. +type evalCmd struct { + expr string + start time.Time + line int + + fail, ordered bool + + metrics map[uint64]labels.Labels + expected map[uint64]entry + m3query *m3queryClient +} + +type entry struct { + pos int + vals []parser.SequenceValue +} + +func (e entry) String() string { + return fmt.Sprintf("%d: %s", e.pos, e.vals) +} + +func newEvalCmd(expr string, start time.Time, line int) *evalCmd { + return &evalCmd{ + expr: expr, + start: start, + line: line, + + metrics: map[uint64]labels.Labels{}, + expected: map[uint64]entry{}, + m3query: newM3QueryClient("localhost", 7201), + } +} + +func (ev *evalCmd) String() string { + return "eval" +} + +// expect adds a new metric with a sequence of values to the set of expected +// results for the query. +func (ev *evalCmd) expect(pos int, m labels.Labels, vals ...parser.SequenceValue) { + if m == nil { + ev.expected[0] = entry{pos: pos, vals: vals} + return + } + h := m.Hash() + ev.metrics[h] = m + ev.expected[h] = entry{pos: pos, vals: vals} +} + +const sep = '\xff' + +// Hash returns a hash value for the label set. +func hash(ls model.Metric) uint64 { + lbs := make(labels.Labels, 0, len(ls)) + for k, v := range ls { + lbs = append(lbs, labels.Label{ + Name: string(k), + Value: string(v), + }) + } + + sort.Slice(lbs[:], func(i, j int) bool { + return lbs[i].Name < lbs[j].Name + }) + + return lbs.Hash() +} + +// QueryResponse defines a structure for expected response. +type QueryResponse struct { + Status string + Data struct { + ResultType string + Result model.Samples + } +} + +// compareResult compares the result value with the defined expectation. +func (ev *evalCmd) compareResult(j []byte) error { + var result QueryResponse + err := json.Unmarshal(j, &result) + if err != nil { + return err + } + + if result.Status != "success" { + return fmt.Errorf("unsuccess status received: %s", result.Status) + } + + switch result.Data.ResultType { + case "matrix": + return errors.New("received range result on instant evaluation") + + case "vector": + seen := map[uint64]bool{} + for pos, v := range result.Data.Result { + fp := hash(v.Metric) + if _, ok := ev.metrics[fp]; !ok { + return errors.Errorf("unexpected metric %s in result", v.Metric) + } + + exp := ev.expected[fp] + if ev.ordered && exp.pos != pos+1 { + return errors.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric, exp.vals, exp.pos, pos+1) + } + if !almostEqual(exp.vals[0].Value, float64(v.Value)) { + return errors.Errorf("expected %v for %s but got %v", exp.vals[0].Value, v.Metric, v.Value) + } + seen[fp] = true + } + + for fp, expVals := range ev.expected { + if !seen[fp] { + fmt.Println("vector result", len(result.Data.Result), ev.expr) + for _, ss := range result.Data.Result { + fmt.Println(" ", ss.Metric, ss.Value) + } + return errors.Errorf("expected metric %s with %v not found", ev.metrics[fp], expVals) + } + } + + // TODO: Untested code. Uncomment when m3query will support scalars and fix it if needed. + //case "scalar": + // if !almostEqual(ev.expected[0].vals[0].Value, val.V) { + // return errors.Errorf("expected Scalar %v but got %v", val.V, ev.expected[0].vals[0].Value) + // } + + default: + panic(errors.Errorf("promql.Test.compareResult: unexpected result type %T", result)) + } + return nil +} + +// clearCmd is a command that wipes the test's storage state. +type clearCmd struct{} + +func (cmd clearCmd) String() string { + return "clear" +} + +// Run executes the command sequence of the test. Until the maximum error number +// is reached, evaluation errors do not terminate execution. +func (t *Test) Run() error { + for _, cmd := range t.cmds { + // TODO(fabxc): aggregate command errors, yield diffs for result + // comparison errors. + if err := t.exec(cmd); err != nil { + return err + } + } + return nil +} + +// exec processes a single step of the test. +func (t *Test) exec(tc testCommand) error { + switch cmd := tc.(type) { + case *clearCmd: + return t.clear() + + case *loadCmd: + return cmd.append() + + case *evalCmd: + expr, err := parser.ParseExpr(cmd.expr) + if err != nil { + return err + } + + t := time.Unix(0, startingTime+(cmd.start.Unix()*1000000000)) + bodyBytes, err := cmd.m3query.query(expr.String(), t) + if err != nil { + if cmd.fail { + return nil + } + return errors.Wrapf(err, "error in %s %s, line %d", cmd, cmd.expr, cmd.line) + } + if cmd.fail { + return fmt.Errorf("expected to fail at %s %s, line %d", cmd, cmd.expr, cmd.line) + } + + err = cmd.compareResult(bodyBytes) + if err != nil { + return errors.Wrapf(err, "error in %s %s, line %d. m3query response: %s", cmd, cmd.expr, cmd.line, string(bodyBytes)) + } + + default: + panic("promql.Test.exec: unknown test command type") + } + return nil +} + +// clear the current test storage of all inserted samples. +func (t *Test) clear() error { + t.m3comparator.clear() + return nil +} + +// Close closes resources associated with the Test. +func (t *Test) Close() { +} + +// samplesAlmostEqual returns true if the two sample lines only differ by a +// small relative error in their sample value. +func almostEqual(a, b float64) bool { + // NaN has no equality but for testing we still want to know whether both values + // are NaN. + if math.IsNaN(a) && math.IsNaN(b) { + return true + } + + // Cf. http://floating-point-gui.de/errors/comparison/ + if a == b { + return true + } + + diff := math.Abs(a - b) + + if a == 0 || b == 0 || diff < minNormal { + return diff < epsilon*minNormal + } + return diff/(math.Abs(a)+math.Abs(b)) < epsilon +} + +func parseNumber(s string) (float64, error) { + n, err := strconv.ParseInt(s, 0, 64) + f := float64(n) + if err != nil { + f, err = strconv.ParseFloat(s, 64) + } + if err != nil { + return 0, errors.Wrap(err, "error parsing number") + } + return f, nil +} diff --git a/src/query/test/testdata/LICENSE.txt b/src/query/test/testdata/LICENSE.txt new file mode 100644 index 0000000000..6133dce0f9 --- /dev/null +++ b/src/query/test/testdata/LICENSE.txt @@ -0,0 +1,34 @@ +// Modifications Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Copyright 2015 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// This code was taken from prometheus repo: https://github.com/prometheus/prometheus/tree/master/promql/testdata diff --git a/src/query/test/testdata/aggregators.test b/src/query/test/testdata/aggregators.test new file mode 100644 index 0000000000..8505cf18f1 --- /dev/null +++ b/src/query/test/testdata/aggregators.test @@ -0,0 +1,304 @@ +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+10x10 + http_requests{job="api-server", instance="1", group="production"} 0+20x10 + http_requests{job="api-server", instance="0", group="canary"} 0+30x10 + http_requests{job="api-server", instance="1", group="canary"} 0+40x10 + http_requests{job="app-server", instance="0", group="production"} 0+50x10 + http_requests{job="app-server", instance="1", group="production"} 0+60x10 + http_requests{job="app-server", instance="0", group="canary"} 0+70x10 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + +load 5m + foo{job="api-server", instance="0", region="europe"} 0+90x10 + foo{job="api-server"} 0+100x10 + +# Simple sum. +eval instant at 50m SUM BY (group) (http_requests{job="api-server"}) + {group="canary"} 700 + {group="production"} 300 + +eval instant at 50m SUM BY (group) (((http_requests{job="api-server"}))) + {group="canary"} 700 + {group="production"} 300 + +# Test alternative "by"-clause order. +eval instant at 50m sum by (group) (http_requests{job="api-server"}) + {group="canary"} 700 + {group="production"} 300 + +# Simple average. +eval instant at 50m avg by (group) (http_requests{job="api-server"}) + {group="canary"} 350 + {group="production"} 150 + +# Simple count. +eval instant at 50m count by (group) (http_requests{job="api-server"}) + {group="canary"} 2 + {group="production"} 2 + +# FAILING issue #9. Simple without. +#eval instant at 50m sum without (instance) (http_requests{job="api-server"}) +# {group="canary",job="api-server"} 700 +# {group="production",job="api-server"} 300 + +# Empty by. +eval instant at 50m sum by () (http_requests{job="api-server"}) + {} 1000 + +# No by/without. +eval instant at 50m sum(http_requests{job="api-server"}) + {} 1000 + +# FAILING issue #9. Empty without. +#eval instant at 50m sum without () (http_requests{job="api-server",group="production"}) +# {group="production",job="api-server",instance="0"} 100 +# {group="production",job="api-server",instance="1"} 200 + +# FAILING issue #9. Without with mismatched and missing labels. Do not do this. +#eval instant at 50m sum without (instance) (http_requests{job="api-server"} or foo) +# {group="canary",job="api-server"} 700 +# {group="production",job="api-server"} 300 +# {region="europe",job="api-server"} 900 +# {job="api-server"} 1000 + +# Lower-cased aggregation operators should work too. +eval instant at 50m sum(http_requests) by (job) + min(http_requests) by (job) + max(http_requests) by (job) + avg(http_requests) by (job) + {job="app-server"} 4550 + {job="api-server"} 1750 + +# Test alternative "by"-clause order. +eval instant at 50m sum by (group) (http_requests{job="api-server"}) + {group="canary"} 700 + {group="production"} 300 + +# Test both alternative "by"-clause orders in one expression. +# Public health warning: stick to one form within an expression (or even +# in an organization), or risk serious user confusion. +eval instant at 50m sum(sum by (group) (http_requests{job="api-server"})) by (job) + {} 1000 + + + +# Standard deviation and variance. +eval instant at 50m stddev(http_requests) + {} 229.12878474779 + +eval instant at 50m stddev by (instance)(http_requests) + {instance="0"} 223.60679774998 + {instance="1"} 223.60679774998 + +eval instant at 50m stdvar(http_requests) + {} 52500 + +eval instant at 50m stdvar by (instance)(http_requests) + {instance="0"} 50000 + {instance="1"} 50000 + +# Float precision test for standard deviation and variance +clear +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+1.33x10 + http_requests{job="api-server", instance="1", group="production"} 0+1.33x10 + http_requests{job="api-server", instance="0", group="canary"} 0+1.33x10 + +#eval instant at 50m stddev(http_requests) +# {} 0.0 + +#eval instant at 50m stdvar(http_requests) +# {} 0.0 + + + +# Regression test for missing separator byte in labelsToGroupingKey. +clear +load 5m + label_grouping_test{a="aa", b="bb"} 0+10x10 + label_grouping_test{a="a", b="abb"} 0+20x10 + +# FAILING +#eval instant at 50m sum(label_grouping_test) by (a, b) +# {a="a", b="abb"} 200 +# {a="aa", b="bb"} 100 + + + +# Tests for min/max. +clear +load 5m + http_requests{job="api-server", instance="0", group="production"} 1 + http_requests{job="api-server", instance="1", group="production"} 2 + http_requests{job="api-server", instance="0", group="canary"} NaN + http_requests{job="api-server", instance="1", group="canary"} 3 + http_requests{job="api-server", instance="2", group="canary"} 4 + +eval instant at 0m max(http_requests) + {} 4 + +eval instant at 0m min(http_requests) + {} 1 + +eval instant at 0m max by (group) (http_requests) + {group="production"} 2 + {group="canary"} 4 + +eval instant at 0m min by (group) (http_requests) + {group="production"} 1 + {group="canary"} 3 + +clear + +# Tests for topk/bottomk. +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+10x10 + http_requests{job="api-server", instance="1", group="production"} 0+20x10 + http_requests{job="api-server", instance="2", group="production"} NaN NaN NaN NaN NaN NaN NaN NaN NaN NaN + http_requests{job="api-server", instance="0", group="canary"} 0+30x10 + http_requests{job="api-server", instance="1", group="canary"} 0+40x10 + http_requests{job="app-server", instance="0", group="production"} 0+50x10 + http_requests{job="app-server", instance="1", group="production"} 0+60x10 + http_requests{job="app-server", instance="0", group="canary"} 0+70x10 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + foo 3+0x10 + +# FAILING issue #12. All topk and bottomk tests are failing. +#eval_ordered instant at 50m topk(3, http_requests) +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="production", instance="1", job="app-server"} 600 + +#eval_ordered instant at 50m topk((3), (http_requests)) +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="production", instance="1", job="app-server"} 600 + +#eval_ordered instant at 50m topk(5, http_requests{group="canary",job="app-server"}) +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="0", job="app-server"} 700 + +#eval_ordered instant at 50m bottomk(3, http_requests) +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="canary", instance="0", job="api-server"} 300 + +#eval_ordered instant at 50m bottomk(5, http_requests{group="canary",job="app-server"}) +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +#eval instant at 50m topk by (group) (1, http_requests) +# http_requests{group="production", instance="1", job="app-server"} 600 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +#eval instant at 50m bottomk by (group) (2, http_requests) +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="1", job="api-server"} 200 + +#eval_ordered instant at 50m bottomk by (group) (2, http_requests{group="production"}) +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="1", job="api-server"} 200 + +# Test NaN is sorted away from the top/bottom. +#eval_ordered instant at 50m topk(3, http_requests{job="api-server",group="production"}) +# http_requests{job="api-server", instance="1", group="production"} 200 +# http_requests{job="api-server", instance="0", group="production"} 100 +# http_requests{job="api-server", instance="2", group="production"} NaN + +#eval_ordered instant at 50m bottomk(3, http_requests{job="api-server",group="production"}) +# http_requests{job="api-server", instance="0", group="production"} 100 +# http_requests{job="api-server", instance="1", group="production"} 200 +# http_requests{job="api-server", instance="2", group="production"} NaN + +# Test topk and bottomk allocate min(k, input_vector) for results vector +#eval_ordered instant at 50m bottomk(9999999999, http_requests{job="app-server",group="canary"}) +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +#eval_ordered instant at 50m topk(9999999999, http_requests{job="api-server",group="production"}) +# http_requests{job="api-server", instance="1", group="production"} 200 +# http_requests{job="api-server", instance="0", group="production"} 100 +# http_requests{job="api-server", instance="2", group="production"} NaN + +# Bug #5276. +#eval_ordered instant at 50m topk(scalar(foo), http_requests) +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="production", instance="1", job="app-server"} 600 + +clear + +# Tests for count_values. +load 5m + version{job="api-server", instance="0", group="production"} 6 + version{job="api-server", instance="1", group="production"} 6 + version{job="api-server", instance="2", group="production"} 6 + version{job="api-server", instance="0", group="canary"} 8 + version{job="api-server", instance="1", group="canary"} 8 + version{job="app-server", instance="0", group="production"} 6 + version{job="app-server", instance="1", group="production"} 6 + version{job="app-server", instance="0", group="canary"} 7 + version{job="app-server", instance="1", group="canary"} 7 + +# FAILING issue #14 +#eval instant at 5m count_values("version", version) +# {version="6"} 5 +# {version="7"} 2 +# {version="8"} 2 + + +#eval instant at 5m count_values(((("version"))), version) +# {version="6"} 5 +# {version="7"} 2 +# {version="8"} 2 + + +#eval instant at 5m count_values without (instance)("version", version) +# {job="api-server", group="production", version="6"} 3 +# {job="api-server", group="canary", version="8"} 2 +# {job="app-server", group="production", version="6"} 2 +# {job="app-server", group="canary", version="7"} 2 + +# Overwrite label with output. Don't do this. +#eval instant at 5m count_values without (instance)("job", version) +# {job="6", group="production"} 5 +# {job="8", group="canary"} 2 +# {job="7", group="canary"} 2 + +# Overwrite label with output. Don't do this. +#eval instant at 5m count_values by (job, group)("job", version) +# {job="6", group="production"} 5 +# {job="8", group="canary"} 2 +# {job="7", group="canary"} 2 + + +# Tests for quantile. +clear + +load 10s + data{test="two samples",point="a"} 0 + data{test="two samples",point="b"} 1 + data{test="three samples",point="a"} 0 + data{test="three samples",point="b"} 1 + data{test="three samples",point="c"} 2 + data{test="uneven samples",point="a"} 0 + data{test="uneven samples",point="b"} 1 + data{test="uneven samples",point="c"} 4 + foo .8 + +# FAILING issue #8 +#eval instant at 1m quantile without(point)(0.8, data) +# {test="two samples"} 0.8 +# {test="three samples"} 1.6 +# {test="uneven samples"} 2.8 + +# Bug #5276. +#eval instant at 1m quantile without(point)(scalar(foo), data) +# {test="two samples"} 0.8 +# {test="three samples"} 1.6 +# {test="uneven samples"} 2.8 + + +#eval instant at 1m quantile without(point)((scalar(foo)), data) +# {test="two samples"} 0.8 +# {test="three samples"} 1.6 +# {test="uneven samples"} 2.8 \ No newline at end of file diff --git a/src/query/test/testdata/functions.test b/src/query/test/testdata/functions.test new file mode 100644 index 0000000000..25561f9e30 --- /dev/null +++ b/src/query/test/testdata/functions.test @@ -0,0 +1,644 @@ +# Testdata for resets() and changes(). +load 5m + http_requests{path="/foo"} 1 2 3 0 1 0 0 1 2 0 + http_requests{path="/bar"} 1 2 3 4 5 1 2 3 4 5 + http_requests{path="/biz"} 0 0 0 0 0 1 1 1 1 1 + +# Tests for resets(). +# FAILING issue #16 +#eval instant at 50m resets(http_requests[5m]) +# {path="/foo"} 0 +# {path="/bar"} 0 +# {path="/biz"} 0 + +eval instant at 50m resets(http_requests[20m]) + {path="/foo"} 1 + {path="/bar"} 0 + {path="/biz"} 0 + +eval instant at 50m resets(http_requests[30m]) + {path="/foo"} 2 + {path="/bar"} 1 + {path="/biz"} 0 + +eval instant at 50m resets(http_requests[50m]) + {path="/foo"} 3 + {path="/bar"} 1 + {path="/biz"} 0 + +# FAILING. eval instant at 50m resets(nonexistent_metric[50m]) + +# Tests for changes(). +# FAILING issue #17 +#eval instant at 50m changes(http_requests[5m]) +# {path="/foo"} 0 +# {path="/bar"} 0 +# {path="/biz"} 0 + +eval instant at 50m changes(http_requests[20m]) + {path="/foo"} 3 + {path="/bar"} 3 + {path="/biz"} 0 + +eval instant at 50m changes(http_requests[30m]) + {path="/foo"} 4 + {path="/bar"} 5 + {path="/biz"} 1 + +#eval instant at 50m changes(http_requests[50m]) +# {path="/foo"} 8 +# {path="/bar"} 9 +# {path="/biz"} 1 + +#eval instant at 50m changes((http_requests[50m])) +# {path="/foo"} 8 +# {path="/bar"} 9 +# {path="/biz"} 1 + +# FAILING. eval instant at 50m changes(nonexistent_metric[50m]) + +clear + +load 5m + x{a="b"} NaN NaN NaN + x{a="c"} 0 NaN 0 + +# FAILING +#eval instant at 15m changes(x[15m]) +# {a="b"} 0 +# {a="c"} 2 + +clear + +# Tests for increase(). +load 5m + http_requests{path="/foo"} 0+10x10 + http_requests{path="/bar"} 0+10x5 0+10x5 + +# Tests for increase(). +eval instant at 50m increase(http_requests[50m]) + {path="/foo"} 100 + {path="/bar"} 90 + +eval instant at 50m increase(http_requests[100m]) + {path="/foo"} 100 + {path="/bar"} 90 + +clear + +# Test for increase() with counter reset. +# When the counter is reset, it always starts at 0. +# So the sequence 3 2 (decreasing counter = reset) is interpreted the same as 3 0 1 2. +# Prometheus assumes it missed the intermediate values 0 and 1. +load 5m + http_requests{path="/foo"} 0 1 2 3 2 3 4 + +eval instant at 30m increase(http_requests[30m]) + {path="/foo"} 7 + +clear + +# Tests for irate(). +load 5m + http_requests{path="/foo"} 0+10x10 + http_requests{path="/bar"} 0+10x5 0+10x5 + +eval instant at 50m irate(http_requests[50m]) + {path="/foo"} .03333333333333333333 + {path="/bar"} .03333333333333333333 + +# Counter reset. +eval instant at 30m irate(http_requests[50m]) + {path="/foo"} .03333333333333333333 + {path="/bar"} 0 + +clear + +# Tests for delta(). +load 5m + http_requests{path="/foo"} 0 50 100 150 200 + http_requests{path="/bar"} 200 150 100 50 0 + +eval instant at 20m delta(http_requests[20m]) + {path="/foo"} 200 + {path="/bar"} -200 + +clear + +# Tests for idelta(). +load 5m + http_requests{path="/foo"} 0 50 100 150 + http_requests{path="/bar"} 0 50 100 50 + +eval instant at 20m idelta(http_requests[20m]) + {path="/foo"} 50 + {path="/bar"} -50 + +clear + +# Tests for deriv() and predict_linear(). +load 5m + testcounter_reset_middle 0+10x4 0+10x5 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + +# FAILING TODO: should not fail, need to check +# deriv should return the same as rate in simple cases. +eval instant at 50m rate(http_requests{group="canary", instance="1", job="app-server"}[50m]) + {group="canary", instance="1", job="app-server"} 0.26666666666666666 + +# FAILING TODO: should not fail, need to check +eval instant at 50m deriv(http_requests{group="canary", instance="1", job="app-server"}[50m]) + {group="canary", instance="1", job="app-server"} 0.26666666666666666 + +# deriv should return correct result. +eval instant at 50m deriv(testcounter_reset_middle[100m]) + {} 0.010606060606060607 + +# predict_linear should return correct result. +# X/s = [ 0, 300, 600, 900,1200,1500,1800,2100,2400,2700,3000] +# Y = [ 0, 10, 20, 30, 40, 0, 10, 20, 30, 40, 50] +# sumX = 16500 +# sumY = 250 +# sumXY = 480000 +# sumX2 = 34650000 +# n = 11 +# covXY = 105000 +# varX = 9900000 +# slope = 0.010606060606060607 +# intercept at t=0: 6.818181818181818 +# intercept at t=3000: 38.63636363636364 +# intercept at t=3000+3600: 76.81818181818181 +eval instant at 50m predict_linear(testcounter_reset_middle[100m], 3600) + {} 76.81818181818181 + +# With http_requests, there is a sample value exactly at the end of +# the range, and it has exactly the predicted value, so predict_linear +# can be emulated with deriv. +eval instant at 50m predict_linear(http_requests[50m], 3600) - (http_requests + deriv(http_requests[50m]) * 3600) + {group="canary", instance="1", job="app-server"} 0 + +clear + +# Tests for label_replace. +load 5m + testmetric{src="source-value-10",dst="original-destination-value"} 0 + testmetric{src="source-value-20",dst="original-destination-value"} 1 + +# label_replace does a full-string match and replace. +eval instant at 0m label_replace(testmetric, "dst", "destination-value-$1", "src", "source-value-(.*)") + testmetric{src="source-value-10",dst="destination-value-10"} 0 + testmetric{src="source-value-20",dst="destination-value-20"} 1 + +# FAILING. label_replace does not do a sub-string match. +#eval instant at 0m label_replace(testmetric, "dst", "destination-value-$1", "src", "value-(.*)") +# testmetric{src="source-value-10",dst="original-destination-value"} 0 +# testmetric{src="source-value-20",dst="original-destination-value"} 1 + +# label_replace works with multiple capture groups. +eval instant at 0m label_replace(testmetric, "dst", "$1-value-$2", "src", "(.*)-value-(.*)") + testmetric{src="source-value-10",dst="source-value-10"} 0 + testmetric{src="source-value-20",dst="source-value-20"} 1 + +# label_replace does not overwrite the destination label if the source label +# does not exist. +eval instant at 0m label_replace(testmetric, "dst", "value-$1", "nonexistent-src", "source-value-(.*)") + testmetric{src="source-value-10",dst="original-destination-value"} 0 + testmetric{src="source-value-20",dst="original-destination-value"} 1 + +# FAILING. label_replace overwrites the destination label if the source label is empty, +# but matched. +#eval instant at 0m label_replace(testmetric, "dst", "value-$1", "nonexistent-src", "(.*)") +# testmetric{src="source-value-10",dst="value-"} 0 +# testmetric{src="source-value-20",dst="value-"} 1 + +# label_replace does not overwrite the destination label if the source label +# is not matched. +eval instant at 0m label_replace(testmetric, "dst", "value-$1", "src", "non-matching-regex") + testmetric{src="source-value-10",dst="original-destination-value"} 0 + testmetric{src="source-value-20",dst="original-destination-value"} 1 + +# FAILING. eval instant at 0m label_replace((((testmetric))), (("dst")), (("value-$1")), (("src")), (("non-matching-regex"))) +# testmetric{src="source-value-10",dst="original-destination-value"} 0 +# testmetric{src="source-value-20",dst="original-destination-value"} 1 + +# FAILING. label_replace drops labels that are set to empty values. +#eval instant at 0m label_replace(testmetric, "dst", "", "dst", ".*") +# testmetric{src="source-value-10"} 0 +# testmetric{src="source-value-20"} 1 + +# label_replace fails when the regex is invalid. +eval_fail instant at 0m label_replace(testmetric, "dst", "value-$1", "src", "(.*") + +# FAILING. label_replace fails when the destination label name is not a valid Prometheus label name. +#eval_fail instant at 0m label_replace(testmetric, "invalid-label-name", "", "src", "(.*)") + +# FAILING. label_replace fails when there would be duplicated identical output label sets. +#eval_fail instant at 0m label_replace(testmetric, "src", "", "", "") + +clear + +# Tests for vector, time and timestamp. +load 10s + metric 1 1 + +# FAILING. eval instant at 0s timestamp(metric) +# {} 0 + +# FAILING. eval instant at 5s timestamp(metric) +# {} 0 + +# FAILING. eval instant at 10s timestamp(metric) +# {} 10 + +# FAILING. eval instant at 10s timestamp(((metric))) +# {} 10 + +# Tests for label_join. +load 5m + testmetric{src="a",src1="b",src2="c",dst="original-destination-value"} 0 + testmetric{src="d",src1="e",src2="f",dst="original-destination-value"} 1 + +# label_join joins all src values in order. +eval instant at 0m label_join(testmetric, "dst", "-", "src", "src1", "src2") + testmetric{src="a",src1="b",src2="c",dst="a-b-c"} 0 + testmetric{src="d",src1="e",src2="f",dst="d-e-f"} 1 + +# FAILING. label_join treats non existent src labels as empty strings. +#eval instant at 0m label_join(testmetric, "dst", "-", "src", "src3", "src1") +# testmetric{src="a",src1="b",src2="c",dst="a--b"} 0 +# testmetric{src="d",src1="e",src2="f",dst="d--e"} 1 + +# FAILING. label_join overwrites the destination label even if the resulting dst label is empty string +#eval instant at 0m label_join(testmetric, "dst", "", "emptysrc", "emptysrc1", "emptysrc2") +# testmetric{src="a",src1="b",src2="c"} 0 +# testmetric{src="d",src1="e",src2="f"} 1 + +# test without src label for label_join +# FAILING. eval instant at 0m label_join(testmetric, "dst", ", ") +# testmetric{src="a",src1="b",src2="c"} 0 +# testmetric{src="d",src1="e",src2="f"} 1 + +# test without dst label for label_join +load 5m + testmetric1{src="foo",src1="bar",src2="foobar"} 0 + testmetric1{src="fizz",src1="buzz",src2="fizzbuzz"} 1 + +# label_join creates dst label if not present. +eval instant at 0m label_join(testmetric1, "dst", ", ", "src", "src1", "src2") + testmetric1{src="foo",src1="bar",src2="foobar",dst="foo, bar, foobar"} 0 + testmetric1{src="fizz",src1="buzz",src2="fizzbuzz",dst="fizz, buzz, fizzbuzz"} 1 + +clear + +# Tests for vector. +eval instant at 0m vector(1) + {} 1 + +eval instant at 0s vector(time()) + {} 0 + +# FAILING. eval instant at 5s vector(time()) +# {} 5 + +# FAILING. eval instant at 60m vector(time()) +# {} 3600 + + +# Tests for clamp_max and clamp_min(). +load 5m + test_clamp{src="clamp-a"} -50 + test_clamp{src="clamp-b"} 0 + test_clamp{src="clamp-c"} 100 + +eval instant at 0m clamp_max(test_clamp, 75) + {src="clamp-a"} -50 + {src="clamp-b"} 0 + {src="clamp-c"} 75 + +eval instant at 0m clamp_min(test_clamp, -25) + {src="clamp-a"} -25 + {src="clamp-b"} 0 + {src="clamp-c"} 100 + +eval instant at 0m clamp_max(clamp_min(test_clamp, -20), 70) + {src="clamp-a"} -20 + {src="clamp-b"} 0 + {src="clamp-c"} 70 + +eval instant at 0m clamp_max((clamp_min(test_clamp, (-20))), (70)) + {src="clamp-a"} -20 + {src="clamp-b"} 0 + {src="clamp-c"} 70 + + +# Tests for sort/sort_desc. +clear +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+10x10 + http_requests{job="api-server", instance="1", group="production"} 0+20x10 + http_requests{job="api-server", instance="0", group="canary"} 0+30x10 + http_requests{job="api-server", instance="1", group="canary"} 0+40x10 + http_requests{job="api-server", instance="2", group="canary"} NaN NaN NaN NaN NaN NaN NaN NaN NaN NaN + http_requests{job="app-server", instance="0", group="production"} 0+50x10 + http_requests{job="app-server", instance="1", group="production"} 0+60x10 + http_requests{job="app-server", instance="0", group="canary"} 0+70x10 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + +# FAILING. eval_ordered instant at 50m sort(http_requests) +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="app-server"} 600 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="2", job="api-server"} NaN + +#eval_ordered instant at 50m sort_desc(http_requests) +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="production", instance="1", job="app-server"} 600 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="canary", instance="2", job="api-server"} NaN + +# Tests for holt_winters +clear + +# positive trends +load 10s + http_requests{job="api-server", instance="0", group="production"} 0+10x1000 100+30x1000 + http_requests{job="api-server", instance="1", group="production"} 0+20x1000 200+30x1000 + http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300+80x1000 + http_requests{job="api-server", instance="1", group="canary"} 0+40x2000 + +eval instant at 8000s holt_winters(http_requests[1m], 0.01, 0.1) + {job="api-server", instance="0", group="production"} 8000 + {job="api-server", instance="1", group="production"} 16000 + {job="api-server", instance="0", group="canary"} 24000 + {job="api-server", instance="1", group="canary"} 32000 + +# negative trends +clear +load 10s + http_requests{job="api-server", instance="0", group="production"} 8000-10x1000 + http_requests{job="api-server", instance="1", group="production"} 0-20x1000 + http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300-80x1000 + http_requests{job="api-server", instance="1", group="canary"} 0-40x1000 0+40x1000 + +eval instant at 8000s holt_winters(http_requests[1m], 0.01, 0.1) + {job="api-server", instance="0", group="production"} 0 + {job="api-server", instance="1", group="production"} -16000 + {job="api-server", instance="0", group="canary"} 24000 + {job="api-server", instance="1", group="canary"} -32000 + +# Tests for avg_over_time +clear +load 10s + metric 1 2 3 4 5 + +eval instant at 1m avg_over_time(metric[1m]) + {} 3 + +# Tests for stddev_over_time and stdvar_over_time. +clear +load 10s + metric 0 8 8 2 3 + +eval instant at 1m stdvar_over_time(metric[1m]) + {} 10.56 + +eval instant at 1m stddev_over_time(metric[1m]) + {} 3.249615 + +# FAILING. eval instant at 1m stddev_over_time((metric[1m])) +# {} 3.249615 + +# Tests for stddev_over_time and stdvar_over_time #4927. +clear +load 10s + metric 1.5990505637277868 1.5990505637277868 1.5990505637277868 + +eval instant at 1m stdvar_over_time(metric[1m]) + {} 0 + +eval instant at 1m stddev_over_time(metric[1m]) + {} 0 + +# Tests for quantile_over_time +clear + +load 10s + data{test="two samples"} 0 1 + data{test="three samples"} 0 1 2 + data{test="uneven samples"} 0 1 4 + +eval instant at 1m quantile_over_time(0, data[1m]) + {test="two samples"} 0 + {test="three samples"} 0 + {test="uneven samples"} 0 + +eval instant at 1m quantile_over_time(0.5, data[1m]) + {test="two samples"} 0.5 + {test="three samples"} 1 + {test="uneven samples"} 1 + +eval instant at 1m quantile_over_time(0.75, data[1m]) + {test="two samples"} 0.75 + {test="three samples"} 1.5 + {test="uneven samples"} 2.5 + +eval instant at 1m quantile_over_time(0.8, data[1m]) + {test="two samples"} 0.8 + {test="three samples"} 1.6 + {test="uneven samples"} 2.8 + +eval instant at 1m quantile_over_time(1, data[1m]) + {test="two samples"} 1 + {test="three samples"} 2 + {test="uneven samples"} 4 + +eval instant at 1m quantile_over_time(-1, data[1m]) + {test="two samples"} -Inf + {test="three samples"} -Inf + {test="uneven samples"} -Inf + +eval instant at 1m quantile_over_time(2, data[1m]) + {test="two samples"} +Inf + {test="three samples"} +Inf + {test="uneven samples"} +Inf + +# FAILING. eval instant at 1m (quantile_over_time(2, (data[1m]))) +# {test="two samples"} +Inf +# {test="three samples"} +Inf +# {test="uneven samples"} +Inf + +clear + +# FAILING. Test time-related functions. +#eval instant at 0m year() +# {} 1970 + +#eval instant at 1ms time() +# 0.001 + +eval instant at 0m year(vector(1136239445)) + {} 2006 + +#eval instant at 0m month() +# {} 1 + +eval instant at 0m month(vector(1136239445)) + {} 1 + +#eval instant at 0m day_of_month() +# {} 1 + +eval instant at 0m day_of_month(vector(1136239445)) + {} 2 + +# Thursday. +#eval instant at 0m day_of_week() +# {} 4 + +eval instant at 0m day_of_week(vector(1136239445)) + {} 1 + +#eval instant at 0m hour() +# {} 0 + +eval instant at 0m hour(vector(1136239445)) + {} 22 + +#eval instant at 0m minute() +# {} 0 + +eval instant at 0m minute(vector(1136239445)) + {} 4 + +# 2008-12-31 23:59:59 just before leap second. +eval instant at 0m year(vector(1230767999)) + {} 2008 + +# 2009-01-01 00:00:00 just after leap second. +eval instant at 0m year(vector(1230768000)) + {} 2009 + +# 2016-02-29 23:59:59 February 29th in leap year. +eval instant at 0m month(vector(1456790399)) + day_of_month(vector(1456790399)) / 100 + {} 2.29 + +# 2016-03-01 00:00:00 March 1st in leap year. +eval instant at 0m month(vector(1456790400)) + day_of_month(vector(1456790400)) / 100 + {} 3.01 + +# February 1st 2016 in leap year. +eval instant at 0m days_in_month(vector(1454284800)) + {} 29 + +# February 1st 2017 not in leap year. +eval instant at 0m days_in_month(vector(1485907200)) + {} 28 + +clear + +# Test duplicate labelset in promql output. +load 5m + testmetric1{src="a",dst="b"} 0 + testmetric2{src="a",dst="b"} 1 + +# FAILING. eval_fail instant at 0m changes({__name__=~'testmetric1|testmetric2'}[5m]) + +# Tests for *_over_time +clear + +load 10s + data{type="numbers"} 2 0 3 + data{type="some_nan"} 2 0 NaN + data{type="some_nan2"} 2 NaN 1 + data{type="some_nan3"} NaN 0 1 + data{type="only_nan"} NaN NaN NaN + +# Failing with keepNaN feature. eval instant at 1m min_over_time(data[1m]) +# {type="numbers"} 0 +# {type="some_nan"} 0 +# {type="some_nan2"} 1 +# {type="some_nan3"} 0 +# {type="only_nan"} NaN + +# Failing with keepNaN feature. eval instant at 1m max_over_time(data[1m]) +# {type="numbers"} 3 +# {type="some_nan"} 2 +# {type="some_nan2"} 2 +# {type="some_nan3"} 1 +# {type="only_nan"} NaN + +clear + +# FAILING. Testdata for absent_over_time() +#eval instant at 1m absent_over_time(http_requests[5m]) +# {} 1 + +# FAILING. eval instant at 1m absent_over_time(http_requests{handler="/foo"}[5m]) +# {handler="/foo"} 1 + +# FAILING. eval instant at 1m absent_over_time(http_requests{handler!="/foo"}[5m]) +# {} 1 + +# FAILING. eval instant at 1m absent_over_time(http_requests{handler="/foo", handler="/bar", handler="/foobar"}[5m]) +# {} 1 + +# FAILING. eval instant at 1m absent_over_time(rate(nonexistant[5m])[5m:]) +# {} 1 + +# FAILING. eval instant at 1m absent_over_time(http_requests{handler="/foo", handler="/bar", instance="127.0.0.1"}[5m]) +# {instance="127.0.0.1"} 1 + +load 1m + http_requests{path="/foo",instance="127.0.0.1",job="httpd"} 1+1x10 + http_requests{path="/bar",instance="127.0.0.1",job="httpd"} 1+1x10 + httpd_handshake_failures_total{instance="127.0.0.1",job="node"} 1+1x15 + httpd_log_lines_total{instance="127.0.0.1",job="node"} 1 + ssl_certificate_expiry_seconds{job="ingress"} NaN NaN NaN NaN NaN + +# FAILING. eval instant at 5m absent_over_time(http_requests[5m]) + +# FAILING. eval instant at 5m absent_over_time(rate(http_requests[5m])[5m:1m]) + +# FAILING. eval instant at 0m absent_over_time(httpd_log_lines_total[30s]) + +# FAILING. eval instant at 1m absent_over_time(httpd_log_lines_total[30s]) +# {} 1 + +# FAILING. eval instant at 15m absent_over_time(http_requests[5m]) + +# FAILING. eval instant at 16m absent_over_time(http_requests[5m]) +# {} 1 + +# FAILING. eval instant at 16m absent_over_time(http_requests[6m]) + +# FAILING. eval instant at 16m absent_over_time(httpd_handshake_failures_total[1m]) + +# FAILING. eval instant at 16m absent_over_time({instance="127.0.0.1"}[5m]) + +# FAILING. eval instant at 16m absent_over_time({instance="127.0.0.1"}[5m]) + +# FAILING. eval instant at 21m absent_over_time({instance="127.0.0.1"}[5m]) +# FAILING. {instance="127.0.0.1"} 1 + +# FAILING. eval instant at 21m absent_over_time({instance="127.0.0.1"}[20m]) + +# FAILING. eval instant at 21m absent_over_time({job="grok"}[20m]) +# FAILING. {job="grok"} 1 + +# FAILING. eval instant at 30m absent_over_time({instance="127.0.0.1"}[5m:5s]) +# FAILING. {} 1 + +# FAILING. eval instant at 5m absent_over_time({job="ingress"}[4m]) + +# FAILING. eval instant at 10m absent_over_time({job="ingress"}[4m]) +# FAILING. {job="ingress"} 1 diff --git a/src/query/test/testdata/histograms.test b/src/query/test/testdata/histograms.test new file mode 100644 index 0000000000..88fe483e64 --- /dev/null +++ b/src/query/test/testdata/histograms.test @@ -0,0 +1,181 @@ +# Two histograms with 4 buckets each (x_sum and x_count not included, +# only buckets). Lowest bucket for one histogram < 0, for the other > +# 0. They have the same name, just separated by label. Not useful in +# practice, but can happen (if clients change bucketing), and the +# server has to cope with it. + +# Test histogram. +load 5m + testhistogram_bucket{le="0.1", start="positive"} 0+5x10 + testhistogram_bucket{le=".2", start="positive"} 0+7x10 + testhistogram_bucket{le="1e0", start="positive"} 0+11x10 + testhistogram_bucket{le="+Inf", start="positive"} 0+12x10 + testhistogram_bucket{le="-.2", start="negative"} 0+1x10 + testhistogram_bucket{le="-0.1", start="negative"} 0+2x10 + testhistogram_bucket{le="0.3", start="negative"} 0+2x10 + testhistogram_bucket{le="+Inf", start="negative"} 0+3x10 + + +# Now a more realistic histogram per job and instance to test aggregation. +load 5m + request_duration_seconds_bucket{job="job1", instance="ins1", le="0.1"} 0+1x10 + request_duration_seconds_bucket{job="job1", instance="ins1", le="0.2"} 0+3x10 + request_duration_seconds_bucket{job="job1", instance="ins1", le="+Inf"} 0+4x10 + request_duration_seconds_bucket{job="job1", instance="ins2", le="0.1"} 0+2x10 + request_duration_seconds_bucket{job="job1", instance="ins2", le="0.2"} 0+5x10 + request_duration_seconds_bucket{job="job1", instance="ins2", le="+Inf"} 0+6x10 + request_duration_seconds_bucket{job="job2", instance="ins1", le="0.1"} 0+3x10 + request_duration_seconds_bucket{job="job2", instance="ins1", le="0.2"} 0+4x10 + request_duration_seconds_bucket{job="job2", instance="ins1", le="+Inf"} 0+6x10 + request_duration_seconds_bucket{job="job2", instance="ins2", le="0.1"} 0+4x10 + request_duration_seconds_bucket{job="job2", instance="ins2", le="0.2"} 0+7x10 + request_duration_seconds_bucket{job="job2", instance="ins2", le="+Inf"} 0+9x10 + +# Different le representations in one histogram. +load 5m + mixed_bucket{job="job1", instance="ins1", le="0.1"} 0+1x10 + mixed_bucket{job="job1", instance="ins1", le="0.2"} 0+1x10 + mixed_bucket{job="job1", instance="ins1", le="2e-1"} 0+1x10 + mixed_bucket{job="job1", instance="ins1", le="2.0e-1"} 0+1x10 + mixed_bucket{job="job1", instance="ins1", le="+Inf"} 0+4x10 + mixed_bucket{job="job1", instance="ins2", le="+inf"} 0+0x10 + mixed_bucket{job="job1", instance="ins2", le="+Inf"} 0+0x10 + +# Quantile too low. +eval instant at 50m histogram_quantile(-0.1, testhistogram_bucket) + {start="positive"} -Inf + {start="negative"} -Inf + +# Quantile too high. +eval instant at 50m histogram_quantile(1.01, testhistogram_bucket) + {start="positive"} +Inf + {start="negative"} +Inf + +# Quantile value in lowest bucket, which is positive. +eval instant at 50m histogram_quantile(0, testhistogram_bucket{start="positive"}) + {start="positive"} 0 + +# Quantile value in lowest bucket, which is negative. +eval instant at 50m histogram_quantile(0, testhistogram_bucket{start="negative"}) + {start="negative"} -0.2 + +# Quantile value in highest bucket. +eval instant at 50m histogram_quantile(1, testhistogram_bucket) + {start="positive"} 1 + {start="negative"} 0.3 + +# Finally some useful quantiles. +eval instant at 50m histogram_quantile(0.2, testhistogram_bucket) + {start="positive"} 0.048 + {start="negative"} -0.2 + + +eval instant at 50m histogram_quantile(0.5, testhistogram_bucket) + {start="positive"} 0.15 + {start="negative"} -0.15 + +eval instant at 50m histogram_quantile(0.8, testhistogram_bucket) + {start="positive"} 0.72 + {start="negative"} 0.3 + +# More realistic with rates. +eval instant at 50m histogram_quantile(0.2, rate(testhistogram_bucket[5m])) + {start="positive"} 0.048 + {start="negative"} -0.2 + +eval instant at 50m histogram_quantile(0.5, rate(testhistogram_bucket[5m])) + {start="positive"} 0.15 + {start="negative"} -0.15 + +eval instant at 50m histogram_quantile(0.8, rate(testhistogram_bucket[5m])) + {start="positive"} 0.72 + {start="negative"} 0.3 + +# Aggregated histogram: Everything in one. +eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le)) + {} 0.075 + +eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le)) + {} 0.1277777777777778 + +# Aggregated histogram: Everything in one. Now with avg, which does not change anything. +eval instant at 50m histogram_quantile(0.3, avg(rate(request_duration_seconds_bucket[5m])) by (le)) + {} 0.075 + +eval instant at 50m histogram_quantile(0.5, avg(rate(request_duration_seconds_bucket[5m])) by (le)) + {} 0.12777777777777778 + +# Aggregated histogram: By job. +eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance)) + {instance="ins1"} 0.075 + {instance="ins2"} 0.075 + +eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance)) + {instance="ins1"} 0.1333333333 + {instance="ins2"} 0.125 + +# Aggregated histogram: By instance. +eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job)) + {job="job1"} 0.1 + {job="job2"} 0.0642857142857143 + +eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job)) + {job="job1"} 0.14 + {job="job2"} 0.1125 + +# Aggregated histogram: By job and instance. +eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance)) + {instance="ins1", job="job1"} 0.11 + {instance="ins2", job="job1"} 0.09 + {instance="ins1", job="job2"} 0.06 + {instance="ins2", job="job2"} 0.0675 + +eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance)) + {instance="ins1", job="job1"} 0.15 + {instance="ins2", job="job1"} 0.1333333333333333 + {instance="ins1", job="job2"} 0.1 + {instance="ins2", job="job2"} 0.1166666666666667 + +# The unaggregated histogram for comparison. Same result as the previous one. +eval instant at 50m histogram_quantile(0.3, rate(request_duration_seconds_bucket[5m])) + {instance="ins1", job="job1"} 0.11 + {instance="ins2", job="job1"} 0.09 + {instance="ins1", job="job2"} 0.06 + {instance="ins2", job="job2"} 0.0675 + +eval instant at 50m histogram_quantile(0.5, rate(request_duration_seconds_bucket[5m])) + {instance="ins1", job="job1"} 0.15 + {instance="ins2", job="job1"} 0.13333333333333333 + {instance="ins1", job="job2"} 0.1 + {instance="ins2", job="job2"} 0.11666666666666667 + +# A histogram with nonmonotonic bucket counts. This may happen when recording +# rule evaluation or federation races scrape ingestion, causing some buckets +# counts to be derived from fewer samples. The wrong answer we want to avoid +# is for histogram_quantile(0.99, nonmonotonic_bucket) to return ~1000 instead +# of 1. + +load 5m + nonmonotonic_bucket{le="0.1"} 0+1x10 + nonmonotonic_bucket{le="1"} 0+9x10 + nonmonotonic_bucket{le="10"} 0+8x10 + nonmonotonic_bucket{le="100"} 0+8x10 + nonmonotonic_bucket{le="1000"} 0+9x10 + nonmonotonic_bucket{le="+Inf"} 0+9x10 + +# Nonmonotonic buckets +# FAILING issue #48. eval instant at 50m histogram_quantile(0.99, nonmonotonic_bucket) +# {} 0.989875 + +# FAILING issue #48. Buckets with different representations of the same upper bound. +# eval instant at 50m histogram_quantile(0.5, rate(mixed_bucket[5m])) +# {instance="ins1", job="job1"} 0.15 +# {instance="ins2", job="job1"} NaN + +# Failing with keepNaN feature. eval instant at 50m histogram_quantile(0.75, rate(mixed_bucket[5m])) +# {instance="ins1", job="job1"} 0.2 +# {instance="ins2", job="job1"} NaN + +# Failing with keepNaN feature. eval instant at 50m histogram_quantile(1, rate(mixed_bucket[5m])) +# {instance="ins1", job="job1"} 0.2 +# {instance="ins2", job="job1"} NaN diff --git a/src/query/test/testdata/legacy.test b/src/query/test/testdata/legacy.test new file mode 100644 index 0000000000..b60a352ed1 --- /dev/null +++ b/src/query/test/testdata/legacy.test @@ -0,0 +1,391 @@ +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+10x10 + http_requests{job="api-server", instance="1", group="production"} 0+20x10 + http_requests{job="api-server", instance="0", group="canary"} 0+30x10 + http_requests{job="api-server", instance="1", group="canary"} 0+40x10 + http_requests{job="app-server", instance="0", group="production"} 0+50x10 + http_requests{job="app-server", instance="1", group="production"} 0+60x10 + http_requests{job="app-server", instance="0", group="canary"} 0+70x10 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + +load 5m + x{y="testvalue"} 0+10x10 + +load 5m + testcounter_reset_middle 0+10x4 0+10x5 + testcounter_reset_end 0+10x9 0 10 + +load 4m + testcounter_zero_cutoff{start="0m"} 0+240x10 + testcounter_zero_cutoff{start="1m"} 60+240x10 + testcounter_zero_cutoff{start="2m"} 120+240x10 + testcounter_zero_cutoff{start="3m"} 180+240x10 + testcounter_zero_cutoff{start="4m"} 240+240x10 + testcounter_zero_cutoff{start="5m"} 300+240x10 + +load 5m + label_grouping_test{a="aa", b="bb"} 0+10x10 + label_grouping_test{a="a", b="abb"} 0+20x10 + +load 5m + vector_matching_a{l="x"} 0+1x100 + vector_matching_a{l="y"} 0+2x50 + vector_matching_b{l="x"} 0+4x25 + +load 5m + cpu_count{instance="0", type="numa"} 0+30x10 + cpu_count{instance="0", type="smp"} 0+10x20 + cpu_count{instance="1", type="smp"} 0+20x10 + + +eval instant at 50m SUM(http_requests) + {} 3600 + +eval instant at 50m SUM(http_requests{instance="0"}) BY(job) + {job="api-server"} 400 + {job="app-server"} 1200 + +eval instant at 50m SUM(http_requests) BY (job) + {job="api-server"} 1000 + {job="app-server"} 2600 + +# Non-existent labels mentioned in BY-clauses shouldn't propagate to output. +eval instant at 50m SUM(http_requests) BY (job, nonexistent) + {job="api-server"} 1000 + {job="app-server"} 2600 + + +eval instant at 50m COUNT(http_requests) BY (job) + {job="api-server"} 4 + {job="app-server"} 4 + + +eval instant at 50m SUM(http_requests) BY (job, group) + {group="canary", job="api-server"} 700 + {group="canary", job="app-server"} 1500 + {group="production", job="api-server"} 300 + {group="production", job="app-server"} 1100 + + +eval instant at 50m AVG(http_requests) BY (job) + {job="api-server"} 250 + {job="app-server"} 650 + + +eval instant at 50m MIN(http_requests) BY (job) + {job="api-server"} 100 + {job="app-server"} 500 + + +eval instant at 50m MAX(http_requests) BY (job) + {job="api-server"} 400 + {job="app-server"} 800 + + +# Single-letter label names and values. +eval instant at 50m x{y="testvalue"} + x{y="testvalue"} 100 + + +# Rates should calculate per-second rates. +eval instant at 50m rate(http_requests{group="canary", instance="1", job="app-server"}[50m]) + {group="canary", instance="1", job="app-server"} 0.26666666666666666 + + +# Counter resets at in the middle of range are handled correctly by rate(). +eval instant at 50m rate(testcounter_reset_middle[50m]) + {} 0.03 + + +# Counter resets at end of range are ignored by rate(). +eval instant at 50m rate(testcounter_reset_end[5m]) + {} 0 + + +# Zero cutoff for left-side extrapolation. +eval instant at 10m rate(testcounter_zero_cutoff[20m]) + {start="0m"} 0.5 + {start="1m"} 0.55 + {start="2m"} 0.6 + {start="3m"} 0.65 + {start="4m"} 0.7 + {start="5m"} 0.6 + +# Normal half-interval cutoff for left-side extrapolation. +eval instant at 50m rate(testcounter_zero_cutoff[20m]) + {start="0m"} 0.6 + {start="1m"} 0.6 + {start="2m"} 0.6 + {start="3m"} 0.6 + {start="4m"} 0.6 + {start="5m"} 0.6 + + +eval instant at 50m http_requests{group!="canary"} + http_requests{group="production", instance="1", job="app-server"} 600 + http_requests{group="production", instance="0", job="app-server"} 500 + http_requests{group="production", instance="1", job="api-server"} 200 + http_requests{group="production", instance="0", job="api-server"} 100 + +eval instant at 50m http_requests{job=~".+-server",group!="canary"} + http_requests{group="production", instance="1", job="app-server"} 600 + http_requests{group="production", instance="0", job="app-server"} 500 + http_requests{group="production", instance="1", job="api-server"} 200 + http_requests{group="production", instance="0", job="api-server"} 100 + +eval instant at 50m http_requests{job!~"api-.+",group!="canary"} + http_requests{group="production", instance="1", job="app-server"} 600 + http_requests{group="production", instance="0", job="app-server"} 500 + +eval instant at 50m http_requests{group="production",job=~"api-.+"} + http_requests{group="production", instance="0", job="api-server"} 100 + http_requests{group="production", instance="1", job="api-server"} 200 + +eval instant at 50m abs(-1 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 100 + {group="production", instance="1", job="api-server"} 200 + +eval instant at 50m floor(0.004 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 0 + {group="production", instance="1", job="api-server"} 0 + +eval instant at 50m ceil(0.004 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 1 + {group="production", instance="1", job="api-server"} 1 + +eval instant at 50m round(0.004 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 0 + {group="production", instance="1", job="api-server"} 1 + +# Round should correctly handle negative numbers. +eval instant at 50m round(-1 * (0.004 * http_requests{group="production",job="api-server"})) + {group="production", instance="0", job="api-server"} 0 + {group="production", instance="1", job="api-server"} -1 + +# Round should round half up. +eval instant at 50m round(0.005 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 1 + {group="production", instance="1", job="api-server"} 1 + +eval instant at 50m round(-1 * (0.005 * http_requests{group="production",job="api-server"})) + {group="production", instance="0", job="api-server"} 0 + {group="production", instance="1", job="api-server"} -1 + +eval instant at 50m round(1 + 0.005 * http_requests{group="production",job="api-server"}) + {group="production", instance="0", job="api-server"} 2 + {group="production", instance="1", job="api-server"} 2 + +eval instant at 50m round(-1 * (1 + 0.005 * http_requests{group="production",job="api-server"})) + {group="production", instance="0", job="api-server"} -1 + {group="production", instance="1", job="api-server"} -2 + +# Round should accept the number to round nearest to. +eval instant at 50m round(0.0005 * http_requests{group="production",job="api-server"}, 0.1) + {group="production", instance="0", job="api-server"} 0.1 + {group="production", instance="1", job="api-server"} 0.1 + +eval instant at 50m round(2.1 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1) + {group="production", instance="0", job="api-server"} 2.2 + {group="production", instance="1", job="api-server"} 2.2 + +eval instant at 50m round(5.2 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1) + {group="production", instance="0", job="api-server"} 5.3 + {group="production", instance="1", job="api-server"} 5.3 + +# Round should work correctly with negative numbers and multiple decimal places. +eval instant at 50m round(-1 * (5.2 + 0.0005 * http_requests{group="production",job="api-server"}), 0.1) + {group="production", instance="0", job="api-server"} -5.2 + {group="production", instance="1", job="api-server"} -5.3 + +# Round should work correctly with big toNearests. +eval instant at 50m round(0.025 * http_requests{group="production",job="api-server"}, 5) + {group="production", instance="0", job="api-server"} 5 + {group="production", instance="1", job="api-server"} 5 + +eval instant at 50m round(0.045 * http_requests{group="production",job="api-server"}, 5) + {group="production", instance="0", job="api-server"} 5 + {group="production", instance="1", job="api-server"} 10 + +eval instant at 50m avg_over_time(http_requests{group="production",job="api-server"}[1h]) + {group="production", instance="0", job="api-server"} 50 + {group="production", instance="1", job="api-server"} 100 + +eval instant at 50m count_over_time(http_requests{group="production",job="api-server"}[1h]) + {group="production", instance="0", job="api-server"} 11 + {group="production", instance="1", job="api-server"} 11 + +eval instant at 50m max_over_time(http_requests{group="production",job="api-server"}[1h]) + {group="production", instance="0", job="api-server"} 100 + {group="production", instance="1", job="api-server"} 200 + +eval instant at 50m min_over_time(http_requests{group="production",job="api-server"}[1h]) + {group="production", instance="0", job="api-server"} 0 + {group="production", instance="1", job="api-server"} 0 + +eval instant at 50m sum_over_time(http_requests{group="production",job="api-server"}[1h]) + {group="production", instance="0", job="api-server"} 550 + {group="production", instance="1", job="api-server"} 1100 + +# FAILING. eval instant at 50m time() +# 3000 + +# FAILING. eval instant at 50m {__name__=~".+"} +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="1", job="app-server"} 600 +# testcounter_reset_end 0 +# testcounter_reset_middle 50 +# x{y="testvalue"} 100 +# label_grouping_test{a="a", b="abb"} 200 +# label_grouping_test{a="aa", b="bb"} 100 +# vector_matching_a{l="x"} 10 +# vector_matching_a{l="y"} 20 +# vector_matching_b{l="x"} 40 +# cpu_count{instance="1", type="smp"} 200 +# cpu_count{instance="0", type="smp"} 100 +# cpu_count{instance="0", type="numa"} 300 + + +# FAILING. eval instant at 50m {job=~".+-server", job!~"api-.+"} +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="app-server"} 600 + +# FAILING. eval instant at 50m absent(nonexistent) +# {} 1 + +# FAILING. eval instant at 50m absent(nonexistent{job="testjob", instance="testinstance", method=~".x"}) +# {instance="testinstance", job="testjob"} 1 + +# FAILING. eval instant at 50m absent(nonexistent{job="testjob",job="testjob2",foo="bar"}) +# {foo="bar"} 1 + +# FAILING. eval instant at 50m absent(nonexistent{job="testjob",job="testjob2",job="three",foo="bar"}) +# {foo="bar"} 1 + +# FAILING. eval instant at 50m absent(nonexistent{job="testjob",job=~"testjob2",foo="bar"}) +# {foo="bar"} 1 + +eval instant at 50m absent(http_requests) + +eval instant at 50m absent(sum(http_requests)) + +# FAILING. eval instant at 50m absent(sum(nonexistent{job="testjob", instance="testinstance"})) +# {} 1 + +# FAILING. eval instant at 50m absent(max(nonexistant)) +# {} 1 + +# FAILING. eval instant at 50m absent(nonexistant > 1) +# {} 1 + +# FAILING. eval instant at 50m absent(a + b) +# {} 1 + +# FAILING. eval instant at 50m absent(a and b) +# {} 1 + +# FAILING. eval instant at 50m absent(rate(nonexistant[5m])) +# {} 1 + +eval instant at 50m http_requests{group="production",job="api-server"} offset 5m + http_requests{group="production", instance="0", job="api-server"} 90 + http_requests{group="production", instance="1", job="api-server"} 180 + +eval instant at 50m rate(http_requests{group="production",job="api-server"}[10m] offset 5m) + {group="production", instance="0", job="api-server"} 0.03333333333333333 + {group="production", instance="1", job="api-server"} 0.06666666666666667 + +eval instant at 50m http_requests{group="canary", instance="0", job="api-server"} / 0 + {group="canary", instance="0", job="api-server"} +Inf + +eval instant at 50m -1 * http_requests{group="canary", instance="0", job="api-server"} / 0 + {group="canary", instance="0", job="api-server"} -Inf + +# Failing with keepNaN feature. eval instant at 50m 0 * http_requests{group="canary", instance="0", job="api-server"} / 0 +# {group="canary", instance="0", job="api-server"} NaN + +# Failing with keepNaN feature. eval instant at 50m 0 * http_requests{group="canary", instance="0", job="api-server"} % 0 +# {group="canary", instance="0", job="api-server"} NaN + +# FAILING. eval instant at 50m exp(vector_matching_a) +# {l="x"} 22026.465794806718 +# {l="y"} 485165195.4097903 + +eval instant at 50m exp(vector_matching_a - 10) + {l="y"} 22026.465794806718 + {l="x"} 1 + +eval instant at 50m exp(vector_matching_a - 20) + {l="x"} 4.5399929762484854e-05 + {l="y"} 1 + +# FAILING. eval instant at 50m ln(vector_matching_a) +# {l="x"} 2.302585092994046 +# {l="y"} 2.995732273553991 + +# FAILING. eval instant at 50m ln(vector_matching_a - 10) +# {l="y"} 2.302585092994046 +# {l="x"} -Inf + +# FAILING. eval instant at 50m ln(vector_matching_a - 20) +# {l="y"} -Inf +# {l="x"} NaN + +# FAILING. eval instant at 50m exp(ln(vector_matching_a)) +# {l="y"} 20 +# {l="x"} 10 + +# FAILING. eval instant at 50m sqrt(vector_matching_a) +# {l="x"} 3.1622776601683795 +# {l="y"} 4.47213595499958 + +# FAILING. eval instant at 50m log2(vector_matching_a) +# {l="x"} 3.3219280948873626 +# {l="y"} 4.321928094887363 + +# FAILING. eval instant at 50m log2(vector_matching_a - 10) +# {l="y"} 3.3219280948873626 +# {l="x"} -Inf + +# FAILING. eval instant at 50m log2(vector_matching_a - 20) +# {l="x"} NaN +# {l="y"} -Inf + +# FAILING. eval instant at 50m log10(vector_matching_a) +# {l="x"} 1 +# {l="y"} 1.301029995663981 + +# FAILING. eval instant at 50m log10(vector_matching_a - 10) +# {l="y"} 1 +# {l="x"} -Inf + +# FAILING. eval instant at 50m log10(vector_matching_a - 20) +# {l="x"} NaN +# {l="y"} -Inf + + +# Matrix tests. +clear +load 1h + testmetric{aa="bb"} 1 + testmetric{a="abb"} 2 + +eval instant at 0h testmetric + testmetric{aa="bb"} 1 + testmetric{a="abb"} 2 + +clear + +# Test duplicate labelset in promql output. +load 5m + testmetric1{src="a",dst="b"} 0 + testmetric2{src="a",dst="b"} 1 + +# FAILING. eval_fail instant at 0m ceil({__name__=~'testmetric1|testmetric2'}) diff --git a/src/query/test/testdata/literals.test b/src/query/test/testdata/literals.test new file mode 100644 index 0000000000..344e2a808a --- /dev/null +++ b/src/query/test/testdata/literals.test @@ -0,0 +1,61 @@ +# FAILING issue #46 (we return vector resultType in place of scalar). + +# FAILING. eval instant at 50m 12.34e6 +# 12340000 + +# FAILING. eval instant at 50m 12.34e+6 +# 12340000 + +# FAILING. eval instant at 50m 12.34e-6 +# 0.00001234 + +# FAILING. eval instant at 50m 1+1 +# 2 + +# FAILING. eval instant at 50m 1-1 +# 0 + +# FAILING. eval instant at 50m 1 - -1 +# 2 + +# FAILING. eval instant at 50m .2 +# 0.2 + +# FAILING. eval instant at 50m +0.2 +# 0.2 + +# FAILING. eval instant at 50m -0.2e-6 +# -0.0000002 + +# FAILING. eval instant at 50m +Inf +# +Inf + +# FAILING. eval instant at 50m inF +# +Inf + +# FAILING. eval instant at 50m -inf +# -Inf + +# FAILING. eval instant at 50m NaN +# NaN + +# FAILING. eval instant at 50m nan +# NaN + +# FAILING. eval instant at 50m 2. +# 2 + +# FAILING. eval instant at 50m 1 / 0 +# +Inf + +# FAILING. eval instant at 50m ((1) / (0)) +# +Inf + +# FAILING. eval instant at 50m -1 / 0 +# -Inf + +# FAILING. eval instant at 50m 0 / 0 +# NaN + +# FAILING. eval instant at 50m 1 % 0 +# NaN diff --git a/src/query/test/testdata/operators.test b/src/query/test/testdata/operators.test new file mode 100644 index 0000000000..a0ecb8779e --- /dev/null +++ b/src/query/test/testdata/operators.test @@ -0,0 +1,440 @@ +load 5m + http_requests{job="api-server", instance="0", group="production"} 0+10x10 + http_requests{job="api-server", instance="1", group="production"} 0+20x10 + http_requests{job="api-server", instance="0", group="canary"} 0+30x10 + http_requests{job="api-server", instance="1", group="canary"} 0+40x10 + http_requests{job="app-server", instance="0", group="production"} 0+50x10 + http_requests{job="app-server", instance="1", group="production"} 0+60x10 + http_requests{job="app-server", instance="0", group="canary"} 0+70x10 + http_requests{job="app-server", instance="1", group="canary"} 0+80x10 + +load 5m + vector_matching_a{l="x"} 0+1x100 + vector_matching_a{l="y"} 0+2x50 + vector_matching_b{l="x"} 0+4x25 + + +eval instant at 50m SUM(http_requests) BY (job) - COUNT(http_requests) BY (job) + {job="api-server"} 996 + {job="app-server"} 2596 + +eval instant at 50m 2 - SUM(http_requests) BY (job) + {job="api-server"} -998 + {job="app-server"} -2598 + +# FAILING. eval instant at 50m -http_requests{job="api-server",instance="0",group="production"} +# {job="api-server",instance="0",group="production"} -100 + +eval instant at 50m +http_requests{job="api-server",instance="0",group="production"} + http_requests{job="api-server",instance="0",group="production"} 100 + +eval instant at 50m - - - SUM(http_requests) BY (job) + {job="api-server"} -1000 + {job="app-server"} -2600 + +# FAILING. eval instant at 50m - - - 1 +# -1 + +# FAILING. eval instant at 50m -2^---1*3 +# -1.5 + +# FAILING. eval instant at 50m 2/-2^---1*3+2 +# -10 + +# FAILING. eval instant at 50m -10^3 * - SUM(http_requests) BY (job) ^ -1 +# {job="api-server"} 1 +# {job="app-server"} 0.38461538461538464 + +eval instant at 50m 1000 / SUM(http_requests) BY (job) + {job="api-server"} 1 + {job="app-server"} 0.38461538461538464 + +eval instant at 50m SUM(http_requests) BY (job) - 2 + {job="api-server"} 998 + {job="app-server"} 2598 + +eval instant at 50m SUM(http_requests) BY (job) % 3 + {job="api-server"} 1 + {job="app-server"} 2 + +eval instant at 50m SUM(http_requests) BY (job) % 0.3 + {job="api-server"} 0.1 + {job="app-server"} 0.2 + +eval instant at 50m SUM(http_requests) BY (job) ^ 2 + {job="api-server"} 1000000 + {job="app-server"} 6760000 + +eval instant at 50m SUM(http_requests) BY (job) % 3 ^ 2 + {job="api-server"} 1 + {job="app-server"} 8 + +eval instant at 50m SUM(http_requests) BY (job) % 2 ^ (3 ^ 2) + {job="api-server"} 488 + {job="app-server"} 40 + +eval instant at 50m SUM(http_requests) BY (job) % 2 ^ 3 ^ 2 + {job="api-server"} 488 + {job="app-server"} 40 + +eval instant at 50m SUM(http_requests) BY (job) % 2 ^ 3 ^ 2 ^ 2 + {job="api-server"} 1000 + {job="app-server"} 2600 + +eval instant at 50m COUNT(http_requests) BY (job) ^ COUNT(http_requests) BY (job) + {job="api-server"} 256 + {job="app-server"} 256 + +eval instant at 50m SUM(http_requests) BY (job) / 0 + {job="api-server"} +Inf + {job="app-server"} +Inf + +eval instant at 50m SUM(http_requests) BY (job) + SUM(http_requests) BY (job) + {job="api-server"} 2000 + {job="app-server"} 5200 + +eval instant at 50m (SUM((http_requests)) BY (job)) + SUM(http_requests) BY (job) + {job="api-server"} 2000 + {job="app-server"} 5200 + +eval instant at 50m http_requests{job="api-server", group="canary"} + http_requests{group="canary", instance="0", job="api-server"} 300 + http_requests{group="canary", instance="1", job="api-server"} 400 + +eval instant at 50m http_requests{job="api-server", group="canary"} + rate(http_requests{job="api-server"}[5m]) * 5 * 60 + {group="canary", instance="0", job="api-server"} 330 + {group="canary", instance="1", job="api-server"} 440 + +eval instant at 50m rate(http_requests[25m]) * 25 * 60 + {group="canary", instance="0", job="api-server"} 150 + {group="canary", instance="0", job="app-server"} 350 + {group="canary", instance="1", job="api-server"} 200 + {group="canary", instance="1", job="app-server"} 400 + {group="production", instance="0", job="api-server"} 50 + {group="production", instance="0", job="app-server"} 249.99999999999997 + {group="production", instance="1", job="api-server"} 100 + {group="production", instance="1", job="app-server"} 300 + +# FAILING issue #30. eval instant at 50m (rate((http_requests[25m])) * 25) * 60 +# {group="canary", instance="0", job="api-server"} 150 +# {group="canary", instance="0", job="app-server"} 350 +# {group="canary", instance="1", job="api-server"} 200 +# {group="canary", instance="1", job="app-server"} 400 +# {group="production", instance="0", job="api-server"} 50 +# {group="production", instance="0", job="app-server"} 249.99999999999997 +# {group="production", instance="1", job="api-server"} 100 +# {group="production", instance="1", job="app-server"} 300 + + +# FAILING issue #23. eval instant at 50m http_requests{group="canary"} and http_requests{instance="0"} +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="0", job="app-server"} 700 + +eval instant at 50m (http_requests{group="canary"} + 1) and http_requests{instance="0"} + {group="canary", instance="0", job="api-server"} 301 + {group="canary", instance="0", job="app-server"} 701 + +eval instant at 50m (http_requests{group="canary"} + 1) and on(instance, job) http_requests{instance="0", group="production"} + {group="canary", instance="0", job="api-server"} 301 + {group="canary", instance="0", job="app-server"} 701 + +eval instant at 50m (http_requests{group="canary"} + 1) and on(instance) http_requests{instance="0", group="production"} + {group="canary", instance="0", job="api-server"} 301 + {group="canary", instance="0", job="app-server"} 701 + +eval instant at 50m (http_requests{group="canary"} + 1) and ignoring(group) http_requests{instance="0", group="production"} + {group="canary", instance="0", job="api-server"} 301 + {group="canary", instance="0", job="app-server"} 701 + +eval instant at 50m (http_requests{group="canary"} + 1) and ignoring(group, job) http_requests{instance="0", group="production"} + {group="canary", instance="0", job="api-server"} 301 + {group="canary", instance="0", job="app-server"} 701 + +# FAILING issue #23. eval instant at 50m http_requests{group="canary"} or http_requests{group="production"} +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="1", job="app-server"} 600 + +# On overlap the rhs samples must be dropped. +# FAILING issue 34#. eval instant at 50m (http_requests{group="canary"} + 1) or http_requests{instance="1"} +# {group="canary", instance="0", job="api-server"} 301 +# {group="canary", instance="0", job="app-server"} 701 +# {group="canary", instance="1", job="api-server"} 401 +# {group="canary", instance="1", job="app-server"} 801 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="1", job="app-server"} 600 + + +# Matching only on instance excludes everything that has instance=0/1 but includes +# entries without the instance label. +# FAILING issue 34#. eval instant at 50m (http_requests{group="canary"} + 1) or on(instance) (http_requests or cpu_count or vector_matching_a) +# {group="canary", instance="0", job="api-server"} 301 +# {group="canary", instance="0", job="app-server"} 701 +# {group="canary", instance="1", job="api-server"} 401 +# {group="canary", instance="1", job="app-server"} 801 +# vector_matching_a{l="x"} 10 +# vector_matching_a{l="y"} 20 + +# FAILING issue 34#. eval instant at 50m (http_requests{group="canary"} + 1) or ignoring(l, group, job) (http_requests or cpu_count or vector_matching_a) +# {group="canary", instance="0", job="api-server"} 301 +# {group="canary", instance="0", job="app-server"} 701 +# {group="canary", instance="1", job="api-server"} 401 +# {group="canary", instance="1", job="app-server"} 801 +# vector_matching_a{l="x"} 10 +# vector_matching_a{l="y"} 20 + +# FAILING issue 34#. eval instant at 50m http_requests{group="canary"} unless http_requests{instance="0"} +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +# FAILING issue #35. eval instant at 50m http_requests{group="canary"} unless on(job) http_requests{instance="0"} + +# FAILING issue #34. eval instant at 50m http_requests{group="canary"} unless on(job, instance) http_requests{instance="0"} +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +# FAILING issue #36. eval instant at 50m http_requests{group="canary"} / on(instance,job) http_requests{group="production"} +# {instance="0", job="api-server"} 3 +# {instance="0", job="app-server"} 1.4 +# {instance="1", job="api-server"} 2 +# {instance="1", job="app-server"} 1.3333333333333333 + +# FAILING issue #35. eval instant at 50m http_requests{group="canary"} unless ignoring(group, instance) http_requests{instance="0"} + +# FAILING. eval instant at 50m http_requests{group="canary"} unless ignoring(group) http_requests{instance="0"} +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 + +# FAILING. eval instant at 50m http_requests{group="canary"} / ignoring(group) http_requests{group="production"} +# {instance="0", job="api-server"} 3 +# {instance="0", job="app-server"} 1.4 +# {instance="1", job="api-server"} 2 +# {instance="1", job="app-server"} 1.3333333333333333 + +# https://github.com/prometheus/prometheus/issues/1489 +# FAILING. eval instant at 50m http_requests AND ON (dummy) vector(1) +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="1", job="app-server"} 600 + +# FAILING. eval instant at 50m http_requests AND IGNORING (group, instance, job) vector(1) +# http_requests{group="canary", instance="0", job="api-server"} 300 +# http_requests{group="canary", instance="0", job="app-server"} 700 +# http_requests{group="canary", instance="1", job="api-server"} 400 +# http_requests{group="canary", instance="1", job="app-server"} 800 +# http_requests{group="production", instance="0", job="api-server"} 100 +# http_requests{group="production", instance="0", job="app-server"} 500 +# http_requests{group="production", instance="1", job="api-server"} 200 +# http_requests{group="production", instance="1", job="app-server"} 600 + + +# Comparisons. +# FAILING issue #37. eval instant at 50m SUM(http_requests) BY (job) > 1000 +# {job="app-server"} 2600 + +# FAILING issue #37. eval instant at 50m 1000 < SUM(http_requests) BY (job) +# {job="app-server"} 2600 + +# FAILING issue #37. eval instant at 50m SUM(http_requests) BY (job) <= 1000 +# {job="api-server"} 1000 + +# FAILING issue #37. eval instant at 50m SUM(http_requests) BY (job) != 1000 +# {job="app-server"} 2600 + +# FAILING issue #37. eval instant at 50m SUM(http_requests) BY (job) == 1000 +# {job="api-server"} 1000 + +eval instant at 50m SUM(http_requests) BY (job) == bool 1000 + {job="api-server"} 1 + {job="app-server"} 0 + +eval instant at 50m SUM(http_requests) BY (job) == bool SUM(http_requests) BY (job) + {job="api-server"} 1 + {job="app-server"} 1 + +eval instant at 50m SUM(http_requests) BY (job) != bool SUM(http_requests) BY (job) + {job="api-server"} 0 + {job="app-server"} 0 + +# FAILING issue #31. eval instant at 50m 0 == bool 1 +# 0 + +# FAILING issue #31. eval instant at 50m 1 == bool 1 +# 1 + +eval instant at 50m http_requests{job="api-server", instance="0", group="production"} == bool 100 + {job="api-server", instance="0", group="production"} 1 + +# group_left/group_right. + +clear + +load 5m + node_var{instance="abc",job="node"} 2 + node_role{instance="abc",job="node",role="prometheus"} 1 + +load 5m + node_cpu{instance="abc",job="node",mode="idle"} 3 + node_cpu{instance="abc",job="node",mode="user"} 1 + node_cpu{instance="def",job="node",mode="idle"} 8 + node_cpu{instance="def",job="node",mode="user"} 2 + +load 5m + random{foo="bar"} 1 + +load 5m + threshold{instance="abc",job="node",target="a@b.com"} 0 + +# Copy machine role to node variable. +eval instant at 5m node_role * on (instance) group_right (role) node_var + {instance="abc",job="node",role="prometheus"} 2 + +# FAILING. eval instant at 5m node_var * on (instance) group_left (role) node_role +# {instance="abc",job="node",role="prometheus"} 2 + +# FAILING. eval instant at 5m node_var * ignoring (role) group_left (role) node_role +# {instance="abc",job="node",role="prometheus"} 2 + +eval instant at 5m node_role * ignoring (role) group_right (role) node_var + {instance="abc",job="node",role="prometheus"} 2 + +# Copy machine role to node variable with instrumentation labels. +# FAILING. eval instant at 5m node_cpu * ignoring (role, mode) group_left (role) node_role +# {instance="abc",job="node",mode="idle",role="prometheus"} 3 +# {instance="abc",job="node",mode="user",role="prometheus"} 1 + +# FAILING. eval instant at 5m node_cpu * on (instance) group_left (role) node_role +# {instance="abc",job="node",mode="idle",role="prometheus"} 3 +# {instance="abc",job="node",mode="user",role="prometheus"} 1 + + +# Ratio of total. +eval instant at 5m node_cpu / on (instance) group_left sum by (instance,job)(node_cpu) + {instance="abc",job="node",mode="idle"} .75 + {instance="abc",job="node",mode="user"} .25 + {instance="def",job="node",mode="idle"} .80 + {instance="def",job="node",mode="user"} .20 + +eval instant at 5m sum by (mode, job)(node_cpu) / on (job) group_left sum by (job)(node_cpu) + {job="node",mode="idle"} 0.7857142857142857 + {job="node",mode="user"} 0.21428571428571427 + +eval instant at 5m sum(sum by (mode, job)(node_cpu) / on (job) group_left sum by (job)(node_cpu)) + {} 1.0 + + +eval instant at 5m node_cpu / ignoring (mode) group_left sum without (mode)(node_cpu) + {instance="abc",job="node",mode="idle"} .75 + {instance="abc",job="node",mode="user"} .25 + {instance="def",job="node",mode="idle"} .80 + {instance="def",job="node",mode="user"} .20 + +eval instant at 5m node_cpu / ignoring (mode) group_left(dummy) sum without (mode)(node_cpu) + {instance="abc",job="node",mode="idle"} .75 + {instance="abc",job="node",mode="user"} .25 + {instance="def",job="node",mode="idle"} .80 + {instance="def",job="node",mode="user"} .20 + +eval instant at 5m sum without (instance)(node_cpu) / ignoring (mode) group_left sum without (instance, mode)(node_cpu) + {job="node",mode="idle"} 0.7857142857142857 + {job="node",mode="user"} 0.21428571428571427 + +eval instant at 5m sum(sum without (instance)(node_cpu) / ignoring (mode) group_left sum without (instance, mode)(node_cpu)) + {} 1.0 + + +# Copy over label from metric with no matching labels, without having to list cross-job target labels ('job' here). +# FAILING. eval instant at 5m node_cpu + on(dummy) group_left(foo) random*0 +# {instance="abc",job="node",mode="idle",foo="bar"} 3 +# {instance="abc",job="node",mode="user",foo="bar"} 1 +# {instance="def",job="node",mode="idle",foo="bar"} 8 +# {instance="def",job="node",mode="user",foo="bar"} 2 + + +# Use threshold from metric, and copy over target. +# FAILING. eval instant at 5m node_cpu > on(job, instance) group_left(target) threshold +# node_cpu{instance="abc",job="node",mode="idle",target="a@b.com"} 3 +# node_cpu{instance="abc",job="node",mode="user",target="a@b.com"} 1 + +# Use threshold from metric, and a default (1) if it's not present. +# FAILING. eval instant at 5m node_cpu > on(job, instance) group_left(target) (threshold or on (job, instance) (sum by (job, instance)(node_cpu) * 0 + 1)) +# node_cpu{instance="abc",job="node",mode="idle",target="a@b.com"} 3 +# node_cpu{instance="abc",job="node",mode="user",target="a@b.com"} 1 +# node_cpu{instance="def",job="node",mode="idle"} 8 +# node_cpu{instance="def",job="node",mode="user"} 2 + + +# Check that binops drop the metric name. +eval instant at 5m node_cpu + 2 + {instance="abc",job="node",mode="idle"} 5 + {instance="abc",job="node",mode="user"} 3 + {instance="def",job="node",mode="idle"} 10 + {instance="def",job="node",mode="user"} 4 + +eval instant at 5m node_cpu - 2 + {instance="abc",job="node",mode="idle"} 1 + {instance="abc",job="node",mode="user"} -1 + {instance="def",job="node",mode="idle"} 6 + {instance="def",job="node",mode="user"} 0 + +eval instant at 5m node_cpu / 2 + {instance="abc",job="node",mode="idle"} 1.5 + {instance="abc",job="node",mode="user"} 0.5 + {instance="def",job="node",mode="idle"} 4 + {instance="def",job="node",mode="user"} 1 + +eval instant at 5m node_cpu * 2 + {instance="abc",job="node",mode="idle"} 6 + {instance="abc",job="node",mode="user"} 2 + {instance="def",job="node",mode="idle"} 16 + {instance="def",job="node",mode="user"} 4 + +eval instant at 5m node_cpu ^ 2 + {instance="abc",job="node",mode="idle"} 9 + {instance="abc",job="node",mode="user"} 1 + {instance="def",job="node",mode="idle"} 64 + {instance="def",job="node",mode="user"} 4 + +eval instant at 5m node_cpu % 2 + {instance="abc",job="node",mode="idle"} 1 + {instance="abc",job="node",mode="user"} 1 + {instance="def",job="node",mode="idle"} 0 + {instance="def",job="node",mode="user"} 0 + + +clear + +load 5m + random{foo="bar"} 2 + metricA{baz="meh"} 3 + metricB{baz="meh"} 4 + +# On with no labels, for metrics with no common labels. +# FAILING. eval instant at 5m random + on() metricA +# {} 5 + +# Ignoring with no labels is the same as no ignoring. +eval instant at 5m metricA + ignoring() metricB + {baz="meh"} 7 + +eval instant at 5m metricA + metricB + {baz="meh"} 7 + +clear + +# Test duplicate labelset in promql output. +load 5m + testmetric1{src="a",dst="b"} 0 + testmetric2{src="a",dst="b"} 1 + +# FAILING issue #32. eval_fail instant at 0m -{__name__=~'testmetric1|testmetric2'} diff --git a/src/query/test/testdata/selectors.test b/src/query/test/testdata/selectors.test new file mode 100644 index 0000000000..27c5e77f2b --- /dev/null +++ b/src/query/test/testdata/selectors.test @@ -0,0 +1,59 @@ +load 10s + http_requests{job="api-server", instance="0", group="production"} 0+10x1000 100+30x1000 + http_requests{job="api-server", instance="1", group="production"} 0+20x1000 200+30x1000 + http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300+80x1000 + http_requests{job="api-server", instance="1", group="canary"} 0+40x2000 + +eval instant at 8000s rate(http_requests[1m]) + {job="api-server", instance="0", group="production"} 1 + {job="api-server", instance="1", group="production"} 2 + {job="api-server", instance="0", group="canary"} 3 + {job="api-server", instance="1", group="canary"} 4 + +eval instant at 18000s rate(http_requests[1m]) + {job="api-server", instance="0", group="production"} 3 + {job="api-server", instance="1", group="production"} 3 + {job="api-server", instance="0", group="canary"} 8 + {job="api-server", instance="1", group="canary"} 4 + +eval instant at 8000s rate(http_requests{group=~"pro.*"}[1m]) + {job="api-server", instance="0", group="production"} 1 + {job="api-server", instance="1", group="production"} 2 + +eval instant at 18000s rate(http_requests{group=~".*ry", instance="1"}[1m]) + {job="api-server", instance="1", group="canary"} 4 + +eval instant at 18000s rate(http_requests{instance!="3"}[1m] offset 10000s) + {job="api-server", instance="0", group="production"} 1 + {job="api-server", instance="1", group="production"} 2 + {job="api-server", instance="0", group="canary"} 3 + {job="api-server", instance="1", group="canary"} 4 + +eval instant at 18000s rate(http_requests[40s]) - rate(http_requests[1m] offset 10000s) + {job="api-server", instance="0", group="production"} 2 + {job="api-server", instance="1", group="production"} 1 + {job="api-server", instance="0", group="canary"} 5 + {job="api-server", instance="1", group="canary"} 0 + +# https://github.com/prometheus/prometheus/issues/3575 +eval instant at 0s http_requests{foo!="bar"} + http_requests{job="api-server", instance="0", group="production"} 0 + http_requests{job="api-server", instance="1", group="production"} 0 + http_requests{job="api-server", instance="0", group="canary"} 0 + http_requests{job="api-server", instance="1", group="canary"} 0 + +eval instant at 0s http_requests{foo!="bar", job="api-server"} + http_requests{job="api-server", instance="0", group="production"} 0 + http_requests{job="api-server", instance="1", group="production"} 0 + http_requests{job="api-server", instance="0", group="canary"} 0 + http_requests{job="api-server", instance="1", group="canary"} 0 + +eval instant at 0s http_requests{foo!~"bar", job="api-server"} + http_requests{job="api-server", instance="0", group="production"} 0 + http_requests{job="api-server", instance="1", group="production"} 0 + http_requests{job="api-server", instance="0", group="canary"} 0 + http_requests{job="api-server", instance="1", group="canary"} 0 + +eval instant at 0s http_requests{foo!~"bar", job="api-server", instance="1", x!="y", z="", group!=""} + http_requests{job="api-server", instance="1", group="production"} 0 + http_requests{job="api-server", instance="1", group="canary"} 0 diff --git a/src/query/test/testdata/staleness.test b/src/query/test/testdata/staleness.test new file mode 100644 index 0000000000..a950a31dd2 --- /dev/null +++ b/src/query/test/testdata/staleness.test @@ -0,0 +1,51 @@ +load 10s + metric 0 1 stale 2 + +# Instant vector doesn't return series when stale. +eval instant at 10s metric + {__name__="metric"} 1 + +#eval instant at 20s metric + +eval instant at 30s metric + {__name__="metric"} 2 + +eval instant at 40s metric + {__name__="metric"} 2 + +# It goes stale 5 minutes after the last sample. +eval instant at 330s metric + {__name__="metric"} 2 + +#eval instant at 331s metric + + +# Range vector ignores stale sample. +eval instant at 30s count_over_time(metric[1m]) + {} 3 + +eval instant at 10s count_over_time(metric[1s]) + {} 1 + +# FAILING. eval instant at 20s count_over_time(metric[1s]) + +eval instant at 20s count_over_time(metric[10s]) + {} 1 + + +clear + +load 10s + metric 0 + +# Series with single point goes stale after 5 minutes. +eval instant at 0s metric + {__name__="metric"} 0 + +eval instant at 150s metric + {__name__="metric"} 0 + +eval instant at 300s metric + {__name__="metric"} 0 + +# FAILING. eval instant at 301s metric diff --git a/src/query/test/testdata/subquery.test b/src/query/test/testdata/subquery.test new file mode 100644 index 0000000000..376f066087 --- /dev/null +++ b/src/query/test/testdata/subquery.test @@ -0,0 +1,117 @@ +load 10s + metric 1 2 + +# Evaluation before 0s gets no sample. +# FAILING issue #38. eval instant at 10s sum_over_time(metric[50s:10s]) +# {} 3 + +# FAILING issue #38. eval instant at 10s sum_over_time(metric[50s:5s]) +# {} 4 + +# Every evaluation yields the last value, i.e. 2 +# FAILING issue #38. eval instant at 5m sum_over_time(metric[50s:10s]) +# {} 12 + +# Series becomes stale at 5m10s (5m after last sample) +# Hence subquery gets a single sample at 6m-50s=5m10s. +# FAILING issue #38. eval instant at 6m sum_over_time(metric[50s:10s]) +# {} 2 + +# FAILING issue #38. eval instant at 10s rate(metric[20s:10s]) +# {} 0.1 + +# FAILING issue #38. eval instant at 20s rate(metric[20s:5s]) +# {} 0.05 + +clear + +load 10s + http_requests{job="api-server", instance="1", group="production"} 0+20x1000 200+30x1000 + http_requests{job="api-server", instance="0", group="production"} 0+10x1000 100+30x1000 + http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300+80x1000 + http_requests{job="api-server", instance="1", group="canary"} 0+40x2000 + +# FAILING issue #38. eval instant at 8000s rate(http_requests{group=~"pro.*"}[1m:10s]) +# {job="api-server", instance="0", group="production"} 1 +# {job="api-server", instance="1", group="production"} 2 + +# FAILING issue #38. eval instant at 20000s avg_over_time(rate(http_requests[1m])[1m:1s]) +# {job="api-server", instance="0", group="canary"} 8 +# {job="api-server", instance="1", group="canary"} 4 +# {job="api-server", instance="1", group="production"} 3 +# {job="api-server", instance="0", group="production"} 3 + +clear + +load 10s + metric1 0+1x1000 + metric2 0+2x1000 + metric3 0+3x1000 + +# FAILING issue #38. eval instant at 1000s sum_over_time(metric1[30s:10s]) +# {} 394 + +# This is (394*2 - 100), because other than the last 100 at 1000s, +# everything else is repeated with the 5s step. +# FAILING issue #38. eval instant at 1000s sum_over_time(metric1[30s:5s]) +# {} 688 + +# Offset is aligned with the step. +# FAILING issue #38. eval instant at 1010s sum_over_time(metric1[30s:10s] offset 10s) +# {} 394 + +# Same result for different offsets due to step alignment. +# FAILING issue #38. eval instant at 1010s sum_over_time(metric1[30s:10s] offset 9s) +# {} 297 + +# FAILING issue #38. eval instant at 1010s sum_over_time(metric1[30s:10s] offset 7s) +# {} 297 + +# FAILING issue #38. eval instant at 1010s sum_over_time(metric1[30s:10s] offset 5s) +# {} 297 + +# FAILING issue #38. eval instant at 1010s sum_over_time(metric1[30s:10s] offset 3s) +# {} 297 + +# FAILING issue #38. eval instant at 1010s sum_over_time((metric1)[30s:10s] offset 3s) +# {} 297 + +# Nested subqueries +# FAILING issue #38. eval instant at 1000s rate(sum_over_time(metric1[30s:10s])[50s:10s]) +# {} 0.4 + +# FAILING issue #38. eval instant at 1000s rate(sum_over_time(metric2[30s:10s])[50s:10s]) +# {} 0.8 + +# FAILING issue #38. eval instant at 1000s rate(sum_over_time(metric3[30s:10s])[50s:10s]) +# {} 1.2 + +# FAILING issue #38. eval instant at 1000s rate(sum_over_time((metric1+metric2+metric3)[30s:10s])[30s:10s]) +# {} 2.4 + +clear + +# Fibonacci sequence, to ensure the rate is not constant. +# Additional note: using subqueries unnecessarily is unwise. +load 7s + metric 1 1 2 3 5 8 13 21 34 55 89 144 233 377 610 987 1597 2584 4181 6765 10946 17711 28657 46368 75025 121393 196418 317811 514229 832040 1346269 2178309 3524578 5702887 9227465 14930352 24157817 39088169 63245986 102334155 165580141 267914296 433494437 701408733 1134903170 1836311903 2971215073 4807526976 7778742049 12586269025 20365011074 32951280099 53316291173 86267571272 139583862445 225851433717 365435296162 591286729879 956722026041 1548008755920 2504730781961 4052739537881 6557470319842 10610209857723 17167680177565 27777890035288 44945570212853 72723460248141 117669030460994 190392490709135 308061521170129 498454011879264 806515533049393 1304969544928657 2111485077978050 3416454622906707 5527939700884757 8944394323791464 14472334024676221 23416728348467685 37889062373143906 61305790721611591 99194853094755497 160500643816367088 259695496911122585 420196140727489673 679891637638612258 1100087778366101931 1779979416004714189 2880067194370816120 4660046610375530309 7540113804746346429 12200160415121876738 19740274219868223167 31940434634990099905 51680708854858323072 83621143489848422977 135301852344706746049 218922995834555169026 354224848179261915075 573147844013817084101 927372692193078999176 1500520536206896083277 2427893228399975082453 3928413764606871165730 6356306993006846248183 10284720757613717413913 16641027750620563662096 26925748508234281076009 43566776258854844738105 70492524767089125814114 114059301025943970552219 184551825793033096366333 298611126818977066918552 483162952612010163284885 781774079430987230203437 1264937032042997393488322 2046711111473984623691759 3311648143516982017180081 5358359254990966640871840 8670007398507948658051921 14028366653498915298923761 22698374052006863956975682 36726740705505779255899443 59425114757512643212875125 96151855463018422468774568 155576970220531065681649693 251728825683549488150424261 407305795904080553832073954 659034621587630041982498215 1066340417491710595814572169 1725375039079340637797070384 2791715456571051233611642553 4517090495650391871408712937 7308805952221443105020355490 11825896447871834976429068427 19134702400093278081449423917 30960598847965113057878492344 50095301248058391139327916261 81055900096023504197206408605 131151201344081895336534324866 212207101440105399533740733471 343358302784187294870275058337 555565404224292694404015791808 898923707008479989274290850145 1454489111232772683678306641953 2353412818241252672952597492098 3807901929474025356630904134051 6161314747715278029583501626149 9969216677189303386214405760200 16130531424904581415797907386349 26099748102093884802012313146549 42230279526998466217810220532898 68330027629092351019822533679447 110560307156090817237632754212345 178890334785183168257455287891792 289450641941273985495088042104137 468340976726457153752543329995929 757791618667731139247631372100066 1226132595394188293000174702095995 1983924214061919432247806074196061 3210056809456107725247980776292056 5193981023518027157495786850488117 8404037832974134882743767626780173 13598018856492162040239554477268290 22002056689466296922983322104048463 35600075545958458963222876581316753 57602132235424755886206198685365216 93202207781383214849429075266681969 150804340016807970735635273952047185 244006547798191185585064349218729154 394810887814999156320699623170776339 638817435613190341905763972389505493 1033628323428189498226463595560281832 1672445759041379840132227567949787325 2706074082469569338358691163510069157 4378519841510949178490918731459856482 7084593923980518516849609894969925639 11463113765491467695340528626429782121 18547707689471986212190138521399707760 + +# Extrapolated from [3@21, 144@77]: (144 - 3) / (77 - 21) +eval instant at 80s rate(metric[1m]) + {} 2.517857143 + +# No extrapolation, [2@20, 144@80]: (144 - 2) / 60 +# FAILING issue #38. eval instant at 80s rate(metric[1m:10s]) +# {} 2.366666667 + +# Only one value between 10s and 20s, 2@14 +eval instant at 20s min_over_time(metric[10s]) + {} 2 + +# min(1@10, 2@20) +# FAILING issue #38. eval instant at 20s min_over_time(metric[10s:10s]) +# {} 1 + +# FAILING issue #38. eval instant at 20m min_over_time(rate(metric[5m])[20m:1m]) +# {} 0.12119047619047618 +